From f3f2cd35e144ca4ab01b7f9c37832555e40bbd97 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 4 Aug 2017 02:04:54 +0900 Subject: [PATCH] Array-based aggregation for groupBy query (#4576) * Array-based aggregation * Fix handling missing grouping key * Handle invalid offset * Fix compilation * Add cardinality check * Fix cardinality check * Address comments * Address comments * Address comments * Address comments * Cleanup GroupByQueryEngineV2.process * Change to Byte.SIZE * Add flatMap --- .../benchmark/query/GroupByBenchmark.java | 28 + docs/content/querying/groupbyquery.md | 4 + ...singSketchMergeAggregatorFactoryTest.java} | 8 +- .../util/common/guava/ConcatSequence.java | 4 +- .../java/util/common/guava/Sequence.java | 13 + .../java/util/common/guava/Sequences.java | 2 +- .../query/groupby/GroupByQueryConfig.java | 12 + ...er.java => AbstractBufferHashGrouper.java} | 32 +- .../epinephelinae/BufferArrayGrouper.java | 284 ++++++++++ ...ferGrouper.java => BufferHashGrouper.java} | 6 +- .../epinephelinae/ByteBufferHashTable.java | 2 +- .../ByteBufferMinMaxOffsetHeap.java | 4 +- .../epinephelinae/ConcurrentGrouper.java | 6 - .../epinephelinae/GroupByQueryEngineV2.java | 533 +++++++++++++----- .../query/groupby/epinephelinae/Grouper.java | 20 +- .../query/groupby/epinephelinae/Groupers.java | 16 + ...per.java => LimitedBufferHashGrouper.java} | 6 +- .../epinephelinae/SpillingGrouper.java | 12 +- ...ngStringGroupByColumnSelectorStrategy.java | 24 + .../DoubleGroupByColumnSelectorStrategy.java | 12 + .../FloatGroupByColumnSelectorStrategy.java | 12 + .../column/GroupByColumnSelectorStrategy.java | 22 +- .../LongGroupByColumnSelectorStrategy.java | 12 + .../StringGroupByColumnSelectorStrategy.java | 18 +- .../groupby/strategy/GroupByStrategyV2.java | 2 +- .../query/groupby/GroupByQueryRunnerTest.java | 4 +- .../epinephelinae/BufferArrayGrouperTest.java | 89 +++ ...erTest.java => BufferHashGrouperTest.java} | 8 +- ...java => LimitedBufferHashGrouperTest.java} | 12 +- 29 files changed, 994 insertions(+), 213 deletions(-) rename extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/{BufferGrouperUsingSketchMergeAggregatorFactoryTest.java => BufferHashGrouperUsingSketchMergeAggregatorFactoryTest.java} (92%) rename processing/src/main/java/io/druid/query/groupby/epinephelinae/{AbstractBufferGrouper.java => AbstractBufferHashGrouper.java} (85%) create mode 100644 processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferArrayGrouper.java rename processing/src/main/java/io/druid/query/groupby/epinephelinae/{BufferGrouper.java => BufferHashGrouper.java} (97%) rename processing/src/main/java/io/druid/query/groupby/epinephelinae/{LimitedBufferGrouper.java => LimitedBufferHashGrouper.java} (99%) create mode 100644 processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferArrayGrouperTest.java rename processing/src/test/java/io/druid/query/groupby/epinephelinae/{BufferGrouperTest.java => BufferHashGrouperTest.java} (97%) rename processing/src/test/java/io/druid/query/groupby/epinephelinae/{LimitedBufferGrouperTest.java => LimitedBufferHashGrouperTest.java} (93%) 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 09f8a1fd2f8..7720cb1358b 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -53,10 +54,13 @@ import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.DoubleMinAggregatorFactory; +import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DimensionSpec; +import io.druid.query.filter.BoundDimFilter; import io.druid.query.groupby.GroupByQuery; import io.druid.query.groupby.GroupByQueryConfig; import io.druid.query.groupby.GroupByQueryEngine; @@ -238,6 +242,30 @@ public class GroupByBenchmark basicQueries.put("nested", queryA); } + + { // basic.filter + final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec( + Collections.singletonList(basicSchema.getDataInterval()) + ); + // Use multiple aggregators to see how the number of aggregators impact to the query performance + List queryAggs = ImmutableList.of( + new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"), + new LongSumAggregatorFactory("rows", "rows"), + new DoubleSumAggregatorFactory("sumFloatNormal", "sumFloatNormal"), + new DoubleMinAggregatorFactory("minFloatZipf", "minFloatZipf") + ); + GroupByQuery queryA = GroupByQuery + .builder() + .setDataSource("blah") + .setQuerySegmentSpec(intervalSpec) + .setDimensions(ImmutableList.of(new DefaultDimensionSpec("dimUniform", null))) + .setAggregatorSpecs(queryAggs) + .setGranularity(Granularity.fromString(queryGranularity)) + .setDimFilter(new BoundDimFilter("dimUniform", "0", "100", true, true, null, null, null)) + .build(); + + basicQueries.put("filter", queryA); + } SCHEMA_QUERY_MAP.put("basic", basicQueries); // simple one column schema, for testing performance difference between querying on numeric values as Strings and diff --git a/docs/content/querying/groupbyquery.md b/docs/content/querying/groupbyquery.md index 32bbbf42492..fc323217ec0 100644 --- a/docs/content/querying/groupbyquery.md +++ b/docs/content/querying/groupbyquery.md @@ -153,6 +153,9 @@ threads. You can adjust this as necessary to balance concurrency and memory usag historical nodes. - groupBy v1 supports using [chunkPeriod](query-context.html) to parallelize merging on the broker, whereas groupBy v2 ignores chunkPeriod. +- groupBy v2 supports both array-based aggregation and hash-based aggregation. The array-based aggregation is used only +when the grouping key is a single indexed string column. In array-based aggregation, the dictionary-encoded value is used +as the index, so the aggregated values in the array can be accessed directly without finding buckets based on hashing. #### Memory tuning and resource limits @@ -246,6 +249,7 @@ When using the "v2" strategy, the following query context parameters apply: |`maxOnDiskStorage`|Can be used to lower the value of `druid.query.groupBy.maxOnDiskStorage` for this query.| |`sortByDimsFirst`|Sort the results first by dimension values and then by timestamp.| |`forcePushDownLimit`|When all fields in the orderby are part of the grouping key, the broker will push limit application down to the historical nodes. When the sorting order uses fields that are not in the grouping key, applying this optimization can result in approximate results with unknown accuracy, so this optimization is disabled by default in that case. Enabling this context flag turns on limit push down for limit/orderbys that contain non-grouping key columns.| +|`forceHashAggregation`|Force to use hash-based aggregation.| When using the "v1" strategy, the following query context parameters apply: diff --git a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/BufferGrouperUsingSketchMergeAggregatorFactoryTest.java b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/BufferHashGrouperUsingSketchMergeAggregatorFactoryTest.java similarity index 92% rename from extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/BufferGrouperUsingSketchMergeAggregatorFactoryTest.java rename to extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/BufferHashGrouperUsingSketchMergeAggregatorFactoryTest.java index c70b60e4228..ef4cf90a8de 100644 --- a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/BufferGrouperUsingSketchMergeAggregatorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/BufferHashGrouperUsingSketchMergeAggregatorFactoryTest.java @@ -27,7 +27,7 @@ import com.yahoo.sketches.theta.UpdateSketch; import io.druid.data.input.MapBasedRow; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; -import io.druid.query.groupby.epinephelinae.BufferGrouper; +import io.druid.query.groupby.epinephelinae.BufferHashGrouper; import io.druid.query.groupby.epinephelinae.Grouper; import io.druid.query.groupby.epinephelinae.GrouperTestUtil; import io.druid.query.groupby.epinephelinae.TestColumnSelectorFactory; @@ -36,15 +36,15 @@ import org.junit.Test; import java.nio.ByteBuffer; -public class BufferGrouperUsingSketchMergeAggregatorFactoryTest +public class BufferHashGrouperUsingSketchMergeAggregatorFactoryTest { - private static BufferGrouper makeGrouper( + private static BufferHashGrouper makeGrouper( TestColumnSelectorFactory columnSelectorFactory, int bufferSize, int initialBuckets ) { - final BufferGrouper grouper = new BufferGrouper<>( + final BufferHashGrouper grouper = new BufferHashGrouper<>( Suppliers.ofInstance(ByteBuffer.allocate(bufferSize)), GrouperTestUtil.intKeySerde(), columnSelectorFactory, diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java index 7479b7c0a95..efbab021390 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java @@ -31,10 +31,10 @@ public class ConcatSequence implements Sequence private final Sequence> baseSequences; public ConcatSequence( - Sequence> baseSequences + Sequence> baseSequences ) { - this.baseSequences = baseSequences; + this.baseSequences = (Sequence>) baseSequences; } @Override diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/Sequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/Sequence.java index b3851e706c1..20fab62adc2 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/Sequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/Sequence.java @@ -21,6 +21,7 @@ package io.druid.java.util.common.guava; import com.google.common.collect.Ordering; +import java.io.Closeable; import java.util.concurrent.Executor; import java.util.function.Function; @@ -68,6 +69,13 @@ public interface Sequence return new MappedSequence<>(this, mapper); } + default Sequence flatMap( + Function> mapper + ) + { + return new ConcatSequence<>(this.map(mapper)); + } + default Sequence flatMerge( Function> mapper, Ordering ordering @@ -80,4 +88,9 @@ public interface Sequence { return Sequences.withEffect(this, effect, effectExecutor); } + + default Sequence withBaggage(Closeable baggage) + { + return Sequences.withBaggage(this, baggage); + } } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java index ccbe990c1ba..8d9a77904a8 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java @@ -75,7 +75,7 @@ public class Sequences return concat(Sequences.simple(sequences)); } - public static Sequence concat(Sequence> sequences) + public static Sequence concat(Sequence> sequences) { return new ConcatSequence<>(sequences); } 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 0e1516b1c71..230acbd0a07 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java @@ -36,6 +36,7 @@ public class GroupByQueryConfig private static final String CTX_KEY_BUFFER_GROUPER_MAX_SIZE = "bufferGrouperMaxSize"; private static final String CTX_KEY_MAX_ON_DISK_STORAGE = "maxOnDiskStorage"; private static final String CTX_KEY_MAX_MERGING_DICTIONARY_SIZE = "maxMergingDictionarySize"; + private static final String CTX_KEY_FORCE_HASH_AGGREGATION = "forceHashAggregation"; @JsonProperty private String defaultStrategy = GroupByStrategySelector.STRATEGY_V2; @@ -70,6 +71,9 @@ public class GroupByQueryConfig @JsonProperty private boolean forcePushDownLimit = false; + @JsonProperty + private boolean forceHashAggregation = false; + public String getDefaultStrategy() { return defaultStrategy; @@ -134,6 +138,11 @@ public class GroupByQueryConfig { return forcePushDownLimit; } + + public boolean isForceHashAggregation() + { + return forceHashAggregation; + } public GroupByQueryConfig withOverrides(final GroupByQuery query) { @@ -169,6 +178,7 @@ public class GroupByQueryConfig getMaxMergingDictionarySize() ); newConfig.forcePushDownLimit = query.getContextBoolean(CTX_KEY_FORCE_LIMIT_PUSH_DOWN, isForcePushDownLimit()); + newConfig.forceHashAggregation = query.getContextBoolean(CTX_KEY_FORCE_HASH_AGGREGATION, isForceHashAggregation()); return newConfig; } @@ -185,6 +195,8 @@ public class GroupByQueryConfig ", bufferGrouperInitialBuckets=" + bufferGrouperInitialBuckets + ", maxMergingDictionarySize=" + maxMergingDictionarySize + ", maxOnDiskStorage=" + maxOnDiskStorage + + ", forcePushDownLimit=" + forcePushDownLimit + + ", forceHashAggregation=" + forceHashAggregation + '}'; } } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/AbstractBufferGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/AbstractBufferHashGrouper.java similarity index 85% rename from processing/src/main/java/io/druid/query/groupby/epinephelinae/AbstractBufferGrouper.java rename to processing/src/main/java/io/druid/query/groupby/epinephelinae/AbstractBufferHashGrouper.java index 7d847b2b06e..ef0d57b7331 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/AbstractBufferGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/AbstractBufferHashGrouper.java @@ -28,21 +28,10 @@ import io.druid.query.aggregation.BufferAggregator; import java.nio.ByteBuffer; -public abstract class AbstractBufferGrouper implements Grouper +public abstract class AbstractBufferHashGrouper implements Grouper { - private static final AggregateResult DICTIONARY_FULL = AggregateResult.failure( - "Not enough dictionary space to execute this query. Try increasing " - + "druid.query.groupBy.maxMergingDictionarySize or enable disk spilling by setting " - + "druid.query.groupBy.maxOnDiskStorage to a positive number." - ); - private static final AggregateResult HASHTABLE_FULL = AggregateResult.failure( - "Not enough aggregation table space to execute this query. Try increasing " - + "druid.processing.buffer.sizeBytes or enable disk spilling by setting " - + "druid.query.groupBy.maxOnDiskStorage to a positive number." - ); - protected static final int HASH_SIZE = Ints.BYTES; - protected static final Logger log = new Logger(AbstractBufferGrouper.class); + protected static final Logger log = new Logger(AbstractBufferHashGrouper.class); protected final Supplier bufferSupplier; protected final KeySerde keySerde; @@ -61,7 +50,8 @@ public abstract class AbstractBufferGrouper implements Grouper protected ByteBufferHashTable hashTable; protected ByteBuffer hashTableBuffer; // buffer for the entire hash table (total space, not individual growth) - public AbstractBufferGrouper( + public AbstractBufferHashGrouper( + // the buffer returned from the below supplier can have dirty bits and should be cleared during initialization final Supplier bufferSupplier, final KeySerde keySerde, final AggregatorFactory[] aggregatorFactories, @@ -77,7 +67,7 @@ public abstract class AbstractBufferGrouper implements Grouper } /** - * Called when a new bucket is used for an entry in the hash table. An implementing BufferGrouper class + * Called when a new bucket is used for an entry in the hash table. An implementing BufferHashGrouper class * can use this to update its own state, e.g. tracking bucket offsets in a structure outside of the hash table. * * @param bucketOffset offset of the new bucket, within the buffer returned by hashTable.getTableBuffer() @@ -95,7 +85,7 @@ public abstract class AbstractBufferGrouper implements Grouper public abstract boolean canSkipAggregate(boolean bucketWasUsed, int bucketOffset); /** - * Called after a row is aggregated. An implementing BufferGrouper class can use this to update + * Called after a row is aggregated. An implementing BufferHashGrouper class can use this to update * its own state, e.g. reading the new aggregated values for the row's key and acting on that information. * * @param bucketOffset Offset of the bucket containing the row that was aggregated, @@ -134,7 +124,7 @@ public abstract class AbstractBufferGrouper implements Grouper if (keyBuffer == null) { // This may just trigger a spill and get ignored, which is ok. If it bubbles up to the user, the message will // be correct. - return DICTIONARY_FULL; + return Groupers.DICTIONARY_FULL; } if (keyBuffer.remaining() != keySize) { @@ -150,7 +140,7 @@ public abstract class AbstractBufferGrouper implements Grouper if (bucket < 0) { // This may just trigger a spill and get ignored, which is ok. If it bubbles up to the user, the message will // be correct. - return HASHTABLE_FULL; + return Groupers.HASH_TABLE_FULL; } final int bucketStartOffset = hashTable.getOffsetForBucket(bucket); @@ -181,12 +171,6 @@ public abstract class AbstractBufferGrouper implements Grouper return AggregateResult.ok(); } - @Override - public AggregateResult aggregate(final KeyType key) - { - return aggregate(key, Groupers.hash(key)); - } - @Override public void close() { diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferArrayGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferArrayGrouper.java new file mode 100644 index 00000000000..a17761090c1 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferArrayGrouper.java @@ -0,0 +1,284 @@ +/* + * 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.base.Supplier; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.logger.Logger; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.groupby.epinephelinae.column.GroupByColumnSelectorStrategy; +import io.druid.segment.ColumnSelectorFactory; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.function.ToIntFunction; + +/** + * A buffer grouper for array-based aggregation. This grouper stores aggregated values in the buffer using the grouping + * key as the index. + *

+ * The buffer is divided into 2 separate regions, i.e., used flag buffer and value buffer. The used flag buffer is a + * bit set to represent which keys are valid. If a bit of an index is set, that key is valid. Finally, the value + * buffer is used to store aggregated values. The first index is reserved for + * {@link GroupByColumnSelectorStrategy#GROUP_BY_MISSING_VALUE}. + *

+ * This grouper is available only when the grouping key is a single indexed dimension of a known cardinality because it + * directly uses the dimension value as the index for array access. Since the cardinality for the grouping key across + * different segments cannot be currently retrieved, this grouper can be used only when performing per-segment query + * execution. + */ +public class BufferArrayGrouper implements Grouper +{ + private static final Logger LOG = new Logger(BufferArrayGrouper.class); + + private final Supplier bufferSupplier; + private final BufferAggregator[] aggregators; + private final int[] aggregatorOffsets; + private final int cardinalityWithMissingValue; + private final int recordSize; // size of all aggregated values + + private boolean initialized = false; + private ByteBuffer usedFlagBuffer; + private ByteBuffer valBuffer; + + static int requiredBufferCapacity( + int cardinality, + AggregatorFactory[] aggregatorFactories + ) + { + final int cardinalityWithMissingValue = cardinality + 1; + final int recordSize = Arrays.stream(aggregatorFactories) + .mapToInt(AggregatorFactory::getMaxIntermediateSize) + .sum(); + + return getUsedFlagBufferCapacity(cardinalityWithMissingValue) + // total used flags size + cardinalityWithMissingValue * recordSize; // total values size + } + + /** + * Compute the number of bytes to store all used flag bits. + */ + private static int getUsedFlagBufferCapacity(int cardinalityWithMissingValue) + { + return (cardinalityWithMissingValue + Byte.SIZE - 1) / Byte.SIZE; + } + + public BufferArrayGrouper( + // the buffer returned from the below supplier can have dirty bits and should be cleared during initialization + final Supplier bufferSupplier, + final ColumnSelectorFactory columnSelectorFactory, + final AggregatorFactory[] aggregatorFactories, + final int cardinality + ) + { + Preconditions.checkNotNull(aggregatorFactories, "aggregatorFactories"); + Preconditions.checkArgument(cardinality > 0, "Cardinality must a non-zero positive number"); + + this.bufferSupplier = Preconditions.checkNotNull(bufferSupplier, "bufferSupplier"); + this.aggregators = new BufferAggregator[aggregatorFactories.length]; + this.aggregatorOffsets = new int[aggregatorFactories.length]; + this.cardinalityWithMissingValue = cardinality + 1; + + int offset = 0; + for (int i = 0; i < aggregatorFactories.length; i++) { + aggregators[i] = aggregatorFactories[i].factorizeBuffered(columnSelectorFactory); + aggregatorOffsets[i] = offset; + offset += aggregatorFactories[i].getMaxIntermediateSize(); + } + recordSize = offset; + } + + @Override + public void init() + { + if (!initialized) { + final ByteBuffer buffer = bufferSupplier.get(); + + final int usedFlagBufferEnd = getUsedFlagBufferCapacity(cardinalityWithMissingValue); + buffer.position(0); + buffer.limit(usedFlagBufferEnd); + usedFlagBuffer = buffer.slice(); + + buffer.position(usedFlagBufferEnd); + buffer.limit(buffer.capacity()); + valBuffer = buffer.slice(); + + reset(); + + initialized = true; + } + } + + @Override + public boolean isInitialized() + { + return initialized; + } + + @Override + public AggregateResult aggregate(Integer key, int dimIndex) + { + Preconditions.checkArgument( + dimIndex >= 0 && dimIndex < cardinalityWithMissingValue, + "Invalid dimIndex[%s]", + dimIndex + ); + + Preconditions.checkNotNull(key); + + final int recordOffset = dimIndex * recordSize; + + if (recordOffset + recordSize > valBuffer.capacity()) { + // This error cannot be recoverd, and the query must fail + throw new ISE( + "A record of size [%d] cannot be written to the array buffer at offset[%d] " + + "because it exceeds the buffer capacity[%d]. Try increasing druid.processing.buffer.sizeBytes", + recordSize, + recordOffset, + valBuffer.capacity() + ); + } + + if (!isUsedSlot(dimIndex)) { + initializeSlot(dimIndex); + } + + for (int i = 0; i < aggregators.length; i++) { + aggregators[i].aggregate(valBuffer, recordOffset + aggregatorOffsets[i]); + } + + return AggregateResult.ok(); + } + + private void initializeSlot(int dimIndex) + { + final int index = dimIndex / Byte.SIZE; + final int extraIndex = dimIndex % Byte.SIZE; + usedFlagBuffer.put(index, (byte) (usedFlagBuffer.get(index) | (1 << extraIndex))); + + final int recordOffset = dimIndex * recordSize; + for (int i = 0; i < aggregators.length; i++) { + aggregators[i].init(valBuffer, recordOffset + aggregatorOffsets[i]); + } + } + + private boolean isUsedSlot(int dimIndex) + { + final int index = dimIndex / Byte.SIZE; + final int extraIndex = dimIndex % Byte.SIZE; + final int usedFlagByte = 1 << extraIndex; + return (usedFlagBuffer.get(index) & usedFlagByte) != 0; + } + + @Override + public void reset() + { + // Clear the entire usedFlagBuffer + final int usedFlagBufferCapacity = usedFlagBuffer.capacity(); + + // putLong() instead of put() can boost the performance of clearing the buffer + final int n = (usedFlagBufferCapacity / Long.BYTES) * Long.BYTES; + for (int i = 0; i < n; i += Long.BYTES) { + usedFlagBuffer.putLong(i, 0L); + } + + for (int i = n; i < usedFlagBufferCapacity; i++) { + usedFlagBuffer.put(i, (byte) 0); + } + } + + @Override + public ToIntFunction hashFunction() + { + return key -> key + 1; + } + + @Override + public void close() + { + for (BufferAggregator aggregator : aggregators) { + try { + aggregator.close(); + } + catch (Exception e) { + LOG.warn(e, "Could not close aggregator [%s], skipping.", aggregator); + } + } + } + + @Override + public Iterator> iterator(boolean sorted) + { + if (sorted) { + throw new UnsupportedOperationException("sorted iterator is not supported yet"); + } + + return new Iterator>() + { + int cur = -1; + boolean findNext = false; + + { + cur = findNext(); + } + + @Override + public boolean hasNext() + { + if (findNext) { + cur = findNext(); + findNext = false; + } + return cur >= 0; + } + + private int findNext() + { + for (int i = cur + 1; i < cardinalityWithMissingValue; i++) { + if (isUsedSlot(i)) { + return i; + } + } + return -1; + } + + @Override + public Entry next() + { + if (cur < 0) { + throw new NoSuchElementException(); + } + + findNext = true; + + final Object[] values = new Object[aggregators.length]; + final int recordOffset = cur * recordSize; + for (int i = 0; i < aggregators.length; i++) { + values[i] = aggregators[i].get(valBuffer, recordOffset + aggregatorOffsets[i]); + } + return new Entry<>(cur - 1, values); + } + }; + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferHashGrouper.java similarity index 97% rename from processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferGrouper.java rename to processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferHashGrouper.java index cb738ed4d0c..4b709e3a0db 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferHashGrouper.java @@ -35,9 +35,9 @@ import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; -public class BufferGrouper extends AbstractBufferGrouper +public class BufferHashGrouper extends AbstractBufferHashGrouper { - private static final Logger log = new Logger(BufferGrouper.class); + private static final Logger log = new Logger(BufferHashGrouper.class); private static final int MIN_INITIAL_BUCKETS = 4; private static final int DEFAULT_INITIAL_BUCKETS = 1024; private static final float DEFAULT_MAX_LOAD_FACTOR = 0.7f; @@ -51,7 +51,7 @@ public class BufferGrouper extends AbstractBufferGrouper private ByteBuffer offsetListBuffer; private ByteBufferIntList offsetList; - public BufferGrouper( + public BufferHashGrouper( final Supplier bufferSupplier, final KeySerde keySerde, final ColumnSelectorFactory columnSelectorFactory, diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferHashTable.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferHashTable.java index cd83b229c35..4723f831dba 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferHashTable.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferHashTable.java @@ -238,7 +238,7 @@ public class ByteBufferHashTable { int offset = bucket * bucketSizeWithHash; tableBuffer.position(offset); - tableBuffer.putInt(keyHash | 0x80000000); + tableBuffer.putInt(Groupers.getUsedFlag(keyHash)); tableBuffer.put(keyBuffer); size++; diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferMinMaxOffsetHeap.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferMinMaxOffsetHeap.java index 852177966ea..d0351c301f8 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferMinMaxOffsetHeap.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferMinMaxOffsetHeap.java @@ -41,7 +41,7 @@ public class ByteBufferMinMaxOffsetHeap private final Comparator maxComparator; private final ByteBuffer buf; private final int limit; - private final LimitedBufferGrouper.BufferGrouperOffsetHeapIndexUpdater heapIndexUpdater; + private final LimitedBufferHashGrouper.BufferGrouperOffsetHeapIndexUpdater heapIndexUpdater; private int heapSize; @@ -49,7 +49,7 @@ public class ByteBufferMinMaxOffsetHeap ByteBuffer buf, int limit, Comparator minComparator, - LimitedBufferGrouper.BufferGrouperOffsetHeapIndexUpdater heapIndexUpdater + LimitedBufferHashGrouper.BufferGrouperOffsetHeapIndexUpdater heapIndexUpdater ) { this.buf = buf; 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 index 71d3ff6f154..8bd1b597c41 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouper.java @@ -187,12 +187,6 @@ public class ConcurrentGrouper implements Grouper } } - @Override - public AggregateResult aggregate(KeyType key) - { - return aggregate(key, Groupers.hash(key)); - } - @Override public void reset() { 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 index 82487d860ec..417e7d47b1e 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java @@ -19,7 +19,7 @@ package io.druid.query.groupby.epinephelinae; -import com.google.common.base.Function; +import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.base.Suppliers; import com.google.common.collect.Maps; @@ -30,9 +30,7 @@ import io.druid.data.input.Row; import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.BaseSequence; -import io.druid.java.util.common.guava.CloseQuietly; import io.druid.java.util.common.guava.Sequence; -import io.druid.java.util.common.guava.Sequences; import io.druid.query.ColumnSelectorPlus; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.dimension.ColumnSelectorStrategyFactory; @@ -54,12 +52,13 @@ import io.druid.segment.DimensionSelector; import io.druid.segment.StorageAdapter; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ValueType; +import io.druid.segment.data.IndexedInts; import io.druid.segment.filter.Filters; import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.io.Closeable; -import java.io.IOException; import java.nio.ByteBuffer; import java.util.Iterator; import java.util.List; @@ -113,6 +112,13 @@ public class GroupByQueryEngineV2 null ); + final boolean allSingleValueDims = query + .getDimensions() + .stream() + .allMatch(dimension -> { + final ColumnCapabilities columnCapabilities = storageAdapter.getColumnCapabilities(dimension.getDimension()); + return columnCapabilities != null && !columnCapabilities.hasMultipleValues(); + }); final ResourceHolder bufferHolder = intermediateResultsBufferPool.take(); @@ -124,56 +130,105 @@ public class GroupByQueryEngineV2 ? null : new DateTime(Long.parseLong(fudgeTimestampString)); - return Sequences.concat( - Sequences.withBaggage( - Sequences.map( - cursors, - new Function>() - { - @Override - public Sequence apply(final Cursor cursor) - { - return new BaseSequence<>( - new BaseSequence.IteratorMaker() - { - @Override - public GroupByEngineIterator make() - { - ColumnSelectorPlus[] selectorPlus = DimensionHandlerUtils.createColumnSelectorPluses( - STRATEGY_FACTORY, - query.getDimensions(), - cursor - ); - return new GroupByEngineIterator( - query, - config, - cursor, - bufferHolder.get(), - fudgeTimestamp, - createGroupBySelectorPlus(selectorPlus) - ); - } - - @Override - public void cleanup(GroupByEngineIterator iterFromMake) - { - iterFromMake.close(); - } - } - ); - } - } - ), - new Closeable() + return cursors.flatMap( + cursor -> new BaseSequence<>( + new BaseSequence.IteratorMaker>() { @Override - public void close() throws IOException + public GroupByEngineIterator make() { - CloseQuietly.close(bufferHolder); + ColumnSelectorPlus[] selectorPlus = DimensionHandlerUtils + .createColumnSelectorPluses( + STRATEGY_FACTORY, + query.getDimensions(), + cursor + ); + GroupByColumnSelectorPlus[] dims = createGroupBySelectorPlus(selectorPlus); + + final ByteBuffer buffer = bufferHolder.get(); + + // Check array-based aggregation is applicable + if (isArrayAggregateApplicable(config, query, dims, storageAdapter, buffer)) { + return new ArrayAggregateIterator( + query, + config, + cursor, + buffer, + fudgeTimestamp, + dims, + allSingleValueDims, + // There must be 0 or 1 dimension if isArrayAggregateApplicable() is true + dims.length == 0 ? 1 : storageAdapter.getDimensionCardinality(dims[0].getName()) + ); + } else { + return new HashAggregateIterator( + query, + config, + cursor, + buffer, + fudgeTimestamp, + dims, + allSingleValueDims + ); + } + } + + @Override + public void cleanup(GroupByEngineIterator iterFromMake) + { + iterFromMake.close(); } } ) - ); + ).withBaggage(bufferHolder); + } + + private static boolean isArrayAggregateApplicable( + GroupByQueryConfig config, + GroupByQuery query, + GroupByColumnSelectorPlus[] dims, + StorageAdapter storageAdapter, + ByteBuffer buffer + ) + { + if (config.isForceHashAggregation()) { + return false; + } + + final ColumnCapabilities columnCapabilities; + final int cardinality; + + // Find cardinality + if (dims.length == 0) { + columnCapabilities = null; + cardinality = 1; + } else if (dims.length == 1) { + columnCapabilities = storageAdapter.getColumnCapabilities(dims[0].getName()); + cardinality = storageAdapter.getDimensionCardinality(dims[0].getName()); + } else { + columnCapabilities = null; + cardinality = -1; // ArrayAggregateIterator is not available + } + + // Choose array-based aggregation if the grouping key is a single string dimension of a + // known cardinality + if ((columnCapabilities == null || columnCapabilities.getType().equals(ValueType.STRING)) + && cardinality > 0) { + final AggregatorFactory[] aggregatorFactories = query + .getAggregatorSpecs() + .toArray(new AggregatorFactory[query.getAggregatorSpecs().size()]); + final int requiredBufferCapacity = BufferArrayGrouper.requiredBufferCapacity( + cardinality, + aggregatorFactories + ); + + // Check that all keys and aggregated values can be contained the buffer + if (requiredBufferCapacity <= buffer.capacity()) { + return true; + } + } + + return false; } private static class GroupByStrategyFactory implements ColumnSelectorStrategyFactory @@ -204,22 +259,18 @@ public class GroupByQueryEngineV2 } } - private static class GroupByEngineIterator implements Iterator, Closeable + private abstract static class GroupByEngineIterator implements Iterator, Closeable { - private final GroupByQuery query; - private final GroupByQueryConfig querySpecificConfig; - private final Cursor cursor; - private final ByteBuffer buffer; - private final Grouper.KeySerde keySerde; - private final DateTime timestamp; - private final ByteBuffer keyBuffer; - private final int[] stack; - private final Object[] valuess; - private final GroupByColumnSelectorPlus[] dims; + protected final GroupByQuery query; + protected final GroupByQueryConfig querySpecificConfig; + protected final Cursor cursor; + protected final ByteBuffer buffer; + protected final Grouper.KeySerde keySerde; + protected final GroupByColumnSelectorPlus[] dims; + protected final DateTime timestamp; - private int stackp = Integer.MIN_VALUE; - private boolean currentRowWasPartiallyAggregated = false; - private CloseableGrouperIterator delegate = null; + protected CloseableGrouperIterator delegate = null; + protected final boolean allSingleValueDims; public GroupByEngineIterator( final GroupByQuery query, @@ -227,43 +278,159 @@ public class GroupByQueryEngineV2 final Cursor cursor, final ByteBuffer buffer, final DateTime fudgeTimestamp, - final GroupByColumnSelectorPlus[] dims + final GroupByColumnSelectorPlus[] dims, + final boolean allSingleValueDims ) { - final int dimCount = query.getDimensions().size(); - this.query = query; this.querySpecificConfig = config.withOverrides(query); this.cursor = cursor; this.buffer = buffer; this.keySerde = new GroupByEngineKeySerde(dims); - this.keyBuffer = ByteBuffer.allocate(keySerde.keySize()); this.dims = dims; - this.stack = new int[dimCount]; - this.valuess = new Object[dimCount]; // Time is the same for every row in the cursor this.timestamp = fudgeTimestamp != null ? fudgeTimestamp : cursor.getTime(); + this.allSingleValueDims = allSingleValueDims; + } + + private CloseableGrouperIterator initNewDelegate() + { + final Grouper grouper = newGrouper(); + grouper.init(); + + if (allSingleValueDims) { + aggregateSingleValueDims(grouper); + } else { + aggregateMultiValueDims(grouper); + } + + return new CloseableGrouperIterator<>( + grouper, + false, + entry -> { + Map theMap = Maps.newLinkedHashMap(); + + // Add dimensions. + putToMap(entry.getKey(), theMap); + + convertRowTypesToOutputTypes(query.getDimensions(), theMap); + + // 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); + }, + grouper + ); } @Override public Row next() { - if (delegate != null && delegate.hasNext()) { - return delegate.next(); - } - - if (cursor.isDone()) { + if (delegate == null || !delegate.hasNext()) { throw new NoSuchElementException(); } - // Make a new delegate iterator + return delegate.next(); + } + + @Override + public boolean hasNext() + { + if (delegate != null && delegate.hasNext()) { + return true; + } else { + if (!cursor.isDone()) { + if (delegate != null) { + delegate.close(); + } + delegate = initNewDelegate(); + return true; + } else { + return false; + } + } + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + + @Override + public void close() + { if (delegate != null) { delegate.close(); - delegate = null; } + } - final Grouper grouper = new BufferGrouper<>( + /** + * Create a new grouper. + */ + protected abstract Grouper newGrouper(); + + /** + * Grouping dimensions are all single-valued, and thus the given grouper don't have to worry about multi-valued + * dimensions. + */ + protected abstract void aggregateSingleValueDims(Grouper grouper); + + /** + * Grouping dimensions can be multi-valued, and thus the given grouper should handle them properly during + * aggregation. + */ + protected abstract void aggregateMultiValueDims(Grouper grouper); + + /** + * Add the key to the result map. Some pre-processing like deserialization might be done for the key before + * adding to the map. + */ + protected abstract void putToMap(KeyType key, Map map); + + protected int getSingleValue(IndexedInts indexedInts) + { + Preconditions.checkArgument(indexedInts.size() < 2, "should be single value"); + return indexedInts.size() == 1 ? indexedInts.get(0) : GroupByColumnSelectorStrategy.GROUP_BY_MISSING_VALUE; + } + + } + + private static class HashAggregateIterator extends GroupByEngineIterator + { + private final int[] stack; + private final Object[] valuess; + private final ByteBuffer keyBuffer; + + private int stackPointer = Integer.MIN_VALUE; + protected boolean currentRowWasPartiallyAggregated = false; + + public HashAggregateIterator( + GroupByQuery query, + GroupByQueryConfig config, + Cursor cursor, + ByteBuffer buffer, + DateTime fudgeTimestamp, + GroupByColumnSelectorPlus[] dims, + boolean allSingleValueDims + ) + { + super(query, config, cursor, buffer, fudgeTimestamp, dims, allSingleValueDims); + + final int dimCount = query.getDimensions().size(); + stack = new int[dimCount]; + valuess = new Object[dimCount]; + keyBuffer = ByteBuffer.allocate(keySerde.keySize()); + } + + @Override + protected Grouper newGrouper() + { + return new BufferHashGrouper<>( Suppliers.ofInstance(buffer), keySerde, cursor, @@ -273,13 +440,36 @@ public class GroupByQueryEngineV2 querySpecificConfig.getBufferGrouperMaxLoadFactor(), querySpecificConfig.getBufferGrouperInitialBuckets() ); - grouper.init(); + } -outer: + @Override + protected void aggregateSingleValueDims(Grouper grouper) + { + while (!cursor.isDone()) { + for (int i = 0; i < dims.length; i++) { + final GroupByColumnSelectorStrategy strategy = dims[i].getColumnSelectorStrategy(); + strategy.writeToKeyBuffer( + dims[i].getKeyBufferPosition(), + strategy.getOnlyValue(dims[i].getSelector()), + keyBuffer + ); + } + keyBuffer.rewind(); + + if (!grouper.aggregate(keyBuffer).isOk()) { + return; + } + cursor.advance(); + } + } + + @Override + protected void aggregateMultiValueDims(Grouper grouper) + { while (!cursor.isDone()) { if (!currentRowWasPartiallyAggregated) { // Set up stack, valuess, and first grouping in keyBuffer for this row - stackp = stack.length - 1; + stackPointer = stack.length - 1; for (int i = 0; i < dims.length; i++) { GroupByColumnSelectorStrategy strategy = dims[i].getColumnSelectorStrategy(); @@ -300,29 +490,29 @@ outer: // Aggregate groupings for this row boolean doAggregate = true; - while (stackp >= -1) { + while (stackPointer >= -1) { // Aggregate additional grouping for this row if (doAggregate) { keyBuffer.rewind(); if (!grouper.aggregate(keyBuffer).isOk()) { // Buffer full while aggregating; break out and resume later currentRowWasPartiallyAggregated = true; - break outer; + return; } doAggregate = false; } - if (stackp >= 0) { - doAggregate = dims[stackp].getColumnSelectorStrategy().checkRowIndexAndAddValueToGroupingKey( - dims[stackp].getKeyBufferPosition(), - valuess[stackp], - stack[stackp], + if (stackPointer >= 0) { + doAggregate = dims[stackPointer].getColumnSelectorStrategy().checkRowIndexAndAddValueToGroupingKey( + dims[stackPointer].getKeyBufferPosition(), + valuess[stackPointer], + stack[stackPointer], keyBuffer ); if (doAggregate) { - stack[stackp]++; - for (int i = stackp + 1; i < stack.length; i++) { + stack[stackPointer]++; + for (int i = stackPointer + 1; i < stack.length; i++) { dims[i].getColumnSelectorStrategy().initGroupingKeyColumnValue( dims[i].getKeyBufferPosition(), i, @@ -331,12 +521,12 @@ outer: stack ); } - stackp = stack.length - 1; + stackPointer = stack.length - 1; } else { - stackp--; + stackPointer--; } } else { - stackp--; + stackPointer--; } } @@ -344,66 +534,131 @@ outer: cursor.advance(); currentRowWasPartiallyAggregated = false; } + } - delegate = new CloseableGrouperIterator<>( - grouper, - false, - new Function, Row>() - { - @Override - public Row apply(final Grouper.Entry entry) - { - Map theMap = Maps.newLinkedHashMap(); + @Override + protected void putToMap(ByteBuffer key, Map map) + { + for (GroupByColumnSelectorPlus selectorPlus : dims) { + selectorPlus.getColumnSelectorStrategy().processValueFromGroupingKey( + selectorPlus, + key, + map + ); + } + } + } - // Add dimensions. - for (GroupByColumnSelectorPlus selectorPlus : dims) { - selectorPlus.getColumnSelectorStrategy().processValueFromGroupingKey( - selectorPlus, - entry.getKey(), - theMap - ); - } + private static class ArrayAggregateIterator extends GroupByEngineIterator + { + private final int cardinality; - convertRowTypesToOutputTypes(query.getDimensions(), theMap); + @Nullable + private final GroupByColumnSelectorPlus dim; - // Add aggregations. - for (int i = 0; i < entry.getValues().length; i++) { - theMap.put(query.getAggregatorSpecs().get(i).getName(), entry.getValues()[i]); - } + private IndexedInts multiValues; + private int nextValIndex; - return new MapBasedRow(timestamp, theMap); - } - }, - new Closeable() - { - @Override - public void close() throws IOException - { - grouper.close(); + public ArrayAggregateIterator( + GroupByQuery query, + GroupByQueryConfig config, + Cursor cursor, + ByteBuffer buffer, + DateTime fudgeTimestamp, + GroupByColumnSelectorPlus[] dims, + boolean allSingleValueDims, + int cardinality + ) + { + super(query, config, cursor, buffer, fudgeTimestamp, dims, allSingleValueDims); + this.cardinality = cardinality; + if (dims.length == 1) { + this.dim = dims[0]; + } else if (dims.length == 0) { + this.dim = null; + } else { + throw new IAE("Group key should be a single dimension"); + } + } + + @Override + protected Grouper newGrouper() + { + return new BufferArrayGrouper( + Suppliers.ofInstance(buffer), + cursor, + query.getAggregatorSpecs() + .toArray(new AggregatorFactory[query.getAggregatorSpecs().size()]), + cardinality + ); + } + + @Override + protected void aggregateSingleValueDims(Grouper grouper) + { + while (!cursor.isDone()) { + final int key; + if (dim != null) { + // dim is always an indexed string dimension + final IndexedInts indexedInts = ((DimensionSelector) dim.getSelector()).getRow(); + key = getSingleValue(indexedInts); + } else { + key = 0; + } + if (!grouper.aggregate(key).isOk()) { + return; + } + cursor.advance(); + } + } + + @Override + protected void aggregateMultiValueDims(Grouper grouper) + { + if (dim == null) { + throw new ISE("dim must exist"); + } + + if (multiValues == null) { + // dim is always an indexed string dimension + multiValues = ((DimensionSelector) dim.getSelector()).getRow(); + nextValIndex = 0; + } + + while (!cursor.isDone()) { + if (multiValues.size() == 0) { + if (!grouper.aggregate(GroupByColumnSelectorStrategy.GROUP_BY_MISSING_VALUE).isOk()) { + return; + } + } else { + for (; nextValIndex < multiValues.size(); nextValIndex++) { + if (!grouper.aggregate(multiValues.get(nextValIndex)).isOk()) { + return; } } - ); + } - return delegate.next(); + cursor.advance(); + if (!cursor.isDone()) { + // dim is always an indexed string dimension + multiValues = ((DimensionSelector) dim.getSelector()).getRow(); + nextValIndex = multiValues.size() == 0 ? -1 : 0; + } + } } @Override - public boolean hasNext() + protected void putToMap(Integer key, Map map) { - return (delegate != null && delegate.hasNext()) || !cursor.isDone(); - } - - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } - - @Override - public void close() - { - if (delegate != null) { - delegate.close(); + if (dim != null) { + if (key != -1) { + map.put( + dim.getOutputName(), + ((DimensionSelector) dim.getSelector()).lookupName(key) + ); + } else { + map.put(dim.getOutputName(), ""); + } } } } 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 index 2f6f795fd09..2ca544485eb 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/Grouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/Grouper.java @@ -21,6 +21,7 @@ package io.druid.query.groupby.epinephelinae; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; import io.druid.query.aggregation.AggregatorFactory; import java.io.Closeable; @@ -28,6 +29,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Comparator; import java.util.Iterator; +import java.util.function.ToIntFunction; /** * Groupers aggregate metrics from rows that they typically get from a ColumnSelectorFactory, under @@ -59,7 +61,7 @@ public interface Grouper extends Closeable * some are not. * * @param key key object - * @param keyHash result of {@link Groupers#hash(Object)} on the key + * @param keyHash result of {@link #hashFunction()} on the key * * @return result that is ok if the row was aggregated, not ok if a resource limit was hit */ @@ -73,13 +75,22 @@ public interface Grouper extends Closeable * * @return result that is ok if the row was aggregated, not ok if a resource limit was hit */ - AggregateResult aggregate(KeyType key); + default AggregateResult aggregate(KeyType key) + { + Preconditions.checkNotNull(key, "key"); + return aggregate(key, hashFunction().applyAsInt(key)); + } /** * Reset the grouper to its initial state. */ void reset(); + default ToIntFunction hashFunction() + { + return Groupers::hash; + } + /** * Close the grouper and release associated resources. */ @@ -87,7 +98,7 @@ public interface Grouper extends Closeable void close(); /** - * Iterate through entries. If a comparator is provided, do a sorted iteration. + * Iterate through entries. *

* 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 @@ -96,6 +107,9 @@ public interface Grouper extends Closeable * 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. + *

+ * Callers must process and discard the returned {@link Entry}s immediately because some implementations can reuse the + * key objects. * * @param sorted return sorted results * 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 index eb47abc70cd..d03be91abe3 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/Groupers.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/Groupers.java @@ -31,6 +31,17 @@ public class Groupers // No instantiation } + static final AggregateResult DICTIONARY_FULL = AggregateResult.failure( + "Not enough dictionary space to execute this query. Try increasing " + + "druid.query.groupBy.maxMergingDictionarySize or enable disk spilling by setting " + + "druid.query.groupBy.maxOnDiskStorage to a positive number." + ); + static final AggregateResult HASH_TABLE_FULL = AggregateResult.failure( + "Not enough aggregation buffer space to execute this query. Try increasing " + + "druid.processing.buffer.sizeBytes or enable disk spilling by setting " + + "druid.query.groupBy.maxOnDiskStorage to a positive number." + ); + private static final int C1 = 0xcc9e2d51; private static final int C2 = 0x1b873593; @@ -56,6 +67,11 @@ public class Groupers } + static int getUsedFlag(int keyHash) + { + return keyHash | 0x80000000; + } + public static Iterator> mergeIterators( final Iterable>> iterators, final Comparator> keyTypeComparator diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/LimitedBufferGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/LimitedBufferHashGrouper.java similarity index 99% rename from processing/src/main/java/io/druid/query/groupby/epinephelinae/LimitedBufferGrouper.java rename to processing/src/main/java/io/druid/query/groupby/epinephelinae/LimitedBufferHashGrouper.java index a5513c8503c..30aabc63de9 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/LimitedBufferGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/LimitedBufferHashGrouper.java @@ -35,7 +35,7 @@ import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; -public class LimitedBufferGrouper extends AbstractBufferGrouper +public class LimitedBufferHashGrouper extends AbstractBufferHashGrouper { private static final int MIN_INITIAL_BUCKETS = 4; private static final int DEFAULT_INITIAL_BUCKETS = 1024; @@ -51,7 +51,7 @@ public class LimitedBufferGrouper extends AbstractBufferGrouper extends AbstractBufferGrouper bufferSupplier, final Grouper.KeySerde keySerde, final ColumnSelectorFactory columnSelectorFactory, 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 index 41d8bdba178..60e618def1a 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/SpillingGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/SpillingGrouper.java @@ -46,7 +46,7 @@ import java.util.Iterator; import java.util.List; /** - * Grouper based around a single underlying {@link BufferGrouper}. Not thread-safe. + * Grouper based around a single underlying {@link BufferHashGrouper}. Not thread-safe. * * When the underlying grouper is full, its contents are sorted and written to temporary files using "spillMapper". */ @@ -88,7 +88,7 @@ public class SpillingGrouper implements Grouper this.keyObjComparator = keySerdeFactory.objectComparator(false); this.defaultOrderKeyObjComparator = keySerdeFactory.objectComparator(true); if (limitSpec != null) { - this.grouper = new LimitedBufferGrouper<>( + this.grouper = new LimitedBufferHashGrouper<>( bufferSupplier, keySerde, columnSelectorFactory, @@ -100,7 +100,7 @@ public class SpillingGrouper implements Grouper sortHasNonGroupingFields ); } else { - this.grouper = new BufferGrouper<>( + this.grouper = new BufferHashGrouper<>( bufferSupplier, keySerde, columnSelectorFactory, @@ -153,12 +153,6 @@ public class SpillingGrouper implements Grouper } } - @Override - public AggregateResult aggregate(KeyType key) - { - return aggregate(key, Groupers.hash(key)); - } - @Override public void reset() { diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/DictionaryBuildingStringGroupByColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/DictionaryBuildingStringGroupByColumnSelectorStrategy.java index 528f432baec..503f80b6bd2 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/DictionaryBuildingStringGroupByColumnSelectorStrategy.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/DictionaryBuildingStringGroupByColumnSelectorStrategy.java @@ -19,6 +19,7 @@ package io.druid.query.groupby.epinephelinae.column; +import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import io.druid.segment.ColumnValueSelector; import io.druid.segment.DimensionSelector; @@ -83,4 +84,27 @@ public class DictionaryBuildingStringGroupByColumnSelectorStrategy extends Strin } valuess[columnIndex] = ArrayBasedIndexedInts.of(newIds); } + + @Override + public Object getOnlyValue(ColumnValueSelector selector) + { + final DimensionSelector dimSelector = (DimensionSelector) selector; + final IndexedInts row = dimSelector.getRow(); + + Preconditions.checkState(row.size() < 2, "Not supported for multi-value dimensions"); + + if (row.size() == 0) { + return GROUP_BY_MISSING_VALUE; + } + + final String value = dimSelector.lookupName(row.get(0)); + final int dictId = reverseDictionary.getInt(value); + if (dictId < 0) { + dictionary.add(value); + reverseDictionary.put(value, nextId); + return nextId++; + } else { + return dictId; + } + } } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/DoubleGroupByColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/DoubleGroupByColumnSelectorStrategy.java index 3979f53dd2d..847fb092e4c 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/DoubleGroupByColumnSelectorStrategy.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/DoubleGroupByColumnSelectorStrategy.java @@ -67,4 +67,16 @@ public class DoubleGroupByColumnSelectorStrategy implements GroupByColumnSelecto // this method handles row values after the first in a multivalued row, so just return false return false; } + + @Override + public Object getOnlyValue(ColumnValueSelector selector) + { + return ((DoubleColumnSelector) selector).get(); + } + + @Override + public void writeToKeyBuffer(int keyBufferPosition, Object obj, ByteBuffer keyBuffer) + { + keyBuffer.putDouble(keyBufferPosition, (Double) obj); + } } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/FloatGroupByColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/FloatGroupByColumnSelectorStrategy.java index 3e63be4d59d..f29338a9b18 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/FloatGroupByColumnSelectorStrategy.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/FloatGroupByColumnSelectorStrategy.java @@ -50,6 +50,18 @@ public class FloatGroupByColumnSelectorStrategy implements GroupByColumnSelector valuess[columnIndex] = ((FloatColumnSelector) selector).get(); } + @Override + public Object getOnlyValue(ColumnValueSelector selector) + { + return ((FloatColumnSelector) selector).get(); + } + + @Override + public void writeToKeyBuffer(int keyBufferPosition, Object obj, ByteBuffer keyBuffer) + { + keyBuffer.putFloat(keyBufferPosition, (Float) obj); + } + @Override public void initGroupingKeyColumnValue( int keyBufferPosition, int columnIndex, Object rowObj, ByteBuffer keyBuffer, int[] stack diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorStrategy.java index 7a11258af7e..ceda4275342 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorStrategy.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorStrategy.java @@ -33,6 +33,8 @@ import java.util.Map; */ public interface GroupByColumnSelectorStrategy extends ColumnSelectorStrategy { + int GROUP_BY_MISSING_VALUE = -1; + /** * Return the size, in bytes, of this dimension's values in the grouping key. * @@ -65,7 +67,7 @@ public interface GroupByColumnSelectorStrategy extends ColumnSelectorStrategy ); /** - * Retrieve a row object from the ColumnSelectorPlus and put it in valuess at columnIndex. + * Retrieve a row object from the {@link ColumnValueSelector} and put it in valuess at columnIndex. * * @param selector Value selector for a column. * @param columnIndex Index of the column within the row values array @@ -101,4 +103,22 @@ public interface GroupByColumnSelectorStrategy extends ColumnSelectorStrategy * @return true if rowValIdx < size of rowObj, false otherwise */ boolean checkRowIndexAndAddValueToGroupingKey(int keyBufferPosition, Object rowObj, int rowValIdx, ByteBuffer keyBuffer); + + /** + * Retrieve a single object using the {@link ColumnValueSelector}. The reading column must have a single value. + * + * @param selector Value selector for a column + * + * @return an object retrieved from the column + */ + Object getOnlyValue(ColumnValueSelector selector); + + /** + * Write a given object to the keyBuffer at keyBufferPosition. + * + * @param keyBufferPosition starting offset for this column's value within the grouping key + * @param obj row value object retrieved from {@link #getOnlyValue(ColumnValueSelector)} + * @param keyBuffer grouping key + */ + void writeToKeyBuffer(int keyBufferPosition, Object obj, ByteBuffer keyBuffer); } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/LongGroupByColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/LongGroupByColumnSelectorStrategy.java index 2e5fed32194..178828076c6 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/LongGroupByColumnSelectorStrategy.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/LongGroupByColumnSelectorStrategy.java @@ -50,6 +50,18 @@ public class LongGroupByColumnSelectorStrategy implements GroupByColumnSelectorS valuess[columnIndex] = ((LongColumnSelector) selector).get(); } + @Override + public Object getOnlyValue(ColumnValueSelector selector) + { + return ((LongColumnSelector) selector).get(); + } + + @Override + public void writeToKeyBuffer(int keyBufferPosition, Object obj, ByteBuffer keyBuffer) + { + keyBuffer.putLong(keyBufferPosition, (Long) obj); + } + @Override public void initGroupingKeyColumnValue( int keyBufferPosition, int columnIndex, Object rowObj, ByteBuffer keyBuffer, int[] stack diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/StringGroupByColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/StringGroupByColumnSelectorStrategy.java index c531437b2b1..9cabaa5aa1c 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/StringGroupByColumnSelectorStrategy.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/StringGroupByColumnSelectorStrategy.java @@ -19,6 +19,7 @@ package io.druid.query.groupby.epinephelinae.column; +import com.google.common.base.Preconditions; import com.google.common.primitives.Ints; import io.druid.segment.ColumnValueSelector; import io.druid.segment.DimensionSelector; @@ -29,8 +30,6 @@ import java.util.Map; public class StringGroupByColumnSelectorStrategy implements GroupByColumnSelectorStrategy { - private static final int GROUP_BY_MISSING_VALUE = -1; - @Override public int getGroupingKeySize() { @@ -61,6 +60,21 @@ public class StringGroupByColumnSelectorStrategy implements GroupByColumnSelecto valuess[columnIndex] = row; } + @Override + public Object getOnlyValue(ColumnValueSelector selector) + { + final DimensionSelector dimSelector = (DimensionSelector) selector; + final IndexedInts row = dimSelector.getRow(); + Preconditions.checkState(row.size() < 2, "Not supported for multi-value dimensions"); + return row.size() == 1 ? row.get(0) : GROUP_BY_MISSING_VALUE; + } + + @Override + public void writeToKeyBuffer(int keyBufferPosition, Object obj, ByteBuffer keyBuffer) + { + keyBuffer.putInt(keyBufferPosition, (int) obj); + } + @Override public void initGroupingKeyColumnValue(int keyBufferPosition, int columnIndex, Object rowObj, ByteBuffer keyBuffer, int[] stack) { 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 index add5387e75b..6381fdebd32 100644 --- a/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java +++ b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java @@ -287,7 +287,7 @@ public class GroupByStrategyV2 implements GroupByStrategy } ); - // Don't apply limit here for inner results, that will be pushed down to the BufferGrouper + // Don't apply limit here for inner results, that will be pushed down to the BufferHashGrouper if (query.getContextBoolean(CTX_KEY_OUTERMOST, true)) { return query.postProcess(rowSequence); } else { 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 c9fdbe94e27..0a1d327a5a0 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -1401,7 +1401,7 @@ public class GroupByQueryRunnerTest List expectedResults = null; if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { expectedException.expect(ResourceLimitExceededException.class); - expectedException.expectMessage("Not enough aggregation table space to execute this query"); + expectedException.expectMessage("Not enough aggregation buffer space to execute this query"); } else { expectedResults = Arrays.asList( GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), @@ -1579,7 +1579,7 @@ public class GroupByQueryRunnerTest GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); } else { expectedException.expect(ResourceLimitExceededException.class); - expectedException.expectMessage("Not enough aggregation table space to execute this query"); + expectedException.expectMessage("Not enough aggregation buffer space to execute this query"); GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); } } diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferArrayGrouperTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferArrayGrouperTest.java new file mode 100644 index 00000000000..83a16a7a45f --- /dev/null +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferArrayGrouperTest.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 com.google.common.base.Suppliers; +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 io.druid.query.groupby.epinephelinae.Grouper.Entry; +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.util.Comparator; +import java.util.List; + +public class BufferArrayGrouperTest +{ + @Test + public void testAggregate() + { + final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory(); + final Grouper grouper = newGrouper(columnSelectorFactory, 1024); + + 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(6); + + final List> expected = ImmutableList.of( + new Grouper.Entry<>(6, new Object[]{30L, 3L}), + new Grouper.Entry<>(10, new Object[]{10L, 1L}), + new Grouper.Entry<>(12, new Object[]{20L, 2L}) + ); + final List> unsortedEntries = Lists.newArrayList(grouper.iterator(false)); + + Assert.assertEquals( + expected, + Ordering.from((Comparator>) (o1, o2) -> Ints.compare(o1.getKey(), o2.getKey())) + .sortedCopy(unsortedEntries) + ); + } + + private BufferArrayGrouper newGrouper( + TestColumnSelectorFactory columnSelectorFactory, + int bufferSize + ) + { + final ByteBuffer buffer = ByteBuffer.allocate(bufferSize); + + final BufferArrayGrouper grouper = new BufferArrayGrouper( + Suppliers.ofInstance(buffer), + columnSelectorFactory, + new AggregatorFactory[]{ + new LongSumAggregatorFactory("valueSum", "value"), + new CountAggregatorFactory("count") + }, + 1000 + ); + grouper.init(); + return grouper; + } +} diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferGrouperTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferHashGrouperTest.java similarity index 97% rename from processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferGrouperTest.java rename to processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferHashGrouperTest.java index e7ca10b7171..e28ed1aebc3 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferGrouperTest.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferHashGrouperTest.java @@ -43,7 +43,7 @@ import java.nio.channels.FileChannel; import java.util.Comparator; import java.util.List; -public class BufferGrouperTest +public class BufferHashGrouperTest { @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -52,7 +52,7 @@ public class BufferGrouperTest public void testSimple() { final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory(); - final Grouper grouper = new BufferGrouper<>( + final Grouper grouper = new BufferHashGrouper<>( Suppliers.ofInstance(ByteBuffer.allocate(1000)), GrouperTestUtil.intKeySerde(), columnSelectorFactory, @@ -182,7 +182,7 @@ public class BufferGrouperTest Assert.assertEquals(expected, Lists.newArrayList(grouper.iterator(true))); } - private BufferGrouper makeGrouper( + private BufferHashGrouper makeGrouper( TestColumnSelectorFactory columnSelectorFactory, int bufferSize, int initialBuckets @@ -197,7 +197,7 @@ public class BufferGrouperTest throw Throwables.propagate(e); } - final BufferGrouper grouper = new BufferGrouper<>( + final BufferHashGrouper grouper = new BufferHashGrouper<>( Suppliers.ofInstance(buffer), GrouperTestUtil.intKeySerde(), columnSelectorFactory, diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/LimitedBufferGrouperTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/LimitedBufferHashGrouperTest.java similarity index 93% rename from processing/src/test/java/io/druid/query/groupby/epinephelinae/LimitedBufferGrouperTest.java rename to processing/src/test/java/io/druid/query/groupby/epinephelinae/LimitedBufferHashGrouperTest.java index 7cd746379ee..cdab2215801 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/LimitedBufferGrouperTest.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/LimitedBufferHashGrouperTest.java @@ -35,7 +35,7 @@ import org.junit.rules.ExpectedException; import java.nio.ByteBuffer; import java.util.List; -public class LimitedBufferGrouperTest +public class LimitedBufferHashGrouperTest { @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -46,7 +46,7 @@ public class LimitedBufferGrouperTest final int limit = 100; final int keyBase = 100000; final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory(); - final LimitedBufferGrouper grouper = makeGrouper(columnSelectorFactory, 20000, 2, limit); + final LimitedBufferHashGrouper grouper = makeGrouper(columnSelectorFactory, 20000, 2, limit); final int numRows = 1000; columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 10L))); @@ -100,7 +100,7 @@ public class LimitedBufferGrouperTest { expectedException.expect(IAE.class); final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory(); - final LimitedBufferGrouper grouper = makeGrouper(columnSelectorFactory, 10, 2, 100); + final LimitedBufferHashGrouper grouper = makeGrouper(columnSelectorFactory, 10, 2, 100); } @Test @@ -109,7 +109,7 @@ public class LimitedBufferGrouperTest final int limit = 100; final int keyBase = 100000; final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory(); - final LimitedBufferGrouper grouper = makeGrouper(columnSelectorFactory, 11716, 2, limit); + final LimitedBufferHashGrouper grouper = makeGrouper(columnSelectorFactory, 11716, 2, limit); final int numRows = 1000; columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 10L))); @@ -146,14 +146,14 @@ public class LimitedBufferGrouperTest Assert.assertEquals(expected, Lists.newArrayList(grouper.iterator(true))); } - private static LimitedBufferGrouper makeGrouper( + private static LimitedBufferHashGrouper makeGrouper( TestColumnSelectorFactory columnSelectorFactory, int bufferSize, int initialBuckets, int limit ) { - LimitedBufferGrouper grouper = new LimitedBufferGrouper<>( + LimitedBufferHashGrouper grouper = new LimitedBufferHashGrouper<>( Suppliers.ofInstance(ByteBuffer.allocate(bufferSize)), GrouperTestUtil.intKeySerde(), columnSelectorFactory,