From 14022a7ed317e8b2b1935eb915ae0eccfc153a28 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 19 May 2016 15:52:15 -0700 Subject: [PATCH] Alternative groupBy strategy. This patch introduces a GroupByStrategy concept and two strategies: "v1" is the current groupBy strategy and "v2" is a new one. It also introduces a merge buffers concept in DruidProcessingModule, to try to better manage memory used for merging. Both of these are described in more detail in #2987. There are two goals of this patch: 1. Make it possible for historical/realtime nodes to return larger groupBy result sets, faster, with better memory management. 2. Make it possible for brokers to merge streams when there are no order-by columns, avoiding materialization. This patch does not do anything to help with memory management on the broker when there are order-by columns or when there are nested queries. That could potentially be done in a future patch. --- .../benchmark/query/GroupByBenchmark.java | 96 ++- .../druid/benchmark/query/TopNBenchmark.java | 5 +- .../io/druid/collections/BlockingPool.java | 119 +++ .../druid/collections/LoadBalancingPool.java | 4 +- .../io/druid/collections/ResourceHolder.java | 3 + .../java/io/druid/collections/StupidPool.java | 2 +- .../collections/StupidResourceHolder.java | 2 +- docs/content/configuration/broker.md | 12 +- docs/content/configuration/historical.md | 12 +- docs/content/configuration/realtime.md | 12 +- docs/content/querying/groupbyquery.md | 103 ++- .../DistinctCountGroupByQueryTest.java | 28 +- .../ApproximateHistogramGroupByQueryTest.java | 97 +-- .../io/druid/guice/annotations/Merging.java | 34 + .../io/druid/query/DruidProcessingConfig.java | 9 +- .../io/druid/query/groupby/GroupByQuery.java | 70 ++ .../query/groupby/GroupByQueryConfig.java | 44 ++ .../query/groupby/GroupByQueryHelper.java | 41 + .../groupby/GroupByQueryQueryToolChest.java | 86 +- .../groupby/GroupByQueryRunnerFactory.java | 51 +- .../groupby/epinephelinae/BufferGrouper.java | 492 ++++++++++++ .../CloseableGrouperIterator.java | 67 ++ .../epinephelinae/ConcurrentGrouper.java | 143 ++++ .../epinephelinae/GroupByBinaryFnV2.java | 84 ++ .../GroupByMergingQueryRunnerV2.java | 673 ++++++++++++++++ .../epinephelinae/GroupByQueryEngineV2.java | 392 ++++++++++ .../query/groupby/epinephelinae/Grouper.java | 220 ++++++ .../query/groupby/epinephelinae/Groupers.java | 63 ++ .../LimitedTemporaryStorage.java | 171 ++++ .../epinephelinae/SpillingGrouper.java | 205 +++++ .../groupby/orderby/DefaultLimitSpec.java | 41 +- .../query/groupby/orderby/LimitSpec.java | 9 + .../groupby/strategy/GroupByStrategy.java | 48 ++ .../strategy/GroupByStrategySelector.java | 65 ++ .../groupby/strategy/GroupByStrategyV1.java | 129 +++ .../groupby/strategy/GroupByStrategyV2.java | 211 +++++ .../data/CompressedObjectStrategy.java | 16 +- .../incremental/OffheapIncrementalIndex.java | 22 +- .../aggregation/AggregationTestHelper.java | 37 +- .../GroupByQueryRunnerFactoryTest.java | 40 +- .../query/groupby/GroupByQueryRunnerTest.java | 738 ++++++++++++++---- .../GroupByTimeseriesQueryRunnerTest.java | 27 +- .../epinephelinae/BufferGrouperTest.java | 162 ++++ .../epinephelinae/GrouperTestUtil.java | 38 + .../groupby/epinephelinae/IntKeySerde.java | 83 ++ .../TestColumnSelectorFactory.java | 89 +++ .../groupby/orderby/DefaultLimitSpecTest.java | 34 +- .../java/io/druid/segment/TestHelper.java | 55 +- .../io/druid/client/cache/MemcachedCache.java | 9 - .../io/druid/guice/DruidProcessingModule.java | 44 +- ...rPool.java => OffheapBufferGenerator.java} | 42 +- .../client/CachingClusteredClientTest.java | 78 +- .../segment/realtime/RealtimeManagerTest.java | 25 +- 53 files changed, 4690 insertions(+), 692 deletions(-) create mode 100644 common/src/main/java/io/druid/collections/BlockingPool.java create mode 100644 processing/src/main/java/io/druid/guice/annotations/Merging.java create mode 100644 processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferGrouper.java create mode 100644 processing/src/main/java/io/druid/query/groupby/epinephelinae/CloseableGrouperIterator.java create mode 100644 processing/src/main/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouper.java create mode 100644 processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByBinaryFnV2.java create mode 100644 processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java create mode 100644 processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java create mode 100644 processing/src/main/java/io/druid/query/groupby/epinephelinae/Grouper.java create mode 100644 processing/src/main/java/io/druid/query/groupby/epinephelinae/Groupers.java create mode 100644 processing/src/main/java/io/druid/query/groupby/epinephelinae/LimitedTemporaryStorage.java create mode 100644 processing/src/main/java/io/druid/query/groupby/epinephelinae/SpillingGrouper.java create mode 100644 processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategy.java create mode 100644 processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategySelector.java create mode 100644 processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV1.java create mode 100644 processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java create mode 100644 processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferGrouperTest.java create mode 100644 processing/src/test/java/io/druid/query/groupby/epinephelinae/GrouperTestUtil.java create mode 100644 processing/src/test/java/io/druid/query/groupby/epinephelinae/IntKeySerde.java create mode 100644 processing/src/test/java/io/druid/query/groupby/epinephelinae/TestColumnSelectorFactory.java rename server/src/main/java/io/druid/offheap/{OffheapBufferPool.java => OffheapBufferGenerator.java} (58%) diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java index 693653c5f4ed..a5c3254cd4e8 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java @@ -20,6 +20,7 @@ package io.druid.benchmark.query; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.collect.Lists; @@ -32,13 +33,16 @@ import io.druid.benchmark.datagen.BenchmarkDataGenerator; import io.druid.benchmark.datagen.BenchmarkSchemaInfo; import io.druid.benchmark.datagen.BenchmarkSchemas; +import io.druid.collections.BlockingPool; +import io.druid.collections.StupidPool; import io.druid.concurrent.Execs; import io.druid.data.input.InputRow; import io.druid.data.input.Row; import io.druid.data.input.impl.DimensionsSpec; import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; -import io.druid.offheap.OffheapBufferPool; +import io.druid.offheap.OffheapBufferGenerator; +import io.druid.query.DruidProcessingConfig; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; @@ -54,6 +58,9 @@ import io.druid.query.groupby.GroupByQueryEngine; import io.druid.query.groupby.GroupByQueryQueryToolChest; import io.druid.query.groupby.GroupByQueryRunnerFactory; +import io.druid.query.groupby.strategy.GroupByStrategySelector; +import io.druid.query.groupby.strategy.GroupByStrategyV1; +import io.druid.query.groupby.strategy.GroupByStrategyV2; import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.query.spec.QuerySegmentSpec; import io.druid.segment.IncrementalIndexSegment; @@ -67,7 +74,6 @@ import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.incremental.OnheapIncrementalIndex; import io.druid.segment.serde.ComplexMetrics; - import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -83,6 +89,7 @@ import java.io.File; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.LinkedHashMap; @@ -100,12 +107,21 @@ public class GroupByBenchmark @Param({"4"}) private int numSegments; + @Param({"4"}) + private int numProcessingThreads; + + @Param({"-1"}) + private int initialBuckets; + @Param({"100000"}) private int rowsPerSegment; @Param({"basic.A"}) private String schemaAndQuery; + @Param({"v1", "v2"}) + private String defaultStrategy; + private static final Logger log = new Logger(GroupByBenchmark.class); private static final int RNG_SEED = 9999; private static final IndexMergerV9 INDEX_MERGER_V9; @@ -186,7 +202,7 @@ public void setup() throws IOException if (ComplexMetrics.getSerdeForType("hyperUnique") == null) { ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde(Hashing.murmur3_128())); } - executorService = Execs.multiThreaded(numSegments, "GroupByThreadPool"); + executorService = Execs.multiThreaded(numProcessingThreads, "GroupByThreadPool[%d]"); setupQueries(); @@ -237,25 +253,75 @@ public void setup() throws IOException qIndexes.add(qIndex); } - OffheapBufferPool bufferPool = new OffheapBufferPool(250000000, Integer.MAX_VALUE); - OffheapBufferPool bufferPool2 = new OffheapBufferPool(250000000, Integer.MAX_VALUE); - final GroupByQueryConfig config = new GroupByQueryConfig(); + StupidPool bufferPool = new StupidPool<>( + new OffheapBufferGenerator("compute", 250000000), + Integer.MAX_VALUE + ); + BlockingPool mergePool = new BlockingPool<>( + new OffheapBufferGenerator("merge", 250000000), + 1 + ); + final GroupByQueryConfig config = new GroupByQueryConfig() + { + @Override + public String getDefaultStrategy() + { + return defaultStrategy; + } + + @Override + public int getBufferGrouperInitialBuckets() + { + return initialBuckets; + } + }; config.setSingleThreaded(false); - config.setMaxIntermediateRows(1000000); - config.setMaxResults(1000000); + config.setMaxIntermediateRows(Integer.MAX_VALUE); + config.setMaxResults(Integer.MAX_VALUE); + + DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig() + { + @Override + public int getNumThreads() + { + // Used by "v2" strategy for concurrencyHint + return numProcessingThreads; + } + + @Override + public String getFormatString() + { + return null; + } + }; final Supplier configSupplier = Suppliers.ofInstance(config); - final GroupByQueryEngine engine = new GroupByQueryEngine(configSupplier, bufferPool); + final GroupByStrategySelector strategySelector = new GroupByStrategySelector( + configSupplier, + new GroupByStrategyV1( + configSupplier, + new GroupByQueryEngine(configSupplier, bufferPool), + QueryBenchmarkUtil.NOOP_QUERYWATCHER, + bufferPool + ), + new GroupByStrategyV2( + druidProcessingConfig, + configSupplier, + bufferPool, + mergePool, + new ObjectMapper(new SmileFactory()), + QueryBenchmarkUtil.NOOP_QUERYWATCHER + ) + ); factory = new GroupByQueryRunnerFactory( - engine, - QueryBenchmarkUtil.NOOP_QUERYWATCHER, - configSupplier, + strategySelector, new GroupByQueryQueryToolChest( - configSupplier, JSON_MAPPER, engine, bufferPool2, + configSupplier, + strategySelector, + bufferPool, QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator() - ), - bufferPool2 + ) ); } diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/TopNBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/TopNBenchmark.java index 12646f6e867c..50a7eecb164b 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/TopNBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/TopNBenchmark.java @@ -30,12 +30,13 @@ import io.druid.benchmark.datagen.BenchmarkDataGenerator; import io.druid.benchmark.datagen.BenchmarkSchemaInfo; import io.druid.benchmark.datagen.BenchmarkSchemas; +import io.druid.collections.StupidPool; import io.druid.concurrent.Execs; import io.druid.data.input.InputRow; import io.druid.data.input.impl.DimensionsSpec; import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; -import io.druid.offheap.OffheapBufferPool; +import io.druid.offheap.OffheapBufferGenerator; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; @@ -236,7 +237,7 @@ public void setup() throws IOException } factory = new TopNQueryRunnerFactory( - new OffheapBufferPool(250000000, Integer.MAX_VALUE), + new StupidPool<>(new OffheapBufferGenerator("compute", 250000000), Integer.MAX_VALUE), new TopNQueryQueryToolChest(new TopNQueryConfig(), QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()), QueryBenchmarkUtil.NOOP_QUERYWATCHER ); diff --git a/common/src/main/java/io/druid/collections/BlockingPool.java b/common/src/main/java/io/druid/collections/BlockingPool.java new file mode 100644 index 000000000000..5dbb366304bb --- /dev/null +++ b/common/src/main/java/io/druid/collections/BlockingPool.java @@ -0,0 +1,119 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.collections; + +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; + +import java.io.IOException; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Pool that pre-generates objects up to a limit, then permits possibly-blocking "take" operations. + */ +public class BlockingPool +{ + private static final Logger log = new Logger(BlockingPool.class); + + private final BlockingQueue objects; + + public BlockingPool( + Supplier generator, + int limit + ) + { + this.objects = limit > 0 ? new ArrayBlockingQueue(limit) : null; + + for (int i = 0; i < limit; i++) { + objects.add(generator.get()); + } + } + + /** + * Take a resource from the pool. + * + * @param timeout maximum time to wait for a resource, in milliseconds. Negative means do not use a timeout. + * + * @return a resource, or null if the timeout was reached + * + * @throws InterruptedException if interrupted while waiting for a resource to become available + */ + public ResourceHolder take(final long timeout) throws InterruptedException + { + Preconditions.checkState(objects != null, "Pool was initialized with limit = 0, there are no objects to take."); + final T theObject = timeout >= 0 ? objects.poll(timeout, TimeUnit.MILLISECONDS) : objects.take(); + return theObject == null ? null : new ObjectResourceHolder(theObject); + } + + /** + * Similar to StupidPool.ObjectResourceHolder, except this one has no objectsCacheMaxCount, and it returns objects + * to the pool on finalize. + */ + private class ObjectResourceHolder implements ResourceHolder + { + private AtomicBoolean closed = new AtomicBoolean(false); + private final T object; + + public ObjectResourceHolder(final T object) + { + this.object = object; + } + + // WARNING: it is entirely possible for a caller to hold onto the object and call "close", then still use that + // object even though it will be offered to someone else in BlockingPool.take + @Override + public T get() + { + if (closed.get()) { + throw new ISE("Already Closed!"); + } + + return object; + } + + @Override + public void close() + { + if (!closed.compareAndSet(false, true)) { + log.warn(new ISE("Already Closed!"), "Already closed"); + return; + } + if (!objects.offer(object)) { + throw new ISE("WTF?! Queue offer failed"); + } + } + + @Override + protected void finalize() throws Throwable + { + if (closed.compareAndSet(false, true)) { + log.warn("Not closed! Object was[%s]. Returning to pool.", object); + if (!objects.offer(object)) { + log.error("WTF?! Queue offer failed during finalize, uh oh..."); + } + } + } + } +} diff --git a/common/src/main/java/io/druid/collections/LoadBalancingPool.java b/common/src/main/java/io/druid/collections/LoadBalancingPool.java index fd6f0e6d91d1..6b6bbc87097f 100644 --- a/common/src/main/java/io/druid/collections/LoadBalancingPool.java +++ b/common/src/main/java/io/druid/collections/LoadBalancingPool.java @@ -104,11 +104,9 @@ public T get() /** * Not idempotent, should only be called once when done using the resource - * - * @throws IOException */ @Override - public void close() throws IOException + public void close() { // ensures count always gets adjusted while item is removed from the queue synchronized (this) { diff --git a/common/src/main/java/io/druid/collections/ResourceHolder.java b/common/src/main/java/io/druid/collections/ResourceHolder.java index b5b827a10b58..5d09be70471b 100644 --- a/common/src/main/java/io/druid/collections/ResourceHolder.java +++ b/common/src/main/java/io/druid/collections/ResourceHolder.java @@ -26,4 +26,7 @@ public interface ResourceHolder extends Closeable { T get(); + + @Override + void close(); } diff --git a/common/src/main/java/io/druid/collections/StupidPool.java b/common/src/main/java/io/druid/collections/StupidPool.java index 8c5937ad3359..bb60707139e5 100644 --- a/common/src/main/java/io/druid/collections/StupidPool.java +++ b/common/src/main/java/io/druid/collections/StupidPool.java @@ -87,7 +87,7 @@ public T get() } @Override - public void close() throws IOException + public void close() { if (!closed.compareAndSet(false, true)) { log.warn(new ISE("Already Closed!"), "Already closed"); diff --git a/common/src/main/java/io/druid/collections/StupidResourceHolder.java b/common/src/main/java/io/druid/collections/StupidResourceHolder.java index 594134115618..6e975c652f2c 100644 --- a/common/src/main/java/io/druid/collections/StupidResourceHolder.java +++ b/common/src/main/java/io/druid/collections/StupidResourceHolder.java @@ -46,7 +46,7 @@ public T get() } @Override - public void close() throws IOException + public void close() { // Do nothing } diff --git a/docs/content/configuration/broker.md b/docs/content/configuration/broker.md index a924ed28b32a..230de4c4e8f4 100644 --- a/docs/content/configuration/broker.md +++ b/docs/content/configuration/broker.md @@ -51,24 +51,26 @@ Druid broker can optionally retry queries internally for transient errors. The broker uses processing configs for nested groupBy queries. And, optionally, Long-interval queries (of any type) can be broken into shorter interval queries and processed in parallel inside this thread pool. For more details, see "chunkPeriod" in [Query Context](../querying/query-context.html) doc. - |Property|Description|Default| |--------|-----------|-------| |`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|1073741824 (1GB)| |`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE| |`druid.processing.formatString`|Realtime and historical nodes use this format string to name their processing threads.|processing-%s| +|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. By default, no queries use these buffers, so the default pool size is zero.|0| |`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)| |`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require additional garbage collection tuning to avoid long GC pauses.|`0` (disabled)| |`druid.processing.fifo`|If the processing queue should treat tasks of equal priority in a FIFO manner|`false`| +The amount of direct memory needed by Druid is at least +`druid.processing.buffer.sizeBytes * (druid.processing.numMergeBuffers + druid.processing.numThreads + 1)`. You can +ensure at least this amount of direct memory is available by providing `-XX:MaxDirectMemorySize=` at the command +line. + #### General Query Configuration ##### GroupBy Query Config -|Property|Description|Default| -|--------|-----------|-------| -|`druid.query.groupBy.maxIntermediateRows`|Maximum number of intermediate rows. This can be lowered at query time by `maxIntermediateRows` attribute in query context.|50000| -|`druid.query.groupBy.maxResults`|Maximum number of results. This can be lowered at query time by `maxResults` attribute in query context.|500000| +See [groupBy server configuration](../querying/groupbyquery.html#server-configuration). ##### Search Query Config diff --git a/docs/content/configuration/historical.md b/docs/content/configuration/historical.md index bd0417f01c69..b8d2645e217f 100644 --- a/docs/content/configuration/historical.md +++ b/docs/content/configuration/historical.md @@ -56,19 +56,21 @@ Druid uses Jetty to serve HTTP requests. |`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|1073741824 (1GB)| |`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE| |`druid.processing.formatString`|Realtime and historical nodes use this format string to name their processing threads.|processing-%s| +|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. By default, no queries use these buffers, so the default pool size is zero.|0| |`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)| |`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require additional garbage collection tuning to avoid long GC pauses.|`0` (disabled)| |`druid.processing.fifo`|If the processing queue should treat tasks of equal priority in a FIFO manner|`false`| +The amount of direct memory needed by Druid is at least +`druid.processing.buffer.sizeBytes * (druid.processing.numMergeBuffers + druid.processing.numThreads + 1)`. You can +ensure at least this amount of direct memory is available by providing `-XX:MaxDirectMemorySize=` at the command +line. + #### General Query Configuration ##### GroupBy Query Config -|Property|Description|Default| -|--------|-----------|-------| -|`druid.query.groupBy.singleThreaded`|Run single threaded group By queries.|false| -|`druid.query.groupBy.maxIntermediateRows`|Maximum number of intermediate rows. This can be lowered at query time by `maxIntermediateRows` attribute in query context.|50000| -|`druid.query.groupBy.maxResults`|Maximum number of results. This can be lowered at query time by `maxResults` attribute in query context.|500000| +See [groupBy server configuration](../querying/groupbyquery.html#server-configuration). ##### Search Query Config diff --git a/docs/content/configuration/realtime.md b/docs/content/configuration/realtime.md index 6ca00ae2b5df..a1e18b3dadf8 100644 --- a/docs/content/configuration/realtime.md +++ b/docs/content/configuration/realtime.md @@ -41,18 +41,20 @@ The realtime node uses several of the global configs in [Configuration](../confi |--------|-----------|-------| |`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|1073741824 (1GB)| |`druid.processing.formatString`|Realtime and historical nodes use this format string to name their processing threads.|processing-%s| +|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. By default, no queries use these buffers, so the default pool size is zero.|0| |`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)| |`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require additional garbage collection tuning to avoid long GC pauses.|`0` (disabled)| +The amount of direct memory needed by Druid is at least +`druid.processing.buffer.sizeBytes * (druid.processing.numMergeBuffers + druid.processing.numThreads + 1)`. You can +ensure at least this amount of direct memory is available by providing `-XX:MaxDirectMemorySize=` at the command +line. + #### General Query Configuration ##### GroupBy Query Config -|Property|Description|Default| -|--------|-----------|-------| -|`druid.query.groupBy.singleThreaded`|Run single threaded group By queries.|false| -|`druid.query.groupBy.maxIntermediateRows`|Maximum number of intermediate rows.|50000| -|`druid.query.groupBy.maxResults`|Maximum number of results.|500000| +See [groupBy server configuration](../querying/groupbyquery.html#server-configuration). ##### Search Query Config diff --git a/docs/content/querying/groupbyquery.md b/docs/content/querying/groupbyquery.md index 7a1c17a232ad..62000c561fde 100644 --- a/docs/content/querying/groupbyquery.md +++ b/docs/content/querying/groupbyquery.md @@ -2,10 +2,16 @@ layout: doc_page --- # groupBy Queries + These types of queries take a groupBy query object and return an array of JSON objects where each object represents a -grouping asked for by the query. Note: If you only want to do straight aggregates for some time range, we highly recommend -using [TimeseriesQueries](../querying/timeseriesquery.html) instead. The performance will be substantially better. If you want to -do an ordered groupBy over a single dimension, please look at [TopN](../querying/topnquery.html) queries. The performance for that use case is also substantially better. +grouping asked for by the query. + +
+Note: If you are doing aggregations with time as your only grouping, or an ordered groupBy over a single dimension, +consider Timeseries and TopN queries as well as +groupBy. Their performance may be better in some cases. See Alternatives below for more details. +
+ An example groupBy query object is shown below: ``` json @@ -106,3 +112,94 @@ your filter, you can use a [filtered dimensionSpec](dimensionspecs.html#filtered improve performance. See [Multi-value dimensions](multi-value-dimensions.html) for more details. + +### Implementation details + +#### Strategies + +GroupBy queries can be executed using two different strategies. The default strategy for a cluster is determined by the +"druid.query.groupBy.defaultStrategy" runtime property on the broker. This can be overridden using "groupByStrategy" in +the query context. If neither the context field nor the property is set, the "v1" strategy will be used. + +- "v1", the default, generates per-segment results on data nodes (historical, realtime, middleManager) using a map which +is partially on-heap (dimension keys and the map itself) and partially off-heap (the aggregated values). Data nodes then +merge the per-segment results using Druid's indexing mechanism. This merging is multi-threaded by default, but can +optionally be single-threaded. The broker merges the final result set using Druid's indexing mechanism again. The broker +merging is always single-threaded. Because the broker merges results using the indexing mechanism, it must materialize +the full result set before returning any results. On both the data nodes and the broker, the merging index is fully +on-heap by default, but it can optionally store aggregated values off-heap. + +- "v2" (experimental) is designed to offer better performance and memory management. This strategy generates +per-segment results using a fully off-heap map. Data nodes merge the per-segment results using a fully off-heap +concurrent facts map combined with an on-heap string dictionary. This may optionally involve spilling to disk. Data +nodes return sorted results to the broker, which merges result streams using an N-way merge. The broker materializes +the results if necessary (e.g. if the query sorts on columns other than its dimensions). Otherwise, it streams results +back as they are merged. + +#### Alternatives + +There are some situations where other query types may be a better choice than groupBy. + +- For queries with no "dimensions" (i.e. grouping by time only) the [Timeseries query](timeseriesquery.html) will +generally be faster than groupBy. The major differences are that it is implemented in a fully streaming manner (taking +advantage of the fact that segments are already sorted on time) and does not need to use a hash table for merging. + +- For queries with a single "dimensions" element (i.e. grouping by one string dimension), the [TopN query](topnquery.html) +will sometimes be faster than groupBy. This is especially true if you are ordering by a metric and find approximate +results acceptable. + +#### Nested groupBys + +Nested groupBys (dataSource of type "query") are performed the same way for both "v1" and "v2". The broker runs the +inner groupBy query in the usual way, then materializes the inner query's results, then runs the outer query on those +materialized results. In particular, the outer query is not distributed at all; it takes place completely on the broker. +Currently the materialized results are stored on-heap in the broker, and the outer query is done in a single-threaded +fashion. + +#### Server configuration + +When using the "v1" strategy, the following runtime properties apply: + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.query.groupBy.defaultStrategy`|Default groupBy query strategy.|v1| +|`druid.query.groupBy.maxIntermediateRows`|Maximum number of intermediate rows for the per-segment grouping engine. This is a tuning parameter that does not impose a hard limit; rather, it potentially shifts merging work from the per-segment engine to the overall merging index. Queries that exceed this limit will not fail.|50000| +|`druid.query.groupBy.maxResults`|Maximum number of results. Queries that exceed this limit will fail.|500000| +|`druid.query.groupBy.singleThreaded`|Merge results using a single thread.|false| + +When using the "v2" strategy, the following runtime properties apply: + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.query.groupBy.defaultStrategy`|Default groupBy query strategy.|v1| +|`druid.query.groupBy.bufferGrouperInitialBuckets`|Initial number of buckets in the off-heap hash table used for grouping results. Set to -1 to use a reasonable default.|-1| +|`druid.query.groupBy.maxMergingDictionarySize`|Maximum amount of heap space (approximately) to use for the string dictionary during merging. When the dictionary exceeds this size, a spill to disk will be triggered.|25000000| +|`druid.query.groupBy.maxOnDiskStorage`|Maximum amount of disk space to use, per-query, for spilling result sets to disk when either the merging buffer or the dictionary fills up. Queries that exceed this limit will fail. Set to zero to disable disk spilling.|0 (disabled)| + +Additionally, the "v2" strategy uses merging buffers for merging. It is currently the only query implementation that +does so. By default, Druid is configured without any merging buffer pool, so to use the "v2" strategy you must also +set `druid.processing.numMergeBuffers` to some non-zero number. + +This may require allocating more direct memory. The amount of direct memory needed by Druid is at least +`druid.processing.buffer.sizeBytes * (druid.processing.numMergeBuffers + druid.processing.numThreads + 1)`. You can +ensure at least this amount of direct memory is available by providing `-XX:MaxDirectMemorySize=` at the command +line. + +#### Query context + +When using the "v1" strategy, the following query context parameters apply: + +|Property|Description| +|--------|-----------| +|`groupByStrategy`|Overrides the value of `druid.query.groupBy.defaultStrategy` for this query.| +|`groupByIsSingleThreaded`|Overrides the value of `druid.query.groupBy.singleThreaded` for this query.| +|`maxIntermediateRows`|Can be used to lower the value of `druid.query.groupBy.maxIntermediateRows` for this query.| +|`maxResults`|Can be used to lower the value of `druid.query.groupBy.maxResults` for this query.| +|`useOffheap`|Set to true to store aggregations off-heap when merging results.| + +When using the "v2" strategy, the following query context parameters apply: + +|Property|Description| +|--------|-----------| +|`groupByStrategy`|Overrides the value of `druid.query.groupBy.defaultStrategy` for this query.| +|`bufferGrouperInitialBuckets`|Overrides the value of `druid.query.groupBy.bufferGrouperInitialBuckets` for this query.| diff --git a/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java b/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java index 81a2319c00e8..28e7ac878e7d 100644 --- a/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java +++ b/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java @@ -39,6 +39,7 @@ import io.druid.query.groupby.GroupByQueryEngine; import io.druid.query.groupby.GroupByQueryQueryToolChest; import io.druid.query.groupby.GroupByQueryRunnerFactory; +import io.druid.query.groupby.GroupByQueryRunnerTest; import io.druid.query.groupby.GroupByQueryRunnerTestHelper; import io.druid.query.groupby.orderby.DefaultLimitSpec; import io.druid.query.groupby.orderby.OrderByColumnSpec; @@ -59,34 +60,9 @@ public class DistinctCountGroupByQueryTest @Test public void testGroupByWithDistinctCountAgg() throws Exception { - final ObjectMapper mapper = new DefaultObjectMapper(); - final StupidPool pool = new StupidPool( - new Supplier() - { - @Override - public ByteBuffer get() - { - return ByteBuffer.allocate(1024 * 1024); - } - } - ); - final GroupByQueryConfig config = new GroupByQueryConfig(); config.setMaxIntermediateRows(10000); - - final Supplier configSupplier = Suppliers.ofInstance(config); - final GroupByQueryEngine engine = new GroupByQueryEngine(configSupplier, pool); - - final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( - engine, - QueryRunnerTestHelper.NOOP_QUERYWATCHER, - configSupplier, - new GroupByQueryQueryToolChest( - configSupplier, mapper, engine, pool, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ), - pool - ); + final GroupByQueryRunnerFactory factory = GroupByQueryRunnerTest.makeQueryRunnerFactory(config); IncrementalIndex index = new OnheapIncrementalIndex( 0, QueryGranularities.SECOND, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, 1000 diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java index a7de51d4afb7..c846acd906ec 100644 --- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java +++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java @@ -19,15 +19,9 @@ package io.druid.query.aggregation.histogram; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; -import com.google.common.base.Supplier; -import com.google.common.base.Suppliers; -import com.google.common.collect.Iterables; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import io.druid.collections.StupidPool; import io.druid.data.input.Row; -import io.druid.jackson.DefaultObjectMapper; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.aggregation.PostAggregator; @@ -35,20 +29,18 @@ import io.druid.query.dimension.DimensionSpec; import io.druid.query.groupby.GroupByQuery; import io.druid.query.groupby.GroupByQueryConfig; -import io.druid.query.groupby.GroupByQueryEngine; -import io.druid.query.groupby.GroupByQueryQueryToolChest; import io.druid.query.groupby.GroupByQueryRunnerFactory; +import io.druid.query.groupby.GroupByQueryRunnerTest; import io.druid.query.groupby.GroupByQueryRunnerTestHelper; import io.druid.query.groupby.orderby.DefaultLimitSpec; import io.druid.query.groupby.orderby.OrderByColumnSpec; +import io.druid.query.groupby.strategy.GroupByStrategySelector; import io.druid.segment.TestHelper; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import javax.annotation.Nullable; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; @@ -63,36 +55,8 @@ public class ApproximateHistogramGroupByQueryTest @Parameterized.Parameters public static Iterable constructorFeeder() throws IOException { - final ObjectMapper mapper = new DefaultObjectMapper(); - final StupidPool pool = new StupidPool( - new Supplier() - { - @Override - public ByteBuffer get() - { - return ByteBuffer.allocate(1024 * 1024); - } - } - ); - - final GroupByQueryConfig config = new GroupByQueryConfig(); - config.setMaxIntermediateRows(10000); - - final Supplier configSupplier = Suppliers.ofInstance(config); - final GroupByQueryEngine engine = new GroupByQueryEngine(configSupplier, pool); - - final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( - engine, - QueryRunnerTestHelper.NOOP_QUERYWATCHER, - configSupplier, - new GroupByQueryQueryToolChest( - configSupplier, mapper, engine, pool, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ), - pool - ); - - GroupByQueryConfig singleThreadedConfig = new GroupByQueryConfig() + final GroupByQueryConfig defaultConfig = new GroupByQueryConfig(); + final GroupByQueryConfig singleThreadedConfig = new GroupByQueryConfig() { @Override public boolean isSingleThreaded() @@ -100,44 +64,33 @@ public boolean isSingleThreaded() return true; } }; - singleThreadedConfig.setMaxIntermediateRows(10000); - - final Supplier singleThreadedConfigSupplier = Suppliers.ofInstance(singleThreadedConfig); - final GroupByQueryEngine singleThreadEngine = new GroupByQueryEngine(singleThreadedConfigSupplier, pool); - - final GroupByQueryRunnerFactory singleThreadFactory = new GroupByQueryRunnerFactory( - singleThreadEngine, - QueryRunnerTestHelper.NOOP_QUERYWATCHER, - singleThreadedConfigSupplier, - new GroupByQueryQueryToolChest( - singleThreadedConfigSupplier, mapper, singleThreadEngine, pool, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ), - pool - ); - - - final Function function = new Function() + final GroupByQueryConfig v2Config = new GroupByQueryConfig() { @Override - public Object[] apply(@Nullable Object input) + public String getDefaultStrategy() { - return new Object[]{factory, input}; + return GroupByStrategySelector.STRATEGY_V2; } }; - return Lists.newArrayList( - Iterables.concat( - Iterables.transform( - QueryRunnerTestHelper.makeQueryRunners(factory), - function - ), - Iterables.transform( - QueryRunnerTestHelper.makeQueryRunners(singleThreadFactory), - function - ) - ) + defaultConfig.setMaxIntermediateRows(10000); + singleThreadedConfig.setMaxIntermediateRows(10000); + + final List constructors = Lists.newArrayList(); + final List configs = ImmutableList.of( + defaultConfig, + singleThreadedConfig, + v2Config ); + + for (GroupByQueryConfig config : configs) { + final GroupByQueryRunnerFactory factory = GroupByQueryRunnerTest.makeQueryRunnerFactory(config); + for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners(factory)) { + constructors.add(new Object[]{factory, runner}); + } + } + + return constructors; } public ApproximateHistogramGroupByQueryTest(GroupByQueryRunnerFactory factory, QueryRunner runner) diff --git a/processing/src/main/java/io/druid/guice/annotations/Merging.java b/processing/src/main/java/io/druid/guice/annotations/Merging.java new file mode 100644 index 000000000000..35da2ab5e0ee --- /dev/null +++ b/processing/src/main/java/io/druid/guice/annotations/Merging.java @@ -0,0 +1,34 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.guice.annotations; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@BindingAnnotation +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +public @interface Merging +{ +} diff --git a/processing/src/main/java/io/druid/query/DruidProcessingConfig.java b/processing/src/main/java/io/druid/query/DruidProcessingConfig.java index 73f7b007a373..c4cca2e83c6d 100644 --- a/processing/src/main/java/io/druid/query/DruidProcessingConfig.java +++ b/processing/src/main/java/io/druid/query/DruidProcessingConfig.java @@ -37,7 +37,8 @@ public int poolCacheMaxCount() return Integer.MAX_VALUE; } - @Override @Config(value = "${base_path}.numThreads") + @Override + @Config(value = "${base_path}.numThreads") public int getNumThreads() { // default to leaving one core for background tasks @@ -45,6 +46,12 @@ public int getNumThreads() return processors > 1 ? processors - 1 : processors; } + @Config("${base_path}.numMergeBuffers") + public int getNumMergeBuffers() + { + return 0; + } + @Config(value = "${base_path}.columnCache.sizeBytes") public int columnCacheSizeBytes() { diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java index 9abc6d6f317c..a21dc12f3f07 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java @@ -27,6 +27,8 @@ import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; +import com.google.common.collect.Ordering; +import com.google.common.primitives.Longs; import com.metamx.common.ISE; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; @@ -52,6 +54,7 @@ import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.Interval; +import java.util.Comparator; import java.util.List; import java.util.Map; @@ -218,6 +221,73 @@ public String getType() return GROUP_BY; } + @Override + public Ordering getResultOrdering() + { + final Comparator naturalNullsFirst = Ordering.natural().nullsFirst(); + final Ordering rowOrdering = getRowOrdering(false); + + return Ordering.from( + new Comparator() + { + @Override + public int compare(Object lhs, Object rhs) + { + if (lhs instanceof Row) { + return rowOrdering.compare((Row) lhs, (Row) rhs); + } else { + // Probably bySegment queries + return naturalNullsFirst.compare(lhs, rhs); + } + } + } + ); + } + + public Ordering getRowOrdering(final boolean granular) + { + final Comparator naturalNullsFirst = Ordering.natural().nullsFirst(); + + return Ordering.from( + new Comparator() + { + @Override + public int compare(Row lhs, Row rhs) + { + final int timeCompare; + + if (granular) { + timeCompare = Longs.compare( + granularity.truncate(lhs.getTimestampFromEpoch()), + granularity.truncate(rhs.getTimestampFromEpoch()) + ); + } else { + timeCompare = Longs.compare( + lhs.getTimestampFromEpoch(), + rhs.getTimestampFromEpoch() + ); + } + + if (timeCompare != 0) { + return timeCompare; + } + + for (DimensionSpec dimension : dimensions) { + final int dimCompare = naturalNullsFirst.compare( + lhs.getRaw(dimension.getOutputName()), + rhs.getRaw(dimension.getOutputName()) + ); + if (dimCompare != 0) { + return dimCompare; + } + } + + return 0; + } + } + ); + } + public Sequence applyLimit(Sequence results) { return limitFn.apply(results); diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java index 830e41ebb60b..6a669e443c5c 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java @@ -20,11 +20,15 @@ package io.druid.query.groupby; import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.query.groupby.strategy.GroupByStrategySelector; /** */ public class GroupByQueryConfig { + @JsonProperty + private String defaultStrategy = GroupByStrategySelector.STRATEGY_V1; + @JsonProperty private boolean singleThreaded = false; @@ -34,6 +38,26 @@ public class GroupByQueryConfig @JsonProperty private int maxResults = 500000; + @JsonProperty + // Not documented, only used for tests to force spilling + private int bufferGrouperMaxSize = Integer.MAX_VALUE; + + @JsonProperty + private int bufferGrouperInitialBuckets = -1; + + @JsonProperty + // Size of on-heap string dictionary for merging, per-query; when exceeded, partial results will be spilled to disk + private long maxMergingDictionarySize = 25_000_000L; + + @JsonProperty + // Max on-disk temporary storage, per-query; when exceeded, the query fails + private long maxOnDiskStorage = 0L; + + public String getDefaultStrategy() + { + return defaultStrategy; + } + public boolean isSingleThreaded() { return singleThreaded; @@ -63,4 +87,24 @@ public void setMaxResults(int maxResults) { this.maxResults = maxResults; } + + public int getBufferGrouperMaxSize() + { + return bufferGrouperMaxSize; + } + + public int getBufferGrouperInitialBuckets() + { + return bufferGrouperInitialBuckets; + } + + public long getMaxMergingDictionarySize() + { + return maxMergingDictionarySize; + } + + public long getMaxOnDiskStorage() + { + return maxOnDiskStorage; + } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java index 637dce3f077d..37ad1505d5a5 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java @@ -24,9 +24,12 @@ import com.metamx.common.ISE; import com.metamx.common.Pair; import com.metamx.common.guava.Accumulator; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; import io.druid.collections.StupidPool; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.MapBasedRow; +import io.druid.data.input.Row; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.dimension.DimensionSpec; @@ -159,4 +162,42 @@ public Queue accumulate(Queue accumulated, T in) }; return new Pair<>(init, accumulator); } + + // Used by GroupByQueryQueryToolChest, GroupByStrategyV1 + public static IncrementalIndex makeIncrementalIndex( + GroupByQuery query, + GroupByQueryConfig config, + StupidPool bufferPool, + Sequence rows + ) + { + Pair> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair( + query, + config, + bufferPool + ); + + return rows.accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); + } + + // Used by GroupByQueryQueryToolChest, GroupByStrategyV1 + public static Sequence postAggregate(final GroupByQuery query, IncrementalIndex index) + { + return Sequences.map( + Sequences.simple(index.iterableWithPostAggregations(query.getPostAggregatorSpecs(), query.isDescending())), + new Function() + { + @Override + public Row apply(Row input) + { + final MapBasedRow row = (MapBasedRow) input; + return new MapBasedRow( + query.getGranularity() + .toDateTime(row.getTimestampFromEpoch()), + row.getEvent() + ); + } + } + ); + } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index 8482fdd9e738..dd66d743f02d 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -20,7 +20,6 @@ package io.druid.query.groupby; import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Predicate; import com.google.common.base.Supplier; @@ -35,8 +34,6 @@ import com.google.inject.Inject; import com.metamx.common.IAE; import com.metamx.common.ISE; -import com.metamx.common.Pair; -import com.metamx.common.guava.Accumulator; import com.metamx.common.guava.ResourceClosingSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; @@ -64,6 +61,8 @@ import io.druid.query.dimension.DimensionSpec; import io.druid.query.extraction.ExtractionFn; import io.druid.query.filter.DimFilter; +import io.druid.query.groupby.strategy.GroupByStrategy; +import io.druid.query.groupby.strategy.GroupByStrategySelector; import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexStorageAdapter; @@ -92,28 +91,23 @@ public class GroupByQueryQueryToolChest extends QueryToolChest TYPE_REFERENCE = new TypeReference() { }; - private static final String GROUP_BY_MERGE_KEY = "groupByMerge"; + public static final String GROUP_BY_MERGE_KEY = "groupByMerge"; private final Supplier configSupplier; - + private final GroupByStrategySelector strategySelector; private final StupidPool bufferPool; - private final ObjectMapper jsonMapper; - private GroupByQueryEngine engine; // For running the outer query around a subquery - private final IntervalChunkingQueryRunnerDecorator intervalChunkingQueryRunnerDecorator; @Inject public GroupByQueryQueryToolChest( Supplier configSupplier, - ObjectMapper jsonMapper, - GroupByQueryEngine engine, + GroupByStrategySelector strategySelector, @Global StupidPool bufferPool, IntervalChunkingQueryRunnerDecorator intervalChunkingQueryRunnerDecorator ) { this.configSupplier = configSupplier; - this.jsonMapper = jsonMapper; - this.engine = engine; + this.strategySelector = strategySelector; this.bufferPool = bufferPool; this.intervalChunkingQueryRunnerDecorator = intervalChunkingQueryRunnerDecorator; } @@ -224,6 +218,7 @@ public boolean apply(AggregatorFactory agg) //is ensured by QuerySegmentSpec. //GroupByQueryEngine can only process one interval at a time, so we need to call it once per interval //and concatenate the results. + final GroupByStrategy strategy = strategySelector.strategize(query); final IncrementalIndex outerQueryResultIndex = makeIncrementalIndex( outerQuery, Sequences.concat( @@ -234,7 +229,7 @@ public boolean apply(AggregatorFactory agg) @Override public Sequence apply(Interval interval) { - return engine.process( + return strategy.process( outerQuery.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(ImmutableList.of(interval)) ), @@ -249,74 +244,17 @@ public Sequence apply(Interval interval) innerQueryResultIndex.close(); return new ResourceClosingSequence<>( - outerQuery.applyLimit(postAggregate(query, outerQueryResultIndex)), + outerQuery.applyLimit(GroupByQueryHelper.postAggregate(query, outerQueryResultIndex)), outerQueryResultIndex ); - } else { - final IncrementalIndex index = makeIncrementalIndex( - query, runner.run( - new GroupByQuery( - query.getDataSource(), - query.getQuerySegmentSpec(), - query.getDimFilter(), - query.getGranularity(), - query.getDimensions(), - query.getAggregatorSpecs(), - // Don't do post aggs until the end of this method. - ImmutableList.of(), - // Don't do "having" clause until the end of this method. - null, - null, - query.getContext() - ).withOverriddenContext( - ImmutableMap.of( - "finalize", false, - //setting sort to false avoids unnecessary sorting while merging results. we only need to sort - //in the end when returning results to user. - GroupByQueryHelper.CTX_KEY_SORT_RESULTS, false, - //no merging needed at historicals because GroupByQueryRunnerFactory.mergeRunners(..) would return - //merged results - GROUP_BY_MERGE_KEY, false - ) - ) - , context - ) - ); - return new ResourceClosingSequence<>(query.applyLimit(postAggregate(query, index)), index); + return strategySelector.strategize(query).mergeResults(runner, query, context); } } - private Sequence postAggregate(final GroupByQuery query, IncrementalIndex index) - { - return Sequences.map( - Sequences.simple(index.iterableWithPostAggregations(query.getPostAggregatorSpecs(), query.isDescending())), - new Function() - { - @Override - public Row apply(Row input) - { - final MapBasedRow row = (MapBasedRow) input; - return new MapBasedRow( - query.getGranularity() - .toDateTime(row.getTimestampFromEpoch()), - row.getEvent() - ); - } - } - ); - } - private IncrementalIndex makeIncrementalIndex(GroupByQuery query, Sequence rows) { - final GroupByQueryConfig config = configSupplier.get(); - Pair> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair( - query, - config, - bufferPool - ); - - return rows.accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); + return GroupByQueryHelper.makeIncrementalIndex(query, configSupplier.get(), bufferPool, rows); } @Override @@ -428,7 +366,7 @@ public QueryRunner preMergeQueryDecoration(final QueryRunner runner) public Sequence run(Query query, Map responseContext) { GroupByQuery groupByQuery = (GroupByQuery) query; - if (groupByQuery.getDimFilter() != null){ + if (groupByQuery.getDimFilter() != null) { groupByQuery = groupByQuery.withDimFilter(groupByQuery.getDimFilter().optimize()); } final GroupByQuery delegateGroupByQuery = groupByQuery; diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java index bdafd6fef646..fc5d669662d0 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java @@ -19,26 +19,20 @@ package io.druid.query.groupby; -import com.google.common.base.Supplier; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.guava.Sequence; -import com.metamx.common.logger.Logger; -import io.druid.collections.StupidPool; import io.druid.data.input.Row; -import io.druid.guice.annotations.Global; -import io.druid.query.GroupByMergedQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; -import io.druid.query.QueryWatcher; +import io.druid.query.groupby.strategy.GroupByStrategySelector; import io.druid.segment.Segment; import io.druid.segment.StorageAdapter; -import java.nio.ByteBuffer; import java.util.Map; import java.util.concurrent.ExecutorService; @@ -46,41 +40,42 @@ */ public class GroupByQueryRunnerFactory implements QueryRunnerFactory { - private static final Logger log = new Logger(GroupByQueryRunnerFactory.class); - private final GroupByQueryEngine engine; - private final QueryWatcher queryWatcher; - private final Supplier config; + private final GroupByStrategySelector strategySelector; private final GroupByQueryQueryToolChest toolChest; - private final StupidPool computationBufferPool; @Inject public GroupByQueryRunnerFactory( - GroupByQueryEngine engine, - QueryWatcher queryWatcher, - Supplier config, - GroupByQueryQueryToolChest toolChest, - @Global StupidPool computationBufferPool + GroupByStrategySelector strategySelector, + GroupByQueryQueryToolChest toolChest ) { - this.engine = engine; - this.queryWatcher = queryWatcher; - this.config = config; + this.strategySelector = strategySelector; this.toolChest = toolChest; - this.computationBufferPool = computationBufferPool; } @Override public QueryRunner createRunner(final Segment segment) { - return new GroupByQueryRunner(segment, engine); + return new GroupByQueryRunner(segment, strategySelector); } @Override - public QueryRunner mergeRunners(final ExecutorService exec, Iterable> queryRunners) + public QueryRunner mergeRunners(final ExecutorService exec, final Iterable> queryRunners) { // mergeRunners should take ListeningExecutorService at some point final ListeningExecutorService queryExecutor = MoreExecutors.listeningDecorator(exec); - return new GroupByMergedQueryRunner(queryExecutor, config, queryWatcher, computationBufferPool, queryRunners); + + return new QueryRunner() + { + @Override + public Sequence run(Query query, Map responseContext) + { + return strategySelector.strategize((GroupByQuery) query).mergeRunners(queryExecutor, queryRunners).run( + query, + responseContext + ); + } + }; } @Override @@ -92,12 +87,12 @@ public QueryToolChest getToolchest() private static class GroupByQueryRunner implements QueryRunner { private final StorageAdapter adapter; - private final GroupByQueryEngine engine; + private final GroupByStrategySelector strategySelector; - public GroupByQueryRunner(Segment segment, final GroupByQueryEngine engine) + public GroupByQueryRunner(Segment segment, final GroupByStrategySelector strategySelector) { this.adapter = segment.asStorageAdapter(); - this.engine = engine; + this.strategySelector = strategySelector; } @Override @@ -107,7 +102,7 @@ public Sequence run(Query input, Map responseContext) throw new ISE("Got a [%s] which isn't a %s", input.getClass(), GroupByQuery.class); } - return engine.process((GroupByQuery) input, adapter); + return strategySelector.strategize((GroupByQuery) input).process((GroupByQuery) input, adapter); } } } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferGrouper.java new file mode 100644 index 000000000000..592ff2720be1 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferGrouper.java @@ -0,0 +1,492 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.epinephelinae; + +import com.google.common.base.Preconditions; +import com.google.common.primitives.Ints; +import com.metamx.common.IAE; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.ColumnSelectorFactory; + +import java.nio.ByteBuffer; +import java.util.AbstractList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; + +/** + * Grouper based around a hash table and companion array in a single ByteBuffer. Not thread-safe. + * + * The buffer has two parts: a table arena (offset 0 to tableArenaSize) and an array containing pointers objects in + * the table (tableArenaSize until the end of the buffer). + * + * The table uses open addressing with linear probing on collisions. Each bucket contains the key hash (with the high + * bit set to signify the bucket is used), the serialized key (which are a fixed size) and scratch space for + * BufferAggregators (which is also fixed size). The actual table is represented by "tableBuffer", which points to the + * same memory as positions "tableStart" through "tableStart + buckets * bucketSize" of "buffer". Everything else in + * the table arena is potentially junk. + * + * The array of pointers starts out ordered by insertion order, but might be sorted on calls to + * {@link #iterator(boolean)}. This sorting is done in-place to avoid materializing the full array of pointers. The + * first "size" pointers in the array of pointers are valid; everything else is potentially junk. + * + * The table is periodically grown to accommodate more keys. Even though starting small is not necessary to control + * memory use (we already have the entire buffer allocated) or iteration speed (iteration is fast due to the array + * of pointers) it still helps significantly on initialization times. Otherwise, we'd need to clear the used bits of + * each bucket in the entire buffer, which is a lot of writes if the buckets are small. + */ +public class BufferGrouper> implements Grouper +{ + private static final Logger log = new Logger(BufferGrouper.class); + + private static final int DEFAULT_INITIAL_BUCKETS = 1024; + private static final float MAX_LOAD_FACTOR = 0.75f; + private static final int HASH_SIZE = Ints.BYTES; + + private final ByteBuffer buffer; + private final KeySerde keySerde; + private final int keySize; + private final BufferAggregator[] aggregators; + private final int[] aggregatorOffsets; + private final int initialBuckets; + private final int bucketSize; + private final int tableArenaSize; + private final int bufferGrouperMaxSize; // Integer.MAX_VALUE in production, only used for unit tests + + // Buffer pointing to the current table (it moves around as the table grows) + private ByteBuffer tableBuffer; + + // Offset of tableBuffer within the larger buffer + private int tableStart; + + // Current number of buckets in the table + private int buckets; + + // Number of elements in the table right now + private int size; + + // Maximum number of elements in the table before it must be resized + private int maxSize; + + public BufferGrouper( + final ByteBuffer buffer, + final KeySerde keySerde, + final ColumnSelectorFactory columnSelectorFactory, + final AggregatorFactory[] aggregatorFactories, + final int bufferGrouperMaxSize, + final int initialBuckets + ) + { + this.buffer = buffer; + this.keySerde = keySerde; + this.keySize = keySerde.keySize(); + this.aggregators = new BufferAggregator[aggregatorFactories.length]; + this.aggregatorOffsets = new int[aggregatorFactories.length]; + this.bufferGrouperMaxSize = bufferGrouperMaxSize; + this.initialBuckets = initialBuckets > 0 ? initialBuckets : DEFAULT_INITIAL_BUCKETS; + + int offset = HASH_SIZE + keySize; + for (int i = 0; i < aggregatorFactories.length; i++) { + aggregators[i] = aggregatorFactories[i].factorizeBuffered(columnSelectorFactory); + aggregatorOffsets[i] = offset; + offset += aggregatorFactories[i].getMaxIntermediateSize(); + } + + this.bucketSize = offset; + this.tableArenaSize = (buffer.capacity() / (bucketSize + Ints.BYTES)) * bucketSize; + + reset(); + } + + @Override + public boolean aggregate(KeyType key, int keyHash) + { + final ByteBuffer keyBuffer = keySerde.toByteBuffer(key); + if (keyBuffer == null) { + return false; + } + + Preconditions.checkArgument( + keyBuffer.remaining() == keySize, + "keySerde.toByteBuffer(key).remaining[%s] != keySerde.keySize[%s], buffer was the wrong size?!", + keyBuffer.remaining(), + keySize + ); + + int bucket = findBucket( + tableBuffer, + buckets, + bucketSize, + size < Math.min(maxSize, bufferGrouperMaxSize), + keyBuffer, + keySize, + keyHash + ); + + if (bucket < 0) { + if (size < bufferGrouperMaxSize) { + growIfPossible(); + bucket = findBucket(tableBuffer, buckets, bucketSize, size < maxSize, keyBuffer, keySize, keyHash); + } + + if (bucket < 0) { + return false; + } + } + + final int offset = bucket * bucketSize; + + // Set up key if this is a new bucket. + if (!isUsed(bucket)) { + tableBuffer.position(offset); + tableBuffer.putInt(keyHash | 0x80000000); + tableBuffer.put(keyBuffer); + + for (int i = 0; i < aggregators.length; i++) { + aggregators[i].init(tableBuffer, offset + aggregatorOffsets[i]); + } + + buffer.putInt(tableArenaSize + size * Ints.BYTES, offset); + size++; + } + + // Aggregate the current row. + for (int i = 0; i < aggregators.length; i++) { + aggregators[i].aggregate(tableBuffer, offset + aggregatorOffsets[i]); + } + + return true; + } + + @Override + public boolean aggregate(final KeyType key) + { + return aggregate(key, Groupers.hash(key)); + } + + @Override + public void reset() + { + size = 0; + buckets = Math.min(tableArenaSize / bucketSize, initialBuckets); + maxSize = maxSizeForBuckets(buckets); + + if (buckets < 1) { + throw new IAE( + "Not enough capacity for even one row! Need[%,d] but have[%,d].", + bucketSize + Ints.BYTES, + buffer.capacity() + ); + } + + // Start table part-way through the buffer so the last growth can start from zero and thereby use more space. + tableStart = tableArenaSize - buckets * bucketSize; + int nextBuckets = buckets * 2; + while (true) { + final int nextTableStart = tableStart - nextBuckets * bucketSize; + if (nextTableStart > tableArenaSize / 2) { + tableStart = nextTableStart; + nextBuckets = nextBuckets * 2; + } else { + break; + } + } + + if (tableStart < tableArenaSize / 2) { + tableStart = 0; + } + + final ByteBuffer bufferDup = buffer.duplicate(); + bufferDup.position(tableStart); + bufferDup.limit(tableStart + buckets * bucketSize); + tableBuffer = bufferDup.slice(); + + // Clear used bits of new table + for (int i = 0; i < buckets; i++) { + tableBuffer.put(i * bucketSize, (byte) 0); + } + + keySerde.reset(); + } + + @Override + public Iterator> iterator(final boolean sorted) + { + if (sorted) { + final List wrappedOffsets = new AbstractList() + { + @Override + public Integer get(int index) + { + return buffer.getInt(tableArenaSize + index * Ints.BYTES); + } + + @Override + public Integer set(int index, Integer element) + { + final Integer oldValue = get(index); + buffer.putInt(tableArenaSize + index * Ints.BYTES, element); + return oldValue; + } + + @Override + public int size() + { + return size; + } + }; + + final KeyComparator comparator = keySerde.comparator(); + + // Sort offsets in-place. + Collections.sort( + wrappedOffsets, + new Comparator() + { + @Override + public int compare(Integer lhs, Integer rhs) + { + return comparator.compare( + tableBuffer, + tableBuffer, + lhs + HASH_SIZE, + rhs + HASH_SIZE + ); + } + } + ); + + return new Iterator>() + { + int curr = 0; + + @Override + public boolean hasNext() + { + return curr < size; + } + + @Override + public Entry next() + { + return bucketEntryForOffset(wrappedOffsets.get(curr++)); + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + }; + } else { + // Unsorted iterator + return new Iterator>() + { + int curr = 0; + + @Override + public boolean hasNext() + { + return curr < size; + } + + @Override + public Entry next() + { + final int offset = buffer.getInt(tableArenaSize + curr * Ints.BYTES); + final Entry entry = bucketEntryForOffset(offset); + curr++; + + return entry; + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + }; + } + } + + @Override + public void close() + { + for (BufferAggregator aggregator : aggregators) { + try { + aggregator.close(); + } + catch (Exception e) { + log.warn(e, "Could not close aggregator, skipping.", aggregator); + } + } + } + + private boolean isUsed(final int bucket) + { + return (tableBuffer.get(bucket * bucketSize) & 0x80) == 0x80; + } + + private Entry bucketEntryForOffset(final int bucketOffset) + { + final KeyType key = keySerde.fromByteBuffer(tableBuffer, bucketOffset + HASH_SIZE); + final Object[] values = new Object[aggregators.length]; + for (int i = 0; i < aggregators.length; i++) { + values[i] = aggregators[i].get(tableBuffer, bucketOffset + aggregatorOffsets[i]); + } + + return new Entry<>(key, values); + } + + private void growIfPossible() + { + if (tableStart == 0) { + // tableStart = 0 is the last growth; no further growing is possible. + return; + } + + final int newBuckets; + final int newMaxSize; + final int newTableStart; + + if ((tableStart + buckets * 3 * bucketSize) > tableArenaSize) { + // Not enough space to grow upwards, start back from zero + newTableStart = 0; + newBuckets = tableStart / bucketSize; + newMaxSize = maxSizeForBuckets(newBuckets); + } else { + newTableStart = tableStart + tableBuffer.limit(); + newBuckets = buckets * 2; + newMaxSize = maxSizeForBuckets(newBuckets); + } + + if (newBuckets < buckets) { + throw new ISE("WTF?! newBuckets[%,d] < buckets[%,d]", newBuckets, buckets); + } + + ByteBuffer newTableBuffer = buffer.duplicate(); + newTableBuffer.position(newTableStart); + newTableBuffer.limit(newTableStart + newBuckets * bucketSize); + newTableBuffer = newTableBuffer.slice(); + + int newSize = 0; + + // Clear used bits of new table + for (int i = 0; i < newBuckets; i++) { + newTableBuffer.put(i * bucketSize, (byte) 0); + } + + // Loop over old buckets and copy to new table + final ByteBuffer entryBuffer = tableBuffer.duplicate(); + final ByteBuffer keyBuffer = tableBuffer.duplicate(); + + for (int oldBucket = 0; oldBucket < buckets; oldBucket++) { + if (isUsed(oldBucket)) { + entryBuffer.limit((oldBucket + 1) * bucketSize); + entryBuffer.position(oldBucket * bucketSize); + keyBuffer.limit(entryBuffer.position() + HASH_SIZE + keySize); + keyBuffer.position(entryBuffer.position() + HASH_SIZE); + + final int keyHash = entryBuffer.getInt(entryBuffer.position()) & 0x7fffffff; + final int newBucket = findBucket(newTableBuffer, newBuckets, bucketSize, true, keyBuffer, keySize, keyHash); + + if (newBucket < 0) { + throw new ISE("WTF?! Couldn't find a bucket while resizing?!"); + } + + newTableBuffer.position(newBucket * bucketSize); + newTableBuffer.put(entryBuffer); + + buffer.putInt(tableArenaSize + newSize * Ints.BYTES, newBucket * bucketSize); + newSize++; + } + } + + buckets = newBuckets; + maxSize = newMaxSize; + tableBuffer = newTableBuffer; + tableStart = newTableStart; + + if (size != newSize) { + throw new ISE("WTF?! size[%,d] != newSize[%,d] after resizing?!", size, maxSize); + } + } + + private static int maxSizeForBuckets(int buckets) + { + return Math.max(1, (int) (buckets * MAX_LOAD_FACTOR)); + } + + /** + * Finds the bucket into which we should insert a key. + * + * @param keyBuffer key, must have exactly keySize bytes remaining. Will not be modified. + * + * @return bucket index for this key, or -1 if no bucket is available due to being full + */ + private static int findBucket( + final ByteBuffer tableBuffer, + final int buckets, + final int bucketSize, + final boolean allowNewBucket, + final ByteBuffer keyBuffer, + final int keySize, + final int keyHash + ) + { + // startBucket will never be negative since keyHash is always positive (see Groupers.hash) + final int startBucket = keyHash % buckets; + int bucket = startBucket; + +outer: + while (true) { + final int bucketOffset = bucket * bucketSize; + + if ((tableBuffer.get(bucketOffset) & 0x80) == 0) { + // Found unused bucket before finding our key + return allowNewBucket ? bucket : -1; + } + + for (int i = bucketOffset + HASH_SIZE, j = keyBuffer.position(); j < keyBuffer.position() + keySize; i++, j++) { + if (tableBuffer.get(i) != keyBuffer.get(j)) { + bucket += 1; + if (bucket == buckets) { + bucket = 0; + } + + if (bucket == startBucket) { + // Came back around to the start without finding a free slot, that was a long trip! + // Should never happen unless buckets == maxSize. + return -1; + } + + continue outer; + } + } + + // Found our key in a used bucket + return bucket; + } + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/CloseableGrouperIterator.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/CloseableGrouperIterator.java new file mode 100644 index 000000000000..b3177f751545 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/CloseableGrouperIterator.java @@ -0,0 +1,67 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.epinephelinae; + +import com.google.common.base.Function; + +import java.io.Closeable; +import java.util.Iterator; + +public class CloseableGrouperIterator, T> implements Iterator, Closeable +{ + private final Grouper grouper; + private final Function, T> transformer; + private final Iterator> iterator; + + public CloseableGrouperIterator( + final Grouper grouper, + final boolean sorted, + final Function, T> transformer + ) + { + this.grouper = grouper; + this.transformer = transformer; + this.iterator = grouper.iterator(sorted); + } + + @Override + public T next() + { + return transformer.apply(iterator.next()); + } + + @Override + public boolean hasNext() + { + return iterator.hasNext(); + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + + @Override + public void close() + { + grouper.close(); + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouper.java new file mode 100644 index 000000000000..d597ecb51c82 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouper.java @@ -0,0 +1,143 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.epinephelinae; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.common.ISE; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.ColumnSelectorFactory; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +/** + * Grouper based around a set of underlying {@link SpillingGrouper} instances. Thread-safe. + * + * The passed-in buffer is cut up into "concurrencyHint" slices, and each slice is passed to a different underlying + * grouper. Access to each slice is separately synchronized. + */ +public class ConcurrentGrouper> implements Grouper +{ + private final List> groupers; + private volatile boolean closed = false; + + public ConcurrentGrouper( + final ByteBuffer buffer, + final int concurrencyHint, + final LimitedTemporaryStorage temporaryStorage, + final ObjectMapper spillMapper, + final int bufferGrouperMaxSize, + final int bufferGrouperInitialBuckets, + final KeySerdeFactory keySerdeFactory, + final ColumnSelectorFactory columnSelectorFactory, + final AggregatorFactory[] aggregatorFactories + ) + { + this.groupers = new ArrayList<>(concurrencyHint); + + final int sliceSize = (buffer.capacity() / concurrencyHint); + for (int i = 0; i < concurrencyHint; i++) { + final ByteBuffer slice = buffer.duplicate(); + slice.position(sliceSize * i); + slice.limit(slice.position() + sliceSize); + groupers.add( + new SpillingGrouper<>( + slice.slice(), + keySerdeFactory, + columnSelectorFactory, + aggregatorFactories, + temporaryStorage, + spillMapper, + bufferGrouperMaxSize, + bufferGrouperInitialBuckets + ) + ); + } + } + + @Override + public boolean aggregate(KeyType key, int keyHash) + { + if (closed) { + throw new ISE("Grouper is closed"); + } + + final Grouper grouper = groupers.get(grouperNumberForKeyHash(keyHash)); + synchronized (grouper) { + return grouper.aggregate(key, keyHash); + } + } + + @Override + public boolean aggregate(KeyType key) + { + return aggregate(key, Groupers.hash(key)); + } + + @Override + public void reset() + { + if (closed) { + throw new ISE("Grouper is closed"); + } + + for (Grouper grouper : groupers) { + synchronized (grouper) { + grouper.reset(); + } + } + } + + @Override + public Iterator> iterator(final boolean sorted) + { + if (closed) { + throw new ISE("Grouper is closed"); + } + + final List>> iterators = new ArrayList<>(groupers.size()); + + for (Grouper grouper : groupers) { + synchronized (grouper) { + iterators.add(grouper.iterator(sorted)); + } + } + + return Groupers.mergeIterators(iterators, sorted); + } + + @Override + public void close() + { + closed = true; + for (Grouper grouper : groupers) { + synchronized (grouper) { + grouper.close(); + } + } + } + + private int grouperNumberForKeyHash(int keyHash) + { + return keyHash % groupers.size(); + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByBinaryFnV2.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByBinaryFnV2.java new file mode 100644 index 000000000000..663920cbca7d --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByBinaryFnV2.java @@ -0,0 +1,84 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.epinephelinae; + +import com.google.common.collect.Maps; +import com.metamx.common.guava.nary.BinaryFn; +import io.druid.data.input.MapBasedRow; +import io.druid.data.input.Row; +import io.druid.granularity.AllGranularity; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.groupby.GroupByQuery; +import org.joda.time.DateTime; + +import java.util.Map; + +public class GroupByBinaryFnV2 implements BinaryFn +{ + private final GroupByQuery query; + + public GroupByBinaryFnV2(GroupByQuery query) + { + this.query = query; + } + + @Override + public Row apply(final Row arg1, final Row arg2) + { + if (arg1 == null) { + return arg2; + } else if (arg2 == null) { + return arg1; + } + + final Map newMap = Maps.newHashMapWithExpectedSize( + query.getDimensions().size() + + query.getAggregatorSpecs().size() + ); + + // Add dimensions + for (DimensionSpec dimension : query.getDimensions()) { + newMap.put(dimension.getOutputName(), arg1.getRaw(dimension.getOutputName())); + } + + // Add aggregations + for (AggregatorFactory aggregatorFactory : query.getAggregatorSpecs()) { + newMap.put( + aggregatorFactory.getName(), + aggregatorFactory.combine( + arg1.getRaw(aggregatorFactory.getName()), + arg2.getRaw(aggregatorFactory.getName()) + ) + ); + } + + return new MapBasedRow(adjustTimestamp(arg1), newMap); + } + + private DateTime adjustTimestamp(final Row row) + { + if (query.getGranularity() instanceof AllGranularity) { + return row.getTimestamp(); + } else { + return query.getGranularity().toDateTime(query.getGranularity().truncate(row.getTimestamp().getMillis())); + } + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java new file mode 100644 index 000000000000..285c87a5a1a2 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java @@ -0,0 +1,673 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.epinephelinae; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.base.Predicates; +import com.google.common.base.Strings; +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.Maps; +import com.google.common.primitives.Chars; +import com.google.common.primitives.Ints; +import com.google.common.primitives.Longs; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.metamx.common.ISE; +import com.metamx.common.guava.Accumulator; +import com.metamx.common.guava.BaseSequence; +import com.metamx.common.guava.CloseQuietly; +import com.metamx.common.guava.ResourceClosingSequence; +import com.metamx.common.guava.Sequence; +import com.metamx.common.logger.Logger; +import io.druid.collections.BlockingPool; +import io.druid.collections.ResourceHolder; +import io.druid.data.input.MapBasedRow; +import io.druid.data.input.Row; +import io.druid.query.AbstractPrioritizedCallable; +import io.druid.query.BaseQuery; +import io.druid.query.ChainedExecutionQueryRunner; +import io.druid.query.Query; +import io.druid.query.QueryContextKeys; +import io.druid.query.QueryInterruptedException; +import io.druid.query.QueryRunner; +import io.druid.query.QueryWatcher; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.groupby.GroupByQuery; +import io.druid.query.groupby.GroupByQueryConfig; +import io.druid.query.groupby.strategy.GroupByStrategyV2; +import io.druid.segment.ColumnSelectorFactory; +import io.druid.segment.DimensionSelector; +import io.druid.segment.FloatColumnSelector; +import io.druid.segment.LongColumnSelector; +import io.druid.segment.ObjectColumnSelector; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.CancellationException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +public class GroupByMergingQueryRunnerV2 implements QueryRunner +{ + private static final Logger log = new Logger(GroupByMergingQueryRunnerV2.class); + private static final String CTX_KEY_MERGE_RUNNERS_USING_CHAINED_EXECUTION = "mergeRunnersUsingChainedExecution"; + + private final GroupByQueryConfig config; + private final Iterable> queryables; + private final ListeningExecutorService exec; + private final QueryWatcher queryWatcher; + private final int concurrencyHint; + private final BlockingPool mergeBufferPool; + private final ObjectMapper spillMapper; + + public GroupByMergingQueryRunnerV2( + GroupByQueryConfig config, + ExecutorService exec, + QueryWatcher queryWatcher, + Iterable> queryables, + int concurrencyHint, + BlockingPool mergeBufferPool, + ObjectMapper spillMapper + ) + { + this.config = config; + this.exec = MoreExecutors.listeningDecorator(exec); + this.queryWatcher = queryWatcher; + this.queryables = Iterables.unmodifiableIterable(Iterables.filter(queryables, Predicates.notNull())); + this.concurrencyHint = concurrencyHint; + this.mergeBufferPool = mergeBufferPool; + this.spillMapper = spillMapper; + } + + @Override + public Sequence run(final Query queryParam, final Map responseContext) + { + final GroupByQuery query = (GroupByQuery) queryParam; + + // CTX_KEY_MERGE_RUNNERS_USING_CHAINED_EXECUTION is here because realtime servers use nested mergeRunners calls + // (one for the entire query and one for each sink). We only want the outer call to actually do merging with a + // merge buffer, otherwise the query will allocate too many merge buffers. This is potentially sub-optimal as it + // will involve materializing the results for each sink before starting to feed them into the outer merge buffer. + // I'm not sure of a better way to do this without tweaking how realtime servers do queries. + final boolean forceChainedExecution = query.getContextBoolean( + CTX_KEY_MERGE_RUNNERS_USING_CHAINED_EXECUTION, + false + ); + final GroupByQuery queryForRunners = query.withOverriddenContext( + ImmutableMap.of(CTX_KEY_MERGE_RUNNERS_USING_CHAINED_EXECUTION, true) + ); + + if (BaseQuery.getContextBySegment(query, false) || forceChainedExecution) { + return new ChainedExecutionQueryRunner(exec, queryWatcher, queryables).run(query, responseContext); + } + + final AggregatorFactory[] combiningAggregatorFactories = new AggregatorFactory[query.getAggregatorSpecs().size()]; + for (int i = 0; i < query.getAggregatorSpecs().size(); i++) { + combiningAggregatorFactories[i] = query.getAggregatorSpecs().get(i).getCombiningFactory(); + } + + final GroupByMergingKeySerdeFactory keySerdeFactory = new GroupByMergingKeySerdeFactory( + query.getDimensions().size(), + config.getMaxMergingDictionarySize() / concurrencyHint + ); + final GroupByMergingColumnSelectorFactory columnSelectorFactory = new GroupByMergingColumnSelectorFactory(); + + final File temporaryStorageDirectory = new File( + System.getProperty("java.io.tmpdir"), + String.format("druid-groupBy-%s_%s", UUID.randomUUID(), query.getId()) + ); + + final LimitedTemporaryStorage temporaryStorage = new LimitedTemporaryStorage( + temporaryStorageDirectory, + config.getMaxOnDiskStorage() + ); + + // Figure out timeoutAt time now, so we can apply the timeout to both the mergeBufferPool.take and the actual + // query processing together. + final long startTime = System.currentTimeMillis(); + final Number timeout = query.getContextValue(QueryContextKeys.TIMEOUT, null); + final long timeoutAt = timeout == null ? -1L : startTime + timeout.longValue(); + + final ResourceHolder mergeBufferHolder; + + try { + mergeBufferHolder = mergeBufferPool.take(timeout != null && timeout.longValue() > 0 ? timeout.longValue() : -1); + } + catch (InterruptedException e) { + CloseQuietly.close(temporaryStorage); + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + + final long processingStartTime = System.currentTimeMillis(); + + try { + return new ResourceClosingSequence<>( + new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public CloseableGrouperIterator make() + { + final Grouper grouper = new ConcurrentGrouper<>( + mergeBufferHolder.get(), + concurrencyHint, + temporaryStorage, + spillMapper, + config.getBufferGrouperMaxSize(), + GroupByStrategyV2.getBufferGrouperInitialBuckets(config, query), + keySerdeFactory, + columnSelectorFactory, + combiningAggregatorFactories + ); + + final Accumulator, Row> accumulator = new Accumulator, Row>() + { + @Override + public Grouper accumulate( + final Grouper theGrouper, + final Row row + ) + { + final long timestamp = row.getTimestampFromEpoch(); + + final String[] dimensions = new String[query.getDimensions().size()]; + for (int i = 0; i < dimensions.length; i++) { + final Object dimValue = row.getRaw(query.getDimensions().get(i).getOutputName()); + dimensions[i] = Strings.nullToEmpty((String) dimValue); + } + + columnSelectorFactory.setRow(row); + final boolean didAggregate = theGrouper.aggregate(new GroupByMergingKey(timestamp, dimensions)); + if (!didAggregate) { + throw new ISE("Grouping resources exhausted"); + } + columnSelectorFactory.setRow(null); + + return theGrouper; + } + }; + + final int priority = BaseQuery.getContextPriority(query, 0); + + ListenableFuture> futures = Futures.allAsList( + Lists.newArrayList( + Iterables.transform( + queryables, + new Function, ListenableFuture>() + { + @Override + public ListenableFuture apply(final QueryRunner input) + { + if (input == null) { + throw new ISE( + "Null queryRunner! Looks to be some segment unmapping action happening" + ); + } + + return exec.submit( + new AbstractPrioritizedCallable(priority) + { + @Override + public Void call() throws Exception + { + try { + input.run(queryForRunners, responseContext) + .accumulate(grouper, accumulator); + return null; + } + catch (QueryInterruptedException e) { + throw Throwables.propagate(e); + } + catch (Exception e) { + log.error(e, "Exception with one of the sequences!"); + throw Throwables.propagate(e); + } + } + } + ); + } + } + ) + ) + ); + + try { + waitForFutureCompletion(query, futures, timeoutAt - processingStartTime); + } + catch (Exception e) { + grouper.close(); + throw e; + } + + return new CloseableGrouperIterator<>( + grouper, + true, + new Function, Row>() + { + @Override + public Row apply(Grouper.Entry entry) + { + Map theMap = Maps.newLinkedHashMap(); + + // Add dimensions. + for (int i = 0; i < entry.getKey().getDimensions().length; i++) { + theMap.put( + query.getDimensions().get(i).getOutputName(), + Strings.emptyToNull(entry.getKey().getDimensions()[i]) + ); + } + + // Add aggregations. + for (int i = 0; i < entry.getValues().length; i++) { + theMap.put(query.getAggregatorSpecs().get(i).getName(), entry.getValues()[i]); + } + + return new MapBasedRow( + query.getGranularity().toDateTime(entry.getKey().getTimestamp()), + theMap + ); + } + } + ); + } + + @Override + public void cleanup(CloseableGrouperIterator iterFromMake) + { + iterFromMake.close(); + } + } + ), + new Closeable() + { + @Override + public void close() throws IOException + { + mergeBufferHolder.close(); + CloseQuietly.close(temporaryStorage); + } + } + ); + } + catch (Exception e) { + // Exception caught while creating the sequence; release resources. + mergeBufferHolder.close(); + CloseQuietly.close(temporaryStorage); + throw e; + } + } + + private void waitForFutureCompletion( + GroupByQuery query, + ListenableFuture future, + long timeout + ) + { + try { + if (queryWatcher != null) { + queryWatcher.registerQuery(query, future); + } + if (timeout <= 0) { + future.get(); + } else { + future.get(timeout, TimeUnit.MILLISECONDS); + } + } + catch (InterruptedException e) { + log.warn(e, "Query interrupted, cancelling pending results, query id [%s]", query.getId()); + future.cancel(true); + throw new QueryInterruptedException(e); + } + catch (CancellationException e) { + throw new QueryInterruptedException(e); + } + catch (TimeoutException e) { + log.info("Query timeout, cancelling pending results for query id [%s]", query.getId()); + future.cancel(true); + throw new QueryInterruptedException(e); + } + catch (ExecutionException e) { + throw Throwables.propagate(e.getCause()); + } + } + + private static class GroupByMergingKey implements Comparable + { + private final long timestamp; + private final String[] dimensions; + + @JsonCreator + public GroupByMergingKey( + // Using short key names to reduce serialized size when spilling to disk. + @JsonProperty("t") long timestamp, + @JsonProperty("d") String[] dimensions + ) + { + this.timestamp = timestamp; + this.dimensions = dimensions; + } + + @JsonProperty("t") + public long getTimestamp() + { + return timestamp; + } + + @JsonProperty("d") + public String[] getDimensions() + { + return dimensions; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + GroupByMergingKey that = (GroupByMergingKey) o; + + if (timestamp != that.timestamp) { + return false; + } + // Probably incorrect - comparing Object[] arrays with Arrays.equals + return Arrays.equals(dimensions, that.dimensions); + + } + + @Override + public int hashCode() + { + int result = (int) (timestamp ^ (timestamp >>> 32)); + result = 31 * result + Arrays.hashCode(dimensions); + return result; + } + + @Override + public int compareTo(GroupByMergingKey other) + { + final int timeCompare = Longs.compare(timestamp, other.getTimestamp()); + if (timeCompare != 0) { + return timeCompare; + } + + for (int i = 0; i < dimensions.length; i++) { + final int cmp = dimensions[i].compareTo(other.getDimensions()[i]); + if (cmp != 0) { + return cmp; + } + } + + return 0; + } + + @Override + public String toString() + { + return "GroupByMergingKey{" + + "timestamp=" + timestamp + + ", dimensions=" + Arrays.toString(dimensions) + + '}'; + } + } + + private static class GroupByMergingKeySerdeFactory implements Grouper.KeySerdeFactory + { + private final int dimCount; + private final long maxDictionarySize; + + public GroupByMergingKeySerdeFactory(int dimCount, long maxDictionarySize) + { + this.dimCount = dimCount; + this.maxDictionarySize = maxDictionarySize; + } + + @Override + public Grouper.KeySerde factorize() + { + return new GroupByMergingKeySerde(dimCount, maxDictionarySize); + } + } + + private static class GroupByMergingKeySerde implements Grouper.KeySerde + { + // Entry in dictionary, node pointer in reverseDictionary, hash + k/v/next pointer in reverseDictionary nodes + private static final int ROUGH_OVERHEAD_PER_DICTIONARY_ENTRY = Longs.BYTES * 5 + Ints.BYTES; + + private final int dimCount; + private final int keySize; + private final ByteBuffer keyBuffer; + private final List dictionary = Lists.newArrayList(); + private final Map reverseDictionary = Maps.newHashMap(); + + // Size limiting for the dictionary, in (roughly estimated) bytes. + private final long maxDictionarySize; + private long currentEstimatedSize = 0; + + // dictionary id -> its position if it were sorted by dictionary value + private int[] sortableIds = null; + + public GroupByMergingKeySerde(final int dimCount, final long maxDictionarySize) + { + this.dimCount = dimCount; + this.maxDictionarySize = maxDictionarySize; + this.keySize = Longs.BYTES + dimCount * Ints.BYTES; + this.keyBuffer = ByteBuffer.allocate(keySize); + } + + @Override + public int keySize() + { + return keySize; + } + + @Override + public Class keyClazz() + { + return GroupByMergingKey.class; + } + + @Override + public ByteBuffer toByteBuffer(GroupByMergingKey key) + { + keyBuffer.rewind(); + keyBuffer.putLong(key.getTimestamp()); + for (int i = 0; i < key.getDimensions().length; i++) { + final int id = addToDictionary(key.getDimensions()[i]); + if (id < 0) { + return null; + } + keyBuffer.putInt(id); + } + keyBuffer.flip(); + return keyBuffer; + } + + @Override + public GroupByMergingKey fromByteBuffer(ByteBuffer buffer, int position) + { + final long timestamp = buffer.getLong(position); + final String[] dimensions = new String[dimCount]; + for (int i = 0; i < dimensions.length; i++) { + dimensions[i] = dictionary.get(buffer.getInt(position + Longs.BYTES + (Ints.BYTES * i))); + } + return new GroupByMergingKey(timestamp, dimensions); + } + + @Override + public Grouper.KeyComparator comparator() + { + if (sortableIds == null) { + Map sortedMap = Maps.newTreeMap(); + for (int id = 0; id < dictionary.size(); id++) { + sortedMap.put(dictionary.get(id), id); + } + sortableIds = new int[dictionary.size()]; + int index = 0; + for (final Integer id : sortedMap.values()) { + sortableIds[id] = index++; + } + } + + return new Grouper.KeyComparator() + { + @Override + public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) + { + final int timeCompare = Longs.compare(lhsBuffer.getLong(lhsPosition), rhsBuffer.getLong(rhsPosition)); + if (timeCompare != 0) { + return timeCompare; + } + + for (int i = 0; i < dimCount; i++) { + final int cmp = Ints.compare( + sortableIds[lhsBuffer.getInt(lhsPosition + Longs.BYTES + (Ints.BYTES * i))], + sortableIds[rhsBuffer.getInt(rhsPosition + Longs.BYTES + (Ints.BYTES * i))] + ); + + if (cmp != 0) { + return cmp; + } + } + + return 0; + } + }; + } + + @Override + public void reset() + { + dictionary.clear(); + reverseDictionary.clear(); + sortableIds = null; + currentEstimatedSize = 0; + } + + /** + * Adds s to the dictionary. If the dictionary's size limit would be exceeded by adding this key, then + * this returns -1. + * + * @param s a string + * + * @return id for this string, or -1 + */ + private int addToDictionary(final String s) + { + Integer idx = reverseDictionary.get(s); + if (idx == null) { + final long additionalEstimatedSize = (long) s.length() * Chars.BYTES + ROUGH_OVERHEAD_PER_DICTIONARY_ENTRY; + if (currentEstimatedSize + additionalEstimatedSize > maxDictionarySize) { + return -1; + } + + idx = dictionary.size(); + reverseDictionary.put(s, idx); + dictionary.add(s); + currentEstimatedSize += additionalEstimatedSize; + } + return idx; + } + } + + private static class GroupByMergingColumnSelectorFactory implements ColumnSelectorFactory + { + private ThreadLocal row = new ThreadLocal<>(); + + public void setRow(Row row) + { + this.row.set(row); + } + + @Override + public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) + { + // Combining factories shouldn't need dimension selectors, that'd be weird. + throw new UnsupportedOperationException(); + } + + @Override + public FloatColumnSelector makeFloatColumnSelector(final String columnName) + { + return new FloatColumnSelector() + { + @Override + public float get() + { + return row.get().getFloatMetric(columnName); + } + }; + } + + @Override + public LongColumnSelector makeLongColumnSelector(final String columnName) + { + return new LongColumnSelector() + { + @Override + public long get() + { + return row.get().getLongMetric(columnName); + } + }; + } + + @Override + public ObjectColumnSelector makeObjectColumnSelector(final String columnName) + { + return new ObjectColumnSelector() + { + @Override + public Class classOfObject() + { + return Object.class; + } + + @Override + public Object get() + { + return row.get().getRaw(columnName); + } + }; + } + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java new file mode 100644 index 000000000000..b2c25673900f --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java @@ -0,0 +1,392 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.epinephelinae; + +import com.google.common.base.Function; +import com.google.common.base.Strings; +import com.google.common.collect.Maps; +import com.google.common.primitives.Ints; +import com.metamx.common.IAE; +import com.metamx.common.ISE; +import com.metamx.common.guava.BaseSequence; +import com.metamx.common.guava.CloseQuietly; +import com.metamx.common.guava.ResourceClosingSequence; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import io.druid.collections.ResourceHolder; +import io.druid.collections.StupidPool; +import io.druid.data.input.MapBasedRow; +import io.druid.data.input.Row; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.groupby.GroupByQuery; +import io.druid.query.groupby.GroupByQueryConfig; +import io.druid.query.groupby.strategy.GroupByStrategyV2; +import io.druid.segment.Cursor; +import io.druid.segment.DimensionSelector; +import io.druid.segment.StorageAdapter; +import io.druid.segment.data.EmptyIndexedInts; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.filter.Filters; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; + +public class GroupByQueryEngineV2 +{ + private GroupByQueryEngineV2() + { + // No instantiation + } + + public static Sequence process( + final GroupByQuery query, + final StorageAdapter storageAdapter, + final StupidPool intermediateResultsBufferPool, + final GroupByQueryConfig config + ) + { + if (storageAdapter == null) { + throw new ISE( + "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + ); + } + + final List intervals = query.getQuerySegmentSpec().getIntervals(); + if (intervals.size() != 1) { + throw new IAE("Should only have one interval, got[%s]", intervals); + } + + final Sequence cursors = storageAdapter.makeCursors( + Filters.toFilter(query.getDimFilter()), + intervals.get(0), + query.getGranularity(), + false + ); + + final Grouper.KeySerde keySerde = new GroupByEngineKeySerde(query.getDimensions().size()); + final ResourceHolder bufferHolder = intermediateResultsBufferPool.take(); + + final String fudgeTimestampString = Strings.emptyToNull( + query.getContextValue(GroupByStrategyV2.CTX_KEY_FUDGE_TIMESTAMP, "") + ); + + final DateTime fudgeTimestamp = fudgeTimestampString == null + ? null + : new DateTime(Long.parseLong(fudgeTimestampString)); + + return Sequences.concat( + new ResourceClosingSequence<>( + Sequences.map( + cursors, + new Function>() + { + @Override + public Sequence apply(final Cursor cursor) + { + return new BaseSequence<>( + new BaseSequence.IteratorMaker() + { + @Override + public GroupByEngineIterator make() + { + return new GroupByEngineIterator( + query, + config, + cursor, + bufferHolder.get(), + keySerde, + fudgeTimestamp + ); + } + + @Override + public void cleanup(GroupByEngineIterator iterFromMake) + { + iterFromMake.close(); + } + } + ); + } + } + ), + new Closeable() + { + @Override + public void close() throws IOException + { + CloseQuietly.close(bufferHolder); + } + } + ) + ); + } + + private static class GroupByEngineIterator implements Iterator, Closeable + { + private final GroupByQuery query; + private final GroupByQueryConfig config; + private final Cursor cursor; + private final ByteBuffer buffer; + private final Grouper.KeySerde keySerde; + private final DateTime timestamp; + private final DimensionSelector[] selectors; + private final ByteBuffer keyBuffer; + private final int[] stack; + private final IndexedInts[] valuess; + + private int stackp = Integer.MIN_VALUE; + private boolean currentRowWasPartiallyAggregated = false; + private CloseableGrouperIterator delegate = null; + + public GroupByEngineIterator( + final GroupByQuery query, + final GroupByQueryConfig config, + final Cursor cursor, + final ByteBuffer buffer, + final Grouper.KeySerde keySerde, + final DateTime fudgeTimestamp + ) + { + final int dimCount = query.getDimensions().size(); + + this.query = query; + this.config = config; + this.cursor = cursor; + this.buffer = buffer; + this.keySerde = keySerde; + this.keyBuffer = ByteBuffer.allocate(keySerde.keySize()); + this.selectors = new DimensionSelector[dimCount]; + for (int i = 0; i < dimCount; i++) { + this.selectors[i] = cursor.makeDimensionSelector(query.getDimensions().get(i)); + } + this.stack = new int[dimCount]; + this.valuess = new IndexedInts[dimCount]; + + // Time is the same for every row in the cursor + this.timestamp = fudgeTimestamp != null ? fudgeTimestamp : cursor.getTime(); + } + + @Override + public Row next() + { + if (delegate != null && delegate.hasNext()) { + return delegate.next(); + } + + if (cursor.isDone()) { + throw new NoSuchElementException(); + } + + // Make a new delegate iterator + if (delegate != null) { + delegate.close(); + delegate = null; + } + + final Grouper grouper = new BufferGrouper<>( + buffer, + keySerde, + cursor, + query.getAggregatorSpecs() + .toArray(new AggregatorFactory[query.getAggregatorSpecs().size()]), + config.getBufferGrouperMaxSize(), + GroupByStrategyV2.getBufferGrouperInitialBuckets(config, query) + ); + +outer: + while (!cursor.isDone()) { + if (!currentRowWasPartiallyAggregated) { + // Set up stack, valuess, and first grouping in keyBuffer for this row + stackp = stack.length - 1; + + for (int i = 0; i < selectors.length; i++) { + final DimensionSelector selector = selectors[i]; + + valuess[i] = selector == null ? EmptyIndexedInts.EMPTY_INDEXED_INTS : selector.getRow(); + + final int position = Ints.BYTES * i; + if (valuess[i].size() == 0) { + stack[i] = 0; + keyBuffer.putInt(position, -1); + } else { + stack[i] = 1; + keyBuffer.putInt(position, valuess[i].get(0)); + } + } + } + + // Aggregate groupings for this row + boolean doAggregate = true; + while (stackp >= -1) { + // Aggregate additional grouping for this row + if (doAggregate) { + keyBuffer.rewind(); + if (!grouper.aggregate(keyBuffer)) { + // Buffer full while aggregating; break out and resume later + currentRowWasPartiallyAggregated = true; + break outer; + } + doAggregate = false; + } + + if (stackp >= 0 && stack[stackp] < valuess[stackp].size()) { + // Load next value for current slot + keyBuffer.putInt( + Ints.BYTES * stackp, + valuess[stackp].get(stack[stackp]) + ); + stack[stackp]++; + + // Reset later slots + for (int i = stackp + 1; i < stack.length; i++) { + final int position = Ints.BYTES * i; + if (valuess[i].size() == 0) { + stack[i] = 0; + keyBuffer.putInt(position, -1); + } else { + stack[i] = 1; + keyBuffer.putInt(position, valuess[i].get(0)); + } + } + + stackp = stack.length - 1; + doAggregate = true; + } else { + stackp--; + } + } + + // Advance to next row + cursor.advance(); + currentRowWasPartiallyAggregated = false; + } + + delegate = new CloseableGrouperIterator<>( + grouper, + false, + new Function, Row>() + { + @Override + public Row apply(final Grouper.Entry entry) + { + Map theMap = Maps.newLinkedHashMap(); + + // Add dimensions. + for (int i = 0; i < selectors.length; i++) { + final int id = entry.getKey().getInt(Ints.BYTES * i); + + if (id >= 0) { + theMap.put( + query.getDimensions().get(i).getOutputName(), + selectors[i].lookupName(id) + ); + } + } + + // Add aggregations. + for (int i = 0; i < entry.getValues().length; i++) { + theMap.put(query.getAggregatorSpecs().get(i).getName(), entry.getValues()[i]); + } + + return new MapBasedRow(timestamp, theMap); + } + } + ); + + return delegate.next(); + } + + @Override + public boolean hasNext() + { + return (delegate != null && delegate.hasNext()) || !cursor.isDone(); + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + + @Override + public void close() + { + if (delegate != null) { + delegate.close(); + } + } + } + + private static class GroupByEngineKeySerde implements Grouper.KeySerde + { + private final int keySize; + + public GroupByEngineKeySerde(final int dimCount) + { + this.keySize = dimCount * Ints.BYTES; + } + + @Override + public int keySize() + { + return keySize; + } + + @Override + public Class keyClazz() + { + return ByteBuffer.class; + } + + @Override + public ByteBuffer toByteBuffer(ByteBuffer key) + { + return key; + } + + @Override + public ByteBuffer fromByteBuffer(ByteBuffer buffer, int position) + { + final ByteBuffer dup = buffer.duplicate(); + dup.position(position).limit(position + keySize); + return dup.slice(); + } + + @Override + public Grouper.KeyComparator comparator() + { + // No sorting, let mergeRunners handle that + return null; + } + + @Override + public void reset() + { + // No state, nothing to reset + } + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/Grouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/Grouper.java new file mode 100644 index 000000000000..68570429212b --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/Grouper.java @@ -0,0 +1,220 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.epinephelinae; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Iterator; + +/** + * Groupers aggregate metrics from rows that they typically get from a ColumnSelectorFactory, under + * grouping keys that some outside driver is passing in. They can also iterate over the grouped + * rows after the aggregation is done. + * + * They work sort of like a map of KeyType to aggregated values, except they don't support + * random lookups. + * + * @param type of the key that will be passed in + */ +public interface Grouper> +{ + /** + * Aggregate the current row with the provided key. Some implementations are thread-safe and + * some are not. + * + * @param key key object + * @param keyHash result of {@link Groupers#hash(Object)} on the key + * + * @return true if the row was aggregated, false if not due to hitting resource limits + */ + boolean aggregate(KeyType key, int keyHash); + + /** + * Aggregate the current row with the provided key. Some implementations are thread-safe and + * some are not. + * + * @param key key + * + * @return true if the row was aggregated, false if not due to hitting resource limits + */ + boolean aggregate(KeyType key); + + /** + * Reset the grouper to its initial state. + */ + void reset(); + + /** + * Close the grouper and release associated resources. + */ + void close(); + + /** + * Iterate through entries. If a comparator is provided, do a sorted iteration. + * + * Once this method is called, writes are no longer safe. After you are done with the iterator returned by this + * method, you should either call {@link #close()} (if you are done with the Grouper), {@link #reset()} (if you + * want to reuse it), or {@link #iterator(boolean)} again if you want another iterator. + * + * If "sorted" is true then the iterator will return sorted results. It will use KeyType's natural ordering on + * deserialized objects, and will use the {@link KeySerde#comparator()} on serialized objects. Woe be unto you + * if these comparators are not equivalent. + * + * @param sorted return sorted results + * + * @return entry iterator + */ + Iterator> iterator(final boolean sorted); + + class Entry + { + final T key; + final Object[] values; + + @JsonCreator + public Entry( + @JsonProperty("k") T key, + @JsonProperty("v") Object[] values + ) + { + this.key = key; + this.values = values; + } + + @JsonProperty("k") + public T getKey() + { + return key; + } + + @JsonProperty("v") + public Object[] getValues() + { + return values; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + Entry entry = (Entry) o; + + if (!key.equals(entry.key)) { + return false; + } + // Probably incorrect - comparing Object[] arrays with Arrays.equals + return Arrays.equals(values, entry.values); + + } + + @Override + public int hashCode() + { + int result = key.hashCode(); + result = 31 * result + Arrays.hashCode(values); + return result; + } + + @Override + public String toString() + { + return "Entry{" + + "key=" + key + + ", values=" + Arrays.toString(values) + + '}'; + } + } + + interface KeySerdeFactory + { + /** + * Create a new KeySerde, which may be stateful. + */ + KeySerde factorize(); + } + + /** + * Possibly-stateful object responsible for serde and comparison of keys. Does not need to be thread-safe. + */ + interface KeySerde + { + /** + * Size of the keys returned by {@link #toByteBuffer(Object)} (which must be a fixed size) + */ + int keySize(); + + /** + * Class of the keys. + */ + Class keyClazz(); + + /** + * Serialize a key. This will be called by the {@link #aggregate(Comparable)} method. The buffer will not + * be retained after the aggregate method returns, so reusing buffers is OK. + * + * This method may return null, which indicates that some internal resource limit has been reached and + * no more keys can be generated. In this situation you can call {@link #reset()} and try again, although + * beware the caveats on that method. + * + * @param key key object + * + * @return serialized key, or null if we are unable to serialize more keys due to resource limits + */ + ByteBuffer toByteBuffer(T key); + + /** + * Deserialize a key from a buffer. Will be called by the {@link #iterator(boolean)} method. + * + * @param buffer buffer containing the key + * @param position key start position in the buffer + * + * @return key object + */ + T fromByteBuffer(ByteBuffer buffer, int position); + + /** + * Return an object that knows how to compare two serialized keys. Will be called by the + * {@link #iterator(boolean)} method if sorting is enabled. + * + * @return comparator for keys + */ + KeyComparator comparator(); + + /** + * Reset the keySerde to its initial state. After this method is called, {@link #fromByteBuffer(ByteBuffer, int)} + * and {@link #comparator()} may no longer work properly on previously-serialized keys. + */ + void reset(); + } + + interface KeyComparator + { + int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition); + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/Groupers.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/Groupers.java new file mode 100644 index 000000000000..db0a6149a6a7 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/Groupers.java @@ -0,0 +1,63 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.epinephelinae; + +import com.google.common.collect.Iterators; + +import java.util.Comparator; +import java.util.Iterator; + +public class Groupers +{ + private static final Comparator> ENTRY_COMPARATOR = new Comparator>() + { + @Override + public int compare( + final Grouper.Entry lhs, + final Grouper.Entry rhs + ) + { + return lhs.getKey().compareTo(rhs.getKey()); + } + }; + + private Groupers() + { + // No instantiation + } + + public static int hash(final Object obj) + { + // Mask off the high bit so we can use that to determine if a bucket is used or not. + return obj.hashCode() & 0x7fffffff; + } + + public static > Iterator> mergeIterators( + final Iterable>> iterators, + final boolean sorted + ) + { + if (sorted) { + return Iterators.mergeSorted(iterators, ENTRY_COMPARATOR); + } else { + return Iterators.concat(iterators.iterator()); + } + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/LimitedTemporaryStorage.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/LimitedTemporaryStorage.java new file mode 100644 index 000000000000..cfb9cf507be8 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/LimitedTemporaryStorage.java @@ -0,0 +1,171 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.epinephelinae; + +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; + +import java.io.Closeable; +import java.io.File; +import java.io.FilterOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.channels.Channels; +import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.nio.file.StandardOpenOption; +import java.util.EnumSet; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; + +/** + * An area for limited temporary storage on disk. Limits are checked when opening files and on all writes to those + * files. Thread-safe. + */ +public class LimitedTemporaryStorage implements Closeable +{ + private static final Logger log = new Logger(LimitedTemporaryStorage.class); + + private final File storageDirectory; + private final long maxBytesUsed; + + private final AtomicLong bytesUsed = new AtomicLong(); + private final Set files = Sets.newTreeSet(); + + private volatile boolean closed = false; + + public LimitedTemporaryStorage(File storageDirectory, long maxBytesUsed) + { + this.storageDirectory = storageDirectory; + this.maxBytesUsed = maxBytesUsed; + } + + public LimitedOutputStream createFile() throws IOException + { + if (bytesUsed.get() >= maxBytesUsed) { + throwFullError(); + } + + synchronized (files) { + if (closed) { + throw new ISE("Closed"); + } + + if (!storageDirectory.exists() && !storageDirectory.mkdir()) { + throw new IOException(String.format("Cannot create storageDirectory: %s", storageDirectory)); + } + + final File theFile = new File(storageDirectory, String.format("%08d.tmp", files.size())); + final EnumSet openOptions = EnumSet.of( + StandardOpenOption.CREATE_NEW, + StandardOpenOption.WRITE + ); + + final FileChannel channel = FileChannel.open(theFile.toPath(), openOptions); + files.add(theFile); + return new LimitedOutputStream(theFile, Channels.newOutputStream(channel)); + } + } + + public void delete(final File file) + { + synchronized (files) { + if (files.contains(file)) { + try { + Files.delete(file.toPath()); + } + catch (IOException e) { + log.warn(e, "Cannot delete file: %s", file); + } + files.remove(file); + } + } + } + + @Override + public void close() + { + synchronized (files) { + if (closed) { + return; + } + closed = true; + for (File file : ImmutableSet.copyOf(files)) { + delete(file); + } + files.clear(); + if (storageDirectory.exists() && !storageDirectory.delete()) { + log.warn("Cannot delete storageDirectory: %s", storageDirectory); + } + } + } + + public class LimitedOutputStream extends FilterOutputStream + { + private final File file; + + private LimitedOutputStream(File file, OutputStream out) + { + super(out); + this.file = file; + } + + @Override + public void write(int b) throws IOException + { + grab(1); + super.write(b); + } + + @Override + public void write(byte[] b) throws IOException + { + grab(b.length); + super.write(b); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException + { + grab(len); + super.write(b, off, len); + } + + public File getFile() + { + return file; + } + + private void grab(int n) throws IOException + { + if (bytesUsed.addAndGet(n) > maxBytesUsed) { + throwFullError(); + } + } + + } + + private void throwFullError() throws IOException + { + throw new IOException(String.format("Cannot write to disk, hit limit of %,d bytes.", maxBytesUsed)); + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/SpillingGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/SpillingGrouper.java new file mode 100644 index 000000000000..6123d9732f9f --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/SpillingGrouper.java @@ -0,0 +1,205 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.epinephelinae; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.MappingIterator; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.base.Throwables; +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; +import com.metamx.common.guava.CloseQuietly; +import com.metamx.common.logger.Logger; +import io.druid.query.QueryInterruptedException; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.ColumnSelectorFactory; +import net.jpountz.lz4.LZ4BlockInputStream; +import net.jpountz.lz4.LZ4BlockOutputStream; + +import java.io.Closeable; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +/** + * Grouper based around a single underlying {@link BufferGrouper}. Not thread-safe. + * + * When the underlying grouper is full, its contents are sorted and written to temporary files using "spillMapper". + */ +public class SpillingGrouper> implements Grouper +{ + private static final Logger log = new Logger(SpillingGrouper.class); + + private final BufferGrouper grouper; + private final KeySerde keySerde; + private final LimitedTemporaryStorage temporaryStorage; + private final ObjectMapper spillMapper; + private final AggregatorFactory[] aggregatorFactories; + + private final List files = Lists.newArrayList(); + private final List closeables = Lists.newArrayList(); + + public SpillingGrouper( + final ByteBuffer buffer, + final KeySerdeFactory keySerdeFactory, + final ColumnSelectorFactory columnSelectorFactory, + final AggregatorFactory[] aggregatorFactories, + final LimitedTemporaryStorage temporaryStorage, + final ObjectMapper spillMapper, + final int bufferGrouperMaxSize, + final int bufferGrouperInitialBuckets + ) + { + this.keySerde = keySerdeFactory.factorize(); + this.grouper = new BufferGrouper<>( + buffer, + keySerde, + columnSelectorFactory, + aggregatorFactories, + bufferGrouperMaxSize, + bufferGrouperInitialBuckets + ); + this.aggregatorFactories = aggregatorFactories; + this.temporaryStorage = temporaryStorage; + this.spillMapper = spillMapper; + } + + @Override + public boolean aggregate(KeyType key, int keyHash) + { + if (grouper.aggregate(key, keyHash)) { + return true; + } else { + // Warning: this can potentially block up a processing thread for a while. + spill(); + return grouper.aggregate(key, keyHash); + } + } + + @Override + public boolean aggregate(KeyType key) + { + return aggregate(key, Groupers.hash(key)); + } + + @Override + public void reset() + { + grouper.reset(); + deleteFiles(); + } + + @Override + public void close() + { + grouper.close(); + deleteFiles(); + } + + @Override + public Iterator> iterator(final boolean sorted) + { + final List>> iterators = new ArrayList<>(1 + files.size()); + + iterators.add(grouper.iterator(sorted)); + + for (final File file : files) { + final MappingIterator> fileIterator = read(file, keySerde.keyClazz()); + iterators.add( + Iterators.transform( + fileIterator, + new Function, Entry>() + { + @Override + public Entry apply(Entry entry) + { + final Object[] deserializedValues = new Object[entry.getValues().length]; + for (int i = 0; i < deserializedValues.length; i++) { + deserializedValues[i] = aggregatorFactories[i].deserialize(entry.getValues()[i]); + if (deserializedValues[i] instanceof Integer) { + // Hack to satisfy the groupBy unit tests; perhaps we could do better by adjusting Jackson config. + deserializedValues[i] = ((Integer) deserializedValues[i]).longValue(); + } + } + return new Entry<>(entry.getKey(), deserializedValues); + } + } + ) + ); + closeables.add(fileIterator); + } + + return Groupers.mergeIterators(iterators, sorted); + } + + private void spill() + { + try ( + final LimitedTemporaryStorage.LimitedOutputStream out = temporaryStorage.createFile(); + final LZ4BlockOutputStream compressedOut = new LZ4BlockOutputStream(out); + final JsonGenerator jsonGenerator = spillMapper.getFactory().createGenerator(compressedOut) + ) { + files.add(out.getFile()); + final Iterator> it = grouper.iterator(true); + while (it.hasNext()) { + if (Thread.interrupted()) { + throw new QueryInterruptedException(new InterruptedException()); + } + + jsonGenerator.writeObject(it.next()); + } + } + catch (IOException e) { + throw Throwables.propagate(e); + } + + grouper.reset(); + } + + private MappingIterator> read(final File file, final Class keyClazz) + { + try { + return spillMapper.readValues( + spillMapper.getFactory().createParser(new LZ4BlockInputStream(new FileInputStream(file))), + spillMapper.getTypeFactory().constructParametricType(Entry.class, keyClazz) + ); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + + private void deleteFiles() + { + for (Closeable closeable : closeables) { + // CloseQuietly is OK on readable streams + CloseQuietly.close(closeable); + } + for (final File file : files) { + temporaryStorage.delete(file); + } + files.clear(); + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java index 10190fb925fa..18b59f3cc67a 100644 --- a/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java @@ -24,9 +24,12 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; +import com.google.common.collect.Sets; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import com.metamx.common.ISE; @@ -36,6 +39,7 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DimensionSpec; +import io.druid.query.ordering.StringComparators; import io.druid.query.ordering.StringComparators.StringComparator; import javax.annotation.Nullable; @@ -44,6 +48,7 @@ import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.Set; /** */ @@ -80,10 +85,42 @@ public int getLimit() @Override public Function, Sequence> build( - List dimensions, List aggs, List postAggs + List dimensions, + List aggs, + List postAggs ) { - if (columns.isEmpty()) { + // Can avoid materialization if the natural ordering is good enough. + + boolean materializationNeeded = false; + + if (dimensions.size() < columns.size()) { + materializationNeeded = true; + } + + final Set aggAndPostAggNames = Sets.newHashSet(); + for (AggregatorFactory agg : aggs) { + aggAndPostAggNames.add(agg.getName()); + } + for (PostAggregator postAgg : postAggs) { + aggAndPostAggNames.add(postAgg.getName()); + } + + if (!materializationNeeded) { + for (int i = 0; i < columns.size(); i++) { + final OrderByColumnSpec columnSpec = columns.get(i); + + if (columnSpec.getDirection() != OrderByColumnSpec.Direction.ASCENDING + || !columnSpec.getDimensionComparator().equals(StringComparators.LEXICOGRAPHIC) + || !columnSpec.getDimension().equals(dimensions.get(i).getOutputName()) + || aggAndPostAggNames.contains(columnSpec.getDimension())) { + materializationNeeded = true; + break; + } + } + } + + if (!materializationNeeded) { return new LimitingFn(limit); } diff --git a/processing/src/main/java/io/druid/query/groupby/orderby/LimitSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/LimitSpec.java index 3319c574575f..1177422f08d7 100644 --- a/processing/src/main/java/io/druid/query/groupby/orderby/LimitSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/LimitSpec.java @@ -38,6 +38,15 @@ }) public interface LimitSpec { + /** + * Returns a function that applies a limit to an input sequence that is assumed to be sorted on dimensions. + * + * @param dimensions query dimensions + * @param aggs query aggregators + * @param postAggs query postAggregators + * + * @return limit function + */ public Function, Sequence> build( List dimensions, List aggs, diff --git a/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategy.java b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategy.java new file mode 100644 index 000000000000..758325a62319 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategy.java @@ -0,0 +1,48 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.strategy; + +import com.google.common.util.concurrent.ListeningExecutorService; +import com.metamx.common.guava.Sequence; +import io.druid.data.input.Row; +import io.druid.query.QueryRunner; +import io.druid.query.groupby.GroupByQuery; +import io.druid.segment.StorageAdapter; + +import java.util.Map; + +public interface GroupByStrategy +{ + Sequence mergeResults( + QueryRunner baseRunner, + GroupByQuery query, + Map responseContext + ); + + QueryRunner mergeRunners( + ListeningExecutorService exec, + Iterable> queryRunners + ); + + Sequence process( + GroupByQuery query, + StorageAdapter storageAdapter + ); +} diff --git a/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategySelector.java b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategySelector.java new file mode 100644 index 000000000000..220418747a2d --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategySelector.java @@ -0,0 +1,65 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.strategy; + +import com.google.common.base.Supplier; +import com.google.inject.Inject; +import com.metamx.common.ISE; +import io.druid.query.groupby.GroupByQuery; +import io.druid.query.groupby.GroupByQueryConfig; + +public class GroupByStrategySelector +{ + public static final String CTX_KEY_STRATEGY = "groupByStrategy"; + public static final String STRATEGY_V2 = "v2"; + public static final String STRATEGY_V1 = "v1"; + + private final GroupByQueryConfig config; + private final GroupByStrategyV1 strategyV1; + private final GroupByStrategyV2 strategyV2; + + @Inject + public GroupByStrategySelector( + Supplier configSupplier, + GroupByStrategyV1 strategyV1, + GroupByStrategyV2 strategyV2 + ) + { + this.config = configSupplier.get(); + this.strategyV1 = strategyV1; + this.strategyV2 = strategyV2; + } + + public GroupByStrategy strategize(GroupByQuery query) + { + final String strategyString = query.getContextValue(CTX_KEY_STRATEGY, config.getDefaultStrategy()); + + switch (strategyString) { + case STRATEGY_V2: + return strategyV2; + + case STRATEGY_V1: + return strategyV1; + + default: + throw new ISE("No such strategy[%s]", strategyString); + } + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV1.java b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV1.java new file mode 100644 index 000000000000..1e17084c0dd1 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV1.java @@ -0,0 +1,129 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.strategy; + +import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.inject.Inject; +import com.metamx.common.guava.ResourceClosingSequence; +import com.metamx.common.guava.Sequence; +import io.druid.collections.StupidPool; +import io.druid.data.input.Row; +import io.druid.guice.annotations.Global; +import io.druid.query.GroupByMergedQueryRunner; +import io.druid.query.QueryRunner; +import io.druid.query.QueryWatcher; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.groupby.GroupByQuery; +import io.druid.query.groupby.GroupByQueryConfig; +import io.druid.query.groupby.GroupByQueryEngine; +import io.druid.query.groupby.GroupByQueryHelper; +import io.druid.query.groupby.GroupByQueryQueryToolChest; +import io.druid.segment.StorageAdapter; +import io.druid.segment.incremental.IncrementalIndex; + +import java.nio.ByteBuffer; +import java.util.Map; + +public class GroupByStrategyV1 implements GroupByStrategy +{ + private final Supplier configSupplier; + private final GroupByQueryEngine engine; + private final QueryWatcher queryWatcher; + private final StupidPool bufferPool; + + @Inject + public GroupByStrategyV1( + Supplier configSupplier, + GroupByQueryEngine engine, + QueryWatcher queryWatcher, + @Global StupidPool bufferPool + ) + { + this.configSupplier = configSupplier; + this.engine = engine; + this.queryWatcher = queryWatcher; + this.bufferPool = bufferPool; + } + + @Override + public Sequence mergeResults( + final QueryRunner baseRunner, + final GroupByQuery query, + final Map responseContext + ) + { + final IncrementalIndex index = GroupByQueryHelper.makeIncrementalIndex( + query, + configSupplier.get(), + bufferPool, + baseRunner.run( + new GroupByQuery( + query.getDataSource(), + query.getQuerySegmentSpec(), + query.getDimFilter(), + query.getGranularity(), + query.getDimensions(), + query.getAggregatorSpecs(), + // Don't do post aggs until the end of this method. + ImmutableList.of(), + // Don't do "having" clause until the end of this method. + null, + null, + query.getContext() + ).withOverriddenContext( + ImmutableMap.of( + "finalize", false, + //setting sort to false avoids unnecessary sorting while merging results. we only need to sort + //in the end when returning results to user. + GroupByQueryHelper.CTX_KEY_SORT_RESULTS, false, + //no merging needed at historicals because GroupByQueryRunnerFactory.mergeRunners(..) would return + //merged results + GroupByQueryQueryToolChest.GROUP_BY_MERGE_KEY, false, + GroupByStrategySelector.CTX_KEY_STRATEGY, GroupByStrategySelector.STRATEGY_V1 + ) + ), + responseContext + ) + ); + + return new ResourceClosingSequence<>(query.applyLimit(GroupByQueryHelper.postAggregate(query, index)), index); + } + + @Override + public QueryRunner mergeRunners( + final ListeningExecutorService exec, + final Iterable> queryRunners + ) + { + return new GroupByMergedQueryRunner<>(exec, configSupplier, queryWatcher, bufferPool, queryRunners); + } + + @Override + public Sequence process( + final GroupByQuery query, + final StorageAdapter storageAdapter + ) + { + return engine.process(query, storageAdapter); + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java new file mode 100644 index 000000000000..a207bd5ff254 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java @@ -0,0 +1,211 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.strategy; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; +import com.google.common.collect.Ordering; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.inject.Inject; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import com.metamx.common.guava.nary.BinaryFn; +import io.druid.collections.BlockingPool; +import io.druid.collections.StupidPool; +import io.druid.data.input.MapBasedRow; +import io.druid.data.input.Row; +import io.druid.granularity.AllGranularity; +import io.druid.granularity.QueryGranularity; +import io.druid.guice.annotations.Global; +import io.druid.guice.annotations.Merging; +import io.druid.guice.annotations.Smile; +import io.druid.query.DruidProcessingConfig; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryWatcher; +import io.druid.query.ResultMergeQueryRunner; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.groupby.GroupByQuery; +import io.druid.query.groupby.GroupByQueryConfig; +import io.druid.query.groupby.epinephelinae.GroupByBinaryFnV2; +import io.druid.query.groupby.epinephelinae.GroupByMergingQueryRunnerV2; +import io.druid.query.groupby.epinephelinae.GroupByQueryEngineV2; +import io.druid.segment.StorageAdapter; +import org.joda.time.DateTime; + +import java.nio.ByteBuffer; +import java.util.Map; + +public class GroupByStrategyV2 implements GroupByStrategy +{ + public static final String CTX_KEY_FUDGE_TIMESTAMP = "fudgeTimestamp"; + + private final DruidProcessingConfig processingConfig; + private final Supplier configSupplier; + private final StupidPool bufferPool; + private final BlockingPool mergeBufferPool; + private final ObjectMapper spillMapper; + private final QueryWatcher queryWatcher; + + @Inject + public GroupByStrategyV2( + DruidProcessingConfig processingConfig, + Supplier configSupplier, + @Global StupidPool bufferPool, + @Merging BlockingPool mergeBufferPool, + @Smile ObjectMapper spillMapper, + QueryWatcher queryWatcher + ) + { + this.processingConfig = processingConfig; + this.configSupplier = configSupplier; + this.bufferPool = bufferPool; + this.mergeBufferPool = mergeBufferPool; + this.spillMapper = spillMapper; + this.queryWatcher = queryWatcher; + } + + @Override + public Sequence mergeResults( + final QueryRunner baseRunner, + final GroupByQuery query, + final Map responseContext + ) + { + // Merge streams using ResultMergeQueryRunner, then apply postaggregators, then apply limit (which may + // involve materialization) + + final ResultMergeQueryRunner mergingQueryRunner = new ResultMergeQueryRunner(baseRunner) + { + @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. Necessary to keep timestamps in sync across partial queries. + final QueryGranularity gran = query.getGranularity(); + final String fudgeTimestamp; + if (query.getContextValue(CTX_KEY_FUDGE_TIMESTAMP, "").isEmpty() && gran instanceof AllGranularity) { + final long timeStart = query.getIntervals().get(0).getStartMillis(); + fudgeTimestamp = String.valueOf( + new DateTime(gran.iterable(timeStart, timeStart + 1).iterator().next()).getMillis() + ); + } else { + fudgeTimestamp = query.getContextValue(CTX_KEY_FUDGE_TIMESTAMP, ""); + } + + return query.applyLimit( + Sequences.map( + mergingQueryRunner.run( + new GroupByQuery( + query.getDataSource(), + query.getQuerySegmentSpec(), + query.getDimFilter(), + gran, + query.getDimensions(), + query.getAggregatorSpecs(), + // Don't do post aggs until the end of this method. + ImmutableList.of(), + // Don't do "having" clause until the end of this method. + null, + null, + query.getContext() + ).withOverriddenContext( + ImmutableMap.of( + "finalize", false, + GroupByStrategySelector.CTX_KEY_STRATEGY, GroupByStrategySelector.STRATEGY_V2, + CTX_KEY_FUDGE_TIMESTAMP, fudgeTimestamp + ) + ), + responseContext + ), + new Function() + { + @Override + public Row apply(final Row row) + { + // Maybe apply postAggregators. + + if (query.getPostAggregatorSpecs().isEmpty()) { + return row; + } + + final Map newMap; + + if (query.getPostAggregatorSpecs().isEmpty()) { + newMap = ((MapBasedRow) row).getEvent(); + } else { + newMap = Maps.newLinkedHashMap(((MapBasedRow) row).getEvent()); + + for (PostAggregator postAggregator : query.getPostAggregatorSpecs()) { + newMap.put(postAggregator.getName(), postAggregator.compute(newMap)); + } + } + + return new MapBasedRow(row.getTimestamp(), newMap); + } + } + ) + ); + } + + @Override + public QueryRunner mergeRunners( + ListeningExecutorService exec, + Iterable> queryRunners + ) + { + return new GroupByMergingQueryRunnerV2( + configSupplier.get(), + exec, + queryWatcher, + queryRunners, + processingConfig.getNumThreads(), + mergeBufferPool, + spillMapper + ); + } + + @Override + public Sequence process( + GroupByQuery query, + StorageAdapter storageAdapter + ) + { + return GroupByQueryEngineV2.process(query, storageAdapter, bufferPool, configSupplier.get()); + } + + public static int getBufferGrouperInitialBuckets(final GroupByQueryConfig config, final GroupByQuery query) + { + return query.getContextValue("bufferGrouperInitialBuckets", config.getBufferGrouperInitialBuckets()); + } +} diff --git a/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java index 925eddae7f8b..c6dd4f9b3781 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java @@ -209,14 +209,9 @@ public static class LZFCompressor implements Compressor @Override public byte[] compress(byte[] bytes) { - try (final ResourceHolder bufferRecycler = CompressedPools.getBufferRecycler()) { return LZFEncoder.encode(bytes, 0, bytes.length, bufferRecycler.get()); } - catch (IOException e) { - log.error(e, "Error compressing data"); - throw Throwables.propagate(e); - } } } @@ -231,7 +226,14 @@ public void decompress(ByteBuffer in, int numBytes, ByteBuffer out) { // Since decompressed size is NOT known, must use lz4Safe // lz4Safe.decompress does not modify buffer positions - final int numDecompressedBytes = lz4Safe.decompress(in, in.position(), numBytes, out, out.position(), out.remaining()); + final int numDecompressedBytes = lz4Safe.decompress( + in, + in.position(), + numBytes, + out, + out.position(), + out.remaining() + ); out.limit(out.position() + numDecompressedBytes); } @@ -298,7 +300,7 @@ public T get() } @Override - public void close() throws IOException + public void close() { bufHolder.close(); } diff --git a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java index 35fe24260244..6dbb924a8f89 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java @@ -93,13 +93,8 @@ public OffheapIncrementalIndex( //check that stupid pool gives buffers that can hold at least one row's aggregators ResourceHolder bb = bufferPool.take(); if (bb.get().capacity() < aggsTotalSize) { - RuntimeException ex = new IAE("bufferPool buffers capacity must be >= [%s]", aggsTotalSize); - try { - bb.close(); - } catch(IOException ioe){ - ex.addSuppressed(ioe); - } - throw ex; + bb.close(); + throw new IAE("bufferPool buffers capacity must be >= [%s]", aggsTotalSize); } aggBuffers.add(bb); } @@ -372,19 +367,8 @@ public void close() RuntimeException ex = null; for (ResourceHolder buffHolder : aggBuffers) { - try { - buffHolder.close(); - } catch(IOException ioe) { - if (ex == null) { - ex = Throwables.propagate(ioe); - } else { - ex.addSuppressed(ioe); - } - } + buffHolder.close(); } aggBuffers.clear(); - if (ex != null) { - throw ex; - } } } diff --git a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java index 19b9b9c6f6df..08abea147c0b 100644 --- a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java @@ -61,6 +61,7 @@ import io.druid.query.groupby.GroupByQueryEngine; import io.druid.query.groupby.GroupByQueryQueryToolChest; import io.druid.query.groupby.GroupByQueryRunnerFactory; +import io.druid.query.groupby.GroupByQueryRunnerTest; import io.druid.query.select.SelectQueryEngine; import io.druid.query.select.SelectQueryQueryToolChest; import io.druid.query.select.SelectQueryRunnerFactory; @@ -133,39 +134,7 @@ public static final AggregationTestHelper createGroupByQueryAggregationTestHelpe ) { ObjectMapper mapper = new DefaultObjectMapper(); - - Supplier configSupplier = Suppliers.ofInstance(new GroupByQueryConfig()); - StupidPool pool = new StupidPool<>( - new Supplier() - { - @Override - public ByteBuffer get() - { - return ByteBuffer.allocate(1024 * 1024); - } - }); - - QueryWatcher noopQueryWatcher = new QueryWatcher() - { - @Override - public void registerQuery(Query query, ListenableFuture future) - { - - } - }; - - GroupByQueryEngine engine = new GroupByQueryEngine(configSupplier, pool); - GroupByQueryQueryToolChest toolchest = new GroupByQueryQueryToolChest( - configSupplier, mapper, engine, pool, - NoopIntervalChunkingQueryRunnerDecorator() - ); - GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( - engine, - noopQueryWatcher, - configSupplier, - toolchest, - pool - ); + GroupByQueryRunnerFactory factory = GroupByQueryRunnerTest.makeQueryRunnerFactory(new GroupByQueryConfig()); IndexIO indexIO = new IndexIO( mapper, @@ -183,7 +152,7 @@ public int columnCacheSizeBytes() mapper, new IndexMerger(mapper, indexIO), indexIO, - toolchest, + factory.getToolchest(), factory, tempFolder, jsonModulesToRegister diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java index eb76d935b1e1..5f1236ed22c3 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java @@ -74,7 +74,7 @@ public class GroupByQueryRunnerFactoryTest @Test public void testMergeRunnersEnsureGroupMerging() throws Exception { - QueryRunnerFactory factory = createFactory(); + QueryRunnerFactory factory = GroupByQueryRunnerTest.makeQueryRunnerFactory(new GroupByQueryConfig()); QueryRunner mergedRunner = factory.mergeRunners( Executors.newSingleThreadExecutor(), ImmutableList.of( @@ -147,42 +147,4 @@ private Segment createSegment() throws Exception return new IncrementalIndexSegment(incrementalIndex, "test"); } - - private GroupByQueryRunnerFactory createFactory() - { - ObjectMapper mapper = new DefaultObjectMapper(); - - Supplier configSupplier = Suppliers.ofInstance(new GroupByQueryConfig()); - StupidPool pool = new StupidPool<>( - new Supplier() - { - @Override - public ByteBuffer get() - { - return ByteBuffer.allocate(1024 * 1024); - } - }); - - QueryWatcher noopQueryWatcher = new QueryWatcher() - { - @Override - public void registerQuery(Query query, ListenableFuture future) - { - - } - }; - - GroupByQueryEngine engine = new GroupByQueryEngine(configSupplier, pool); - GroupByQueryQueryToolChest toolchest = new GroupByQueryQueryToolChest( - configSupplier, mapper, engine, pool, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ); - return new GroupByQueryRunnerFactory( - engine, - noopQueryWatcher, - configSupplier, - toolchest, - pool - ); - } } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 31e016897079..7c5365ab432a 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -19,8 +19,7 @@ package io.druid.query.groupby; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; @@ -29,10 +28,13 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; +import com.google.common.util.concurrent.MoreExecutors; import com.metamx.common.ISE; +import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.common.parsers.ParseException; +import io.druid.collections.BlockingPool; import io.druid.collections.StupidPool; import io.druid.data.input.Row; import io.druid.granularity.PeriodGranularity; @@ -41,6 +43,7 @@ import io.druid.js.JavaScriptConfig; import io.druid.query.BySegmentResultValue; import io.druid.query.BySegmentResultValueClass; +import io.druid.query.DruidProcessingConfig; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; @@ -48,7 +51,6 @@ import io.druid.query.QueryRunnerTestHelper; import io.druid.query.QueryToolChest; import io.druid.query.Result; -import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleMaxAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -86,6 +88,9 @@ import io.druid.query.groupby.orderby.DefaultLimitSpec; import io.druid.query.groupby.orderby.LimitSpec; import io.druid.query.groupby.orderby.OrderByColumnSpec; +import io.druid.query.groupby.strategy.GroupByStrategySelector; +import io.druid.query.groupby.strategy.GroupByStrategyV1; +import io.druid.query.groupby.strategy.GroupByStrategyV2; import io.druid.query.lookup.LookupExtractionFn; import io.druid.query.ordering.StringComparators; import io.druid.query.search.search.ContainsSearchQuerySpec; @@ -97,7 +102,6 @@ import org.joda.time.Interval; import org.joda.time.Period; import org.junit.Assert; -import org.junit.Before; import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; @@ -105,7 +109,6 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -124,50 +127,84 @@ public class GroupByQueryRunnerTest { private final QueryRunner runner; private GroupByQueryRunnerFactory factory; - private Supplier configSupplier; + private GroupByQueryConfig config; @Rule public ExpectedException expectedException = ExpectedException.none(); - @Before - public void setUp() throws Exception + public static GroupByQueryRunnerFactory makeQueryRunnerFactory( + final GroupByQueryConfig config + ) { - configSupplier = Suppliers.ofInstance(new GroupByQueryConfig()); - } - - @Parameterized.Parameters - public static Collection constructorFeeder() throws IOException - { - final ObjectMapper mapper = new DefaultObjectMapper(); - final StupidPool pool = new StupidPool<>( + final Supplier configSupplier = Suppliers.ofInstance(config); + final StupidPool bufferPool = new StupidPool<>( new Supplier() { @Override public ByteBuffer get() { - return ByteBuffer.allocate(1024 * 1024); + return ByteBuffer.allocate(10 * 1024 * 1024); } } ); - - final GroupByQueryConfig config = new GroupByQueryConfig(); - config.setMaxIntermediateRows(10000); - - final Supplier configSupplier = Suppliers.ofInstance(config); - final GroupByQueryEngine engine = new GroupByQueryEngine(configSupplier, pool); - - final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( - engine, - QueryRunnerTestHelper.NOOP_QUERYWATCHER, + final BlockingPool mergeBufferPool = new BlockingPool<>( + new Supplier() + { + @Override + public ByteBuffer get() + { + return ByteBuffer.allocate(10 * 1024 * 1024); + } + }, + 4 + ); + final GroupByStrategySelector strategySelector = new GroupByStrategySelector( configSupplier, - new GroupByQueryQueryToolChest( - configSupplier, mapper, engine, TestQueryRunners.pool, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + new GroupByStrategyV1( + configSupplier, + new GroupByQueryEngine(configSupplier, bufferPool), + QueryRunnerTestHelper.NOOP_QUERYWATCHER, + bufferPool ), - TestQueryRunners.pool + new GroupByStrategyV2( + new DruidProcessingConfig() + { + @Override + public String getFormatString() + { + return null; + } + + @Override + public int getNumThreads() + { + return 2; + } + }, + configSupplier, + bufferPool, + mergeBufferPool, + new DefaultObjectMapper(new SmileFactory()), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) + ); + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest( + configSupplier, + strategySelector, + bufferPool, + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ); + return new GroupByQueryRunnerFactory( + strategySelector, + toolChest ); + } - GroupByQueryConfig singleThreadedConfig = new GroupByQueryConfig() + @Parameterized.Parameters + public static Collection constructorFeeder() throws IOException + { + final GroupByQueryConfig defaultConfig = new GroupByQueryConfig(); + final GroupByQueryConfig singleThreadedConfig = new GroupByQueryConfig() { @Override public boolean isSingleThreaded() @@ -175,50 +212,82 @@ public boolean isSingleThreaded() return true; } }; - singleThreadedConfig.setMaxIntermediateRows(10000); + final GroupByQueryConfig v2Config = new GroupByQueryConfig() + { + @Override + public String getDefaultStrategy() + { + return GroupByStrategySelector.STRATEGY_V2; + } + }; + final GroupByQueryConfig v2SmallBufferConfig = new GroupByQueryConfig() + { + @Override + public String getDefaultStrategy() + { + return GroupByStrategySelector.STRATEGY_V2; + } - final Supplier singleThreadedConfigSupplier = Suppliers.ofInstance(singleThreadedConfig); - final GroupByQueryEngine singleThreadEngine = new GroupByQueryEngine(singleThreadedConfigSupplier, pool); + @Override + public int getBufferGrouperMaxSize() + { + return 2; + } - final GroupByQueryRunnerFactory singleThreadFactory = new GroupByQueryRunnerFactory( - singleThreadEngine, - QueryRunnerTestHelper.NOOP_QUERYWATCHER, - singleThreadedConfigSupplier, - new GroupByQueryQueryToolChest( - singleThreadedConfigSupplier, mapper, singleThreadEngine, pool, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ), - pool - ); + @Override + public long getMaxOnDiskStorage() + { + return 10L * 1024 * 1024; + } + }; + final GroupByQueryConfig epinephelinaeSmallDictionaryConfig = new GroupByQueryConfig() + { + @Override + public String getDefaultStrategy() + { + return GroupByStrategySelector.STRATEGY_V2; + } + @Override + public long getMaxMergingDictionarySize() + { + return 400; + } - Function function = new Function() - { @Override - public Object apply(@Nullable Object input) + public long getMaxOnDiskStorage() { - return new Object[]{factory, input}; + return 10L * 1024 * 1024; } }; - return Lists.newArrayList( - Iterables.concat( - Iterables.transform( - QueryRunnerTestHelper.makeQueryRunners(factory), - function - ), - Iterables.transform( - QueryRunnerTestHelper.makeQueryRunners(singleThreadFactory), - function - ) - ) + defaultConfig.setMaxIntermediateRows(10000); + singleThreadedConfig.setMaxIntermediateRows(10000); + + final List constructors = Lists.newArrayList(); + final List configs = ImmutableList.of( + defaultConfig, + singleThreadedConfig, + v2Config, + v2SmallBufferConfig, + epinephelinaeSmallDictionaryConfig ); + + for (GroupByQueryConfig config : configs) { + final GroupByQueryRunnerFactory factory = makeQueryRunnerFactory(config); + for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners(factory)) { + constructors.add(new Object[]{config, factory, runner}); + } + } + + return constructors; } - public GroupByQueryRunnerTest(GroupByQueryRunnerFactory factory, QueryRunner runner) + public GroupByQueryRunnerTest(GroupByQueryConfig config, GroupByQueryRunnerFactory factory, QueryRunner runner) { + this.config = config; this.factory = factory; - this.runner = runner; + this.runner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.>of(runner)); } @Test @@ -318,24 +387,204 @@ public void testMultipleDimensionsOneOfWhichIsMultiValue1() .build(); List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "automotive", "alias", "a", "rows", 2L, "idx", 282L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "business", "alias", "b", "rows", 2L, "idx", 230L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "entertainment", "alias", "e", "rows", 2L, "idx", 324L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "health", "alias", "h", "rows", 2L, "idx", 233L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "mezzanine", "alias", "m", "rows", 6L, "idx", 5317L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "news", "alias", "n", "rows", 2L, "idx", 235L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "premium", "alias", "p", "rows", 6L, "idx", 5405L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "automotive", "alias", "preferred", "rows", 2L, "idx", 282L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "business", "alias", "preferred", "rows", 2L, "idx", 230L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "entertainment", "alias", "preferred", "rows", 2L, "idx", 324L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "health", "alias", "preferred", "rows", 2L, "idx", 233L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "mezzanine", "alias", "preferred", "rows", 6L, "idx", 5317L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "news", "alias", "preferred", "rows", 2L, "idx", 235L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "premium", "alias", "preferred", "rows", 6L, "idx", 5405L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "technology", "alias", "preferred", "rows", 2L, "idx", 175L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "travel", "alias", "preferred", "rows", 2L, "idx", 245L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "technology", "alias", "t", "rows", 2L, "idx", 175L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "travel", "alias", "t", "rows", 2L, "idx", 245L) + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "automotive", + "alias", + "a", + "rows", + 2L, + "idx", + 282L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "business", + "alias", + "b", + "rows", + 2L, + "idx", + 230L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "entertainment", + "alias", + "e", + "rows", + 2L, + "idx", + 324L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "health", + "alias", + "h", + "rows", + 2L, + "idx", + 233L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "mezzanine", + "alias", + "m", + "rows", + 6L, + "idx", + 5317L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "news", + "alias", + "n", + "rows", + 2L, + "idx", + 235L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "premium", + "alias", + "p", + "rows", + 6L, + "idx", + 5405L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "automotive", + "alias", + "preferred", + "rows", + 2L, + "idx", + 282L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "business", + "alias", + "preferred", + "rows", + 2L, + "idx", + 230L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "entertainment", + "alias", + "preferred", + "rows", + 2L, + "idx", + 324L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "health", + "alias", + "preferred", + "rows", + 2L, + "idx", + 233L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "mezzanine", + "alias", + "preferred", + "rows", + 6L, + "idx", + 5317L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "news", + "alias", + "preferred", + "rows", + 2L, + "idx", + 235L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "premium", + "alias", + "preferred", + "rows", + 6L, + "idx", + 5405L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "technology", + "alias", + "preferred", + "rows", + 2L, + "idx", + 175L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "travel", + "alias", + "preferred", + "rows", + 2L, + "idx", + 245L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "technology", + "alias", + "t", + "rows", + 2L, + "idx", + 175L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "travel", + "alias", + "t", + "rows", + 2L, + "idx", + 245L + ) ); Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); @@ -363,31 +612,211 @@ public void testMultipleDimensionsOneOfWhichIsMultiValueDifferentOrder() .build(); List expectedResults = Arrays.asList( - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "automotive", "alias", "a", "rows", 2L, "idx", 282L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "automotive", "alias", "preferred", "rows", 2L, "idx", 282L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "business", "alias", "b", "rows", 2L, "idx", 230L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "business", "alias", "preferred", "rows", 2L, "idx", 230L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "entertainment", "alias", "e", "rows", 2L, "idx", 324L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "entertainment", "alias", "preferred", "rows", 2L, "idx", 324L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "health", "alias", "h", "rows", 2L, "idx", 233L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "health", "alias", "preferred", "rows", 2L, "idx", 233L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "mezzanine", "alias", "m", "rows", 6L, "idx", 5317L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "mezzanine", "alias", "preferred", "rows", 6L, "idx", 5317L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "news", "alias", "n", "rows", 2L, "idx", 235L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "news", "alias", "preferred", "rows", 2L, "idx", 235L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "premium", "alias", "p", "rows", 6L, "idx", 5405L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "premium", "alias", "preferred", "rows", 6L, "idx", 5405L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "technology", "alias", "preferred", "rows", 2L, "idx", 175L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "technology", "alias", "t", "rows", 2L, "idx", 175L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "travel", "alias", "preferred", "rows", 2L, "idx", 245L), - GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "travel", "alias", "t", "rows", 2L, "idx", 245L) + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "automotive", + "alias", + "a", + "rows", + 2L, + "idx", + 282L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "automotive", + "alias", + "preferred", + "rows", + 2L, + "idx", + 282L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "business", + "alias", + "b", + "rows", + 2L, + "idx", + 230L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "business", + "alias", + "preferred", + "rows", + 2L, + "idx", + 230L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "entertainment", + "alias", + "e", + "rows", + 2L, + "idx", + 324L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "entertainment", + "alias", + "preferred", + "rows", + 2L, + "idx", + 324L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "health", + "alias", + "h", + "rows", + 2L, + "idx", + 233L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "health", + "alias", + "preferred", + "rows", + 2L, + "idx", + 233L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "mezzanine", + "alias", + "m", + "rows", + 6L, + "idx", + 5317L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "mezzanine", + "alias", + "preferred", + "rows", + 6L, + "idx", + 5317L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "news", + "alias", + "n", + "rows", + 2L, + "idx", + 235L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "news", + "alias", + "preferred", + "rows", + 2L, + "idx", + 235L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "premium", + "alias", + "p", + "rows", + 6L, + "idx", + 5405L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "premium", + "alias", + "preferred", + "rows", + 6L, + "idx", + 5405L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "technology", + "alias", + "preferred", + "rows", + 2L, + "idx", + 175L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "technology", + "alias", + "t", + "rows", + 2L, + "idx", + 175L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "travel", + "alias", + "preferred", + "rows", + 2L, + "idx", + 245L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "2011-04-01", + "quality", + "travel", + "alias", + "t", + "rows", + 2L, + "idx", + 245L + ) ); Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } - @Test(expected = ISE.class) + @Test public void testGroupByMaxRowsLimitContextOverrid() { GroupByQuery query = GroupByQuery @@ -405,6 +834,10 @@ public void testGroupByMaxRowsLimitContextOverrid() .setContext(ImmutableMap.of("maxResults", 1)) .build(); + if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { + expectedException.expect(ISE.class); + } + GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); } @@ -962,6 +1395,7 @@ public String apply(String dimValue) } @Test + @Ignore /** * This test exists only to show what the current behavior is and not necessarily to define that this is * correct behavior. In fact, the behavior when returning the empty string from a DimExtractionFn is, by @@ -1267,7 +1701,12 @@ public Sequence run( final Query query2 = query.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) ); - return Sequences.concat(runner.run(query1, responseContext), runner.run(query2, responseContext)); + return new MergeSequence( + query.getResultOrdering(), + Sequences.simple( + Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext)) + ) + ); } } ); @@ -1285,7 +1724,6 @@ public Sequence run( ); Map context = Maps.newHashMap(); - TestHelper.assertExpectedObjects(expectedResults, runner.run(fullQuery, context), "direct"); TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged"); List allGranExpectedResults = Arrays.asList( @@ -1300,7 +1738,6 @@ public Sequence run( GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L) ); - TestHelper.assertExpectedObjects(allGranExpectedResults, runner.run(allGranQuery, context), "direct"); TestHelper.assertExpectedObjects(allGranExpectedResults, mergedRunner.run(allGranQuery, context), "merged"); } @@ -1511,7 +1948,12 @@ public Sequence run( final Query query2 = query.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) ); - return Sequences.concat(runner.run(query1, responseContext), runner.run(query2, responseContext)); + return new MergeSequence( + query.getResultOrdering(), + Sequences.simple( + Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext)) + ) + ); } } ); @@ -2282,7 +2724,12 @@ public Sequence run( final Query query2 = query.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) ); - return Sequences.concat(runner.run(query1, responseContext), runner.run(query2, responseContext)); + return new MergeSequence( + query.getResultOrdering(), + Sequences.simple( + Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext)) + ) + ); } } ); @@ -2519,7 +2966,12 @@ public Sequence run( final Query query2 = query.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) ); - return Sequences.concat(runner.run(query1, responseContext), runner.run(query2, responseContext)); + return new MergeSequence( + query.getResultOrdering(), + Sequences.simple( + Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext)) + ) + ); } } ); @@ -2623,9 +3075,11 @@ public Sequence run( final Query query2 = query.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) ); - return Sequences.concat( - runner.run(query1, responseContext), - runner.run(query2, responseContext) + return new MergeSequence( + query.getResultOrdering(), + Sequences.simple( + Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext)) + ) ); } } @@ -2666,27 +3120,7 @@ public void testGroupByWithRegEx() throws Exception GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "automotive", "rows", 2L) ); - final GroupByQueryEngine engine = new GroupByQueryEngine( - configSupplier, - new StupidPool<>( - new Supplier() - { - @Override - public ByteBuffer get() - { - return ByteBuffer.allocate(1024 * 1024); - } - } - ) - ); - - QueryRunner mergeRunner = new GroupByQueryQueryToolChest( - configSupplier, - new DefaultObjectMapper(), - engine, - TestQueryRunners.pool, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ).mergeResults(runner); + QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); Map context = Maps.newHashMap(); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); } @@ -2746,28 +3180,7 @@ public void testGroupByWithMetricColumnDisappears() throws Exception ); Map context = Maps.newHashMap(); - TestHelper.assertExpectedObjects(expectedResults, runner.run(query, context), "normal"); - final GroupByQueryEngine engine = new GroupByQueryEngine( - configSupplier, - new StupidPool<>( - new Supplier() - { - @Override - public ByteBuffer get() - { - return ByteBuffer.allocate(1024 * 1024); - } - } - ) - ); - - QueryRunner mergeRunner = new GroupByQueryQueryToolChest( - configSupplier, - new DefaultObjectMapper(), - engine, - TestQueryRunners.pool, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ).mergeResults(runner); + QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); } @@ -2826,28 +3239,7 @@ public void testGroupByWithNonexistentDimension() throws Exception ); Map context = Maps.newHashMap(); - TestHelper.assertExpectedObjects(expectedResults, runner.run(query, context), "normal"); - final GroupByQueryEngine engine = new GroupByQueryEngine( - configSupplier, - new StupidPool<>( - new Supplier() - { - @Override - public ByteBuffer get() - { - return ByteBuffer.allocate(1024 * 1024); - } - } - ) - ); - - QueryRunner mergeRunner = new GroupByQueryQueryToolChest( - configSupplier, - new DefaultObjectMapper(), - engine, - TestQueryRunners.pool, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ).mergeResults(runner); + QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit"); } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java index d1655c2a9774..d0e0d97a7dc7 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java @@ -57,32 +57,7 @@ public static Iterable constructorFeeder() throws IOException GroupByQueryConfig config = new GroupByQueryConfig(); config.setMaxIntermediateRows(10000); - final Supplier configSupplier = Suppliers.ofInstance(config); - final GroupByQueryEngine engine = new GroupByQueryEngine( - configSupplier, - new StupidPool( - new Supplier() - { - @Override - public ByteBuffer get() - { - return ByteBuffer.allocate(1024 * 1024); - } - } - ) - ); - - final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( - engine, - QueryRunnerTestHelper.NOOP_QUERYWATCHER, - configSupplier, - new GroupByQueryQueryToolChest( - configSupplier, new DefaultObjectMapper(), - engine, TestQueryRunners.pool, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ), - TestQueryRunners.pool - ); + final GroupByQueryRunnerFactory factory = GroupByQueryRunnerTest.makeQueryRunnerFactory(config); return QueryRunnerTestHelper.transformToConstructionFeeder( Lists.transform( QueryRunnerTestHelper.makeQueryRunners(factory), diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferGrouperTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferGrouperTest.java new file mode 100644 index 000000000000..827167e7ad0f --- /dev/null +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferGrouperTest.java @@ -0,0 +1,162 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.epinephelinae; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Ordering; +import com.google.common.primitives.Ints; +import io.druid.data.input.MapBasedRow; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.util.Comparator; +import java.util.List; + +public class BufferGrouperTest +{ + @Test + public void testSimple() + { + final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory(); + final Grouper grouper = new BufferGrouper<>( + ByteBuffer.allocate(1000), + GrouperTestUtil.intKeySerde(), + columnSelectorFactory, + new AggregatorFactory[]{ + new LongSumAggregatorFactory("valueSum", "value"), + new CountAggregatorFactory("count") + }, + Integer.MAX_VALUE, + -1 + ); + + columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 10L))); + grouper.aggregate(12); + grouper.aggregate(6); + grouper.aggregate(10); + grouper.aggregate(6); + grouper.aggregate(12); + grouper.aggregate(12); + + final List> expected = ImmutableList.of( + new Grouper.Entry<>(6, new Object[]{20L, 2L}), + new Grouper.Entry<>(10, new Object[]{10L, 1L}), + new Grouper.Entry<>(12, new Object[]{30L, 3L}) + ); + final List> unsortedEntries = Lists.newArrayList(grouper.iterator(false)); + final List> sortedEntries = Lists.newArrayList(grouper.iterator(true)); + + Assert.assertEquals(expected, sortedEntries); + Assert.assertEquals( + expected, + Ordering.from( + new Comparator>() + { + @Override + public int compare(Grouper.Entry o1, Grouper.Entry o2) + { + return Ints.compare(o1.getKey(), o2.getKey()); + } + } + ).sortedCopy(unsortedEntries) + ); + } + + @Test + public void testGrowing() + { + final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory(); + final Grouper grouper = makeGrouper(columnSelectorFactory, 10000, 2); + final int expectedMaxSize = 219; + + columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 10L))); + for (int i = 0; i < expectedMaxSize; i++) { + Assert.assertTrue(String.valueOf(i), grouper.aggregate(i)); + } + Assert.assertFalse(grouper.aggregate(expectedMaxSize)); + + // Aggregate slightly different row + columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 11L))); + for (int i = 0; i < expectedMaxSize; i++) { + Assert.assertTrue(String.valueOf(i), grouper.aggregate(i)); + } + Assert.assertFalse(grouper.aggregate(expectedMaxSize)); + + final List> expected = Lists.newArrayList(); + for (int i = 0; i < expectedMaxSize; i++) { + expected.add(new Grouper.Entry<>(i, new Object[]{21L, 2L})); + } + + Assert.assertEquals(expected, Lists.newArrayList(grouper.iterator(true))); + } + + @Test + public void testNoGrowing() + { + final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory(); + final Grouper grouper = makeGrouper(columnSelectorFactory, 10000, Integer.MAX_VALUE); + final int expectedMaxSize = 267; + + columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 10L))); + for (int i = 0; i < expectedMaxSize; i++) { + Assert.assertTrue(String.valueOf(i), grouper.aggregate(i)); + } + Assert.assertFalse(grouper.aggregate(expectedMaxSize)); + + // Aggregate slightly different row + columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 11L))); + for (int i = 0; i < expectedMaxSize; i++) { + Assert.assertTrue(String.valueOf(i), grouper.aggregate(i)); + } + Assert.assertFalse(grouper.aggregate(expectedMaxSize)); + + final List> expected = Lists.newArrayList(); + for (int i = 0; i < expectedMaxSize; i++) { + expected.add(new Grouper.Entry<>(i, new Object[]{21L, 2L})); + } + + Assert.assertEquals(expected, Lists.newArrayList(grouper.iterator(true))); + } + + private static BufferGrouper makeGrouper( + TestColumnSelectorFactory columnSelectorFactory, + int bufferSize, + int initialBuckets + ) + { + return new BufferGrouper<>( + ByteBuffer.allocate(bufferSize), + GrouperTestUtil.intKeySerde(), + columnSelectorFactory, + new AggregatorFactory[]{ + new LongSumAggregatorFactory("valueSum", "value"), + new CountAggregatorFactory("count") + }, + Integer.MAX_VALUE, + initialBuckets + ); + } +} diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/GrouperTestUtil.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/GrouperTestUtil.java new file mode 100644 index 000000000000..89294db2c7e3 --- /dev/null +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/GrouperTestUtil.java @@ -0,0 +1,38 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.epinephelinae; + +public class GrouperTestUtil +{ + private GrouperTestUtil() + { + // No instantiation + } + + public static Grouper.KeySerde intKeySerde() + { + return IntKeySerde.INSTANCE; + } + + public static TestColumnSelectorFactory newColumnSelectorFactory() + { + return new TestColumnSelectorFactory(); + } +} diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/IntKeySerde.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/IntKeySerde.java new file mode 100644 index 000000000000..5fce152cc962 --- /dev/null +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/IntKeySerde.java @@ -0,0 +1,83 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.epinephelinae; + +import com.google.common.primitives.Ints; + +import java.nio.ByteBuffer; + +public class IntKeySerde implements Grouper.KeySerde +{ + public static final Grouper.KeySerde INSTANCE = new IntKeySerde(); + + private IntKeySerde() + { + // No instantiation + } + + private static final Grouper.KeyComparator KEY_COMPARATOR = new Grouper.KeyComparator() + { + @Override + public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) + { + return Ints.compare(lhsBuffer.getInt(lhsPosition), rhsBuffer.getInt(rhsPosition)); + } + }; + + private final ByteBuffer buf = ByteBuffer.allocate(Ints.BYTES); + + @Override + public int keySize() + { + return Ints.BYTES; + } + + @Override + public Class keyClazz() + { + return Integer.class; + } + + @Override + public ByteBuffer toByteBuffer(Integer key) + { + buf.putInt(0, key); + buf.position(0); + return buf; + } + + @Override + public Integer fromByteBuffer(ByteBuffer buffer, int position) + { + return buffer.getInt(position); + } + + @Override + public Grouper.KeyComparator comparator() + { + return KEY_COMPARATOR; + } + + @Override + public void reset() + { + // Nothing to do + } +} diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/TestColumnSelectorFactory.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/TestColumnSelectorFactory.java new file mode 100644 index 000000000000..1b97037c1927 --- /dev/null +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/TestColumnSelectorFactory.java @@ -0,0 +1,89 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.epinephelinae; + +import io.druid.data.input.Row; +import io.druid.query.dimension.DimensionSpec; +import io.druid.segment.ColumnSelectorFactory; +import io.druid.segment.DimensionSelector; +import io.druid.segment.FloatColumnSelector; +import io.druid.segment.LongColumnSelector; +import io.druid.segment.ObjectColumnSelector; + +public class TestColumnSelectorFactory implements ColumnSelectorFactory +{ + private ThreadLocal row = new ThreadLocal<>(); + + public void setRow(Row row) + { + this.row.set(row); + } + + @Override + public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) + { + throw new UnsupportedOperationException(); + } + + @Override + public FloatColumnSelector makeFloatColumnSelector(final String columnName) + { + return new FloatColumnSelector() + { + @Override + public float get() + { + return row.get().getFloatMetric(columnName); + } + }; + } + + @Override + public LongColumnSelector makeLongColumnSelector(final String columnName) + { + return new LongColumnSelector() + { + @Override + public long get() + { + return row.get().getLongMetric(columnName); + } + }; + } + + @Override + public ObjectColumnSelector makeObjectColumnSelector(final String columnName) + { + return new ObjectColumnSelector() + { + @Override + public Class classOfObject() + { + return Object.class; + } + + @Override + public Object get() + { + return row.get().getRaw(columnName); + } + }; + } +} diff --git a/processing/src/test/java/io/druid/query/groupby/orderby/DefaultLimitSpecTest.java b/processing/src/test/java/io/druid/query/groupby/orderby/DefaultLimitSpecTest.java index 499c9e59d669..9bbc3cee1ddb 100644 --- a/processing/src/test/java/io/druid/query/groupby/orderby/DefaultLimitSpecTest.java +++ b/processing/src/test/java/io/druid/query/groupby/orderby/DefaultLimitSpecTest.java @@ -54,9 +54,9 @@ public class DefaultLimitSpecTest public DefaultLimitSpecTest() { testRowsList = ImmutableList.of( - createRow("2011-04-01", "k1", 9.0d, "k2", 2L, "k3", 3L), - createRow("2011-04-01", "k1", 10.0d, "k2", 1L, "k3", 2L), - createRow("2011-04-01", "k1", 20.0d, "k2", 3L, "k3", 1L) + createRow("2011-04-01", "k1", 10.0, "k2", 1L, "k3", 2L), + createRow("2011-04-01", "k1", 20.0, "k2", 3L, "k3", 1L), + createRow("2011-04-01", "k1", 9.0, "k2", 2L, "k3", 3L) ); testRowsSequence = Sequences.simple(testRowsList); @@ -118,6 +118,28 @@ public void testBuildSimple() ); } + @Test + public void testSortDimensionDescending() + { + DefaultLimitSpec limitSpec = new DefaultLimitSpec( + ImmutableList.of(new OrderByColumnSpec("k1", OrderByColumnSpec.Direction.DESCENDING)), + 2 + ); + + Function, Sequence> limitFn = limitSpec.build( + ImmutableList.of(new DefaultDimensionSpec("k1", "k1")), + ImmutableList.of(), + ImmutableList.of() + ); + + // Note: This test encodes the fact that limitSpec sorts numbers like strings; we might want to change this + // in the future. + Assert.assertEquals( + ImmutableList.of(testRowsList.get(2), testRowsList.get(1)), + Sequences.toList(limitFn.apply(testRowsSequence), new ArrayList()) + ); + } + @Test public void testBuildWithExplicitOrder() { @@ -140,7 +162,7 @@ public void testBuildWithExplicitOrder() ) ); Assert.assertEquals( - ImmutableList.of(testRowsList.get(1), testRowsList.get(2)), + ImmutableList.of(testRowsList.get(0), testRowsList.get(1)), Sequences.toList(limitFn.apply(testRowsSequence), new ArrayList()) ); @@ -157,7 +179,7 @@ public void testBuildWithExplicitOrder() ) ); Assert.assertEquals( - ImmutableList.of(testRowsList.get(0), testRowsList.get(1)), + ImmutableList.of(testRowsList.get(2), testRowsList.get(0)), Sequences.toList(limitFn.apply(testRowsSequence), new ArrayList()) ); @@ -180,7 +202,7 @@ public void testBuildWithExplicitOrder() ) ); Assert.assertEquals( - ImmutableList.of(testRowsList.get(0), testRowsList.get(1)), + ImmutableList.of(testRowsList.get(2), testRowsList.get(0)), Sequences.toList(limitFn.apply(testRowsSequence), new ArrayList()) ); } diff --git a/processing/src/test/java/io/druid/segment/TestHelper.java b/processing/src/test/java/io/druid/segment/TestHelper.java index d86ff55ad093..0be1cc96428d 100644 --- a/processing/src/test/java/io/druid/segment/TestHelper.java +++ b/processing/src/test/java/io/druid/segment/TestHelper.java @@ -21,15 +21,19 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Lists; +import com.google.common.math.DoubleMath; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; +import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.Result; import io.druid.segment.column.ColumnConfig; +import org.joda.time.DateTime; import org.junit.Assert; import java.util.Iterator; +import java.util.Map; /** */ @@ -132,8 +136,8 @@ private static void assertResults( if (expectedNext instanceof Row) { // HACK! Special casing for groupBy - Assert.assertEquals(failMsg, expectedNext, next); - Assert.assertEquals(failMsg, expectedNext, next2); + assertRow(failMsg, (Row) expectedNext, (Row) next); + assertRow(failMsg, (Row) expectedNext, (Row) next2); } else { assertResult(failMsg, (Result) expectedNext, (Result) next); assertResult( @@ -180,12 +184,16 @@ private static void assertObjects(Iterable expectedResults, Iterable a final Object next2 = resultsIter2.next(); String failMsg = msg + "-" + index++; - Assert.assertEquals(failMsg, expectedNext, next); - Assert.assertEquals( - String.format("%s: Second iterator bad, multiple calls to iterator() should be safe", failMsg), - expectedNext, - next2 - ); + String failMsg2 = String.format("%s: Second iterator bad, multiple calls to iterator() should be safe", failMsg); + + if (expectedNext instanceof Row) { + // HACK! Special casing for groupBy + assertRow(failMsg, (Row) expectedNext, (Row) next); + assertRow(failMsg2, (Row) expectedNext, (Row) next2); + } else { + Assert.assertEquals(failMsg, expectedNext, next); + Assert.assertEquals(failMsg2, expectedNext, next2); + } } if (resultsIter.hasNext()) { @@ -215,4 +223,35 @@ private static void assertResult(String msg, Result expected, Result actual) { Assert.assertEquals(msg, expected, actual); } + + private static void assertRow(String msg, Row expected, Row actual) + { + // Custom equals check to get fuzzy comparison of numerics, useful because different groupBy strategies don't + // always generate exactly the same results (different merge ordering / float vs double) + Assert.assertEquals(String.format("%s: timestamp", msg), expected.getTimestamp(), actual.getTimestamp()); + + final Map expectedMap = ((MapBasedRow) expected).getEvent(); + final Map actualMap = ((MapBasedRow) actual).getEvent(); + + Assert.assertEquals(String.format("%s: map keys", msg), expectedMap.keySet(), actualMap.keySet()); + for (final String key : expectedMap.keySet()) { + final Object expectedValue = expectedMap.get(key); + final Object actualValue = actualMap.get(key); + + if (expectedValue instanceof Float || expectedValue instanceof Double) { + Assert.assertEquals( + String.format("%s: key[%s]", msg, key), + ((Number) expectedValue).doubleValue(), + ((Number) actualValue).doubleValue(), + ((Number) expectedValue).doubleValue() * 1e-6 + ); + } else { + Assert.assertEquals( + String.format("%s: key[%s]", msg, key), + expectedValue, + actualValue + ); + } + } + } } diff --git a/server/src/main/java/io/druid/client/cache/MemcachedCache.java b/server/src/main/java/io/druid/client/cache/MemcachedCache.java index 26a5d60d6e8a..3316edbd3de1 100644 --- a/server/src/main/java/io/druid/client/cache/MemcachedCache.java +++ b/server/src/main/java/io/druid/client/cache/MemcachedCache.java @@ -473,9 +473,6 @@ public byte[] get(NamedKey key) return null; } } - catch (IOException e) { - throw Throwables.propagate(e); - } } @Override @@ -493,9 +490,6 @@ public void put(NamedKey key, byte[] value) errorCount.incrementAndGet(); log.warn(e, "Unable to queue cache operation"); } - catch (IOException e) { - Throwables.propagate(e); - } } private static byte[] serializeValue(NamedKey key, byte[] value) @@ -589,9 +583,6 @@ public String apply( return results; } } - catch (IOException e) { - throw Throwables.propagate(e); - } } @Override diff --git a/server/src/main/java/io/druid/guice/DruidProcessingModule.java b/server/src/main/java/io/druid/guice/DruidProcessingModule.java index 7b80f801252e..52c8a61b0633 100644 --- a/server/src/main/java/io/druid/guice/DruidProcessingModule.java +++ b/server/src/main/java/io/druid/guice/DruidProcessingModule.java @@ -30,12 +30,14 @@ import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import io.druid.client.cache.CacheConfig; +import io.druid.collections.BlockingPool; import io.druid.collections.StupidPool; import io.druid.common.utils.VMUtils; import io.druid.guice.annotations.BackgroundCaching; import io.druid.guice.annotations.Global; +import io.druid.guice.annotations.Merging; import io.druid.guice.annotations.Processing; -import io.druid.offheap.OffheapBufferPool; +import io.druid.offheap.OffheapBufferGenerator; import io.druid.query.DruidProcessingConfig; import io.druid.query.ExecutorServiceMonitor; import io.druid.query.MetricsEmittingExecutorService; @@ -103,30 +105,52 @@ public ExecutorService getProcessingExecutorService( @LazySingleton @Global public StupidPool getIntermediateResultsPool(DruidProcessingConfig config) + { + verifyDirectMemory(config); + return new StupidPool<>( + new OffheapBufferGenerator("intermediate processing", config.intermediateComputeSizeBytes()), + config.poolCacheMaxCount() + ); + } + + @Provides + @LazySingleton + @Merging + public BlockingPool getMergeBufferPool(DruidProcessingConfig config) + { + verifyDirectMemory(config); + return new BlockingPool<>( + new OffheapBufferGenerator("result merging", config.intermediateComputeSizeBytes()), + config.getNumMergeBuffers() + ); + } + + private void verifyDirectMemory(DruidProcessingConfig config) { try { - long maxDirectMemory = VMUtils.getMaxDirectMemory(); + final long maxDirectMemory = VMUtils.getMaxDirectMemory(); + final long memoryNeeded = (long) config.intermediateComputeSizeBytes() * + (config.getNumMergeBuffers() + config.getNumThreads() + 1); - final long memoryNeeded = (long) config.intermediateComputeSizeBytes() * (config.getNumThreads() + 1); if (maxDirectMemory < memoryNeeded) { throw new ProvisionException( String.format( - "Not enough direct memory. Please adjust -XX:MaxDirectMemorySize, druid.processing.buffer.sizeBytes, or druid.processing.numThreads: " - + "maxDirectMemory[%,d], memoryNeeded[%,d] = druid.processing.buffer.sizeBytes[%,d] * ( druid.processing.numThreads[%,d] + 1 )", + "Not enough direct memory. Please adjust -XX:MaxDirectMemorySize, druid.processing.buffer.sizeBytes, druid.processing.numThreads, or druid.processing.numMergeBuffers: " + + "maxDirectMemory[%,d], memoryNeeded[%,d] = druid.processing.buffer.sizeBytes[%,d] * (druid.processing.numMergeBuffers[%,d] + druid.processing.numThreads[%,d] + 1)", maxDirectMemory, memoryNeeded, config.intermediateComputeSizeBytes(), + config.getNumMergeBuffers(), config.getNumThreads() ) ); } } catch (UnsupportedOperationException e) { - log.info(e.getMessage()); + log.info( + "Could not verify that you have enough direct memory, so I hope you do! Error message was: %s", + e.getMessage() + ); } - - return new OffheapBufferPool(config.intermediateComputeSizeBytes(), config.poolCacheMaxCount()); } - - } diff --git a/server/src/main/java/io/druid/offheap/OffheapBufferPool.java b/server/src/main/java/io/druid/offheap/OffheapBufferGenerator.java similarity index 58% rename from server/src/main/java/io/druid/offheap/OffheapBufferPool.java rename to server/src/main/java/io/druid/offheap/OffheapBufferGenerator.java index aab3ffa7c18e..1edda8cfa580 100644 --- a/server/src/main/java/io/druid/offheap/OffheapBufferPool.java +++ b/server/src/main/java/io/druid/offheap/OffheapBufferGenerator.java @@ -21,34 +21,34 @@ import com.google.common.base.Supplier; import com.metamx.common.logger.Logger; -import io.druid.collections.StupidPool; import java.nio.ByteBuffer; import java.util.concurrent.atomic.AtomicLong; - -public class OffheapBufferPool extends StupidPool +public class OffheapBufferGenerator implements Supplier { - private static final Logger log = new Logger(OffheapBufferPool.class); + private static final Logger log = new Logger(OffheapBufferGenerator.class); + + private final String description; + private final int computationBufferSize; + private final AtomicLong count = new AtomicLong(0); + + public OffheapBufferGenerator(String description, int computationBufferSize) + { + this.description = description; + this.computationBufferSize = computationBufferSize; + } - public OffheapBufferPool(final int computationBufferSize, final int cacheMaxCount) + @Override + public ByteBuffer get() { - super( - new Supplier() - { - final AtomicLong count = new AtomicLong(0); - - @Override - public ByteBuffer get() - { - log.info( - "Allocating new intermediate processing buffer[%,d] of size[%,d]", - count.getAndIncrement(), computationBufferSize - ); - return ByteBuffer.allocateDirect(computationBufferSize); - } - }, - cacheMaxCount + log.info( + "Allocating new %s buffer[%,d] of size[%,d]", + description, + count.getAndIncrement(), + computationBufferSize ); + + return ByteBuffer.allocateDirect(computationBufferSize); } } diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index 1ca49b2a1c40..bbb5ad104bfb 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -92,6 +92,7 @@ import io.druid.query.groupby.GroupByQueryConfig; import io.druid.query.groupby.GroupByQueryEngine; import io.druid.query.groupby.GroupByQueryQueryToolChest; +import io.druid.query.groupby.GroupByQueryRunnerTest; import io.druid.query.search.SearchQueryQueryToolChest; import io.druid.query.search.SearchResultValue; import io.druid.query.search.search.SearchHit; @@ -211,7 +212,6 @@ public class CachingClusteredClientTest private static final DateTimeZone TIMEZONE = DateTimeZone.forID("America/Los_Angeles"); private static final QueryGranularity PT1H_TZ_GRANULARITY = new PeriodGranularity(new Period("PT1H"), null, TIMEZONE); private static final String TOP_DIM = "a_dim"; - private static final Supplier GROUPBY_QUERY_CONFIG_SUPPLIER = Suppliers.ofInstance(new GroupByQueryConfig()); static final QueryToolChestWarehouse WAREHOUSE = new MapQueryToolChestWarehouse( ImmutableMap., QueryToolChest>builder() .put( @@ -241,25 +241,7 @@ SearchQuery.class, new SearchQueryQueryToolChest( ) .put( GroupByQuery.class, - new GroupByQueryQueryToolChest( - GROUPBY_QUERY_CONFIG_SUPPLIER, - jsonMapper, - new GroupByQueryEngine( - GROUPBY_QUERY_CONFIG_SUPPLIER, - new StupidPool<>( - new Supplier() - { - @Override - public ByteBuffer get() - { - return ByteBuffer.allocate(1024 * 1024); - } - } - ) - ), - TestQueryRunners.pool, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ) + GroupByQueryRunnerTest.makeQueryRunnerFactory(new GroupByQueryConfig()).getToolchest() ) .put(TimeBoundaryQuery.class, new TimeBoundaryQueryQueryToolChest()) .build() @@ -1301,35 +1283,9 @@ public void testGroupByCaching() throws Exception ) ); - Supplier configSupplier = new Supplier() - { - @Override - public GroupByQueryConfig get() - { - return new GroupByQueryConfig(); - } - }; QueryRunner runner = new FinalizeResultsQueryRunner( client, - new GroupByQueryQueryToolChest( - configSupplier, - jsonMapper, - new GroupByQueryEngine( - configSupplier, - new StupidPool<>( - new Supplier() - { - @Override - public ByteBuffer get() - { - return ByteBuffer.allocate(1024 * 1024); - } - } - ) - ), - TestQueryRunners.pool, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ) + GroupByQueryRunnerTest.makeQueryRunnerFactory(new GroupByQueryConfig()).getToolchest() ); HashMap context = new HashMap(); TestHelper.assertExpectedObjects( @@ -2552,35 +2508,9 @@ public void testGroupByCachingRenamedAggs() throws Exception ) ); - Supplier configSupplier = new Supplier() - { - @Override - public GroupByQueryConfig get() - { - return new GroupByQueryConfig(); - } - }; QueryRunner runner = new FinalizeResultsQueryRunner( client, - new GroupByQueryQueryToolChest( - configSupplier, - jsonMapper, - new GroupByQueryEngine( - configSupplier, - new StupidPool<>( - new Supplier() - { - @Override - public ByteBuffer get() - { - return ByteBuffer.allocate(1024 * 1024); - } - } - ) - ), - TestQueryRunners.pool, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ) + GroupByQueryRunnerTest.makeQueryRunnerFactory(new GroupByQueryConfig()).getToolchest() ); HashMap context = new HashMap(); TestHelper.assertExpectedObjects( diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index 5e21b355cd32..456185d0e75f 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -61,6 +61,7 @@ import io.druid.query.groupby.GroupByQueryEngine; import io.druid.query.groupby.GroupByQueryQueryToolChest; import io.druid.query.groupby.GroupByQueryRunnerFactory; +import io.druid.query.groupby.GroupByQueryRunnerTest; import io.druid.query.groupby.GroupByQueryRunnerTestHelper; import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.query.spec.MultipleSpecificSegmentSpec; @@ -672,31 +673,9 @@ public void testQueryWithMultipleSegmentSpec() throws IOException, InterruptedEx private static GroupByQueryRunnerFactory initFactory() { - final ObjectMapper mapper = new DefaultObjectMapper(); - final StupidPool pool = new StupidPool<>( - new Supplier() - { - @Override - public ByteBuffer get() - { - return ByteBuffer.allocate(1024 * 1024); - } - } - ); final GroupByQueryConfig config = new GroupByQueryConfig(); config.setMaxIntermediateRows(10000); - final Supplier configSupplier = Suppliers.ofInstance(config); - final GroupByQueryEngine engine = new GroupByQueryEngine(configSupplier, pool); - return new GroupByQueryRunnerFactory( - engine, - QueryRunnerTestHelper.NOOP_QUERYWATCHER, - configSupplier, - new GroupByQueryQueryToolChest( - configSupplier, mapper, engine, TestQueryRunners.pool, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ), - TestQueryRunners.pool - ); + return GroupByQueryRunnerTest.makeQueryRunnerFactory(config); } @After