-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Alternative groupBy strategy. #2998
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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<ByteBuffer> bufferPool = new StupidPool<>( | ||
| new OffheapBufferGenerator("compute", 250000000), | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can these be more descriptive, and are those values required to be fixed? Are the changes in direct memory needserves documented?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is benchmark, nevermind |
||
| Integer.MAX_VALUE | ||
| ); | ||
| BlockingPool<ByteBuffer> 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<GroupByQueryConfig> 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 | ||
| ) | ||
| ); | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -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<T> | ||
| { | ||
| private static final Logger log = new Logger(BlockingPool.class); | ||
|
|
||
| private final BlockingQueue<T> objects; | ||
|
|
||
| public BlockingPool( | ||
| Supplier<T> generator, | ||
| int limit | ||
| ) | ||
| { | ||
| this.objects = limit > 0 ? new ArrayBlockingQueue<T>(limit) : null; | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. preconditions check that limit is set?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nvm i see it below |
||
|
|
||
| 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<T> 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<T> | ||
| { | ||
| 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); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. is it possible here to try to return the same object to pool?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. what do you mean? |
||
| if (!objects.offer(object)) { | ||
| log.error("WTF?! Queue offer failed during finalize, uh oh..."); | ||
| } | ||
| } | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -26,4 +26,7 @@ | |
| public interface ResourceHolder<T> extends Closeable | ||
| { | ||
| T get(); | ||
|
|
||
| @Override | ||
| void close(); | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Most formats are with the thread enumeration after a dash rather than in brackets. Suggest keeping that here.