From 0e68c08d9d0d275132aa31d2d0ace0a872d1594f Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 19 Aug 2017 11:46:53 +0900 Subject: [PATCH 01/44] Add steaming grouper --- docs/content/querying/groupbyquery.md | 1 + .../query/groupby/GroupByQueryConfig.java | 14 + .../epinephelinae/ConcurrentGrouper.java | 423 +++++++++++++++++- .../GroupByMergingQueryRunnerV2.java | 23 + .../query/groupby/epinephelinae/Grouper.java | 8 +- .../epinephelinae/RowBasedGrouperHelper.java | 23 +- .../epinephelinae/SpillingGrouper.java | 2 +- .../StreamingMergeSortedGrouper.java | 378 ++++++++++++++++ .../groupby/strategy/GroupByStrategyV2.java | 1 + .../query/groupby/GroupByQueryRunnerTest.java | 6 + .../epinephelinae/ConcurrentGrouperTest.java | 84 +++- .../StreamingMergeSortedGrouperTest.java | 185 ++++++++ 12 files changed, 1108 insertions(+), 40 deletions(-) create mode 100644 processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java create mode 100644 processing/src/test/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouperTest.java diff --git a/docs/content/querying/groupbyquery.md b/docs/content/querying/groupbyquery.md index fc323217ec06..babc9a7f0c7a 100644 --- a/docs/content/querying/groupbyquery.md +++ b/docs/content/querying/groupbyquery.md @@ -250,6 +250,7 @@ When using the "v2" strategy, the following query context parameters apply: |`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.| +|`forceSingleThreadedCombine`|Force to use single-threaded combine.| When using the "v1" strategy, the following query context parameters apply: 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 85d8b5216ca3..b3d84c794b13 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java @@ -38,6 +38,7 @@ public class GroupByQueryConfig 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"; + private static final String CTX_KEY_FORCE_SINGLE_THREADED_COMBINE = "forceSingleThreadedCombine"; @JsonProperty private String defaultStrategy = GroupByStrategySelector.STRATEGY_V2; @@ -75,6 +76,9 @@ public class GroupByQueryConfig @JsonProperty private boolean forceHashAggregation = false; + @JsonProperty + private boolean forceSingleThreadedCombine = false; + public String getDefaultStrategy() { return defaultStrategy; @@ -144,6 +148,11 @@ public boolean isForceHashAggregation() { return forceHashAggregation; } + + public boolean isForceSingleThreadedCombine() + { + return forceSingleThreadedCombine; + } public GroupByQueryConfig withOverrides(final GroupByQuery query) { @@ -180,6 +189,10 @@ public GroupByQueryConfig withOverrides(final GroupByQuery query) ); newConfig.forcePushDownLimit = query.getContextBoolean(CTX_KEY_FORCE_LIMIT_PUSH_DOWN, isForcePushDownLimit()); newConfig.forceHashAggregation = query.getContextBoolean(CTX_KEY_FORCE_HASH_AGGREGATION, isForceHashAggregation()); + newConfig.forceSingleThreadedCombine = query.getContextBoolean( + CTX_KEY_FORCE_SINGLE_THREADED_COMBINE, + isForceSingleThreadedCombine() + ); return newConfig; } @@ -198,6 +211,7 @@ public String toString() ", maxOnDiskStorage=" + maxOnDiskStorage + ", forcePushDownLimit=" + forcePushDownLimit + ", forceHashAggregation=" + forceHashAggregation + + ", forceSingleThreadedCombine=" + forceSingleThreadedCombine + '}'; } } 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 3f86b120c0da..6f0e8d57479e 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 @@ -26,13 +26,28 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; +import io.druid.collections.ResourceHolder; import io.druid.java.util.common.ISE; +import io.druid.java.util.common.Pair; +import io.druid.java.util.common.logger.Logger; import io.druid.query.AbstractPrioritizedCallable; import io.druid.query.QueryInterruptedException; +import io.druid.query.ResourceLimitExceededException; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.dimension.DimensionSpec; import io.druid.query.groupby.orderby.DefaultLimitSpec; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnSelectorFactory; +import io.druid.segment.DimensionSelector; +import io.druid.segment.DoubleColumnSelector; +import io.druid.segment.FloatColumnSelector; +import io.druid.segment.LongColumnSelector; +import io.druid.segment.ObjectColumnSelector; +import io.druid.segment.column.ColumnCapabilities; +import it.unimi.dsi.fastutil.objects.Object2IntArrayMap; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Comparator; @@ -40,6 +55,7 @@ import java.util.List; import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; @@ -56,6 +72,9 @@ */ public class ConcurrentGrouper implements Grouper { + private static final Logger LOG = new Logger(ConcurrentGrouper.class); + private static final int MINIMUM_COMBINE_DEGREE = 2; + private final List> groupers; private final ThreadLocal> threadLocalGrouper; private final AtomicInteger threadNumber = new AtomicInteger(); @@ -63,6 +82,7 @@ public class ConcurrentGrouper implements Grouper private volatile boolean closed = false; private final Supplier bufferSupplier; + private final Supplier> combineBufferSupplier; private final ColumnSelectorFactory columnSelectorFactory; private final AggregatorFactory[] aggregatorFactories; private final int bufferGrouperMaxSize; @@ -79,11 +99,13 @@ public class ConcurrentGrouper implements Grouper private final int priority; private final boolean hasQueryTimeout; private final long queryTimeoutAt; + private final boolean forceSingleThreadedCombine; private volatile boolean initialized = false; public ConcurrentGrouper( final Supplier bufferSupplier, + final Supplier> combineBufferSupplier, final KeySerdeFactory keySerdeFactory, final ColumnSelectorFactory columnSelectorFactory, final AggregatorFactory[] aggregatorFactories, @@ -98,7 +120,8 @@ public ConcurrentGrouper( final ListeningExecutorService grouperSorter, final int priority, final boolean hasQueryTimeout, - final long queryTimeoutAt + final long queryTimeoutAt, + final boolean forceSingleThreadedCombine ) { Preconditions.checkArgument(concurrencyHint > 0, "concurrencyHint > 0"); @@ -114,6 +137,7 @@ protected SpillingGrouper initialValue() }; this.bufferSupplier = bufferSupplier; + this.combineBufferSupplier = combineBufferSupplier; this.columnSelectorFactory = columnSelectorFactory; this.aggregatorFactories = aggregatorFactories; this.bufferGrouperMaxSize = bufferGrouperMaxSize; @@ -130,6 +154,7 @@ protected SpillingGrouper initialValue() this.priority = priority; this.hasQueryTimeout = hasQueryTimeout; this.queryTimeoutAt = queryTimeoutAt; + this.forceSingleThreadedCombine = forceSingleThreadedCombine; } @Override @@ -142,11 +167,9 @@ public void init() 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); + final ByteBuffer slice = getSlice(buffer, sliceSize, i); final SpillingGrouper grouper = new SpillingGrouper<>( - Suppliers.ofInstance(slice.slice()), + Suppliers.ofInstance(slice), keySerdeFactory, columnSelectorFactory, aggregatorFactories, @@ -221,9 +244,7 @@ public void reset() } for (Grouper grouper : groupers) { - synchronized (grouper) { - grouper.reset(); - } + grouper.reset(); } } @@ -238,13 +259,21 @@ public Iterator> iterator(final boolean sorted) throw new ISE("Grouper is closed"); } - return Groupers.mergeIterators( - sorted && isParallelSortAvailable() ? parallelSortAndGetGroupersIterator() : getGroupersIterator(sorted), - sorted ? keyObjComparator : null - ); + final List>> sortedIterators = sorted && isParallelizable() ? + parallelSortAndGetGroupersIterator() : + getGroupersIterator(sorted); + + if (!forceSingleThreadedCombine && sorted && spilling && isParallelizable()) { + // Parallel combine is used only when data are spilled. This is because ConcurrentGrouper uses two different modes + // depending on data are spilled or not. If data is not spilled, all inputs are completely aggregated and no more + // aggregation is required. + return parallelCombine(sortedIterators); + } else { + return Groupers.mergeIterators(sortedIterators, sorted ? keyObjComparator : null); + } } - private boolean isParallelSortAvailable() + private boolean isParallelizable() { return concurrencyHint > 1; } @@ -292,14 +321,253 @@ private List>> getGroupersIterator(boolean sorted) .collect(Collectors.toList()); } + /** + * Build a combining tree for the input iterators which combine input entries asynchronously. This method is called + * when data are spilled and thus streaming combine is preferred to avoid too many disk accesses. + * + * @param sortedIterators sorted iterators + * + * @return an iterator of the root grouper of the combining tree + */ + private Iterator> parallelCombine(List>> sortedIterators) + { + // CombineBuffer is initialized when this method is called + final ResourceHolder combineBufferHolder = combineBufferSupplier.get(); + final ByteBuffer combineBuffer = combineBufferHolder.get(); + final AggregatorFactory[] combiningFactories = new AggregatorFactory[aggregatorFactories.length]; + for (int i = 0; i < aggregatorFactories.length; i++) { + combiningFactories[i] = aggregatorFactories[i].getCombiningFactory(); + } + final int minimumRequiredBufferCapacity = StreamingMergeSortedGrouper.requiredBufferCapacity( + keySerdeFactory.factorize(), + combiningFactories + ); + final Pair degreeAndBufferNum = findCombineDegreeAndNumBuffers( + combineBuffer, + minimumRequiredBufferCapacity, + concurrencyHint, + sortedIterators.size() + ); + + if (degreeAndBufferNum == null) { + throw new ISE("Cannot find a proper combine degree"); + } + + final int combineDegree = degreeAndBufferNum.lhs; + final int bufferNum = degreeAndBufferNum.rhs; + final int sliceSize = combineBuffer.capacity() / bufferNum; + + final List slices = new ArrayList<>(bufferNum); + for (int i = 0; i < bufferNum; i++) { + slices.add(getSlice(combineBuffer, sliceSize, i)); + } + + final Supplier bufferSupplier = new Supplier() + { + private int i = 0; + + @Override + public ByteBuffer get() + { + if (i < slices.size()) { + return slices.get(i++); + } else { + throw new ISE("Requested number of buffer slices exceeds the planned one"); + } + } + }; + + final List combineFutures = new ArrayList<>(bufferNum); + final Iterator> combinedIterator = buildCombineTree( + sortedIterators, + bufferSupplier, + combiningFactories, + combineDegree, + combineFutures + ); + + return new Iterator>() + { + private boolean closed; + + @Override + public boolean hasNext() + { + if (!combinedIterator.hasNext()) { + if (!closed) { + combineBufferHolder.close(); + closed = true; + + for (Future future : combineFutures) { + try { + // futures should be done before reaching here and throw exceptions if they failed + future.get(); + } + catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + } + } + return false; + } + return true; + } + + @Override + public Entry next() + { + return combinedIterator.next(); + } + }; + } + + /** + * Find a minimum size of the buffer slice and corresponding combining degree and number of slices. Note that each + * node in the combining tree is executed by different threads. This method assumes that higher degree of parallelism + * can exploit better performance and find such a shape of the combining tree. + * + * @param combineBuffer entire buffer used for combining tree + * @param requiredMinimumBufferCapacity minimum buffer capacity for {@link StreamingMergeSortedGrouper} + * @param concurrencyHint available degree of parallelism + * @param maxDegree maximum degree + * + * @return a pair of degree and number of buffers if found. Otherwise null. + */ + @Nullable + private static Pair findCombineDegreeAndNumBuffers( + ByteBuffer combineBuffer, + int requiredMinimumBufferCapacity, + int concurrencyHint, + int maxDegree + ) + { + for (int degree = MINIMUM_COMBINE_DEGREE; degree <= maxDegree; degree++) { + // the number of available combine nodes is concurrencyHint because it's the max concurrency and we don't want to + // parallelize behind that. + final int requiredBufferNum = computeRequiredBufferNum(concurrencyHint, degree); + final int expectedSliceSize = combineBuffer.capacity() / requiredBufferNum; + if (expectedSliceSize > requiredMinimumBufferCapacity) { + return Pair.of(degree, requiredBufferNum); + } + } + return null; + } + + /** + * Recursively compute number of required buffers in a top-down manner. + * + * @param totalNumNodes total number of descendent nodes + * @param degree degree + * + * @return minimum number of buffers required for combining tree + */ + private static int computeRequiredBufferNum(int totalNumNodes, int degree) + { + if (totalNumNodes > degree) { + final int numNodesPerChild = (totalNumNodes + degree - 1) / degree; // ceiling + int sum = 1; // count for the current node + for (int i = 0; i < degree; i++) { + // further compute for child nodes + sum += computeRequiredBufferNum(Math.min(numNodesPerChild, totalNumNodes - i * numNodesPerChild), degree); + } + + return sum; + } else { + return 1; + } + } + + /** + * Recursively build a combining tree in a top-down manner. + * + * @param sortedIterators sorted iterators + * @param bufferSupplier combining buffer supplier + * @param combiningFactories array of combining aggregator factories + * @param combineDegree combining degree + * + * @return an iterator of the root of the combining tree + */ + private Iterator> buildCombineTree( + List>> sortedIterators, + Supplier bufferSupplier, + AggregatorFactory[] combiningFactories, + int combineDegree, + List combineFutures + ) + { + final int numIterators = sortedIterators.size(); + if (numIterators > combineDegree) { + final int iteratorsPerChild = (numIterators + combineDegree - 1) / combineDegree; // ceiling + final List>> childIterators = new ArrayList<>(); + for (int i = 0; i < combineDegree; i++) { + childIterators.add( + buildCombineTree( + sortedIterators.subList(i * iteratorsPerChild, Math.min(numIterators, (i + 1) * iteratorsPerChild)), + bufferSupplier, + combiningFactories, + combineDegree, + combineFutures + ) + ); + } + return runCombiner(childIterators, bufferSupplier.get(), combiningFactories, combineFutures).iterator(); + } else { + return runCombiner(sortedIterators, bufferSupplier.get(), combiningFactories, combineFutures).iterator(); + } + } + + private static ByteBuffer getSlice(ByteBuffer buffer, int sliceSize, int i) + { + final ByteBuffer slice = buffer.duplicate(); + slice.position(sliceSize * i); + slice.limit(slice.position() + sliceSize); + return slice.slice(); + } + + private StreamingMergeSortedGrouper runCombiner( + List>> iterators, + ByteBuffer combineBuffer, + AggregatorFactory[] combiningFactories, + List combineFutures + ) + { + final Iterator> mergedIterator = Groupers.mergeIterators(iterators, keyObjComparator); + final SettableColumnSelectorFactory settableColumnSelectorFactory = + new SettableColumnSelectorFactory(aggregatorFactories); + final StreamingMergeSortedGrouper grouper = new StreamingMergeSortedGrouper<>( + Suppliers.ofInstance(combineBuffer), + keySerdeFactory.factorize(), + settableColumnSelectorFactory, + combiningFactories + ); + grouper.init(); + + ListenableFuture future = grouperSorter.submit(() -> { + while (mergedIterator.hasNext()) { + final Entry next = mergedIterator.next(); + + settableColumnSelectorFactory.set(next.values); + final AggregateResult result = grouper.aggregate(next.key); + if (!result.isOk()) { + throw new ResourceLimitExceededException(result.getReason()); + } + settableColumnSelectorFactory.set(null); + } + + grouper.finish(); + }); + + combineFutures.add(future); + + return grouper; + } + @Override public void close() { closed = true; for (Grouper grouper : groupers) { - synchronized (grouper) { - grouper.close(); - } + grouper.close(); } } @@ -307,4 +575,127 @@ private int grouperNumberForKeyHash(int keyHash) { return keyHash % groupers.size(); } + + private static class SettableColumnSelectorFactory implements ColumnSelectorFactory + { + private static final int UNKNOWN_COLUMN_INDEX = -1; + private final Object2IntMap columnIndexMap; + + private Object[] values; + + SettableColumnSelectorFactory(AggregatorFactory[] aggregatorFactories) + { + columnIndexMap = new Object2IntArrayMap<>(aggregatorFactories.length); + columnIndexMap.defaultReturnValue(UNKNOWN_COLUMN_INDEX); + for (int i = 0; i < aggregatorFactories.length; i++) { + columnIndexMap.put(aggregatorFactories[i].getName(), i); + } + } + + public void set(Object[] values) + { + this.values = values; + } + + private int checkAndGetColumnIndex(String columnName) + { + final int columnIndex = columnIndexMap.getInt(columnName); + Preconditions.checkState( + columnIndex != UNKNOWN_COLUMN_INDEX, + "Cannot find a proper column index for column[%s]", + columnName + ); + return columnIndex; + } + + @Override + public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) + { + throw new UnsupportedOperationException(); + } + + @Override + public FloatColumnSelector makeFloatColumnSelector(String columnName) + { + return new FloatColumnSelector() + { + @Override + public float getFloat() + { + return ((Number) values[checkAndGetColumnIndex(columnName)]).floatValue(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // do nothing + } + }; + } + + @Override + public LongColumnSelector makeLongColumnSelector(String columnName) + { + return new LongColumnSelector() + { + @Override + public long getLong() + { + return ((Number) values[checkAndGetColumnIndex(columnName)]).longValue(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // do nothing + } + }; + } + + @Override + public DoubleColumnSelector makeDoubleColumnSelector(String columnName) + { + return new DoubleColumnSelector() + { + @Override + public double getDouble() + { + return ((Number) values[checkAndGetColumnIndex(columnName)]).doubleValue(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // do nothing + } + }; + } + + @Nullable + @Override + public ObjectColumnSelector makeObjectColumnSelector(String columnName) + { + return new ObjectColumnSelector() + { + @Override + public Class classOfObject() + { + return Object.class; + } + + @Override + public Object get() + { + return values[checkAndGetColumnIndex(columnName)]; + } + }; + } + + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(String column) + { + throw new UnsupportedOperationException(); + } + } } 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 index 1d960e6cf666..1a8725d8e62e 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Predicates; +import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; @@ -33,8 +34,10 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import io.druid.collections.BlockingPool; +import io.druid.collections.NonBlockingPool; import io.druid.collections.ReferenceCountingResourceHolder; import io.druid.collections.Releaser; +import io.druid.collections.ResourceHolder; import io.druid.data.input.Row; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; @@ -80,6 +83,7 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner private final ListeningExecutorService exec; private final QueryWatcher queryWatcher; private final int concurrencyHint; + private final NonBlockingPool processingBufferPool; private final BlockingPool mergeBufferPool; private final ObjectMapper spillMapper; private final String processingTmpDir; @@ -90,6 +94,7 @@ public GroupByMergingQueryRunnerV2( QueryWatcher queryWatcher, Iterable> queryables, int concurrencyHint, + NonBlockingPool processingBufferPool, BlockingPool mergeBufferPool, ObjectMapper spillMapper, String processingTmpDir @@ -100,6 +105,7 @@ public GroupByMergingQueryRunnerV2( this.queryWatcher = queryWatcher; this.queryables = Iterables.unmodifiableIterable(Iterables.filter(queryables, Predicates.notNull())); this.concurrencyHint = concurrencyHint; + this.processingBufferPool = processingBufferPool; this.mergeBufferPool = mergeBufferPool; this.spillMapper = spillMapper; this.processingTmpDir = processingTmpDir; @@ -151,6 +157,22 @@ public Sequence run(final QueryPlus queryPlus, final Map> combineBufferSupplier = new Supplier>() + { + private boolean initialized; + private ResourceHolder buffer; + + @Override + public ResourceHolder get() + { + if (!initialized) { + buffer = processingBufferPool.take(); + initialized = true; + } + return buffer; + } + }; + return new BaseSequence<>( new BaseSequence.IteratorMaker>() { @@ -191,6 +213,7 @@ public CloseableGrouperIterator make() null, config, Suppliers.ofInstance(mergeBufferHolder.get()), + combineBufferSupplier, concurrencyHint, temporaryStorage, spillMapper, 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 7afd95fefe00..dcea34a1e95c 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 @@ -100,10 +100,10 @@ default ToIntFunction hashFunction() /** * 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 - * want to reuse it), or {@link #iterator(boolean)} again if you want another iterator. This method is not thread-safe - * and must not be called by multiple threads concurrently. + * Some implementations allow writes even after this method is called. After you are done with the iterator + * returned by this method, you should either call {@link #close()} (if you are done with the Grouper) or + * {@link #reset()} (if you want to reuse it). Some implmenetations allow calling {@link #iterator(boolean)} again if + * you want another iterator. But, this method must not be called by multiple threads concurrently. *

* 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 diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index 845555020e46..7ac8f957bf90 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -34,6 +34,7 @@ import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import com.google.common.util.concurrent.ListeningExecutorService; +import io.druid.collections.ResourceHolder; import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; import io.druid.java.util.common.IAE; @@ -104,6 +105,7 @@ public static Pair, Accumulator> crea rawInputRowSignature, config, bufferSupplier, + null, SINGLE_THREAD_CONCURRENCY_HINT, temporaryStorage, spillMapper, @@ -126,6 +128,7 @@ public static Pair, Accumulator> crea final Map rawInputRowSignature, final GroupByQueryConfig config, final Supplier bufferSupplier, + final Supplier> combineBufferSupplier, final int concurrencyHint, final LimitedTemporaryStorage temporaryStorage, final ObjectMapper spillMapper, @@ -191,6 +194,7 @@ public static Pair, Accumulator> crea } else { grouper = new ConcurrentGrouper<>( bufferSupplier, + combineBufferSupplier, keySerdeFactory, columnSelectorFactory, aggregatorFactories, @@ -205,7 +209,8 @@ public static Pair, Accumulator> crea grouperSorter, priority, hasQueryTimeout, - queryTimeoutAt + queryTimeoutAt, + config.isForceSingleThreadedCombine() ); } @@ -997,14 +1002,16 @@ public RowBasedKey fromByteBuffer(ByteBuffer buffer, int position) public Grouper.BufferComparator bufferComparator() { if (sortableIds == null) { - Map sortedMap = Maps.newTreeMap(); - for (int id = 0; id < dictionary.size(); id++) { - sortedMap.put(dictionary.get(id), id); + final int dictionarySize = dictionary.size(); + final Pair[] dictAndIds = new Pair[dictionarySize]; + for (int id = 0; id < dictionarySize; id++) { + dictAndIds[id] = new Pair<>(dictionary.get(id), id); } - sortableIds = new int[dictionary.size()]; - int index = 0; - for (final Integer id : sortedMap.values()) { - sortableIds[id] = index++; + Arrays.sort(dictAndIds, Comparator.comparing(pair -> pair.lhs)); + + sortableIds = new int[dictionarySize]; + for (int i = 0; i < dictionarySize; i++) { + sortableIds[dictAndIds[i].rhs] = i; } } 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 60e618def1a9..b94de4471e66 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 @@ -134,7 +134,7 @@ public AggregateResult aggregate(KeyType key, int keyHash) { final AggregateResult result = grouper.aggregate(key, keyHash); - if (result.isOk() || temporaryStorage.maxSize() <= 0 || !spillingAllowed) { + if (result.isOk() || !spillingAllowed || temporaryStorage.maxSize() <= 0) { return result; } else { // Warning: this can potentially block up a processing thread for a while. diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java new file mode 100644 index 000000000000..525ff1e66d9a --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java @@ -0,0 +1,378 @@ +/* + * 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.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import io.druid.java.util.common.IAE; +import io.druid.java.util.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.Iterator; +import java.util.NoSuchElementException; +import java.util.concurrent.TimeoutException; + +/** + * A streaming grouper which can aggregate sorted inputs. This grouper can aggregate while its iterator is consumed. + * Also, the aggregation thread and iterating thread can be different. + * + * This grouper is backed by a circular array off-heap buffer. Reading iterator is able to read from an array slot only + * if the write for that slot is finished. + */ +public class StreamingMergeSortedGrouper implements Grouper +{ + private static final Logger LOG = new Logger(StreamingMergeSortedGrouper.class); + // Timeout for waiting for a slot to be available for read/write. This is required to prevent for the processing + // thread from being blocked if its iterator is not consumed due to some failure. + private static final long DEFAULT_TIMEOUT_MS = 5000L; + + private static final AggregateResult DICTIONARY_FULL = AggregateResult.failure( + "Not enough dictionary space to execute this query. Try increasing " + + "druid.query.groupBy.maxMergingDictionarySize or use single-threaded combine by setting " + + "druid.query.groupBy.forceSingleThreadedCombine to true." + ); + + private final Supplier bufferSupplier; + private final KeySerde keySerde; + private final BufferAggregator[] aggregators; + private final int[] aggregatorOffsets; + private final int keySize; + private final int recordSize; // size of (key + all aggregates) + private final long timeoutMs; + + // Below variables are initialized when init() is called + private ByteBuffer buffer; + private int maxSlotNum; + private boolean initialized; + + // Below variables can be read/written by differernt threads + + /** + * Indicate that this grouper consumed the last input or not. + */ + private volatile boolean finished; + + /** + * Currently writing position. This is always moved ahead of nextReadIndex. + */ + private volatile int curWriteIndex; + + /** + * Next read position. This can be moved to a position only when write for the position is finished. + */ + private volatile int nextReadIndex; + + /** + * Returns the minimum buffer capacity required to use this grouper. This grouper keeps track read/write indexes + * and they cannot point the same position at the same time. Since the read/write indexes circularly, the required + * minimum buffer capacity is 3 * record size. + * + * @return required minimum buffer capacity + */ + public static int requiredBufferCapacity( + KeySerde keySerde, + AggregatorFactory[] aggregatorFactories + ) + { + int recordSize = keySerde.keySize(); + for (AggregatorFactory aggregatorFactory : aggregatorFactories) { + recordSize += aggregatorFactory.getMaxIntermediateSize(); + } + return recordSize * 3; + } + + public StreamingMergeSortedGrouper( + final Supplier bufferSupplier, + final KeySerde keySerde, + final ColumnSelectorFactory columnSelectorFactory, + final AggregatorFactory[] aggregatorFactories + ) + { + this(bufferSupplier, keySerde, columnSelectorFactory, aggregatorFactories, DEFAULT_TIMEOUT_MS); + } + + @VisibleForTesting + StreamingMergeSortedGrouper( + final Supplier bufferSupplier, + final KeySerde keySerde, + final ColumnSelectorFactory columnSelectorFactory, + final AggregatorFactory[] aggregatorFactories, + final long timeoutMs + ) + { + this.bufferSupplier = bufferSupplier; + this.keySerde = keySerde; + this.aggregators = new BufferAggregator[aggregatorFactories.length]; + this.aggregatorOffsets = new int[aggregatorFactories.length]; + + this.keySize = keySerde.keySize(); + int offset = keySize; + for (int i = 0; i < aggregatorFactories.length; i++) { + aggregators[i] = aggregatorFactories[i].factorizeBuffered(columnSelectorFactory); + aggregatorOffsets[i] = offset; + offset += aggregatorFactories[i].getMaxIntermediateSize(); + } + this.recordSize = offset; + this.timeoutMs = timeoutMs; + } + + @Override + public void init() + { + if (!initialized) { + buffer = bufferSupplier.get(); + maxSlotNum = buffer.capacity() / recordSize; + Preconditions.checkState( + maxSlotNum > 2, + "Buffer[%s] should be large enough to store at least three records[%s]", + buffer.capacity(), + recordSize + ); + + reset(); + initialized = true; + } + } + + @Override + public boolean isInitialized() + { + return initialized; + } + + @Override + public AggregateResult aggregate(KeyType key, int notUsed) + { + return aggregate(key); + } + + @Override + public AggregateResult aggregate(KeyType key) + { + final ByteBuffer keyBuffer = keySerde.toByteBuffer(key); + if (keyBuffer == null) { + // This should bubble up to the user, so call finish() here. + finish(); + return DICTIONARY_FULL; + } + + if (keyBuffer.remaining() != keySize) { + throw new IAE( + "keySerde.toByteBuffer(key).remaining[%s] != keySerde.keySize[%s], buffer was the wrong size?!", + keyBuffer.remaining(), + keySize + ); + } + + final int prevRecordOffset = curWriteIndex * recordSize; + if (curWriteIndex == -1 || !keyEquals(keyBuffer, buffer, prevRecordOffset)) { + initNewSlot(keyBuffer); + } + + final int curRecordOffset = curWriteIndex * recordSize; + for (int i = 0; i < aggregatorOffsets.length; i++) { + aggregators[i].aggregate(buffer, curRecordOffset + aggregatorOffsets[i]); + } + + return AggregateResult.ok(); + } + + private boolean keyEquals(ByteBuffer curKeyBuffer, ByteBuffer buffer, int bufferOffset) + { + int i = 0; + for (; i + Long.BYTES <= keySize; i += Long.BYTES) { + if (curKeyBuffer.getLong(i) != buffer.getLong(bufferOffset + i)) { + return false; + } + } + + for (; i + Integer.BYTES <= keySize; i += Integer.BYTES) { + if (curKeyBuffer.getInt(i) != buffer.getInt(bufferOffset + i)) { + return false; + } + } + + for (; i < keySize; i++) { + if (curKeyBuffer.get(i) != buffer.get(bufferOffset + i)) { + return false; + } + } + + return true; + } + + private void initNewSlot(ByteBuffer newKey) + { + increaseWriteIndex(); + + final int recordOffset = recordSize * curWriteIndex; + buffer.position(recordOffset); + buffer.put(newKey); + + for (int i = 0; i < aggregators.length; i++) { + aggregators[i].init(buffer, recordOffset + aggregatorOffsets[i]); + } + } + + /** + * Wait for {@link #nextReadIndex} to be moved if necessary and move {@link #curWriteIndex}. {@link #nextReadIndex} + * is checked in while loops instead of waiting using a lock to avoid frequent thread park. + */ + private void increaseWriteIndex() + { + if (curWriteIndex == maxSlotNum - 1) { + final long startLoopAt = System.currentTimeMillis(); + while (nextReadIndex == -1 || nextReadIndex == 0) { + if (System.currentTimeMillis() - startLoopAt > timeoutMs) { + throw new RuntimeException(new TimeoutException()); + } + } + curWriteIndex = 0; + } else { + final int nextWriteIndex = curWriteIndex + 1; + final long startLoopAt = System.currentTimeMillis(); + while (nextWriteIndex == nextReadIndex) { + if (System.currentTimeMillis() - startLoopAt > timeoutMs) { + throw new RuntimeException(new TimeoutException()); + } + } + curWriteIndex = nextWriteIndex; + } + } + + @Override + public void reset() + { + curWriteIndex = -1; + nextReadIndex = -1; + finished = false; + } + + @Override + public void close() + { + for (BufferAggregator aggregator : aggregators) { + try { + aggregator.close(); + } + catch (Exception e) { + LOG.warn(e, "Could not close aggregator [%s], skipping.", aggregator); + } + } + } + + /** + * Signal that no more inputs are added. Must be called after {@link #aggregate(Object)} is called for the last input. + */ + public void finish() + { + increaseWriteIndex(); + finished = true; + } + + /** + * Return a sorted iterator. This method can be called safely while writing and iterating thread and writing thread + * can be different. The result iterator always returns sorted results. This method should be called only one time + * per grouper. + * + * @return a sorted iterator + */ + public Iterator> iterator() + { + return new Iterator>() + { + { + // Waits for some data to be ready + final long startLoopAt = System.currentTimeMillis(); + while ((curWriteIndex == -1 || curWriteIndex == 0) && !finished) { + if (System.currentTimeMillis() - startLoopAt > timeoutMs) { + throw new RuntimeException(new TimeoutException()); + } + } + nextReadIndex = 0; + } + + @Override + public boolean hasNext() + { + return !finished || remaining() > 0; + } + + private int remaining() + { + if (curWriteIndex >= nextReadIndex) { + return curWriteIndex - nextReadIndex; + } else { + return (maxSlotNum - nextReadIndex) + curWriteIndex; + } + } + + @Override + public Entry next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + final int recordOffset = recordSize * nextReadIndex; + final KeyType key = keySerde.fromByteBuffer(buffer, recordOffset); + + final Object[] values = new Object[aggregators.length]; + for (int i = 0; i < aggregators.length; i++) { + values[i] = aggregators[i].get(buffer, recordOffset + aggregatorOffsets[i]); + } + + final int toBeUpdated = nextReadIndex == maxSlotNum - 1 ? 0 : nextReadIndex + 1; + + if (!finished) { + final long startLoopAt = System.currentTimeMillis(); + while (toBeUpdated == curWriteIndex && !finished) { + if (System.currentTimeMillis() - startLoopAt > timeoutMs) { + throw new RuntimeException(new TimeoutException()); + } + } + } + + nextReadIndex = toBeUpdated; + + return new Entry<>(key, values); + } + }; + } + + /** + * Return a sorted iterator. This method can be called safely while writing and iterating thread and writing thread + * can be different. The result iterator always returns sorted results. This method should be called only one time + * per grouper. + * + * @param sorted not used + * + * @return a sorted iterator + */ + @Override + public Iterator> iterator(boolean sorted) + { + return iterator(); + } +} 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 cabbaf0ea327..488cc527367e 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 @@ -336,6 +336,7 @@ public QueryRunner mergeRunners( queryWatcher, queryRunners, processingConfig.getNumThreads(), + bufferPool, mergeBufferPool, spillMapper, processingConfig.getTmpDir() 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 ddef2a474c78..c7fedc08211a 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -288,6 +288,12 @@ public long getMaxOnDiskStorage() return 10L * 1024 * 1024; } + @Override + public boolean isForceSingleThreadedCombine() + { + return true; + } + @Override public String toString() { diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java index d2162c702a9c..ed0fd0635e7a 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java @@ -20,14 +20,17 @@ package io.druid.query.groupby.epinephelinae; import com.google.common.base.Supplier; +import com.google.common.collect.Lists; import com.google.common.primitives.Longs; import com.google.common.util.concurrent.MoreExecutors; -import io.druid.concurrent.Execs; +import io.druid.collections.ResourceHolder; +import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.IAE; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.dimension.DimensionSpec; import io.druid.query.groupby.epinephelinae.Grouper.BufferComparator; +import io.druid.query.groupby.epinephelinae.Grouper.Entry; import io.druid.query.groupby.epinephelinae.Grouper.KeySerde; import io.druid.query.groupby.epinephelinae.Grouper.KeySerdeFactory; import io.druid.segment.ColumnSelectorFactory; @@ -38,10 +41,16 @@ import io.druid.segment.ObjectColumnSelector; import io.druid.segment.column.ColumnCapabilities; import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import java.io.IOException; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Comparator; +import java.util.List; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -51,6 +60,7 @@ public class ConcurrentGrouperTest { private static final ExecutorService service = Executors.newFixedThreadPool(8); + private static final TestResourceHolder testResourceHolder = new TestResourceHolder(); @AfterClass public static void teardown() @@ -58,21 +68,58 @@ public static void teardown() service.shutdown(); } + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + private static final Supplier bufferSupplier = new Supplier() { private final AtomicBoolean called = new AtomicBoolean(false); + private ByteBuffer buffer; @Override public ByteBuffer get() { if (called.compareAndSet(false, true)) { - return ByteBuffer.allocate(192); + buffer = ByteBuffer.allocate(256); + } + + return buffer; + } + }; + + private static final Supplier> combineBufferSupplier = new Supplier>() + { + private final AtomicBoolean called = new AtomicBoolean(false); + + @Override + public ResourceHolder get() + { + if (called.compareAndSet(false, true)) { + return testResourceHolder; } else { throw new IAE("should be called once"); } } }; + private static class TestResourceHolder implements ResourceHolder + { + private boolean closed; + private ByteBuffer buffer = ByteBuffer.allocate(256); + + @Override + public ByteBuffer get() + { + return buffer; + } + + @Override + public void close() + { + closed = true; + } + } + private static final KeySerdeFactory keySerdeFactory = new KeySerdeFactory() { @Override @@ -189,27 +236,32 @@ public DoubleColumnSelector makeDoubleColumnSelector(String columnName) } }; - @Test(timeout = 5000L) - public void testAggregate() throws InterruptedException, ExecutionException + @Test() + public void testAggregate() throws InterruptedException, ExecutionException, IOException { final ConcurrentGrouper grouper = new ConcurrentGrouper<>( bufferSupplier, + combineBufferSupplier, keySerdeFactory, null_factory, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, 24, 0.7f, 1, - null, - null, + new LimitedTemporaryStorage(temporaryFolder.newFolder(), 1024 * 1024), + new DefaultObjectMapper(), 8, null, false, - MoreExecutors.listeningDecorator(Execs.multiThreaded(4, "concurrent-grouper-test-%d")), + MoreExecutors.listeningDecorator(service), 0, false, - 0 + 0, + false ); + grouper.init(); + + final int numRows = 1000; Future[] futures = new Future[8]; @@ -219,9 +271,8 @@ public void testAggregate() throws InterruptedException, ExecutionException @Override public void run() { - grouper.init(); - for (long i = 0; i < 100; i++) { - grouper.aggregate(0L); + for (long i = 0; i < numRows; i++) { + grouper.aggregate(i); } } }); @@ -231,6 +282,17 @@ public void run() eachFuture.get(); } + final List> actual = Lists.newArrayList(grouper.iterator(true)); + + Assert.assertTrue(testResourceHolder.closed); + + final List> expected = new ArrayList<>(); + for (long i = 0; i < numRows; i++) { + expected.add(new Entry<>(i, new Object[]{8L})); + } + + Assert.assertEquals(expected, actual); + grouper.close(); } } diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouperTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouperTest.java new file mode 100644 index 000000000000..1bcdb3c66642 --- /dev/null +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouperTest.java @@ -0,0 +1,185 @@ +/* + * 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.concurrent.Execs; +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.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeoutException; + +public class StreamingMergeSortedGrouperTest +{ + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Test + public void testAggregate() + { + final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory(); + final StreamingMergeSortedGrouper grouper = newGrouper(columnSelectorFactory, 1024); + + columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 10L))); + grouper.aggregate(6); + grouper.aggregate(6); + grouper.aggregate(6); + grouper.aggregate(10); + grouper.aggregate(12); + grouper.aggregate(12); + + grouper.finish(); + + 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(true)); + + Assert.assertEquals( + expected, + unsortedEntries + ); + } + + @Test(timeout = 5000L) + public void testEmptyIterator() + { + final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory(); + final StreamingMergeSortedGrouper grouper = newGrouper(columnSelectorFactory, 1024); + + grouper.finish(); + + Assert.assertTrue(!grouper.iterator(true).hasNext()); + } + + @Test(timeout = 5000L) + public void testStreamingAggregateWithLargeBuffer() + { + testStreamingAggregate(1024); + } + + @Test(timeout = 5000L) + public void testStreamingAggregateWithMinimumBuffer() + { + testStreamingAggregate(60); + } + + private void testStreamingAggregate(int bufferSize) + { + final ExecutorService exec = Execs.multiThreaded(2, "merge-sorted-grouper-test-%d"); + final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory(); + final StreamingMergeSortedGrouper grouper = newGrouper(columnSelectorFactory, bufferSize); + + final List> expected = new ArrayList<>(1024); + for (int i = 0; i < 1024; i++) { + expected.add(new Entry<>(i, new Object[]{100L, 10L})); + } + + try { + exec.submit(() -> { + columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 10L))); + + for (int i = 0; i < 1024; i++) { + for (int j = 0; j < 10; j++) { + grouper.aggregate(i); + } + } + + grouper.finish(); + }); + + final List> unsortedEntries = Lists.newArrayList(grouper.iterator(true)); + + Assert.assertEquals( + expected, + Ordering.from((Comparator>) (o1, o2) -> Ints.compare(o1.getKey(), o2.getKey())) + .sortedCopy(unsortedEntries) + ); + } + finally { + exec.shutdownNow(); + } + } + + @Test + public void testNotEnoughBuffer() + { + expectedException.expect(IllegalStateException.class); + expectedException.expectMessage("Buffer[50] should be large enough to store at least three records[20]"); + + newGrouper(GrouperTestUtil.newColumnSelectorFactory(), 50); + } + + @Test + public void testTimeout() + { + expectedException.expect(RuntimeException.class); + expectedException.expectCause(CoreMatchers.instanceOf(TimeoutException.class)); + + final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory(); + final StreamingMergeSortedGrouper grouper = newGrouper(columnSelectorFactory, 60); + + columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 10L))); + grouper.aggregate(6); + + grouper.iterator(); + } + + private StreamingMergeSortedGrouper newGrouper( + TestColumnSelectorFactory columnSelectorFactory, + int bufferSize + ) + { + final ByteBuffer buffer = ByteBuffer.allocate(bufferSize); + + final StreamingMergeSortedGrouper grouper = new StreamingMergeSortedGrouper<>( + Suppliers.ofInstance(buffer), + GrouperTestUtil.intKeySerde(), + columnSelectorFactory, + new AggregatorFactory[]{ + new LongSumAggregatorFactory("valueSum", "value"), + new CountAggregatorFactory("count") + }, + 1000 + ); + grouper.init(); + return grouper; + } +} From 30c0038853eb98a9846a4877ab37fe94f55f1f66 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 19 Aug 2017 12:04:28 +0900 Subject: [PATCH 02/44] Fix doc --- .../groupby/epinephelinae/StreamingMergeSortedGrouper.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java index 525ff1e66d9a..d9999b0f1cc3 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java @@ -34,8 +34,8 @@ import java.util.concurrent.TimeoutException; /** - * A streaming grouper which can aggregate sorted inputs. This grouper can aggregate while its iterator is consumed. - * Also, the aggregation thread and iterating thread can be different. + * A streaming grouper which can aggregate sorted inputs. This grouper can aggregate while its iterator is being + * consumed. Also, the aggregation thread and iterating thread can be different. * * This grouper is backed by a circular array off-heap buffer. Reading iterator is able to read from an array slot only * if the write for that slot is finished. From c79959f51c50262eb4c1dbc3dbfeea0902de68fb Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 25 Aug 2017 20:23:59 +0900 Subject: [PATCH 03/44] Use a single dictionary while combining --- .../benchmark/query/GroupByBenchmark.java | 10 +- .../CloseableGrouperIterator.java | 5 +- .../epinephelinae/ConcurrentGrouper.java | 67 +- .../epinephelinae/GroupByQueryEngineV2.java | 7 + .../query/groupby/epinephelinae/Grouper.java | 20 +- .../epinephelinae/RowBasedGrouperHelper.java | 710 ++++++++++-------- .../epinephelinae/RowBasedKeySerdeHelper.java | 73 ++ .../epinephelinae/SpillingGrouper.java | 46 +- .../StreamingMergeSortedGrouper.java | 45 +- .../epinephelinae/ConcurrentGrouperTest.java | 20 + .../groupby/epinephelinae/IntKeySerde.java | 8 + 11 files changed, 625 insertions(+), 386 deletions(-) create mode 100644 processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedKeySerdeHelper.java 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 20f0e46c12bb..32953a83834d 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java @@ -116,10 +116,10 @@ @Measurement(iterations = 30) public class GroupByBenchmark { - @Param({"4"}) + @Param({"8"}) private int numSegments; - @Param({"2", "4"}) + @Param({"8"}) private int numProcessingThreads; @Param({"-1"}) @@ -128,13 +128,13 @@ public class GroupByBenchmark @Param({"100000"}) private int rowsPerSegment; - @Param({"basic.A", "basic.nested"}) + @Param({"basic.A"}) private String schemaAndQuery; - @Param({"v1", "v2"}) + @Param({"v2"}) private String defaultStrategy; - @Param({"all", "day"}) + @Param({"all"}) private String queryGranularity; private static final Logger log = new Logger(GroupByBenchmark.class); 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 index a377f033f196..808b60f75c03 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/CloseableGrouperIterator.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/CloseableGrouperIterator.java @@ -19,16 +19,17 @@ package io.druid.query.groupby.epinephelinae; -import com.google.common.base.Function; import com.google.common.base.Throwables; +import io.druid.query.groupby.epinephelinae.Grouper.Entry; import java.io.Closeable; import java.io.IOException; import java.util.Iterator; +import java.util.function.Function; public class CloseableGrouperIterator implements Iterator, Closeable { - private final Function, T> transformer; + private final Function, T> transformer; private final Closeable closer; private final Iterator> iterator; 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 6f0e8d57479e..593c4ddced17 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 @@ -23,18 +23,19 @@ import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import io.druid.collections.ResourceHolder; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; -import io.druid.java.util.common.logger.Logger; import io.druid.query.AbstractPrioritizedCallable; import io.druid.query.QueryInterruptedException; import io.druid.query.ResourceLimitExceededException; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.dimension.DimensionSpec; +import io.druid.query.groupby.epinephelinae.RowBasedGrouperHelper.RowBasedKeySerde; import io.druid.query.groupby.orderby.DefaultLimitSpec; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnSelectorFactory; @@ -51,8 +52,10 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Comparator; +import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Set; import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; @@ -72,7 +75,6 @@ */ public class ConcurrentGrouper implements Grouper { - private static final Logger LOG = new Logger(ConcurrentGrouper.class); private static final int MINIMUM_COMBINE_DEGREE = 2; private final List> groupers; @@ -92,6 +94,7 @@ public class ConcurrentGrouper implements Grouper private final ObjectMapper spillMapper; private final int concurrencyHint; private final KeySerdeFactory keySerdeFactory; + private final KeySerdeFactory mergingKeySerdeFactory; private final DefaultLimitSpec limitSpec; private final boolean sortHasNonGroupingFields; private final Comparator> keyObjComparator; @@ -107,6 +110,7 @@ public ConcurrentGrouper( final Supplier bufferSupplier, final Supplier> combineBufferSupplier, final KeySerdeFactory keySerdeFactory, + final KeySerdeFactory mergingKeySerdeFactory, final ColumnSelectorFactory columnSelectorFactory, final AggregatorFactory[] aggregatorFactories, final int bufferGrouperMaxSize, @@ -147,6 +151,7 @@ protected SpillingGrouper initialValue() this.spillMapper = spillMapper; this.concurrencyHint = concurrencyHint; this.keySerdeFactory = keySerdeFactory; + this.mergingKeySerdeFactory = mergingKeySerdeFactory; this.limitSpec = limitSpec; this.sortHasNonGroupingFields = sortHasNonGroupingFields; this.keyObjComparator = keySerdeFactory.objectComparator(sortHasNonGroupingFields); @@ -267,10 +272,12 @@ public Iterator> iterator(final boolean sorted) // Parallel combine is used only when data are spilled. This is because ConcurrentGrouper uses two different modes // depending on data are spilled or not. If data is not spilled, all inputs are completely aggregated and no more // aggregation is required. - return parallelCombine(sortedIterators); - } else { - return Groupers.mergeIterators(sortedIterators, sorted ? keyObjComparator : null); + final List mergedDictionary = mergeDictionary(); + if (mergedDictionary != null) { + return parallelCombine(sortedIterators, mergedDictionary); + } } + return Groupers.mergeIterators(sortedIterators, sorted ? keyObjComparator : null); } private boolean isParallelizable() @@ -321,6 +328,31 @@ private List>> getGroupersIterator(boolean sorted) .collect(Collectors.toList()); } + /** + * Merge dictionaries of {@link Grouper.KeySerde}s of {@link Grouper}s. The result dictionary contains unique string + * string keys. + * + * @return merged dictionary if its size does not exceed max dictionary size. Otherwise null. + */ + @Nullable + private List mergeDictionary() + { + final long maxDictionarySize = mergingKeySerdeFactory.getMaxDictionarySize(); + final Set mergedDictionary = new HashSet<>(); + long totalDictionarySize = 0L; + + for (SpillingGrouper grouper : groupers) { + final List dictionary = grouper.getDictionary(); + totalDictionarySize += dictionary.stream().mapToLong(RowBasedKeySerde::estimateStringKeySize).sum(); + if (totalDictionarySize > maxDictionarySize) { + return null; + } + mergedDictionary.addAll(dictionary); + } + + return ImmutableList.copyOf(mergedDictionary); + } + /** * Build a combining tree for the input iterators which combine input entries asynchronously. This method is called * when data are spilled and thus streaming combine is preferred to avoid too many disk accesses. @@ -329,7 +361,10 @@ private List>> getGroupersIterator(boolean sorted) * * @return an iterator of the root grouper of the combining tree */ - private Iterator> parallelCombine(List>> sortedIterators) + private Iterator> parallelCombine( + List>> sortedIterators, + List mergedDictionary + ) { // CombineBuffer is initialized when this method is called final ResourceHolder combineBufferHolder = combineBufferSupplier.get(); @@ -339,7 +374,7 @@ private Iterator> parallelCombine(List>> combiningFactories[i] = aggregatorFactories[i].getCombiningFactory(); } final int minimumRequiredBufferCapacity = StreamingMergeSortedGrouper.requiredBufferCapacity( - keySerdeFactory.factorize(), + mergingKeySerdeFactory.factorizeWithDictionary(mergedDictionary), combiningFactories ); final Pair degreeAndBufferNum = findCombineDegreeAndNumBuffers( @@ -383,7 +418,8 @@ public ByteBuffer get() bufferSupplier, combiningFactories, combineDegree, - combineFutures + combineFutures, + mergedDictionary ); return new Iterator>() @@ -492,7 +528,8 @@ private Iterator> buildCombineTree( Supplier bufferSupplier, AggregatorFactory[] combiningFactories, int combineDegree, - List combineFutures + List combineFutures, + List dictionary ) { final int numIterators = sortedIterators.size(); @@ -506,13 +543,14 @@ private Iterator> buildCombineTree( bufferSupplier, combiningFactories, combineDegree, - combineFutures + combineFutures, + dictionary ) ); } - return runCombiner(childIterators, bufferSupplier.get(), combiningFactories, combineFutures).iterator(); + return runCombiner(childIterators, bufferSupplier.get(), combiningFactories, combineFutures, dictionary).iterator(); } else { - return runCombiner(sortedIterators, bufferSupplier.get(), combiningFactories, combineFutures).iterator(); + return runCombiner(sortedIterators, bufferSupplier.get(), combiningFactories, combineFutures, dictionary).iterator(); } } @@ -528,7 +566,8 @@ private StreamingMergeSortedGrouper runCombiner( List>> iterators, ByteBuffer combineBuffer, AggregatorFactory[] combiningFactories, - List combineFutures + List combineFutures, + List dictionary ) { final Iterator> mergedIterator = Groupers.mergeIterators(iterators, keyObjComparator); @@ -536,7 +575,7 @@ private StreamingMergeSortedGrouper runCombiner( new SettableColumnSelectorFactory(aggregatorFactories); final StreamingMergeSortedGrouper grouper = new StreamingMergeSortedGrouper<>( Suppliers.ofInstance(combineBuffer), - keySerdeFactory.factorize(), + mergingKeySerdeFactory.factorizeWithDictionary(dictionary), settableColumnSelectorFactory, combiningFactories ); 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 30f598684a61..a47a2b7b930e 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 @@ -22,6 +22,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; import io.druid.collections.NonBlockingPool; import io.druid.collections.ResourceHolder; @@ -730,6 +731,12 @@ public Class keyClazz() return ByteBuffer.class; } + @Override + public List getDictionary() + { + return ImmutableList.of(); + } + @Override public ByteBuffer toByteBuffer(ByteBuffer key) { 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 dcea34a1e95c..f33213f19471 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 @@ -29,6 +29,7 @@ import java.util.Arrays; import java.util.Comparator; import java.util.Iterator; +import java.util.List; import java.util.function.ToIntFunction; /** @@ -186,10 +187,22 @@ public String toString() interface KeySerdeFactory { /** - * Create a new KeySerde, which may be stateful. + * Return max dictionary size threshold. + * + * @return max dictionary size + */ + long getMaxDictionarySize(); + + /** + * Create a new {@link KeySerde}, which may be stateful. */ KeySerde factorize(); + /** + * Create a new {@link KeySerde} with the given dictionary. + */ + KeySerde factorizeWithDictionary(List dictionary); + /** * Return an object that knows how to compare two serialized key instances. Will be called by the * {@link #iterator(boolean)} method if sorting is enabled. @@ -217,6 +230,11 @@ interface KeySerde */ Class keyClazz(); + /** + * Return the dictionary of this KeySerde. The return value should not be null. + */ + List getDictionary(); + /** * 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. diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index 7ac8f957bf90..53905ba5278f 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonValue; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.base.Supplier; @@ -38,6 +37,7 @@ import io.druid.data.input.MapBasedRow; 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.Pair; import io.druid.java.util.common.granularity.AllGranularity; import io.druid.java.util.common.guava.Accumulator; @@ -50,6 +50,7 @@ import io.druid.query.groupby.GroupByQuery; import io.druid.query.groupby.GroupByQueryConfig; import io.druid.query.groupby.RowBasedColumnSelectorFactory; +import io.druid.query.groupby.epinephelinae.Grouper.BufferComparator; import io.druid.query.groupby.orderby.DefaultLimitSpec; import io.druid.query.groupby.orderby.OrderByColumnSpec; import io.druid.query.groupby.strategy.GroupByStrategyV2; @@ -65,6 +66,8 @@ import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ValueType; import io.druid.segment.data.IndexedInts; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -77,6 +80,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Function; // this class contains shared code between GroupByMergingQueryRunnerV2 and GroupByRowProcessor public class RowBasedGrouperHelper @@ -192,10 +196,21 @@ public static Pair, Accumulator> crea sortHasNonGroupingFields ); } else { + final Grouper.KeySerdeFactory mergingKeySerdeFactory = new RowBasedKeySerdeFactory( + includeTimestamp, + query.getContextSortByDimsFirst(), + query.getDimensions(), + querySpecificConfig.getMaxMergingDictionarySize(), // use entire dictionary space for one key serde + valueTypes, + aggregatorFactories, + limitSpec + ); + grouper = new ConcurrentGrouper<>( bufferSupplier, combineBufferSupplier, keySerdeFactory, + mergingKeySerdeFactory, columnSelectorFactory, aggregatorFactories, querySpecificConfig.getBufferGrouperMaxSize(), @@ -646,6 +661,12 @@ private static class RowBasedKeySerdeFactory implements Grouper.KeySerdeFactory< this.valueTypes = valueTypes; } + @Override + public long getMaxDictionarySize() + { + return maxDictionarySize; + } + @Override public Grouper.KeySerde factorize() { @@ -655,7 +676,22 @@ public Grouper.KeySerde factorize() dimensions, maxDictionarySize, limitSpec, - valueTypes + valueTypes, + null + ); + } + + @Override + public Grouper.KeySerde factorizeWithDictionary(List dictionary) + { + return new RowBasedKeySerde( + includeTimestamp, + sortByDimsFirst, + dimensions, + maxDictionarySize, + limitSpec, + valueTypes, + dictionary ); } @@ -893,10 +929,12 @@ private static int compareDimsInRowsWithAggs( } } - private static class RowBasedKeySerde implements Grouper.KeySerde + static class RowBasedKeySerde 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 static final int DICTIONARY_INITIAL_CAPACITY = 10000; + private static final int UNKNOWN_DICTIONARY_KEY = -1; private final boolean includeTimestamp; private final boolean sortByDimsFirst; @@ -904,14 +942,18 @@ private static class RowBasedKeySerde implements Grouper.KeySerde dictionary = Lists.newArrayList(); - private final Map reverseDictionary = Maps.newHashMap(); private final List serdeHelpers; private final DefaultLimitSpec limitSpec; private final List valueTypes; + private final boolean enableRuntimeDictionaryGeneration; + + private final List dictionary; + private final Object2IntMap reverseDictionary; + // 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 @@ -923,19 +965,57 @@ private static class RowBasedKeySerde implements Grouper.KeySerde dimensions, final long maxDictionarySize, final DefaultLimitSpec limitSpec, - final List valueTypes + final List valueTypes, + @Nullable final List dictionary ) { this.includeTimestamp = includeTimestamp; this.sortByDimsFirst = sortByDimsFirst; this.dimensions = dimensions; this.dimCount = dimensions.size(); - this.maxDictionarySize = maxDictionarySize; this.valueTypes = valueTypes; this.limitSpec = limitSpec; - this.serdeHelpers = makeSerdeHelpers(); + this.enableRuntimeDictionaryGeneration = dictionary == null; + this.dictionary = enableRuntimeDictionaryGeneration ? new ArrayList<>(DICTIONARY_INITIAL_CAPACITY) : dictionary; + this.reverseDictionary = enableRuntimeDictionaryGeneration ? + new Object2IntOpenHashMap<>(DICTIONARY_INITIAL_CAPACITY) : + new Object2IntOpenHashMap<>(dictionary.size()); + this.reverseDictionary.defaultReturnValue(UNKNOWN_DICTIONARY_KEY); + this.maxDictionarySize = maxDictionarySize; + this.serdeHelpers = makeSerdeHelpers(limitSpec != null, enableRuntimeDictionaryGeneration); this.keySize = (includeTimestamp ? Longs.BYTES : 0) + getTotalKeySize(); this.keyBuffer = ByteBuffer.allocate(keySize); + + if (!enableRuntimeDictionaryGeneration) { + final long initialDictionarySize = dictionary.stream().mapToLong(RowBasedKeySerde::estimateStringKeySize).sum(); + Preconditions.checkState( + maxDictionarySize >= initialDictionarySize, + "Dictionary size[%s] exceeds threshold[%s]", + initialDictionarySize, + maxDictionarySize + ); + + for (int i = 0; i < dictionary.size(); i++) { + reverseDictionary.put(dictionary.get(i), i); + } + + initializeSortableIds(); + } + } + + private void initializeSortableIds() + { + final int dictionarySize = dictionary.size(); + final Pair[] dictAndIds = new Pair[dictionarySize]; + for (int id = 0; id < dictionarySize; id++) { + dictAndIds[id] = new Pair<>(dictionary.get(id), id); + } + Arrays.sort(dictAndIds, Comparator.comparing(pair -> pair.lhs)); + + sortableIds = new int[dictionarySize]; + for (int i = 0; i < dictionarySize; i++) { + sortableIds[dictAndIds[i].rhs] = i; + } } @Override @@ -950,6 +1030,12 @@ public Class keyClazz() return RowBasedKey.class; } + @Override + public List getDictionary() + { + return dictionary; + } + @Override public ByteBuffer toByteBuffer(RowBasedKey key) { @@ -1002,17 +1088,7 @@ public RowBasedKey fromByteBuffer(ByteBuffer buffer, int position) public Grouper.BufferComparator bufferComparator() { if (sortableIds == null) { - final int dictionarySize = dictionary.size(); - final Pair[] dictAndIds = new Pair[dictionarySize]; - for (int id = 0; id < dictionarySize; id++) { - dictAndIds[id] = new Pair<>(dictionary.get(id), id); - } - Arrays.sort(dictAndIds, Comparator.comparing(pair -> pair.lhs)); - - sortableIds = new int[dictionarySize]; - for (int i = 0; i < dictionarySize; i++) { - sortableIds[dictAndIds[i].rhs] = i; - } + initializeSortableIds(); } if (includeTimestamp) { @@ -1024,7 +1100,6 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, { final int cmp = compareDimsInBuffersForNullFudgeTimestamp( serdeHelpers, - sortableIds, dimCount, lhsBuffer, rhsBuffer, @@ -1052,7 +1127,6 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, return compareDimsInBuffersForNullFudgeTimestamp( serdeHelpers, - sortableIds, dimCount, lhsBuffer, rhsBuffer, @@ -1113,35 +1187,19 @@ public Grouper.BufferComparator bufferComparatorWithAggregators( int aggIndex = OrderByColumnSpec.getAggIndexForOrderBy(orderSpec, Arrays.asList(aggregatorFactories)); if (aggIndex >= 0) { final RowBasedKeySerdeHelper serdeHelper; - final StringComparator cmp = orderSpec.getDimensionComparator(); - final boolean cmpIsNumeric = cmp == StringComparators.NUMERIC; + final StringComparator stringComparator = orderSpec.getDimensionComparator(); final String typeName = aggregatorFactories[aggIndex].getTypeName(); final int aggOffset = aggregatorOffsets[aggIndex] - Ints.BYTES; aggCount++; - if (typeName.equals("long")) { - if (cmpIsNumeric) { - serdeHelper = new LongRowBasedKeySerdeHelper(aggOffset); - } else { - serdeHelper = new LimitPushDownLongRowBasedKeySerdeHelper(aggOffset, cmp); - } - } else if (typeName.equals("float")) { - if (cmpIsNumeric) { - serdeHelper = new FloatRowBasedKeySerdeHelper(aggOffset); - } else { - serdeHelper = new LimitPushDownFloatRowBasedKeySerdeHelper(aggOffset, cmp); - } - } else if (typeName.equals("double")) { - if (cmpIsNumeric) { - serdeHelper = new DoubleRowBasedKeySerdeHelper(aggOffset); - } else { - serdeHelper = new LimitPushDownDoubleRowBasedKeySerdeHelper(aggOffset, cmp); - } - } else { + final ValueType valueType = ValueType.fromString(typeName); + if (!ValueType.isNumeric(valueType)) { throw new IAE("Cannot order by a non-numeric aggregator[%s]", orderSpec); } + serdeHelper = makeNumericSerdeHelper(valueType, aggOffset, true, stringComparator); + orderByHelpers.add(serdeHelper); needsReverses.add(needsReverse); } @@ -1244,98 +1302,15 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, } } - private static int compareDimsInBuffersForNullFudgeTimestamp( - List serdeHelpers, - int[] sortableIds, - int dimCount, - ByteBuffer lhsBuffer, - ByteBuffer rhsBuffer, - int lhsPosition, - int rhsPosition - ) - { - for (int i = 0; i < dimCount; i++) { - final int cmp = serdeHelpers.get(i).compare( - lhsBuffer, - rhsBuffer, - lhsPosition + Longs.BYTES, - rhsPosition + Longs.BYTES - ); - if (cmp != 0) { - return cmp; - } - } - - return 0; - } - - private static int compareDimsInBuffersForNullFudgeTimestampForPushDown( - List serdeHelpers, - List needsReverses, - int dimCount, - ByteBuffer lhsBuffer, - ByteBuffer rhsBuffer, - int lhsPosition, - int rhsPosition - ) - { - for (int i = 0; i < dimCount; i++) { - final int cmp; - if (needsReverses.get(i)) { - cmp = serdeHelpers.get(i).compare( - rhsBuffer, - lhsBuffer, - rhsPosition + Longs.BYTES, - lhsPosition + Longs.BYTES - ); - } else { - cmp = serdeHelpers.get(i).compare( - lhsBuffer, - rhsBuffer, - lhsPosition + Longs.BYTES, - rhsPosition + Longs.BYTES - ); - } - 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; + if (enableRuntimeDictionaryGeneration) { + dictionary.clear(); + reverseDictionary.clear(); + sortableIds = null; + currentEstimatedSize = 0; } - return idx; } private int getTotalKeySize() @@ -1347,143 +1322,127 @@ private int getTotalKeySize() return size; } - private List makeSerdeHelpers() + private List makeSerdeHelpers( + boolean pushLimitDown, + boolean enableRuntimeDictionaryGeneration + ) { - if (limitSpec != null) { - return makeSerdeHelpersForLimitPushDown(); - } - - List helpers = new ArrayList<>(); + final List helpers = new ArrayList<>(); int keyBufferPosition = 0; - for (ValueType valType : valueTypes) { - RowBasedKeySerdeHelper helper; - switch (valType) { - case STRING: - helper = new StringRowBasedKeySerdeHelper(keyBufferPosition); - break; - case LONG: - helper = new LongRowBasedKeySerdeHelper(keyBufferPosition); - break; - case FLOAT: - helper = new FloatRowBasedKeySerdeHelper(keyBufferPosition); - break; - case DOUBLE: - helper = new DoubleRowBasedKeySerdeHelper(keyBufferPosition); - break; - default: - throw new IAE("invalid type: %s", valType); + + for (int i = 0; i < dimCount; i++) { + final StringComparator stringComparator; + if (limitSpec != null) { + final String dimName = dimensions.get(i).getOutputName(); + stringComparator = DefaultLimitSpec.getComparatorForDimName(limitSpec, dimName); + } else { + stringComparator = null; } + + RowBasedKeySerdeHelper helper = makeSerdeHelper( + valueTypes.get(i), + keyBufferPosition, + pushLimitDown, + stringComparator, + enableRuntimeDictionaryGeneration + ); + keyBufferPosition += helper.getKeyBufferValueSize(); helpers.add(helper); } + return helpers; } - private List makeSerdeHelpersForLimitPushDown() + private RowBasedKeySerdeHelper makeSerdeHelper( + ValueType valueType, + int keyBufferPosition, + boolean pushLimitDown, + @Nullable StringComparator stringComparator, + boolean enableRuntimeDictionaryGeneration + ) { - List helpers = new ArrayList<>(); - int keyBufferPosition = 0; - - for (int i = 0; i < valueTypes.size(); i++) { - final ValueType valType = valueTypes.get(i); - final String dimName = dimensions.get(i).getOutputName(); - StringComparator cmp = DefaultLimitSpec.getComparatorForDimName(limitSpec, dimName); - final boolean cmpIsNumeric = cmp == StringComparators.NUMERIC; - - RowBasedKeySerdeHelper helper; - switch (valType) { - case STRING: - if (cmp == null) { - cmp = StringComparators.LEXICOGRAPHIC; - } - helper = new LimitPushDownStringRowBasedKeySerdeHelper(keyBufferPosition, cmp); - break; - case LONG: - if (cmp == null || cmpIsNumeric) { - helper = new LongRowBasedKeySerdeHelper(keyBufferPosition); - } else { - helper = new LimitPushDownLongRowBasedKeySerdeHelper(keyBufferPosition, cmp); - } - break; - case FLOAT: - if (cmp == null || cmpIsNumeric) { - helper = new FloatRowBasedKeySerdeHelper(keyBufferPosition); - } else { - helper = new LimitPushDownFloatRowBasedKeySerdeHelper(keyBufferPosition, cmp); - } - break; - case DOUBLE: - if (cmp == null || cmpIsNumeric) { - helper = new DoubleRowBasedKeySerdeHelper(keyBufferPosition); - } else { - helper = new LimitPushDownDoubleRowBasedKeySerdeHelper(keyBufferPosition, cmp); - } - break; - default: - throw new IAE("invalid type: %s", valType); - } - keyBufferPosition += helper.getKeyBufferValueSize(); - helpers.add(helper); + switch (valueType) { + case STRING: + if (enableRuntimeDictionaryGeneration) { + return new DynamicDictionaryStringRowBasedKeySerdeHelper( + keyBufferPosition, + pushLimitDown, + stringComparator + ); + } else { + return new StaticDictionaryStringRowBasedKeySerdeHelper( + keyBufferPosition, + pushLimitDown, + stringComparator + ); + } + case LONG: + case FLOAT: + case DOUBLE: + return makeNumericSerdeHelper(valueType, keyBufferPosition, pushLimitDown, stringComparator); + default: + throw new IAE("invalid type: %s", valueType); } - return helpers; } - private interface RowBasedKeySerdeHelper + private RowBasedKeySerdeHelper makeNumericSerdeHelper( + ValueType valueType, + int keyBufferPosition, + boolean pushLimitDown, + @Nullable StringComparator stringComparator + ) { - /** - * @return The size in bytes for a value of the column handled by this SerdeHelper. - */ - int getKeyBufferValueSize(); - - /** - * Read a value from RowBasedKey at `idx` and put the value at the current position of RowBasedKeySerde's keyBuffer. - * advancing the position by the size returned by getKeyBufferValueSize(). - * - * If an internal resource limit has been reached and the value could not be added to the keyBuffer, - * (e.g., maximum dictionary size exceeded for Strings), this method returns false. - * - * @param key RowBasedKey containing the grouping key values for a row. - * @param idx Index of the grouping key column within that this SerdeHelper handles - * - * @return true if the value was added to the key, false otherwise - */ - boolean putToKeyBuffer(RowBasedKey key, int idx); + switch (valueType) { + case LONG: + return new LongRowBasedKeySerdeHelper(keyBufferPosition, pushLimitDown, stringComparator); + case FLOAT: + return new FloatRowBasedKeySerdeHelper(keyBufferPosition, pushLimitDown, stringComparator); + case DOUBLE: + return new DoubleRowBasedKeySerdeHelper(keyBufferPosition, pushLimitDown, stringComparator); + default: + throw new IAE("invalid type: %s", valueType); + } + } - /** - * Read a value from a ByteBuffer containing a grouping key in the same format as RowBasedKeySerde's keyBuffer and - * put the value in `dimValues` at `dimValIdx`. - * - * The value to be read resides in the buffer at position (`initialOffset` + the SerdeHelper's keyBufferPosition). - * - * @param buffer ByteBuffer containing an array of grouping keys for a row - * @param initialOffset Offset where non-timestamp grouping key columns start, needed because timestamp is not - * always included in the buffer. - * @param dimValIdx Index within dimValues to store the value read from the buffer - * @param dimValues Output array containing grouping key values for a row - */ - void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int dimValIdx, Comparable[] dimValues); + static long estimateStringKeySize(String key) + { + return (long) key.length() * Chars.BYTES + ROUGH_OVERHEAD_PER_DICTIONARY_ENTRY; + } - /** - * Compare the values at lhsBuffer[lhsPosition] and rhsBuffer[rhsPosition] using the natural ordering - * for this SerdeHelper's value type. - * - * @param lhsBuffer ByteBuffer containing an array of grouping keys for a row - * @param rhsBuffer ByteBuffer containing an array of grouping keys for a row - * @param lhsPosition Position of value within lhsBuffer - * @param rhsPosition Position of value within rhsBuffer - * - * @return Negative number if lhs < rhs, positive if lhs > rhs, 0 if lhs == rhs - */ - int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition); + private static boolean isPrimitiveComparable(boolean pushLimitDown, @Nullable StringComparator stringComparator) + { + return !pushLimitDown || stringComparator == null || stringComparator == StringComparators.NUMERIC; } - private class StringRowBasedKeySerdeHelper implements RowBasedKeySerdeHelper + private abstract class AbstractStringRowBasedKeySerdeHelper implements RowBasedKeySerdeHelper { - final int keyBufferPosition; + protected final int keyBufferPosition; + + protected final BufferComparator bufferComparator; - public StringRowBasedKeySerdeHelper(int keyBufferPosition) + protected AbstractStringRowBasedKeySerdeHelper( + int keyBufferPosition, + boolean pushLimitDown, + @Nullable StringComparator stringComparator + ) { this.keyBufferPosition = keyBufferPosition; + if (!pushLimitDown) { + bufferComparator = (lhsBuffer, rhsBuffer, lhsPosition, rhsPosition) -> Ints.compare( + sortableIds[lhsBuffer.getInt(lhsPosition + keyBufferPosition)], + sortableIds[rhsBuffer.getInt(rhsPosition + keyBufferPosition)] + ); + } else { + final StringComparator realComparator = stringComparator == null ? + StringComparators.LEXICOGRAPHIC : + stringComparator; + bufferComparator = (lhsBuffer, rhsBuffer, lhsPosition, rhsPosition) -> { + String lhsStr = dictionary.get(lhsBuffer.getInt(lhsPosition + keyBufferPosition)); + String rhsStr = dictionary.get(rhsBuffer.getInt(rhsPosition + keyBufferPosition)); + return realComparator.compare(lhsStr, rhsStr); + }; + } } @Override @@ -1492,6 +1451,30 @@ public int getKeyBufferValueSize() return Ints.BYTES; } + @Override + public void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int dimValIdx, Comparable[] dimValues) + { + dimValues[dimValIdx] = dictionary.get(buffer.getInt(initialOffset + keyBufferPosition)); + } + + @Override + public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) + { + return bufferComparator.compare(lhsBuffer, rhsBuffer, lhsPosition, rhsPosition); + } + } + + private class DynamicDictionaryStringRowBasedKeySerdeHelper extends AbstractStringRowBasedKeySerdeHelper + { + DynamicDictionaryStringRowBasedKeySerdeHelper( + int keyBufferPosition, + boolean pushLimitDown, + @Nullable StringComparator stringComparator + ) + { + super(keyBufferPosition, pushLimitDown, stringComparator); + } + @Override public boolean putToKeyBuffer(RowBasedKey key, int idx) { @@ -1503,48 +1486,82 @@ public boolean putToKeyBuffer(RowBasedKey key, int idx) return true; } - @Override - public void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int dimValIdx, Comparable[] dimValues) + /** + * 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) { - dimValues[dimValIdx] = dictionary.get(buffer.getInt(initialOffset + keyBufferPosition)); - } + int idx = reverseDictionary.getInt(s); + if (idx == UNKNOWN_DICTIONARY_KEY) { + final long additionalEstimatedSize = estimateStringKeySize(s); + if (currentEstimatedSize + additionalEstimatedSize > maxDictionarySize) { + return -1; + } - @Override - public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) - { - return Ints.compare( - sortableIds[lhsBuffer.getInt(lhsPosition + keyBufferPosition)], - sortableIds[rhsBuffer.getInt(rhsPosition + keyBufferPosition)] - ); + idx = dictionary.size(); + reverseDictionary.put(s, idx); + dictionary.add(s); + currentEstimatedSize += additionalEstimatedSize; + } + return idx; } } - private class LimitPushDownStringRowBasedKeySerdeHelper extends StringRowBasedKeySerdeHelper + private class StaticDictionaryStringRowBasedKeySerdeHelper extends AbstractStringRowBasedKeySerdeHelper { - final StringComparator cmp; - - public LimitPushDownStringRowBasedKeySerdeHelper(int keyBufferPosition, StringComparator cmp) + StaticDictionaryStringRowBasedKeySerdeHelper( + int keyBufferPosition, + boolean pushLimitDown, + @Nullable StringComparator stringComparator + ) { - super(keyBufferPosition); - this.cmp = cmp; + super(keyBufferPosition, pushLimitDown, stringComparator); } @Override - public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) + public boolean putToKeyBuffer(RowBasedKey key, int idx) { - String lhsStr = dictionary.get(lhsBuffer.getInt(lhsPosition + keyBufferPosition)); - String rhsStr = dictionary.get(rhsBuffer.getInt(rhsPosition + keyBufferPosition)); - return cmp.compare(lhsStr, rhsStr); + final String stringKey = (String) key.getKey()[idx]; + + final int dictIndex = reverseDictionary.getInt(stringKey); + if (dictIndex == UNKNOWN_DICTIONARY_KEY) { + throw new ISE("Cannot find key[%s] from dictionary", stringKey); + } + keyBuffer.putInt(dictIndex); + return true; } } private class LongRowBasedKeySerdeHelper implements RowBasedKeySerdeHelper { final int keyBufferPosition; + final BufferComparator bufferComparator; - public LongRowBasedKeySerdeHelper(int keyBufferPosition) + LongRowBasedKeySerdeHelper( + int keyBufferPosition, + boolean pushLimitDown, + @Nullable StringComparator stringComparator + ) { this.keyBufferPosition = keyBufferPosition; + if (isPrimitiveComparable(pushLimitDown, stringComparator)) { + bufferComparator = (lhsBuffer, rhsBuffer, lhsPosition, rhsPosition) -> Longs.compare( + lhsBuffer.getLong(lhsPosition + keyBufferPosition), + rhsBuffer.getLong(rhsPosition + keyBufferPosition) + ); + } else { + bufferComparator = (lhsBuffer, rhsBuffer, lhsPosition, rhsPosition) -> { + long lhs = lhsBuffer.getLong(lhsPosition + keyBufferPosition); + long rhs = rhsBuffer.getLong(rhsPosition + keyBufferPosition); + + return stringComparator.compare(String.valueOf(lhs), String.valueOf(rhs)); + }; + } } @Override @@ -1569,40 +1586,33 @@ public void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int dimValId @Override public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) { - return Longs.compare( - lhsBuffer.getLong(lhsPosition + keyBufferPosition), - rhsBuffer.getLong(rhsPosition + keyBufferPosition) - ); - } - } - - private class LimitPushDownLongRowBasedKeySerdeHelper extends LongRowBasedKeySerdeHelper - { - final StringComparator cmp; - - public LimitPushDownLongRowBasedKeySerdeHelper(int keyBufferPosition, StringComparator cmp) - { - super(keyBufferPosition); - this.cmp = cmp; - } - - @Override - public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) - { - long lhs = lhsBuffer.getLong(lhsPosition + keyBufferPosition); - long rhs = rhsBuffer.getLong(rhsPosition + keyBufferPosition); - - return cmp.compare(String.valueOf(lhs), String.valueOf(rhs)); + return bufferComparator.compare(lhsBuffer, rhsBuffer, lhsPosition, rhsPosition); } } private class FloatRowBasedKeySerdeHelper implements RowBasedKeySerdeHelper { final int keyBufferPosition; + final BufferComparator bufferComparator; - public FloatRowBasedKeySerdeHelper(int keyBufferPosition) + FloatRowBasedKeySerdeHelper( + int keyBufferPosition, + boolean pushLimitDown, + @Nullable StringComparator stringComparator) { this.keyBufferPosition = keyBufferPosition; + if (isPrimitiveComparable(pushLimitDown, stringComparator)) { + bufferComparator = (lhsBuffer, rhsBuffer, lhsPosition, rhsPosition) -> Float.compare( + lhsBuffer.getFloat(lhsPosition + keyBufferPosition), + rhsBuffer.getFloat(rhsPosition + keyBufferPosition) + ); + } else { + bufferComparator = (lhsBuffer, rhsBuffer, lhsPosition, rhsPosition) -> { + float lhs = lhsBuffer.getFloat(lhsPosition + keyBufferPosition); + float rhs = rhsBuffer.getFloat(rhsPosition + keyBufferPosition); + return stringComparator.compare(String.valueOf(lhs), String.valueOf(rhs)); + }; + } } @Override @@ -1627,39 +1637,34 @@ public void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int dimValId @Override public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) { - return Float.compare( - lhsBuffer.getFloat(lhsPosition + keyBufferPosition), - rhsBuffer.getFloat(rhsPosition + keyBufferPosition) - ); - } - } - - private class LimitPushDownFloatRowBasedKeySerdeHelper extends FloatRowBasedKeySerdeHelper - { - final StringComparator cmp; - - public LimitPushDownFloatRowBasedKeySerdeHelper(int keyBufferPosition, StringComparator cmp) - { - super(keyBufferPosition); - this.cmp = cmp; - } - - @Override - public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) - { - float lhs = lhsBuffer.getFloat(lhsPosition + keyBufferPosition); - float rhs = rhsBuffer.getFloat(rhsPosition + keyBufferPosition); - return cmp.compare(String.valueOf(lhs), String.valueOf(rhs)); + return bufferComparator.compare(lhsBuffer, rhsBuffer, lhsPosition, rhsPosition); } } private class DoubleRowBasedKeySerdeHelper implements RowBasedKeySerdeHelper { final int keyBufferPosition; + final BufferComparator bufferComparator; - public DoubleRowBasedKeySerdeHelper(int keyBufferPosition) + DoubleRowBasedKeySerdeHelper( + int keyBufferPosition, + boolean pushLimitDown, + @Nullable StringComparator stringComparator + ) { this.keyBufferPosition = keyBufferPosition; + if (isPrimitiveComparable(pushLimitDown, stringComparator)) { + bufferComparator = (lhsBuffer, rhsBuffer, lhsPosition, rhsPosition) -> Double.compare( + lhsBuffer.getDouble(lhsPosition + keyBufferPosition), + rhsBuffer.getDouble(rhsPosition + keyBufferPosition) + ); + } else { + bufferComparator = (lhsBuffer, rhsBuffer, lhsPosition, rhsPosition) -> { + double lhs = lhsBuffer.getDouble(lhsPosition + keyBufferPosition); + double rhs = rhsBuffer.getDouble(rhsPosition + keyBufferPosition); + return stringComparator.compare(String.valueOf(lhs), String.valueOf(rhs)); + }; + } } @Override @@ -1684,30 +1689,67 @@ public void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int dimValId @Override public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) { - return Double.compare( - lhsBuffer.getDouble(lhsPosition + keyBufferPosition), - rhsBuffer.getDouble(rhsPosition + keyBufferPosition) - ); + return bufferComparator.compare(lhsBuffer, rhsBuffer, lhsPosition, rhsPosition); } } + } - private class LimitPushDownDoubleRowBasedKeySerdeHelper extends DoubleRowBasedKeySerdeHelper - { - final StringComparator cmp; - - public LimitPushDownDoubleRowBasedKeySerdeHelper(int keyBufferPosition, StringComparator cmp) - { - super(keyBufferPosition); - this.cmp = cmp; + static int compareDimsInBuffersForNullFudgeTimestamp( + List serdeHelpers, + int dimCount, + ByteBuffer lhsBuffer, + ByteBuffer rhsBuffer, + int lhsPosition, + int rhsPosition + ) + { + for (int i = 0; i < dimCount; i++) { + final int cmp = serdeHelpers.get(i).compare( + lhsBuffer, + rhsBuffer, + lhsPosition + Longs.BYTES, + rhsPosition + Longs.BYTES + ); + if (cmp != 0) { + return cmp; } + } - @Override - public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) - { - double lhs = lhsBuffer.getDouble(lhsPosition + keyBufferPosition); - double rhs = rhsBuffer.getDouble(rhsPosition + keyBufferPosition); - return cmp.compare(String.valueOf(lhs), String.valueOf(rhs)); + return 0; + } + + static int compareDimsInBuffersForNullFudgeTimestampForPushDown( + List serdeHelpers, + List needsReverses, + int dimCount, + ByteBuffer lhsBuffer, + ByteBuffer rhsBuffer, + int lhsPosition, + int rhsPosition + ) + { + for (int i = 0; i < dimCount; i++) { + final int cmp; + if (needsReverses.get(i)) { + cmp = serdeHelpers.get(i).compare( + rhsBuffer, + lhsBuffer, + rhsPosition + Longs.BYTES, + lhsPosition + Longs.BYTES + ); + } else { + cmp = serdeHelpers.get(i).compare( + lhsBuffer, + rhsBuffer, + lhsPosition + Longs.BYTES, + rhsPosition + Longs.BYTES + ); + } + if (cmp != 0) { + return cmp; } } + + return 0; } } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedKeySerdeHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedKeySerdeHelper.java new file mode 100644 index 000000000000..500d7924715d --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedKeySerdeHelper.java @@ -0,0 +1,73 @@ +/* + * 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.query.groupby.epinephelinae.RowBasedGrouperHelper.RowBasedKey; + +import java.nio.ByteBuffer; + +interface RowBasedKeySerdeHelper +{ + /** + * @return The size in bytes for a value of the column handled by this SerdeHelper. + */ + int getKeyBufferValueSize(); + + /** + * Read a value from RowBasedKey at `idx` and put the value at the current position of RowBasedKeySerde's keyBuffer. + * advancing the position by the size returned by getKeyBufferValueSize(). + * + * If an internal resource limit has been reached and the value could not be added to the keyBuffer, + * (e.g., maximum dictionary size exceeded for Strings), this method returns false. + * + * @param key RowBasedKey containing the grouping key values for a row. + * @param idx Index of the grouping key column within that this SerdeHelper handles + * + * @return true if the value was added to the key, false otherwise + */ + boolean putToKeyBuffer(RowBasedKey key, int idx); + + /** + * Read a value from a ByteBuffer containing a grouping key in the same format as RowBasedKeySerde's keyBuffer and + * put the value in `dimValues` at `dimValIdx`. + * + * The value to be read resides in the buffer at position (`initialOffset` + the SerdeHelper's keyBufferPosition). + * + * @param buffer ByteBuffer containing an array of grouping keys for a row + * @param initialOffset Offset where non-timestamp grouping key columns start, needed because timestamp is not + * always included in the buffer. + * @param dimValIdx Index within dimValues to store the value read from the buffer + * @param dimValues Output array containing grouping key values for a row + */ + void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int dimValIdx, Comparable[] dimValues); + + /** + * Compare the values at lhsBuffer[lhsPosition] and rhsBuffer[rhsPosition] using the natural ordering + * for this SerdeHelper's value type. + * + * @param lhsBuffer ByteBuffer containing an array of grouping keys for a row + * @param rhsBuffer ByteBuffer containing an array of grouping keys for a row + * @param lhsPosition Position of value within lhsBuffer + * @param rhsPosition Position of value within rhsBuffer + * + * @return Negative number if lhs < rhs, positive if lhs > rhs, 0 if lhs == rhs + */ + int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition); +} 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 b94de4471e66..571750a85b60 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 @@ -42,8 +42,10 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Comparator; +import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Set; /** * Grouper based around a single underlying {@link BufferHashGrouper}. Not thread-safe. @@ -64,6 +66,7 @@ public class SpillingGrouper implements Grouper private final Comparator> defaultOrderKeyObjComparator; private final List files = Lists.newArrayList(); + private final List dictionaryFiles = Lists.newArrayList(); private final List closeables = Lists.newArrayList(); private final boolean sortHasNonGroupingFields; @@ -167,6 +170,30 @@ public void close() deleteFiles(); } + public List getDictionary() + { + final Set mergedDictionary = new HashSet<>(); + mergedDictionary.addAll(keySerde.getDictionary()); + + for (File dictFile : dictionaryFiles) { + try { + final MappingIterator dictIterator = spillMapper.readValues( + spillMapper.getFactory().createParser(new LZ4BlockInputStream(new FileInputStream(dictFile))), + spillMapper.getTypeFactory().constructType(String.class) + ); + + while (dictIterator.hasNext()) { + mergedDictionary.add(dictIterator.next()); + } + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + return new ArrayList<>(mergedDictionary); + } + public void setSpillingAllowed(final boolean spillingAllowed) { this.spillingAllowed = spillingAllowed; @@ -214,24 +241,27 @@ public Entry apply(Entry entry) private void spill() throws IOException { - final File outFile; + files.add(spill(grouper.iterator(true))); + dictionaryFiles.add(spill(keySerde.getDictionary().iterator())); + + grouper.reset(); + } + private File spill(Iterator iterator) throws IOException + { try ( final LimitedTemporaryStorage.LimitedOutputStream out = temporaryStorage.createFile(); final LZ4BlockOutputStream compressedOut = new LZ4BlockOutputStream(out); final JsonGenerator jsonGenerator = spillMapper.getFactory().createGenerator(compressedOut) ) { - outFile = out.getFile(); - final Iterator> it = grouper.iterator(true); - while (it.hasNext()) { + while (iterator.hasNext()) { BaseQuery.checkInterrupted(); - jsonGenerator.writeObject(it.next()); + jsonGenerator.writeObject(iterator.next()); } - } - files.add(outFile); - grouper.reset(); + return out.getFile(); + } } private MappingIterator> read(final File file, final Class keyClazz) diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java index d9999b0f1cc3..7a684f036de7 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java @@ -170,32 +170,33 @@ public AggregateResult aggregate(KeyType key, int notUsed) @Override public AggregateResult aggregate(KeyType key) { - final ByteBuffer keyBuffer = keySerde.toByteBuffer(key); - if (keyBuffer == null) { - // This should bubble up to the user, so call finish() here. - finish(); - return DICTIONARY_FULL; - } + try { + final ByteBuffer keyBuffer = keySerde.toByteBuffer(key); + + if (keyBuffer.remaining() != keySize) { + throw new IAE( + "keySerde.toByteBuffer(key).remaining[%s] != keySerde.keySize[%s], buffer was the wrong size?!", + keyBuffer.remaining(), + keySize + ); + } - if (keyBuffer.remaining() != keySize) { - throw new IAE( - "keySerde.toByteBuffer(key).remaining[%s] != keySerde.keySize[%s], buffer was the wrong size?!", - keyBuffer.remaining(), - keySize - ); - } + final int prevRecordOffset = curWriteIndex * recordSize; + if (curWriteIndex == -1 || !keyEquals(keyBuffer, buffer, prevRecordOffset)) { + initNewSlot(keyBuffer); + } - final int prevRecordOffset = curWriteIndex * recordSize; - if (curWriteIndex == -1 || !keyEquals(keyBuffer, buffer, prevRecordOffset)) { - initNewSlot(keyBuffer); - } + final int curRecordOffset = curWriteIndex * recordSize; + for (int i = 0; i < aggregatorOffsets.length; i++) { + aggregators[i].aggregate(buffer, curRecordOffset + aggregatorOffsets[i]); + } - final int curRecordOffset = curWriteIndex * recordSize; - for (int i = 0; i < aggregatorOffsets.length; i++) { - aggregators[i].aggregate(buffer, curRecordOffset + aggregatorOffsets[i]); + return AggregateResult.ok(); + } + catch (Throwable t) { + finish(); + throw t; } - - return AggregateResult.ok(); } private boolean keyEquals(ByteBuffer curKeyBuffer, ByteBuffer buffer, int bufferOffset) diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java index ed0fd0635e7a..e013d6dc461b 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java @@ -20,6 +20,7 @@ package io.druid.query.groupby.epinephelinae; import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.primitives.Longs; import com.google.common.util.concurrent.MoreExecutors; @@ -122,6 +123,12 @@ public void close() private static final KeySerdeFactory keySerdeFactory = new KeySerdeFactory() { + @Override + public long getMaxDictionarySize() + { + return 0; + } + @Override public KeySerde factorize() { @@ -141,6 +148,12 @@ public Class keyClazz() return Long.class; } + @Override + public List getDictionary() + { + return ImmutableList.of(); + } + @Override public ByteBuffer toByteBuffer(Long key) { @@ -183,6 +196,12 @@ public void reset() {} }; } + @Override + public KeySerde factorizeWithDictionary(List dictionary) + { + return factorize(); + } + @Override public Comparator> objectComparator(boolean forceDefaultOrder) { @@ -243,6 +262,7 @@ public void testAggregate() throws InterruptedException, ExecutionException, IOE bufferSupplier, combineBufferSupplier, keySerdeFactory, + keySerdeFactory, null_factory, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, 24, 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 index 8f017caff953..387e14d8406a 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/IntKeySerde.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/IntKeySerde.java @@ -19,11 +19,13 @@ package io.druid.query.groupby.epinephelinae; +import com.google.common.collect.ImmutableList; import com.google.common.primitives.Ints; import io.druid.query.aggregation.AggregatorFactory; import java.nio.ByteBuffer; import java.util.Comparator; +import java.util.List; public class IntKeySerde implements Grouper.KeySerde { @@ -66,6 +68,12 @@ public Class keyClazz() return Integer.class; } + @Override + public List getDictionary() + { + return ImmutableList.of(); + } + @Override public ByteBuffer toByteBuffer(Integer key) { From 21d42ceee230a27afd4b2915abe7f4103e02d835 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 25 Aug 2017 20:55:02 +0900 Subject: [PATCH 04/44] Revert GroupByBenchmark --- .../io/druid/benchmark/query/GroupByBenchmark.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) 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 32953a83834d..20f0e46c12bb 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java @@ -116,10 +116,10 @@ @Measurement(iterations = 30) public class GroupByBenchmark { - @Param({"8"}) + @Param({"4"}) private int numSegments; - @Param({"8"}) + @Param({"2", "4"}) private int numProcessingThreads; @Param({"-1"}) @@ -128,13 +128,13 @@ public class GroupByBenchmark @Param({"100000"}) private int rowsPerSegment; - @Param({"basic.A"}) + @Param({"basic.A", "basic.nested"}) private String schemaAndQuery; - @Param({"v2"}) + @Param({"v1", "v2"}) private String defaultStrategy; - @Param({"all"}) + @Param({"all", "day"}) private String queryGranularity; private static final Logger log = new Logger(GroupByBenchmark.class); From 701adb10ccaccafc599c7f3a029a89bb2b356ae1 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 25 Aug 2017 20:58:50 +0900 Subject: [PATCH 05/44] Removed unused code --- .../groupby/epinephelinae/StreamingMergeSortedGrouper.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java index 7a684f036de7..0f1a674df532 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java @@ -47,12 +47,6 @@ public class StreamingMergeSortedGrouper implements Grouper // thread from being blocked if its iterator is not consumed due to some failure. private static final long DEFAULT_TIMEOUT_MS = 5000L; - private static final AggregateResult DICTIONARY_FULL = AggregateResult.failure( - "Not enough dictionary space to execute this query. Try increasing " - + "druid.query.groupBy.maxMergingDictionarySize or use single-threaded combine by setting " - + "druid.query.groupBy.forceSingleThreadedCombine to true." - ); - private final Supplier bufferSupplier; private final KeySerde keySerde; private final BufferAggregator[] aggregators; From 68ccece4aeca98e78fbf02bbbe90cf20ad923bbe Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 26 Aug 2017 10:03:04 +0900 Subject: [PATCH 06/44] More cleanup --- docs/content/querying/groupbyquery.md | 1 - .../query/groupby/GroupByQueryConfig.java | 13 ------ .../epinephelinae/ConcurrentGrouper.java | 41 +++++++++---------- .../epinephelinae/RowBasedGrouperHelper.java | 34 +++++++-------- .../query/groupby/GroupByQueryRunnerTest.java | 6 --- .../epinephelinae/ConcurrentGrouperTest.java | 3 +- 6 files changed, 38 insertions(+), 60 deletions(-) diff --git a/docs/content/querying/groupbyquery.md b/docs/content/querying/groupbyquery.md index babc9a7f0c7a..fc323217ec06 100644 --- a/docs/content/querying/groupbyquery.md +++ b/docs/content/querying/groupbyquery.md @@ -250,7 +250,6 @@ When using the "v2" strategy, the following query context parameters apply: |`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.| -|`forceSingleThreadedCombine`|Force to use single-threaded combine.| When using the "v1" strategy, the following query context parameters apply: 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 b3d84c794b13..6b208dbc1ed5 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java @@ -76,9 +76,6 @@ public class GroupByQueryConfig @JsonProperty private boolean forceHashAggregation = false; - @JsonProperty - private boolean forceSingleThreadedCombine = false; - public String getDefaultStrategy() { return defaultStrategy; @@ -149,11 +146,6 @@ public boolean isForceHashAggregation() return forceHashAggregation; } - public boolean isForceSingleThreadedCombine() - { - return forceSingleThreadedCombine; - } - public GroupByQueryConfig withOverrides(final GroupByQuery query) { final GroupByQueryConfig newConfig = new GroupByQueryConfig(); @@ -189,10 +181,6 @@ public GroupByQueryConfig withOverrides(final GroupByQuery query) ); newConfig.forcePushDownLimit = query.getContextBoolean(CTX_KEY_FORCE_LIMIT_PUSH_DOWN, isForcePushDownLimit()); newConfig.forceHashAggregation = query.getContextBoolean(CTX_KEY_FORCE_HASH_AGGREGATION, isForceHashAggregation()); - newConfig.forceSingleThreadedCombine = query.getContextBoolean( - CTX_KEY_FORCE_SINGLE_THREADED_COMBINE, - isForceSingleThreadedCombine() - ); return newConfig; } @@ -211,7 +199,6 @@ public String toString() ", maxOnDiskStorage=" + maxOnDiskStorage + ", forcePushDownLimit=" + forcePushDownLimit + ", forceHashAggregation=" + forceHashAggregation + - ", forceSingleThreadedCombine=" + forceSingleThreadedCombine + '}'; } } 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 593c4ddced17..021ff7be48f5 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 @@ -32,10 +32,8 @@ import io.druid.java.util.common.Pair; import io.druid.query.AbstractPrioritizedCallable; import io.druid.query.QueryInterruptedException; -import io.druid.query.ResourceLimitExceededException; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.dimension.DimensionSpec; -import io.druid.query.groupby.epinephelinae.RowBasedGrouperHelper.RowBasedKeySerde; import io.druid.query.groupby.orderby.DefaultLimitSpec; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnSelectorFactory; @@ -94,7 +92,7 @@ public class ConcurrentGrouper implements Grouper private final ObjectMapper spillMapper; private final int concurrencyHint; private final KeySerdeFactory keySerdeFactory; - private final KeySerdeFactory mergingKeySerdeFactory; + private final KeySerdeFactory combineKeySerdeFactory; private final DefaultLimitSpec limitSpec; private final boolean sortHasNonGroupingFields; private final Comparator> keyObjComparator; @@ -102,7 +100,6 @@ public class ConcurrentGrouper implements Grouper private final int priority; private final boolean hasQueryTimeout; private final long queryTimeoutAt; - private final boolean forceSingleThreadedCombine; private volatile boolean initialized = false; @@ -110,7 +107,7 @@ public ConcurrentGrouper( final Supplier bufferSupplier, final Supplier> combineBufferSupplier, final KeySerdeFactory keySerdeFactory, - final KeySerdeFactory mergingKeySerdeFactory, + final KeySerdeFactory combineKeySerdeFactory, final ColumnSelectorFactory columnSelectorFactory, final AggregatorFactory[] aggregatorFactories, final int bufferGrouperMaxSize, @@ -124,8 +121,7 @@ public ConcurrentGrouper( final ListeningExecutorService grouperSorter, final int priority, final boolean hasQueryTimeout, - final long queryTimeoutAt, - final boolean forceSingleThreadedCombine + final long queryTimeoutAt ) { Preconditions.checkArgument(concurrencyHint > 0, "concurrencyHint > 0"); @@ -151,7 +147,7 @@ protected SpillingGrouper initialValue() this.spillMapper = spillMapper; this.concurrencyHint = concurrencyHint; this.keySerdeFactory = keySerdeFactory; - this.mergingKeySerdeFactory = mergingKeySerdeFactory; + this.combineKeySerdeFactory = combineKeySerdeFactory; this.limitSpec = limitSpec; this.sortHasNonGroupingFields = sortHasNonGroupingFields; this.keyObjComparator = keySerdeFactory.objectComparator(sortHasNonGroupingFields); @@ -159,7 +155,6 @@ protected SpillingGrouper initialValue() this.priority = priority; this.hasQueryTimeout = hasQueryTimeout; this.queryTimeoutAt = queryTimeoutAt; - this.forceSingleThreadedCombine = forceSingleThreadedCombine; } @Override @@ -268,10 +263,12 @@ public Iterator> iterator(final boolean sorted) parallelSortAndGetGroupersIterator() : getGroupersIterator(sorted); - if (!forceSingleThreadedCombine && sorted && spilling && isParallelizable()) { - // Parallel combine is used only when data are spilled. This is because ConcurrentGrouper uses two different modes - // depending on data are spilled or not. If data is not spilled, all inputs are completely aggregated and no more - // aggregation is required. + // Parallel combine is used only when data are spilled. This is because ConcurrentGrouper uses two different modes + // depending on data are spilled or not. If data is not spilled, all inputs are completely aggregated and no more + // aggregation is required. + if (sorted && spilling && isParallelizable()) { + // First try to merge dictionaries generated by all underlying groupers. If it is merged successfully, the same + // merged dictionary is used for all combining threads final List mergedDictionary = mergeDictionary(); if (mergedDictionary != null) { return parallelCombine(sortedIterators, mergedDictionary); @@ -337,13 +334,13 @@ private List>> getGroupersIterator(boolean sorted) @Nullable private List mergeDictionary() { - final long maxDictionarySize = mergingKeySerdeFactory.getMaxDictionarySize(); + final long maxDictionarySize = combineKeySerdeFactory.getMaxDictionarySize(); final Set mergedDictionary = new HashSet<>(); long totalDictionarySize = 0L; for (SpillingGrouper grouper : groupers) { final List dictionary = grouper.getDictionary(); - totalDictionarySize += dictionary.stream().mapToLong(RowBasedKeySerde::estimateStringKeySize).sum(); + totalDictionarySize += dictionary.stream().mapToLong(RowBasedGrouperHelper::estimateStringKeySize).sum(); if (totalDictionarySize > maxDictionarySize) { return null; } @@ -374,7 +371,7 @@ private Iterator> parallelCombine( combiningFactories[i] = aggregatorFactories[i].getCombiningFactory(); } final int minimumRequiredBufferCapacity = StreamingMergeSortedGrouper.requiredBufferCapacity( - mergingKeySerdeFactory.factorizeWithDictionary(mergedDictionary), + combineKeySerdeFactory.factorizeWithDictionary(mergedDictionary), combiningFactories ); final Pair degreeAndBufferNum = findCombineDegreeAndNumBuffers( @@ -439,7 +436,10 @@ public boolean hasNext() // futures should be done before reaching here and throw exceptions if they failed future.get(); } - catch (InterruptedException | ExecutionException e) { + catch (InterruptedException e) { + throw new QueryInterruptedException(e); + } + catch (ExecutionException e) { throw new RuntimeException(e); } } @@ -575,7 +575,7 @@ private StreamingMergeSortedGrouper runCombiner( new SettableColumnSelectorFactory(aggregatorFactories); final StreamingMergeSortedGrouper grouper = new StreamingMergeSortedGrouper<>( Suppliers.ofInstance(combineBuffer), - mergingKeySerdeFactory.factorizeWithDictionary(dictionary), + combineKeySerdeFactory.factorizeWithDictionary(dictionary), settableColumnSelectorFactory, combiningFactories ); @@ -586,10 +586,7 @@ private StreamingMergeSortedGrouper runCombiner( final Entry next = mergedIterator.next(); settableColumnSelectorFactory.set(next.values); - final AggregateResult result = grouper.aggregate(next.key); - if (!result.isOk()) { - throw new ResourceLimitExceededException(result.getReason()); - } + grouper.aggregate(next.key); // grouper always returns ok or throws an exception settableColumnSelectorFactory.set(null); } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index 53905ba5278f..a199c3cdfd48 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -85,6 +85,9 @@ // this class contains shared code between GroupByMergingQueryRunnerV2 and GroupByRowProcessor public class RowBasedGrouperHelper { + // 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 static final int SINGLE_THREAD_CONCURRENCY_HINT = -1; private static final int UNKNOWN_THREAD_PRIORITY = -1; private static final long UNKNOWN_TIMEOUT = -1L; @@ -196,11 +199,11 @@ public static Pair, Accumulator> crea sortHasNonGroupingFields ); } else { - final Grouper.KeySerdeFactory mergingKeySerdeFactory = new RowBasedKeySerdeFactory( + final Grouper.KeySerdeFactory combineKeySerdeFactory = new RowBasedKeySerdeFactory( includeTimestamp, query.getContextSortByDimsFirst(), query.getDimensions(), - querySpecificConfig.getMaxMergingDictionarySize(), // use entire dictionary space for one key serde + querySpecificConfig.getMaxMergingDictionarySize(), // use entire dictionary space for combining key serde valueTypes, aggregatorFactories, limitSpec @@ -210,7 +213,7 @@ public static Pair, Accumulator> crea bufferSupplier, combineBufferSupplier, keySerdeFactory, - mergingKeySerdeFactory, + combineKeySerdeFactory, columnSelectorFactory, aggregatorFactories, querySpecificConfig.getBufferGrouperMaxSize(), @@ -224,8 +227,7 @@ public static Pair, Accumulator> crea grouperSorter, priority, hasQueryTimeout, - queryTimeoutAt, - config.isForceSingleThreadedCombine() + queryTimeoutAt ); } @@ -929,10 +931,13 @@ private static int compareDimsInRowsWithAggs( } } - static class RowBasedKeySerde implements Grouper.KeySerde + static long estimateStringKeySize(String key) + { + return (long) key.length() * Chars.BYTES + ROUGH_OVERHEAD_PER_DICTIONARY_ENTRY; + } + + private static class RowBasedKeySerde 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 static final int DICTIONARY_INITIAL_CAPACITY = 10000; private static final int UNKNOWN_DICTIONARY_KEY = -1; @@ -987,7 +992,9 @@ static class RowBasedKeySerde implements Grouper.KeySerde= initialDictionarySize, "Dictionary size[%s] exceeds threshold[%s]", @@ -1405,11 +1412,6 @@ private RowBasedKeySerdeHelper makeNumericSerdeHelper( } } - static long estimateStringKeySize(String key) - { - return (long) key.length() * Chars.BYTES + ROUGH_OVERHEAD_PER_DICTIONARY_ENTRY; - } - private static boolean isPrimitiveComparable(boolean pushLimitDown, @Nullable StringComparator stringComparator) { return !pushLimitDown || stringComparator == null || stringComparator == StringComparators.NUMERIC; @@ -1694,7 +1696,7 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, } } - static int compareDimsInBuffersForNullFudgeTimestamp( + private static int compareDimsInBuffersForNullFudgeTimestamp( List serdeHelpers, int dimCount, ByteBuffer lhsBuffer, @@ -1718,7 +1720,7 @@ static int compareDimsInBuffersForNullFudgeTimestamp( return 0; } - static int compareDimsInBuffersForNullFudgeTimestampForPushDown( + private static int compareDimsInBuffersForNullFudgeTimestampForPushDown( List serdeHelpers, List needsReverses, int dimCount, 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 c7fedc08211a..ddef2a474c78 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -288,12 +288,6 @@ public long getMaxOnDiskStorage() return 10L * 1024 * 1024; } - @Override - public boolean isForceSingleThreadedCombine() - { - return true; - } - @Override public String toString() { diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java index e013d6dc461b..843e16bea43f 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java @@ -276,8 +276,7 @@ public void testAggregate() throws InterruptedException, ExecutionException, IOE MoreExecutors.listeningDecorator(service), 0, false, - 0, - false + 0 ); grouper.init(); From 29a7dfe4fcdf2f07dc78f6b6d6aa24e325568f6b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 29 Aug 2017 20:47:44 +0900 Subject: [PATCH 07/44] Remove unused config --- .../src/main/java/io/druid/query/groupby/GroupByQueryConfig.java | 1 - 1 file changed, 1 deletion(-) 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 6b208dbc1ed5..5ed063db1f44 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java @@ -38,7 +38,6 @@ public class GroupByQueryConfig 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"; - private static final String CTX_KEY_FORCE_SINGLE_THREADED_COMBINE = "forceSingleThreadedCombine"; @JsonProperty private String defaultStrategy = GroupByStrategySelector.STRATEGY_V2; From 48dc2cdd80578cbc69df81f5455cabf43e68d892 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 2 Sep 2017 14:08:34 +0900 Subject: [PATCH 08/44] Fix some typos and bugs --- .../epinephelinae/ConcurrentGrouper.java | 118 +++++++++--------- 1 file changed, 60 insertions(+), 58 deletions(-) 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 021ff7be48f5..99d721e85bd7 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 @@ -96,7 +96,7 @@ public class ConcurrentGrouper implements Grouper private final DefaultLimitSpec limitSpec; private final boolean sortHasNonGroupingFields; private final Comparator> keyObjComparator; - private final ListeningExecutorService grouperSorter; + private final ListeningExecutorService executor; private final int priority; private final boolean hasQueryTimeout; private final long queryTimeoutAt; @@ -118,7 +118,7 @@ public ConcurrentGrouper( final int concurrencyHint, final DefaultLimitSpec limitSpec, final boolean sortHasNonGroupingFields, - final ListeningExecutorService grouperSorter, + final ListeningExecutorService executor, final int priority, final boolean hasQueryTimeout, final long queryTimeoutAt @@ -151,7 +151,7 @@ protected SpillingGrouper initialValue() this.limitSpec = limitSpec; this.sortHasNonGroupingFields = sortHasNonGroupingFields; this.keyObjComparator = keySerdeFactory.objectComparator(sortHasNonGroupingFields); - this.grouperSorter = Preconditions.checkNotNull(grouperSorter); + this.executor = Preconditions.checkNotNull(executor); this.priority = priority; this.hasQueryTimeout = hasQueryTimeout; this.queryTimeoutAt = queryTimeoutAt; @@ -243,9 +243,7 @@ public void reset() throw new ISE("Grouper is closed"); } - for (Grouper grouper : groupers) { - grouper.reset(); - } + groupers.forEach(Grouper::reset); } @Override @@ -263,15 +261,15 @@ public Iterator> iterator(final boolean sorted) parallelSortAndGetGroupersIterator() : getGroupersIterator(sorted); - // Parallel combine is used only when data are spilled. This is because ConcurrentGrouper uses two different modes - // depending on data are spilled or not. If data is not spilled, all inputs are completely aggregated and no more + // Parallel combine is used only when data is spilled. This is because ConcurrentGrouper uses two different modes + // depending on data is spilled or not. If data is not spilled, all inputs are completely aggregated and no more // aggregation is required. if (sorted && spilling && isParallelizable()) { // First try to merge dictionaries generated by all underlying groupers. If it is merged successfully, the same // merged dictionary is used for all combining threads - final List mergedDictionary = mergeDictionary(); - if (mergedDictionary != null) { - return parallelCombine(sortedIterators, mergedDictionary); + final List dictionary = tryMergeDictionary(); + if (dictionary != null) { + return parallelCombine(sortedIterators, dictionary); } } return Groupers.mergeIterators(sortedIterators, sorted ? keyObjComparator : null); @@ -284,11 +282,11 @@ private boolean isParallelizable() private List>> parallelSortAndGetGroupersIterator() { - // The number of groupers is same with the number of processing threads in grouperSorter + // The number of groupers is same with the number of processing threads in the executor final ListenableFuture>>> future = Futures.allAsList( groupers.stream() .map(grouper -> - grouperSorter.submit( + executor.submit( new AbstractPrioritizedCallable>>(priority) { @Override @@ -327,12 +325,12 @@ private List>> getGroupersIterator(boolean sorted) /** * Merge dictionaries of {@link Grouper.KeySerde}s of {@link Grouper}s. The result dictionary contains unique string - * string keys. + * keys. * * @return merged dictionary if its size does not exceed max dictionary size. Otherwise null. */ @Nullable - private List mergeDictionary() + private List tryMergeDictionary() { final long maxDictionarySize = combineKeySerdeFactory.getMaxDictionarySize(); final Set mergedDictionary = new HashSet<>(); @@ -340,11 +338,16 @@ private List mergeDictionary() for (SpillingGrouper grouper : groupers) { final List dictionary = grouper.getDictionary(); - totalDictionarySize += dictionary.stream().mapToLong(RowBasedGrouperHelper::estimateStringKeySize).sum(); - if (totalDictionarySize > maxDictionarySize) { - return null; + + for (String key : dictionary) { + if (!mergedDictionary.contains(key)) { + totalDictionarySize += RowBasedGrouperHelper.estimateStringKeySize(key); + if (totalDictionarySize > maxDictionarySize) { + return null; + } + mergedDictionary.add(key); + } } - mergedDictionary.addAll(dictionary); } return ImmutableList.copyOf(mergedDictionary); @@ -352,9 +355,7 @@ private List mergeDictionary() /** * Build a combining tree for the input iterators which combine input entries asynchronously. This method is called - * when data are spilled and thus streaming combine is preferred to avoid too many disk accesses. - * - * @param sortedIterators sorted iterators + * when data is spilled and thus streaming combine is preferred to avoid too many disk accesses. * * @return an iterator of the root grouper of the combining tree */ @@ -374,42 +375,20 @@ private Iterator> parallelCombine( combineKeySerdeFactory.factorizeWithDictionary(mergedDictionary), combiningFactories ); - final Pair degreeAndBufferNum = findCombineDegreeAndNumBuffers( + final Pair degreeAndNumBuffers = findCombineDegreeAndNumBuffers( combineBuffer, minimumRequiredBufferCapacity, concurrencyHint, sortedIterators.size() ); - if (degreeAndBufferNum == null) { - throw new ISE("Cannot find a proper combine degree"); - } - - final int combineDegree = degreeAndBufferNum.lhs; - final int bufferNum = degreeAndBufferNum.rhs; - final int sliceSize = combineBuffer.capacity() / bufferNum; - - final List slices = new ArrayList<>(bufferNum); - for (int i = 0; i < bufferNum; i++) { - slices.add(getSlice(combineBuffer, sliceSize, i)); - } - - final Supplier bufferSupplier = new Supplier() - { - private int i = 0; + final int combineDegree = degreeAndNumBuffers.lhs; + final int numBuffers = degreeAndNumBuffers.rhs; + final int sliceSize = combineBuffer.capacity() / numBuffers; - @Override - public ByteBuffer get() - { - if (i < slices.size()) { - return slices.get(i++); - } else { - throw new ISE("Requested number of buffer slices exceeds the planned one"); - } - } - }; + final Supplier bufferSupplier = createCombineBufferSupplier(combineBuffer, numBuffers, sliceSize); - final List combineFutures = new ArrayList<>(bufferNum); + final List combineFutures = new ArrayList<>(numBuffers); final Iterator> combinedIterator = buildCombineTree( sortedIterators, bufferSupplier, @@ -457,6 +436,28 @@ public Entry next() }; } + private static Supplier createCombineBufferSupplier( + ByteBuffer combineBuffer, + int numBuffers, + int sliceSize + ) + { + return new Supplier() + { + private int i = 0; + + @Override + public ByteBuffer get() + { + if (i < numBuffers) { + return getSlice(combineBuffer, sliceSize, i++); + } else { + throw new ISE("Requested number of buffer slices exceeds the planned one"); + } + } + }; + } + /** * Find a minimum size of the buffer slice and corresponding combining degree and number of slices. Note that each * node in the combining tree is executed by different threads. This method assumes that higher degree of parallelism @@ -467,9 +468,8 @@ public Entry next() * @param concurrencyHint available degree of parallelism * @param maxDegree maximum degree * - * @return a pair of degree and number of buffers if found. Otherwise null. + * @return a pair of degree and number of buffers if found. */ - @Nullable private static Pair findCombineDegreeAndNumBuffers( ByteBuffer combineBuffer, int requiredMinimumBufferCapacity, @@ -486,7 +486,8 @@ private static Pair findCombineDegreeAndNumBuffers( return Pair.of(degree, requiredBufferNum); } } - return null; + + throw new ISE("Cannot find a proper combine degree"); } /** @@ -579,9 +580,10 @@ private StreamingMergeSortedGrouper runCombiner( settableColumnSelectorFactory, combiningFactories ); - grouper.init(); - ListenableFuture future = grouperSorter.submit(() -> { + ListenableFuture future = executor.submit(() -> { + grouper.init(); + while (mergedIterator.hasNext()) { final Entry next = mergedIterator.next(); @@ -601,9 +603,9 @@ private StreamingMergeSortedGrouper runCombiner( @Override public void close() { - closed = true; - for (Grouper grouper : groupers) { - grouper.close(); + if (!closed) { + closed = true; + groupers.forEach(Grouper::close); } } From 92675a95bd3856a643c60f743809210eb4b9011c Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 2 Sep 2017 14:13:01 +0900 Subject: [PATCH 09/44] Refactor Groupers.mergeIterators() --- .../epinephelinae/ConcurrentGrouper.java | 6 +++++- .../query/groupby/epinephelinae/Groupers.java | 21 ++++++------------- .../epinephelinae/SpillingGrouper.java | 4 +++- 3 files changed, 14 insertions(+), 17 deletions(-) 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 99d721e85bd7..21c0bc0536e4 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 @@ -24,6 +24,7 @@ import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterators; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; @@ -272,7 +273,10 @@ public Iterator> iterator(final boolean sorted) return parallelCombine(sortedIterators, dictionary); } } - return Groupers.mergeIterators(sortedIterators, sorted ? keyObjComparator : null); + + return sorted ? + Groupers.mergeIterators(sortedIterators, keyObjComparator) : + Iterators.concat(sortedIterators.iterator()); } private boolean isParallelizable() 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 d03be91abe35..bd7f6507b6d8 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 @@ -19,6 +19,7 @@ package io.druid.query.groupby.epinephelinae; +import com.google.common.base.Preconditions; import com.google.common.collect.Iterators; import java.util.Comparator; @@ -77,20 +78,10 @@ public static Iterator> mergeIterators( final Comparator> keyTypeComparator ) { - if (keyTypeComparator != null) { - return Iterators.mergeSorted( - iterators, - new Comparator>() - { - @Override - public int compare(Grouper.Entry lhs, Grouper.Entry rhs) - { - return keyTypeComparator.compare(lhs, rhs); - } - } - ); - } else { - return Iterators.concat(iterators.iterator()); - } + Preconditions.checkNotNull(keyTypeComparator); + return Iterators.mergeSorted( + iterators, + keyTypeComparator::compare + ); } } 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 571750a85b60..4fb05df610bc 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 @@ -235,7 +235,9 @@ public Entry apply(Entry entry) if (sortHasNonGroupingFields) { return Groupers.mergeIterators(iterators, defaultOrderKeyObjComparator); } else { - return Groupers.mergeIterators(iterators, sorted ? keyObjComparator : null); + return sorted ? + Groupers.mergeIterators(iterators, keyObjComparator) : + Iterators.concat(iterators.iterator()); } } From a888db088494b146a4ff95a1ed3a96d7d65b1375 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 2 Sep 2017 15:49:03 +0900 Subject: [PATCH 10/44] Add comments for combining tree --- .../epinephelinae/ConcurrentGrouper.java | 62 +++++++++++-------- 1 file changed, 37 insertions(+), 25 deletions(-) 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 21c0bc0536e4..2740cc39c21f 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 @@ -358,8 +358,11 @@ private List tryMergeDictionary() } /** - * Build a combining tree for the input iterators which combine input entries asynchronously. This method is called - * when data is spilled and thus streaming combine is preferred to avoid too many disk accesses. + * Build a combining tree for the input iterators which combine input entries asynchronously. Each node in the tree + * is a combining task which iterates through child iterators, aggregates the inputs from those iterators, and returns + * an iterator for the result of aggregation. + *

+ * This method is called when data is spilled and thus streaming combine is preferred to avoid too many disk accesses. * * @return an iterator of the root grouper of the combining tree */ @@ -379,6 +382,9 @@ private Iterator> parallelCombine( combineKeySerdeFactory.factorizeWithDictionary(mergedDictionary), combiningFactories ); + // We want to maximize the parallelism while the size of buffer slice is greater than the minimum buffer size + // required by StreamingMergeSortedGrouper. Here, we find the degree of the cominbing tree and the required number + // of buffers maximizing the degree of parallelism. final Pair degreeAndNumBuffers = findCombineDegreeAndNumBuffers( combineBuffer, minimumRequiredBufferCapacity, @@ -398,8 +404,8 @@ private Iterator> parallelCombine( bufferSupplier, combiningFactories, combineDegree, - combineFutures, - mergedDictionary + mergedDictionary, + combineFutures ); return new Iterator>() @@ -470,7 +476,7 @@ public ByteBuffer get() * @param combineBuffer entire buffer used for combining tree * @param requiredMinimumBufferCapacity minimum buffer capacity for {@link StreamingMergeSortedGrouper} * @param concurrencyHint available degree of parallelism - * @param maxDegree maximum degree + * @param numLeafNodes number of leaf nodes of combining tree * * @return a pair of degree and number of buffers if found. */ @@ -478,16 +484,16 @@ private static Pair findCombineDegreeAndNumBuffers( ByteBuffer combineBuffer, int requiredMinimumBufferCapacity, int concurrencyHint, - int maxDegree + int numLeafNodes ) { - for (int degree = MINIMUM_COMBINE_DEGREE; degree <= maxDegree; degree++) { - // the number of available combine nodes is concurrencyHint because it's the max concurrency and we don't want to - // parallelize behind that. - final int requiredBufferNum = computeRequiredBufferNum(concurrencyHint, degree); - final int expectedSliceSize = combineBuffer.capacity() / requiredBufferNum; - if (expectedSliceSize > requiredMinimumBufferCapacity) { - return Pair.of(degree, requiredBufferNum); + for (int degree = MINIMUM_COMBINE_DEGREE; degree <= numLeafNodes; degree++) { + final int requiredBufferNum = computeRequiredBufferNum(numLeafNodes, degree); + if (requiredBufferNum <= concurrencyHint) { + final int expectedSliceSize = combineBuffer.capacity() / requiredBufferNum; + if (expectedSliceSize >= requiredMinimumBufferCapacity) { + return Pair.of(degree, requiredBufferNum); + } } } @@ -497,19 +503,22 @@ private static Pair findCombineDegreeAndNumBuffers( /** * Recursively compute number of required buffers in a top-down manner. * - * @param totalNumNodes total number of descendent nodes - * @param degree degree + * @param numLeafNodes number of leaf nodes + * @param combineDegree combine degree * * @return minimum number of buffers required for combining tree */ - private static int computeRequiredBufferNum(int totalNumNodes, int degree) + private static int computeRequiredBufferNum(int numLeafNodes, int combineDegree) { - if (totalNumNodes > degree) { - final int numNodesPerChild = (totalNumNodes + degree - 1) / degree; // ceiling + if (numLeafNodes > combineDegree) { + final int numLeafNodesPerChild = (numLeafNodes + combineDegree - 1) / combineDegree; // ceiling int sum = 1; // count for the current node - for (int i = 0; i < degree; i++) { + for (int i = 0; i < combineDegree; i++) { // further compute for child nodes - sum += computeRequiredBufferNum(Math.min(numNodesPerChild, totalNumNodes - i * numNodesPerChild), degree); + sum += computeRequiredBufferNum( + Math.min(numLeafNodesPerChild, numLeafNodes - i * numLeafNodesPerChild), + combineDegree + ); } return sum; @@ -519,12 +528,15 @@ private static int computeRequiredBufferNum(int totalNumNodes, int degree) } /** - * Recursively build a combining tree in a top-down manner. + * Recursively build a combining tree in a top-down manner. Note that this method executes several combining tasks + * and stores the futures for those tasks in the given combineFutures. (TODO: check this) * * @param sortedIterators sorted iterators * @param bufferSupplier combining buffer supplier * @param combiningFactories array of combining aggregator factories * @param combineDegree combining degree + * @param dictionary merged dictionary + * @param combineFutures list of futures for the combining tasks * * @return an iterator of the root of the combining tree */ @@ -533,8 +545,8 @@ private Iterator> buildCombineTree( Supplier bufferSupplier, AggregatorFactory[] combiningFactories, int combineDegree, - List combineFutures, - List dictionary + List dictionary, + List combineFutures ) { final int numIterators = sortedIterators.size(); @@ -548,8 +560,8 @@ private Iterator> buildCombineTree( bufferSupplier, combiningFactories, combineDegree, - combineFutures, - dictionary + dictionary, + combineFutures ) ); } From 7040ffb2b8e817a9242c5a26b1873b565a9469be Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 2 Sep 2017 16:49:33 +0900 Subject: [PATCH 11/44] Refactor buildCombineTree --- .../epinephelinae/ConcurrentGrouper.java | 78 +++++++++++-------- 1 file changed, 47 insertions(+), 31 deletions(-) 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 2740cc39c21f..5c7f44753f58 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 @@ -50,6 +50,7 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collections; import java.util.Comparator; import java.util.HashSet; import java.util.Iterator; @@ -398,16 +399,17 @@ private Iterator> parallelCombine( final Supplier bufferSupplier = createCombineBufferSupplier(combineBuffer, numBuffers, sliceSize); - final List combineFutures = new ArrayList<>(numBuffers); - final Iterator> combinedIterator = buildCombineTree( + final Pair>, List> combineIteratorAndFutures = buildCombineTree( sortedIterators, bufferSupplier, combiningFactories, combineDegree, - mergedDictionary, - combineFutures + mergedDictionary ); + final Iterator> combineIterator = combineIteratorAndFutures.lhs; + final List combineFutures = combineIteratorAndFutures.rhs; + return new Iterator>() { private boolean closed; @@ -415,7 +417,7 @@ private Iterator> parallelCombine( @Override public boolean hasNext() { - if (!combinedIterator.hasNext()) { + if (!combineIterator.hasNext()) { if (!closed) { combineBufferHolder.close(); closed = true; @@ -441,7 +443,7 @@ public boolean hasNext() @Override public Entry next() { - return combinedIterator.next(); + return combineIterator.next(); } }; } @@ -501,12 +503,16 @@ private static Pair findCombineDegreeAndNumBuffers( } /** - * Recursively compute number of required buffers in a top-down manner. + * Recursively compute the number of required buffers for a combining tree in a top-down manner. Since each node of + * the combining tree represents a combining task and each combining task requires one buffer, the number of required + * buffers is the number of nodes of the combining tree. * * @param numLeafNodes number of leaf nodes * @param combineDegree combine degree * * @return minimum number of buffers required for combining tree + * + * @see {@link #buildCombineTree(List, Supplier, AggregatorFactory[], int, List)} */ private static int computeRequiredBufferNum(int numLeafNodes, int combineDegree) { @@ -528,46 +534,59 @@ private static int computeRequiredBufferNum(int numLeafNodes, int combineDegree) } /** - * Recursively build a combining tree in a top-down manner. Note that this method executes several combining tasks - * and stores the futures for those tasks in the given combineFutures. (TODO: check this) + * Recursively build a combining tree in a top-down manner. Each node of the tree is a task that combines input + * iterators asynchronously. * * @param sortedIterators sorted iterators * @param bufferSupplier combining buffer supplier * @param combiningFactories array of combining aggregator factories * @param combineDegree combining degree * @param dictionary merged dictionary - * @param combineFutures list of futures for the combining tasks * - * @return an iterator of the root of the combining tree + * @return a pair of an iterator of the root of the combining tree and a list of futures of all executed combining + * tasks */ - private Iterator> buildCombineTree( + private Pair>, List> buildCombineTree( List>> sortedIterators, Supplier bufferSupplier, AggregatorFactory[] combiningFactories, int combineDegree, - List dictionary, - List combineFutures + List dictionary ) { final int numIterators = sortedIterators.size(); if (numIterators > combineDegree) { + final List>> childIterators = new ArrayList<>(combineDegree); + final List combineFutures = new ArrayList<>(combineDegree + 1); + final int iteratorsPerChild = (numIterators + combineDegree - 1) / combineDegree; // ceiling - final List>> childIterators = new ArrayList<>(); for (int i = 0; i < combineDegree; i++) { - childIterators.add( - buildCombineTree( - sortedIterators.subList(i * iteratorsPerChild, Math.min(numIterators, (i + 1) * iteratorsPerChild)), - bufferSupplier, - combiningFactories, - combineDegree, - dictionary, - combineFutures - ) + final Pair>, List> childIteratorAndFutures = buildCombineTree( + sortedIterators.subList(i * iteratorsPerChild, Math.min(numIterators, (i + 1) * iteratorsPerChild)), + bufferSupplier, + combiningFactories, + combineDegree, + dictionary ); + childIterators.add(childIteratorAndFutures.lhs); + combineFutures.addAll(childIteratorAndFutures.rhs); } - return runCombiner(childIterators, bufferSupplier.get(), combiningFactories, combineFutures, dictionary).iterator(); + final Pair>, Future> iteratorAndFuture = runCombiner( + childIterators, + bufferSupplier.get(), + combiningFactories, + dictionary + ); + combineFutures.add(iteratorAndFuture.rhs); + return new Pair<>(iteratorAndFuture.lhs, combineFutures); } else { - return runCombiner(sortedIterators, bufferSupplier.get(), combiningFactories, combineFutures, dictionary).iterator(); + final Pair>, Future> iteratorAndFuture = runCombiner( + sortedIterators, + bufferSupplier.get(), + combiningFactories, + dictionary + ); + return new Pair<>(iteratorAndFuture.lhs, Collections.singletonList(iteratorAndFuture.rhs)); } } @@ -579,11 +598,10 @@ private static ByteBuffer getSlice(ByteBuffer buffer, int sliceSize, int i) return slice.slice(); } - private StreamingMergeSortedGrouper runCombiner( + private Pair>, Future> runCombiner( List>> iterators, ByteBuffer combineBuffer, AggregatorFactory[] combiningFactories, - List combineFutures, List dictionary ) { @@ -611,9 +629,7 @@ private StreamingMergeSortedGrouper runCombiner( grouper.finish(); }); - combineFutures.add(future); - - return grouper; + return new Pair<>(grouper.iterator(), future); } @Override From 5c4b846cdffeeda62a4bf480f596eeae8df96fc2 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 2 Sep 2017 18:43:38 +0900 Subject: [PATCH 12/44] Refactor iterator --- .../java/util/common/CloseableIterators.java | 96 ++++++++++++ .../epinephelinae/BufferArrayGrouper.java | 13 +- .../epinephelinae/BufferHashGrouper.java | 24 ++- .../CloseableGrouperIterator.java | 28 ++-- .../epinephelinae/ConcurrentGrouper.java | 138 ++++++++---------- .../query/groupby/epinephelinae/Grouper.java | 4 +- .../query/groupby/epinephelinae/Groupers.java | 18 --- .../LimitedBufferHashGrouper.java | 28 +++- .../epinephelinae/SpillingGrouper.java | 54 ++++--- .../StreamingMergeSortedGrouper.java | 15 +- .../epinephelinae/ConcurrentGrouperTest.java | 5 +- 11 files changed, 272 insertions(+), 151 deletions(-) create mode 100644 java-util/src/main/java/io/druid/java/util/common/CloseableIterators.java diff --git a/java-util/src/main/java/io/druid/java/util/common/CloseableIterators.java b/java-util/src/main/java/io/druid/java/util/common/CloseableIterators.java new file mode 100644 index 000000000000..437e4c8c81e9 --- /dev/null +++ b/java-util/src/main/java/io/druid/java/util/common/CloseableIterators.java @@ -0,0 +1,96 @@ +/* + * 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.java.util.common; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterators; +import io.druid.java.util.common.io.Closer; +import io.druid.java.util.common.parsers.CloseableIterator; + +import javax.annotation.Nullable; +import java.io.Closeable; +import java.io.IOException; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; + +public class CloseableIterators +{ + public static CloseableIterator concat(List> iterators) + { + final Closer closer = Closer.create(); + iterators.forEach(closer::register); + + final Iterator innerIterator = Iterators.concat(iterators.iterator()); + return wrap(innerIterator, closer); + } + + public static CloseableIterator mergeSorted( + List> iterators, + Comparator comparator + ) + { + Preconditions.checkNotNull(comparator); + + final Closer closer = Closer.create(); + iterators.forEach(closer::register); + + final Iterator innerIterator = Iterators.mergeSorted(iterators, comparator); + return wrap(innerIterator, closer); + } + + public static CloseableIterator wrap(Iterator innerIterator, @Nullable Closeable closeable) + { + return new CloseableIterator() + { + private boolean closed; + + @Override + public boolean hasNext() + { + return innerIterator.hasNext(); + } + + @Override + public T next() + { + return innerIterator.next(); + } + + @Override + public void close() throws IOException + { + if (!closed) { + if (closeable != null) { + closeable.close(); + } + closed = true; + } + } + }; + } + + public static CloseableIterator withEmptyBaggage(Iterator innerIterator) + { + return wrap(innerIterator, null); + } + + private CloseableIterators() {} +} 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 index eb5e4d194651..04a7fa5b7c81 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferArrayGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferArrayGrouper.java @@ -21,6 +21,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Supplier; +import io.druid.java.util.common.parsers.CloseableIterator; import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; import io.druid.query.aggregation.AggregatorFactory; @@ -28,9 +29,9 @@ import io.druid.query.groupby.epinephelinae.column.GroupByColumnSelectorStrategy; import io.druid.segment.ColumnSelectorFactory; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; -import java.util.Iterator; import java.util.NoSuchElementException; /** @@ -225,13 +226,13 @@ public void close() } @Override - public Iterator> iterator(boolean sorted) + public CloseableIterator> iterator(boolean sorted) { if (sorted) { throw new UnsupportedOperationException("sorted iterator is not supported yet"); } - return new Iterator>() + return new CloseableIterator>() { int cur = -1; boolean findNext = false; @@ -276,6 +277,12 @@ public Entry next() } return new Entry<>(cur - 1, values); } + + @Override + public void close() throws IOException + { + // do nothing + } }; } } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferHashGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferHashGrouper.java index 4b709e3a0dba..0a59430ef209 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferHashGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferHashGrouper.java @@ -22,16 +22,18 @@ import com.google.common.base.Supplier; import com.google.common.collect.Iterators; import com.google.common.primitives.Ints; +import io.druid.java.util.common.parsers.CloseableIterator; +import io.druid.java.util.common.CloseableIterators; import io.druid.java.util.common.IAE; import io.druid.java.util.common.logger.Logger; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.ColumnSelectorFactory; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.AbstractList; import java.util.Collections; import java.util.Comparator; -import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; @@ -154,12 +156,12 @@ public void reset() } @Override - public Iterator> iterator(boolean sorted) + public CloseableIterator> iterator(boolean sorted) { if (!initialized) { // it's possible for iterator() to be called before initialization when // a nested groupBy's subquery has an empty result set (see testEmptySubquery() in GroupByQueryRunnerTest) - return Iterators.>emptyIterator(); + return CloseableIterators.withEmptyBaggage(Iterators.>emptyIterator()); } if (sorted) { @@ -207,7 +209,7 @@ public int compare(Integer lhs, Integer rhs) } ); - return new Iterator>() + return new CloseableIterator>() { int curr = 0; final int size = getSize(); @@ -232,10 +234,16 @@ public void remove() { throw new UnsupportedOperationException(); } + + @Override + public void close() throws IOException + { + // do nothing + } }; } else { // Unsorted iterator - return new Iterator>() + return new CloseableIterator>() { int curr = 0; final int size = getSize(); @@ -264,6 +272,12 @@ public void remove() { throw new UnsupportedOperationException(); } + + @Override + public void close() throws IOException + { + // do nothing + } }; } } 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 index 808b60f75c03..6accdfd7752b 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/CloseableGrouperIterator.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/CloseableGrouperIterator.java @@ -20,29 +20,33 @@ package io.druid.query.groupby.epinephelinae; import com.google.common.base.Throwables; +import io.druid.java.util.common.io.Closer; +import io.druid.java.util.common.parsers.CloseableIterator; import io.druid.query.groupby.epinephelinae.Grouper.Entry; import java.io.Closeable; import java.io.IOException; -import java.util.Iterator; import java.util.function.Function; -public class CloseableGrouperIterator implements Iterator, Closeable +public class CloseableGrouperIterator implements CloseableIterator { private final Function, T> transformer; - private final Closeable closer; - private final Iterator> iterator; + private final CloseableIterator> iterator; + private final Closer closer; public CloseableGrouperIterator( final Grouper grouper, final boolean sorted, final Function, T> transformer, - final Closeable closer + final Closeable closeable ) { this.transformer = transformer; - this.closer = closer; this.iterator = grouper.iterator(sorted); + this.closer = Closer.create(); + + closer.register(iterator); + closer.register(closeable); } @Override @@ -66,13 +70,11 @@ public void remove() @Override public void close() { - if (closer != null) { - try { - closer.close(); - } - catch (IOException e) { - throw Throwables.propagate(e); - } + try { + closer.close(); + } + catch (IOException e) { + throw Throwables.propagate(e); } } } 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 5c7f44753f58..49349f72a2cd 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 @@ -23,14 +23,17 @@ import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterators; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import io.druid.collections.ResourceHolder; +import io.druid.java.util.common.CloseableIterators; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; +import io.druid.java.util.common.io.Closer; +import io.druid.java.util.common.parsers.CloseableIterator; import io.druid.query.AbstractPrioritizedCallable; import io.druid.query.QueryInterruptedException; import io.druid.query.aggregation.AggregatorFactory; @@ -48,12 +51,12 @@ import it.unimi.dsi.fastutil.objects.Object2IntMap; import javax.annotation.Nullable; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Set; import java.util.concurrent.CancellationException; @@ -129,14 +132,7 @@ public ConcurrentGrouper( Preconditions.checkArgument(concurrencyHint > 0, "concurrencyHint > 0"); this.groupers = new ArrayList<>(concurrencyHint); - this.threadLocalGrouper = new ThreadLocal>() - { - @Override - protected SpillingGrouper initialValue() - { - return groupers.get(threadNumber.getAndIncrement()); - } - }; + this.threadLocalGrouper = ThreadLocal.withInitial(() -> groupers.get(threadNumber.getAndIncrement())); this.bufferSupplier = bufferSupplier; this.combineBufferSupplier = combineBufferSupplier; @@ -249,7 +245,7 @@ public void reset() } @Override - public Iterator> iterator(final boolean sorted) + public CloseableIterator> iterator(final boolean sorted) { if (!initialized) { throw new ISE("Grouper is not initialized"); @@ -259,9 +255,9 @@ public Iterator> iterator(final boolean sorted) throw new ISE("Grouper is closed"); } - final List>> sortedIterators = sorted && isParallelizable() ? - parallelSortAndGetGroupersIterator() : - getGroupersIterator(sorted); + final List>> sortedIterators = sorted && isParallelizable() ? + parallelSortAndGetGroupersIterator() : + getGroupersIterator(sorted); // Parallel combine is used only when data is spilled. This is because ConcurrentGrouper uses two different modes // depending on data is spilled or not. If data is not spilled, all inputs are completely aggregated and no more @@ -276,8 +272,8 @@ public Iterator> iterator(final boolean sorted) } return sorted ? - Groupers.mergeIterators(sortedIterators, keyObjComparator) : - Iterators.concat(sortedIterators.iterator()); + CloseableIterators.mergeSorted(sortedIterators, keyObjComparator) : + CloseableIterators.concat(sortedIterators); } private boolean isParallelizable() @@ -285,17 +281,17 @@ private boolean isParallelizable() return concurrencyHint > 1; } - private List>> parallelSortAndGetGroupersIterator() + private List>> parallelSortAndGetGroupersIterator() { // The number of groupers is same with the number of processing threads in the executor - final ListenableFuture>>> future = Futures.allAsList( + final ListenableFuture>>> future = Futures.allAsList( groupers.stream() .map(grouper -> executor.submit( - new AbstractPrioritizedCallable>>(priority) + new AbstractPrioritizedCallable>>(priority) { @Override - public Iterator> call() throws Exception + public CloseableIterator> call() throws Exception { return grouper.iterator(true); } @@ -321,7 +317,7 @@ public Iterator> call() throws Exception } } - private List>> getGroupersIterator(boolean sorted) + private List>> getGroupersIterator(boolean sorted) { return groupers.stream() .map(grouper -> grouper.iterator(sorted)) @@ -367,8 +363,8 @@ private List tryMergeDictionary() * * @return an iterator of the root grouper of the combining tree */ - private Iterator> parallelCombine( - List>> sortedIterators, + private CloseableIterator> parallelCombine( + List>> sortedIterators, List mergedDictionary ) { @@ -399,7 +395,7 @@ private Iterator> parallelCombine( final Supplier bufferSupplier = createCombineBufferSupplier(combineBuffer, numBuffers, sliceSize); - final Pair>, List> combineIteratorAndFutures = buildCombineTree( + final Pair>, List> combineIteratorAndFutures = buildCombineTree( sortedIterators, bufferSupplier, combiningFactories, @@ -407,45 +403,30 @@ private Iterator> parallelCombine( mergedDictionary ); - final Iterator> combineIterator = combineIteratorAndFutures.lhs; + final CloseableIterator> combineIterator = combineIteratorAndFutures.lhs; final List combineFutures = combineIteratorAndFutures.rhs; - return new Iterator>() - { - private boolean closed; + final Closer closer = Closer.create(); + closer.register(combineBufferHolder); + closer.register(() -> checkCombineFutures(combineFutures)); - @Override - public boolean hasNext() - { - if (!combineIterator.hasNext()) { - if (!closed) { - combineBufferHolder.close(); - closed = true; - - for (Future future : combineFutures) { - try { - // futures should be done before reaching here and throw exceptions if they failed - future.get(); - } - catch (InterruptedException e) { - throw new QueryInterruptedException(e); - } - catch (ExecutionException e) { - throw new RuntimeException(e); - } - } - } - return false; - } - return true; - } + return CloseableIterators.wrap(combineIterator, closer); + } - @Override - public Entry next() - { - return combineIterator.next(); + private static void checkCombineFutures(List combineFutures) + { + for (Future future : combineFutures) { + try { + // futures should be done before reaching here and throw exceptions if they failed + future.get(); } - }; + catch (InterruptedException e) { + throw new QueryInterruptedException(e); + } + catch (ExecutionException e) { + throw new RuntimeException(e); + } + } } private static Supplier createCombineBufferSupplier( @@ -546,8 +527,8 @@ private static int computeRequiredBufferNum(int numLeafNodes, int combineDegree) * @return a pair of an iterator of the root of the combining tree and a list of futures of all executed combining * tasks */ - private Pair>, List> buildCombineTree( - List>> sortedIterators, + private Pair>, List> buildCombineTree( + List>> sortedIterators, Supplier bufferSupplier, AggregatorFactory[] combiningFactories, int combineDegree, @@ -556,12 +537,12 @@ private Pair>, List> buildCombineTree( { final int numIterators = sortedIterators.size(); if (numIterators > combineDegree) { - final List>> childIterators = new ArrayList<>(combineDegree); + final List>> childIterators = new ArrayList<>(combineDegree); final List combineFutures = new ArrayList<>(combineDegree + 1); final int iteratorsPerChild = (numIterators + combineDegree - 1) / combineDegree; // ceiling for (int i = 0; i < combineDegree; i++) { - final Pair>, List> childIteratorAndFutures = buildCombineTree( + final Pair>, List> childIteratorAndFutures = buildCombineTree( sortedIterators.subList(i * iteratorsPerChild, Math.min(numIterators, (i + 1) * iteratorsPerChild)), bufferSupplier, combiningFactories, @@ -571,7 +552,7 @@ private Pair>, List> buildCombineTree( childIterators.add(childIteratorAndFutures.lhs); combineFutures.addAll(childIteratorAndFutures.rhs); } - final Pair>, Future> iteratorAndFuture = runCombiner( + final Pair>, Future> iteratorAndFuture = runCombiner( childIterators, bufferSupplier.get(), combiningFactories, @@ -580,7 +561,7 @@ private Pair>, List> buildCombineTree( combineFutures.add(iteratorAndFuture.rhs); return new Pair<>(iteratorAndFuture.lhs, combineFutures); } else { - final Pair>, Future> iteratorAndFuture = runCombiner( + final Pair>, Future> iteratorAndFuture = runCombiner( sortedIterators, bufferSupplier.get(), combiningFactories, @@ -598,14 +579,13 @@ private static ByteBuffer getSlice(ByteBuffer buffer, int sliceSize, int i) return slice.slice(); } - private Pair>, Future> runCombiner( - List>> iterators, + private Pair>, Future> runCombiner( + List>> iterators, ByteBuffer combineBuffer, AggregatorFactory[] combiningFactories, List dictionary ) { - final Iterator> mergedIterator = Groupers.mergeIterators(iterators, keyObjComparator); final SettableColumnSelectorFactory settableColumnSelectorFactory = new SettableColumnSelectorFactory(aggregatorFactories); final StreamingMergeSortedGrouper grouper = new StreamingMergeSortedGrouper<>( @@ -615,15 +595,25 @@ private Pair>, Future> runCombiner( combiningFactories ); - ListenableFuture future = executor.submit(() -> { + final ListenableFuture future = executor.submit(() -> { grouper.init(); - while (mergedIterator.hasNext()) { - final Entry next = mergedIterator.next(); - - settableColumnSelectorFactory.set(next.values); - grouper.aggregate(next.key); // grouper always returns ok or throws an exception - settableColumnSelectorFactory.set(null); + try ( + CloseableIterator> mergedIterator = CloseableIterators.mergeSorted( + iterators, + keyObjComparator + ) + ) { + while (mergedIterator.hasNext()) { + final Entry next = mergedIterator.next(); + + settableColumnSelectorFactory.set(next.values); + grouper.aggregate(next.key); // grouper always returns ok or throws an exception + settableColumnSelectorFactory.set(null); + } + } + catch (IOException e) { + throw Throwables.propagate(e); } grouper.finish(); 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 f33213f19471..0d75b73308c2 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 @@ -22,13 +22,13 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import io.druid.java.util.common.parsers.CloseableIterator; import io.druid.query.aggregation.AggregatorFactory; import java.io.Closeable; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Comparator; -import java.util.Iterator; import java.util.List; import java.util.function.ToIntFunction; @@ -117,7 +117,7 @@ default ToIntFunction hashFunction() * * @return entry iterator */ - Iterator> iterator(final boolean sorted); + CloseableIterator> iterator(final boolean sorted); class Entry { 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 bd7f6507b6d8..22c8a78cc778 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 @@ -19,12 +19,6 @@ package io.druid.query.groupby.epinephelinae; -import com.google.common.base.Preconditions; -import com.google.common.collect.Iterators; - -import java.util.Comparator; -import java.util.Iterator; - public class Groupers { private Groupers() @@ -72,16 +66,4 @@ static int getUsedFlag(int keyHash) { return keyHash | 0x80000000; } - - public static Iterator> mergeIterators( - final Iterable>> iterators, - final Comparator> keyTypeComparator - ) - { - Preconditions.checkNotNull(keyTypeComparator); - return Iterators.mergeSorted( - iterators, - keyTypeComparator::compare - ); - } } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/LimitedBufferHashGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/LimitedBufferHashGrouper.java index 30aabc63de9e..3b84319a4e28 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/LimitedBufferHashGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/LimitedBufferHashGrouper.java @@ -22,16 +22,18 @@ import com.google.common.base.Supplier; import com.google.common.collect.Iterators; import com.google.common.primitives.Ints; +import io.druid.java.util.common.parsers.CloseableIterator; +import io.druid.java.util.common.CloseableIterators; import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.ColumnSelectorFactory; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.AbstractList; import java.util.Collections; import java.util.Comparator; -import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; @@ -200,13 +202,13 @@ public void reset() } @Override - public Iterator> iterator(boolean sorted) + public CloseableIterator> iterator(boolean sorted) { if (!initialized) { // it's possible for iterator() to be called before initialization when // a nested groupBy's subquery has an empty result set (see testEmptySubqueryWithLimitPushDown() // in GroupByQueryRunnerTest) - return Iterators.>emptyIterator(); + return CloseableIterators.withEmptyBaggage(Iterators.>emptyIterator()); } if (sortHasNonGroupingFields) { @@ -252,7 +254,7 @@ public int getHeapIndexForOffset(int bucketOffset) } } - private Iterator> makeDefaultOrderingIterator() + private CloseableIterator> makeDefaultOrderingIterator() { final int size = offsetHeap.getHeapSize(); @@ -300,7 +302,7 @@ public int compare(Integer lhs, Integer rhs) } ); - return new Iterator>() + return new CloseableIterator>() { int curr = 0; @@ -321,13 +323,19 @@ public void remove() { throw new UnsupportedOperationException(); } + + @Override + public void close() throws IOException + { + // do nothing + } }; } - private Iterator> makeHeapIterator() + private CloseableIterator> makeHeapIterator() { final int initialHeapSize = offsetHeap.getHeapSize(); - return new Iterator>() + return new CloseableIterator>() { int curr = 0; @@ -355,6 +363,12 @@ public void remove() { throw new UnsupportedOperationException(); } + + @Override + public void close() throws IOException + { + // do nothing + } }; } 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 4fb05df610bc..2b3f71368970 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 @@ -27,6 +27,8 @@ import com.google.common.base.Throwables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; +import io.druid.java.util.common.parsers.CloseableIterator; +import io.druid.java.util.common.CloseableIterators; import io.druid.java.util.common.guava.CloseQuietly; import io.druid.query.BaseQuery; import io.druid.query.aggregation.AggregatorFactory; @@ -200,53 +202,57 @@ public void setSpillingAllowed(final boolean spillingAllowed) } @Override - public Iterator> iterator(final boolean sorted) + public CloseableIterator> iterator(final boolean sorted) { - final List>> iterators = new ArrayList<>(1 + files.size()); + 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(); + CloseableIterators.withEmptyBaggage( + 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); } } - return new Entry<>(entry.getKey(), deserializedValues); - } - } + ) ) ); closeables.add(fileIterator); } if (sortHasNonGroupingFields) { - return Groupers.mergeIterators(iterators, defaultOrderKeyObjComparator); + return CloseableIterators.mergeSorted(iterators, defaultOrderKeyObjComparator); } else { return sorted ? - Groupers.mergeIterators(iterators, keyObjComparator) : - Iterators.concat(iterators.iterator()); + CloseableIterators.mergeSorted(iterators, keyObjComparator) : + CloseableIterators.concat(iterators); } } private void spill() throws IOException { - files.add(spill(grouper.iterator(true))); - dictionaryFiles.add(spill(keySerde.getDictionary().iterator())); + try (CloseableIterator> iterator = grouper.iterator(true)) { + files.add(spill(iterator)); + dictionaryFiles.add(spill(keySerde.getDictionary().iterator())); - grouper.reset(); + grouper.reset(); + } } private File spill(Iterator iterator) throws IOException diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java index 0f1a674df532..c8e0c8807626 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java @@ -22,14 +22,15 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Supplier; +import io.druid.java.util.common.parsers.CloseableIterator; import io.druid.java.util.common.IAE; import io.druid.java.util.common.logger.Logger; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.BufferAggregator; import io.druid.segment.ColumnSelectorFactory; +import java.io.IOException; import java.nio.ByteBuffer; -import java.util.Iterator; import java.util.NoSuchElementException; import java.util.concurrent.TimeoutException; @@ -293,9 +294,9 @@ public void finish() * * @return a sorted iterator */ - public Iterator> iterator() + public CloseableIterator> iterator() { - return new Iterator>() + return new CloseableIterator>() { { // Waits for some data to be ready @@ -353,6 +354,12 @@ public Entry next() return new Entry<>(key, values); } + + @Override + public void close() throws IOException + { + // do nothing + } }; } @@ -366,7 +373,7 @@ public Entry next() * @return a sorted iterator */ @Override - public Iterator> iterator(boolean sorted) + public CloseableIterator> iterator(boolean sorted) { return iterator(); } diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java index 843e16bea43f..5a79ef4a6012 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java @@ -26,6 +26,7 @@ import com.google.common.util.concurrent.MoreExecutors; import io.druid.collections.ResourceHolder; import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.parsers.CloseableIterator; import io.druid.java.util.common.IAE; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; @@ -301,7 +302,9 @@ public void run() eachFuture.get(); } - final List> actual = Lists.newArrayList(grouper.iterator(true)); + final CloseableIterator> iterator = grouper.iterator(true); + final List> actual = Lists.newArrayList(iterator); + iterator.close(); Assert.assertTrue(testResourceHolder.closed); From f31f6d203b482ac9a55480bcea3e2b7f32c18a2e Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 2 Sep 2017 19:15:52 +0900 Subject: [PATCH 13/44] Add ParallelCombiner --- .../epinephelinae/ConcurrentGrouper.java | 433 +--------------- .../query/groupby/epinephelinae/Groupers.java | 10 + .../epinephelinae/ParallelCombiner.java | 467 ++++++++++++++++++ 3 files changed, 497 insertions(+), 413 deletions(-) create mode 100644 processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java 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 49349f72a2cd..d1a67629662e 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 @@ -23,7 +23,6 @@ import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -31,37 +30,22 @@ import io.druid.collections.ResourceHolder; import io.druid.java.util.common.CloseableIterators; import io.druid.java.util.common.ISE; -import io.druid.java.util.common.Pair; -import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.parsers.CloseableIterator; import io.druid.query.AbstractPrioritizedCallable; import io.druid.query.QueryInterruptedException; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.dimension.DimensionSpec; import io.druid.query.groupby.orderby.DefaultLimitSpec; -import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnSelectorFactory; -import io.druid.segment.DimensionSelector; -import io.druid.segment.DoubleColumnSelector; -import io.druid.segment.FloatColumnSelector; -import io.druid.segment.LongColumnSelector; -import io.druid.segment.ObjectColumnSelector; -import io.druid.segment.column.ColumnCapabilities; -import it.unimi.dsi.fastutil.objects.Object2IntArrayMap; -import it.unimi.dsi.fastutil.objects.Object2IntMap; import javax.annotation.Nullable; -import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Collections; import java.util.Comparator; import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; @@ -78,8 +62,6 @@ */ public class ConcurrentGrouper implements Grouper { - private static final int MINIMUM_COMBINE_DEGREE = 2; - private final List> groupers; private final ThreadLocal> threadLocalGrouper; private final AtomicInteger threadNumber = new AtomicInteger(); @@ -87,7 +69,6 @@ public class ConcurrentGrouper implements Grouper private volatile boolean closed = false; private final Supplier bufferSupplier; - private final Supplier> combineBufferSupplier; private final ColumnSelectorFactory columnSelectorFactory; private final AggregatorFactory[] aggregatorFactories; private final int bufferGrouperMaxSize; @@ -97,7 +78,6 @@ public class ConcurrentGrouper implements Grouper private final ObjectMapper spillMapper; private final int concurrencyHint; private final KeySerdeFactory keySerdeFactory; - private final KeySerdeFactory combineKeySerdeFactory; private final DefaultLimitSpec limitSpec; private final boolean sortHasNonGroupingFields; private final Comparator> keyObjComparator; @@ -105,6 +85,8 @@ public class ConcurrentGrouper implements Grouper private final int priority; private final boolean hasQueryTimeout; private final long queryTimeoutAt; + private final ParallelCombiner parallelCombiner; + private final long maxDictionarySizeForCombiner; private volatile boolean initialized = false; @@ -135,7 +117,6 @@ public ConcurrentGrouper( this.threadLocalGrouper = ThreadLocal.withInitial(() -> groupers.get(threadNumber.getAndIncrement())); this.bufferSupplier = bufferSupplier; - this.combineBufferSupplier = combineBufferSupplier; this.columnSelectorFactory = columnSelectorFactory; this.aggregatorFactories = aggregatorFactories; this.bufferGrouperMaxSize = bufferGrouperMaxSize; @@ -145,7 +126,6 @@ public ConcurrentGrouper( this.spillMapper = spillMapper; this.concurrencyHint = concurrencyHint; this.keySerdeFactory = keySerdeFactory; - this.combineKeySerdeFactory = combineKeySerdeFactory; this.limitSpec = limitSpec; this.sortHasNonGroupingFields = sortHasNonGroupingFields; this.keyObjComparator = keySerdeFactory.objectComparator(sortHasNonGroupingFields); @@ -153,6 +133,16 @@ public ConcurrentGrouper( this.priority = priority; this.hasQueryTimeout = hasQueryTimeout; this.queryTimeoutAt = queryTimeoutAt; + + this.parallelCombiner = new ParallelCombiner<>( + combineBufferSupplier, + getCombiningFactories(aggregatorFactories), + combineKeySerdeFactory, + executor, + keyObjComparator, + concurrencyHint + ); + this.maxDictionarySizeForCombiner = combineKeySerdeFactory.getMaxDictionarySize(); } @Override @@ -165,7 +155,7 @@ public void init() final int sliceSize = (buffer.capacity() / concurrencyHint); for (int i = 0; i < concurrencyHint; i++) { - final ByteBuffer slice = getSlice(buffer, sliceSize, i); + final ByteBuffer slice = Groupers.getSlice(buffer, sliceSize, i); final SpillingGrouper grouper = new SpillingGrouper<>( Suppliers.ofInstance(slice), keySerdeFactory, @@ -267,7 +257,7 @@ public CloseableIterator> iterator(final boolean sorted) // merged dictionary is used for all combining threads final List dictionary = tryMergeDictionary(); if (dictionary != null) { - return parallelCombine(sortedIterators, dictionary); + return parallelCombiner.combine(sortedIterators, dictionary); } } @@ -333,7 +323,6 @@ private List>> getGroupersIterator(boolean sort @Nullable private List tryMergeDictionary() { - final long maxDictionarySize = combineKeySerdeFactory.getMaxDictionarySize(); final Set mergedDictionary = new HashSet<>(); long totalDictionarySize = 0L; @@ -343,7 +332,7 @@ private List tryMergeDictionary() for (String key : dictionary) { if (!mergedDictionary.contains(key)) { totalDictionarySize += RowBasedGrouperHelper.estimateStringKeySize(key); - if (totalDictionarySize > maxDictionarySize) { + if (totalDictionarySize > maxDictionarySizeForCombiner) { return null; } mergedDictionary.add(key); @@ -354,274 +343,6 @@ private List tryMergeDictionary() return ImmutableList.copyOf(mergedDictionary); } - /** - * Build a combining tree for the input iterators which combine input entries asynchronously. Each node in the tree - * is a combining task which iterates through child iterators, aggregates the inputs from those iterators, and returns - * an iterator for the result of aggregation. - *

- * This method is called when data is spilled and thus streaming combine is preferred to avoid too many disk accesses. - * - * @return an iterator of the root grouper of the combining tree - */ - private CloseableIterator> parallelCombine( - List>> sortedIterators, - List mergedDictionary - ) - { - // CombineBuffer is initialized when this method is called - final ResourceHolder combineBufferHolder = combineBufferSupplier.get(); - final ByteBuffer combineBuffer = combineBufferHolder.get(); - final AggregatorFactory[] combiningFactories = new AggregatorFactory[aggregatorFactories.length]; - for (int i = 0; i < aggregatorFactories.length; i++) { - combiningFactories[i] = aggregatorFactories[i].getCombiningFactory(); - } - final int minimumRequiredBufferCapacity = StreamingMergeSortedGrouper.requiredBufferCapacity( - combineKeySerdeFactory.factorizeWithDictionary(mergedDictionary), - combiningFactories - ); - // We want to maximize the parallelism while the size of buffer slice is greater than the minimum buffer size - // required by StreamingMergeSortedGrouper. Here, we find the degree of the cominbing tree and the required number - // of buffers maximizing the degree of parallelism. - final Pair degreeAndNumBuffers = findCombineDegreeAndNumBuffers( - combineBuffer, - minimumRequiredBufferCapacity, - concurrencyHint, - sortedIterators.size() - ); - - final int combineDegree = degreeAndNumBuffers.lhs; - final int numBuffers = degreeAndNumBuffers.rhs; - final int sliceSize = combineBuffer.capacity() / numBuffers; - - final Supplier bufferSupplier = createCombineBufferSupplier(combineBuffer, numBuffers, sliceSize); - - final Pair>, List> combineIteratorAndFutures = buildCombineTree( - sortedIterators, - bufferSupplier, - combiningFactories, - combineDegree, - mergedDictionary - ); - - final CloseableIterator> combineIterator = combineIteratorAndFutures.lhs; - final List combineFutures = combineIteratorAndFutures.rhs; - - final Closer closer = Closer.create(); - closer.register(combineBufferHolder); - closer.register(() -> checkCombineFutures(combineFutures)); - - return CloseableIterators.wrap(combineIterator, closer); - } - - private static void checkCombineFutures(List combineFutures) - { - for (Future future : combineFutures) { - try { - // futures should be done before reaching here and throw exceptions if they failed - future.get(); - } - catch (InterruptedException e) { - throw new QueryInterruptedException(e); - } - catch (ExecutionException e) { - throw new RuntimeException(e); - } - } - } - - private static Supplier createCombineBufferSupplier( - ByteBuffer combineBuffer, - int numBuffers, - int sliceSize - ) - { - return new Supplier() - { - private int i = 0; - - @Override - public ByteBuffer get() - { - if (i < numBuffers) { - return getSlice(combineBuffer, sliceSize, i++); - } else { - throw new ISE("Requested number of buffer slices exceeds the planned one"); - } - } - }; - } - - /** - * Find a minimum size of the buffer slice and corresponding combining degree and number of slices. Note that each - * node in the combining tree is executed by different threads. This method assumes that higher degree of parallelism - * can exploit better performance and find such a shape of the combining tree. - * - * @param combineBuffer entire buffer used for combining tree - * @param requiredMinimumBufferCapacity minimum buffer capacity for {@link StreamingMergeSortedGrouper} - * @param concurrencyHint available degree of parallelism - * @param numLeafNodes number of leaf nodes of combining tree - * - * @return a pair of degree and number of buffers if found. - */ - private static Pair findCombineDegreeAndNumBuffers( - ByteBuffer combineBuffer, - int requiredMinimumBufferCapacity, - int concurrencyHint, - int numLeafNodes - ) - { - for (int degree = MINIMUM_COMBINE_DEGREE; degree <= numLeafNodes; degree++) { - final int requiredBufferNum = computeRequiredBufferNum(numLeafNodes, degree); - if (requiredBufferNum <= concurrencyHint) { - final int expectedSliceSize = combineBuffer.capacity() / requiredBufferNum; - if (expectedSliceSize >= requiredMinimumBufferCapacity) { - return Pair.of(degree, requiredBufferNum); - } - } - } - - throw new ISE("Cannot find a proper combine degree"); - } - - /** - * Recursively compute the number of required buffers for a combining tree in a top-down manner. Since each node of - * the combining tree represents a combining task and each combining task requires one buffer, the number of required - * buffers is the number of nodes of the combining tree. - * - * @param numLeafNodes number of leaf nodes - * @param combineDegree combine degree - * - * @return minimum number of buffers required for combining tree - * - * @see {@link #buildCombineTree(List, Supplier, AggregatorFactory[], int, List)} - */ - private static int computeRequiredBufferNum(int numLeafNodes, int combineDegree) - { - if (numLeafNodes > combineDegree) { - final int numLeafNodesPerChild = (numLeafNodes + combineDegree - 1) / combineDegree; // ceiling - int sum = 1; // count for the current node - for (int i = 0; i < combineDegree; i++) { - // further compute for child nodes - sum += computeRequiredBufferNum( - Math.min(numLeafNodesPerChild, numLeafNodes - i * numLeafNodesPerChild), - combineDegree - ); - } - - return sum; - } else { - return 1; - } - } - - /** - * Recursively build a combining tree in a top-down manner. Each node of the tree is a task that combines input - * iterators asynchronously. - * - * @param sortedIterators sorted iterators - * @param bufferSupplier combining buffer supplier - * @param combiningFactories array of combining aggregator factories - * @param combineDegree combining degree - * @param dictionary merged dictionary - * - * @return a pair of an iterator of the root of the combining tree and a list of futures of all executed combining - * tasks - */ - private Pair>, List> buildCombineTree( - List>> sortedIterators, - Supplier bufferSupplier, - AggregatorFactory[] combiningFactories, - int combineDegree, - List dictionary - ) - { - final int numIterators = sortedIterators.size(); - if (numIterators > combineDegree) { - final List>> childIterators = new ArrayList<>(combineDegree); - final List combineFutures = new ArrayList<>(combineDegree + 1); - - final int iteratorsPerChild = (numIterators + combineDegree - 1) / combineDegree; // ceiling - for (int i = 0; i < combineDegree; i++) { - final Pair>, List> childIteratorAndFutures = buildCombineTree( - sortedIterators.subList(i * iteratorsPerChild, Math.min(numIterators, (i + 1) * iteratorsPerChild)), - bufferSupplier, - combiningFactories, - combineDegree, - dictionary - ); - childIterators.add(childIteratorAndFutures.lhs); - combineFutures.addAll(childIteratorAndFutures.rhs); - } - final Pair>, Future> iteratorAndFuture = runCombiner( - childIterators, - bufferSupplier.get(), - combiningFactories, - dictionary - ); - combineFutures.add(iteratorAndFuture.rhs); - return new Pair<>(iteratorAndFuture.lhs, combineFutures); - } else { - final Pair>, Future> iteratorAndFuture = runCombiner( - sortedIterators, - bufferSupplier.get(), - combiningFactories, - dictionary - ); - return new Pair<>(iteratorAndFuture.lhs, Collections.singletonList(iteratorAndFuture.rhs)); - } - } - - private static ByteBuffer getSlice(ByteBuffer buffer, int sliceSize, int i) - { - final ByteBuffer slice = buffer.duplicate(); - slice.position(sliceSize * i); - slice.limit(slice.position() + sliceSize); - return slice.slice(); - } - - private Pair>, Future> runCombiner( - List>> iterators, - ByteBuffer combineBuffer, - AggregatorFactory[] combiningFactories, - List dictionary - ) - { - final SettableColumnSelectorFactory settableColumnSelectorFactory = - new SettableColumnSelectorFactory(aggregatorFactories); - final StreamingMergeSortedGrouper grouper = new StreamingMergeSortedGrouper<>( - Suppliers.ofInstance(combineBuffer), - combineKeySerdeFactory.factorizeWithDictionary(dictionary), - settableColumnSelectorFactory, - combiningFactories - ); - - final ListenableFuture future = executor.submit(() -> { - grouper.init(); - - try ( - CloseableIterator> mergedIterator = CloseableIterators.mergeSorted( - iterators, - keyObjComparator - ) - ) { - while (mergedIterator.hasNext()) { - final Entry next = mergedIterator.next(); - - settableColumnSelectorFactory.set(next.values); - grouper.aggregate(next.key); // grouper always returns ok or throws an exception - settableColumnSelectorFactory.set(null); - } - } - catch (IOException e) { - throw Throwables.propagate(e); - } - - grouper.finish(); - }); - - return new Pair<>(grouper.iterator(), future); - } - @Override public void close() { @@ -636,126 +357,12 @@ private int grouperNumberForKeyHash(int keyHash) return keyHash % groupers.size(); } - private static class SettableColumnSelectorFactory implements ColumnSelectorFactory + private AggregatorFactory[] getCombiningFactories(AggregatorFactory[] aggregatorFactories) { - private static final int UNKNOWN_COLUMN_INDEX = -1; - private final Object2IntMap columnIndexMap; - - private Object[] values; - - SettableColumnSelectorFactory(AggregatorFactory[] aggregatorFactories) - { - columnIndexMap = new Object2IntArrayMap<>(aggregatorFactories.length); - columnIndexMap.defaultReturnValue(UNKNOWN_COLUMN_INDEX); - for (int i = 0; i < aggregatorFactories.length; i++) { - columnIndexMap.put(aggregatorFactories[i].getName(), i); - } - } - - public void set(Object[] values) - { - this.values = values; - } - - private int checkAndGetColumnIndex(String columnName) - { - final int columnIndex = columnIndexMap.getInt(columnName); - Preconditions.checkState( - columnIndex != UNKNOWN_COLUMN_INDEX, - "Cannot find a proper column index for column[%s]", - columnName - ); - return columnIndex; - } - - @Override - public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) - { - throw new UnsupportedOperationException(); - } - - @Override - public FloatColumnSelector makeFloatColumnSelector(String columnName) - { - return new FloatColumnSelector() - { - @Override - public float getFloat() - { - return ((Number) values[checkAndGetColumnIndex(columnName)]).floatValue(); - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - // do nothing - } - }; - } - - @Override - public LongColumnSelector makeLongColumnSelector(String columnName) - { - return new LongColumnSelector() - { - @Override - public long getLong() - { - return ((Number) values[checkAndGetColumnIndex(columnName)]).longValue(); - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - // do nothing - } - }; - } - - @Override - public DoubleColumnSelector makeDoubleColumnSelector(String columnName) - { - return new DoubleColumnSelector() - { - @Override - public double getDouble() - { - return ((Number) values[checkAndGetColumnIndex(columnName)]).doubleValue(); - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - // do nothing - } - }; - } - - @Nullable - @Override - public ObjectColumnSelector makeObjectColumnSelector(String columnName) - { - return new ObjectColumnSelector() - { - @Override - public Class classOfObject() - { - return Object.class; - } - - @Override - public Object get() - { - return values[checkAndGetColumnIndex(columnName)]; - } - }; - } - - @Nullable - @Override - public ColumnCapabilities getColumnCapabilities(String column) - { - throw new UnsupportedOperationException(); + final AggregatorFactory[] combiningFactories = new AggregatorFactory[aggregatorFactories.length]; + for (int i = 0; i < aggregatorFactories.length; i++) { + combiningFactories[i] = aggregatorFactories[i].getCombiningFactory(); } + return combiningFactories; } } 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 22c8a78cc778..41b9c3d9c8e2 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 @@ -19,6 +19,8 @@ package io.druid.query.groupby.epinephelinae; +import java.nio.ByteBuffer; + public class Groupers { private Groupers() @@ -66,4 +68,12 @@ static int getUsedFlag(int keyHash) { return keyHash | 0x80000000; } + + public static ByteBuffer getSlice(ByteBuffer buffer, int sliceSize, int i) + { + final ByteBuffer slice = buffer.duplicate(); + slice.position(sliceSize * i); + slice.limit(slice.position() + sliceSize); + return slice.slice(); + } } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java new file mode 100644 index 000000000000..bc6db2f5dee3 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java @@ -0,0 +1,467 @@ +/* + * 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 com.google.common.base.Suppliers; +import com.google.common.base.Throwables; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import io.druid.collections.ResourceHolder; +import io.druid.java.util.common.CloseableIterators; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.Pair; +import io.druid.java.util.common.io.Closer; +import io.druid.java.util.common.parsers.CloseableIterator; +import io.druid.query.QueryInterruptedException; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.groupby.epinephelinae.Grouper.Entry; +import io.druid.query.groupby.epinephelinae.Grouper.KeySerdeFactory; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.ColumnSelectorFactory; +import io.druid.segment.DimensionSelector; +import io.druid.segment.DoubleColumnSelector; +import io.druid.segment.FloatColumnSelector; +import io.druid.segment.LongColumnSelector; +import io.druid.segment.ObjectColumnSelector; +import io.druid.segment.column.ColumnCapabilities; +import it.unimi.dsi.fastutil.objects.Object2IntArrayMap; +import it.unimi.dsi.fastutil.objects.Object2IntMap; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; + +/** + * ParallelCombiner builds a combining tree which asynchronously aggregates input entries. Each node of the combining + * tree is a combining task executed in parallel which aggregates inputs from the child nodes. + */ +public class ParallelCombiner +{ + private static final int MINIMUM_COMBINE_DEGREE = 2; + + private final Supplier> combineBufferSupplier; + private final AggregatorFactory[] combiningFactories; + private final KeySerdeFactory combineKeySerdeFactory; + private final ListeningExecutorService executor; + private final Comparator> keyObjComparator; + private final int concurrencyHint; + + public ParallelCombiner( + Supplier> combineBufferSupplier, + AggregatorFactory[] combiningFactories, + KeySerdeFactory combineKeySerdeFactory, + ListeningExecutorService executor, + Comparator> keyObjComparator, + int concurrencyHint + ) + { + this.combineBufferSupplier = combineBufferSupplier; + this.combiningFactories = combiningFactories; + this.combineKeySerdeFactory = combineKeySerdeFactory; + this.executor = executor; + this.keyObjComparator = keyObjComparator; + this.concurrencyHint = concurrencyHint; + } + + /** + * Build a combining tree for the input iterators which combine input entries asynchronously. Each node in the tree + * is a combining task which iterates through child iterators, aggregates the inputs from those iterators, and returns + * an iterator for the result of aggregation. + *

+ * This method is called when data is spilled and thus streaming combine is preferred to avoid too many disk accesses. + * + * @return an iterator of the root grouper of the combining tree + */ + public CloseableIterator> combine( + List>> sortedIterators, + List mergedDictionary + ) + { + // CombineBuffer is initialized when this method is called + final ResourceHolder combineBufferHolder = combineBufferSupplier.get(); + final ByteBuffer combineBuffer = combineBufferHolder.get(); + final int minimumRequiredBufferCapacity = StreamingMergeSortedGrouper.requiredBufferCapacity( + combineKeySerdeFactory.factorizeWithDictionary(mergedDictionary), + combiningFactories + ); + // We want to maximize the parallelism while the size of buffer slice is greater than the minimum buffer size + // required by StreamingMergeSortedGrouper. Here, we find the degree of the cominbing tree and the required number + // of buffers maximizing the degree of parallelism. + final Pair degreeAndNumBuffers = findCombineDegreeAndNumBuffers( + combineBuffer, + minimumRequiredBufferCapacity, + concurrencyHint, + sortedIterators.size() + ); + + final int combineDegree = degreeAndNumBuffers.lhs; + final int numBuffers = degreeAndNumBuffers.rhs; + final int sliceSize = combineBuffer.capacity() / numBuffers; + + final Supplier bufferSupplier = createCombineBufferSupplier(combineBuffer, numBuffers, sliceSize); + + final Pair>, List> combineIteratorAndFutures = buildCombineTree( + sortedIterators, + bufferSupplier, + combiningFactories, + combineDegree, + mergedDictionary + ); + + final CloseableIterator> combineIterator = combineIteratorAndFutures.lhs; + final List combineFutures = combineIteratorAndFutures.rhs; + + final Closer closer = Closer.create(); + closer.register(combineBufferHolder); + closer.register(() -> checkCombineFutures(combineFutures)); + + return CloseableIterators.wrap(combineIterator, closer); + } + + private static void checkCombineFutures(List combineFutures) + { + for (Future future : combineFutures) { + try { + // futures should be done before reaching here and throw exceptions if they failed + future.get(); + } + catch (InterruptedException e) { + throw new QueryInterruptedException(e); + } + catch (ExecutionException e) { + throw new RuntimeException(e); + } + } + } + + private static Supplier createCombineBufferSupplier( + ByteBuffer combineBuffer, + int numBuffers, + int sliceSize + ) + { + return new Supplier() + { + private int i = 0; + + @Override + public ByteBuffer get() + { + if (i < numBuffers) { + return Groupers.getSlice(combineBuffer, sliceSize, i++); + } else { + throw new ISE("Requested number of buffer slices exceeds the planned one"); + } + } + }; + } + + /** + * Find a minimum size of the buffer slice and corresponding combining degree and number of slices. Note that each + * node in the combining tree is executed by different threads. This method assumes that higher degree of parallelism + * can exploit better performance and find such a shape of the combining tree. + * + * @param combineBuffer entire buffer used for combining tree + * @param requiredMinimumBufferCapacity minimum buffer capacity for {@link StreamingMergeSortedGrouper} + * @param concurrencyHint available degree of parallelism + * @param numLeafNodes number of leaf nodes of combining tree + * + * @return a pair of degree and number of buffers if found. + */ + private static Pair findCombineDegreeAndNumBuffers( + ByteBuffer combineBuffer, + int requiredMinimumBufferCapacity, + int concurrencyHint, + int numLeafNodes + ) + { + for (int degree = MINIMUM_COMBINE_DEGREE; degree <= numLeafNodes; degree++) { + final int requiredBufferNum = computeRequiredBufferNum(numLeafNodes, degree); + if (requiredBufferNum <= concurrencyHint) { + final int expectedSliceSize = combineBuffer.capacity() / requiredBufferNum; + if (expectedSliceSize >= requiredMinimumBufferCapacity) { + return Pair.of(degree, requiredBufferNum); + } + } + } + + throw new ISE("Cannot find a proper combine degree"); + } + + /** + * Recursively compute the number of required buffers for a combining tree in a top-down manner. Since each node of + * the combining tree represents a combining task and each combining task requires one buffer, the number of required + * buffers is the number of nodes of the combining tree. + * + * @param numLeafNodes number of leaf nodes + * @param combineDegree combine degree + * + * @return minimum number of buffers required for combining tree + * + * @see {@link #buildCombineTree(List, Supplier, AggregatorFactory[], int, List)} + */ + private static int computeRequiredBufferNum(int numLeafNodes, int combineDegree) + { + if (numLeafNodes > combineDegree) { + final int numLeafNodesPerChild = (numLeafNodes + combineDegree - 1) / combineDegree; // ceiling + int sum = 1; // count for the current node + for (int i = 0; i < combineDegree; i++) { + // further compute for child nodes + sum += computeRequiredBufferNum( + Math.min(numLeafNodesPerChild, numLeafNodes - i * numLeafNodesPerChild), + combineDegree + ); + } + + return sum; + } else { + return 1; + } + } + + /** + * Recursively build a combining tree in a top-down manner. Each node of the tree is a task that combines input + * iterators asynchronously. + * + * @param sortedIterators sorted iterators + * @param bufferSupplier combining buffer supplier + * @param combiningFactories array of combining aggregator factories + * @param combineDegree combining degree + * @param dictionary merged dictionary + * + * @return a pair of an iterator of the root of the combining tree and a list of futures of all executed combining + * tasks + */ + private Pair>, List> buildCombineTree( + List>> sortedIterators, + Supplier bufferSupplier, + AggregatorFactory[] combiningFactories, + int combineDegree, + List dictionary + ) + { + final int numIterators = sortedIterators.size(); + if (numIterators > combineDegree) { + final List>> childIterators = new ArrayList<>(combineDegree); + final List combineFutures = new ArrayList<>(combineDegree + 1); + + final int iteratorsPerChild = (numIterators + combineDegree - 1) / combineDegree; // ceiling + for (int i = 0; i < combineDegree; i++) { + final Pair>, List> childIteratorAndFutures = buildCombineTree( + sortedIterators.subList(i * iteratorsPerChild, Math.min(numIterators, (i + 1) * iteratorsPerChild)), + bufferSupplier, + combiningFactories, + combineDegree, + dictionary + ); + childIterators.add(childIteratorAndFutures.lhs); + combineFutures.addAll(childIteratorAndFutures.rhs); + } + final Pair>, Future> iteratorAndFuture = runCombiner( + childIterators, + bufferSupplier.get(), + combiningFactories, + dictionary + ); + combineFutures.add(iteratorAndFuture.rhs); + return new Pair<>(iteratorAndFuture.lhs, combineFutures); + } else { + final Pair>, Future> iteratorAndFuture = runCombiner( + sortedIterators, + bufferSupplier.get(), + combiningFactories, + dictionary + ); + return new Pair<>(iteratorAndFuture.lhs, Collections.singletonList(iteratorAndFuture.rhs)); + } + } + + private Pair>, Future> runCombiner( + List>> iterators, + ByteBuffer combineBuffer, + AggregatorFactory[] combiningFactories, + List dictionary + ) + { + final SettableColumnSelectorFactory settableColumnSelectorFactory = + new SettableColumnSelectorFactory(combiningFactories); + final StreamingMergeSortedGrouper grouper = new StreamingMergeSortedGrouper<>( + Suppliers.ofInstance(combineBuffer), + combineKeySerdeFactory.factorizeWithDictionary(dictionary), + settableColumnSelectorFactory, + combiningFactories + ); + + final ListenableFuture future = executor.submit(() -> { + grouper.init(); + + try ( + CloseableIterator> mergedIterator = CloseableIterators.mergeSorted( + iterators, + keyObjComparator + ) + ) { + while (mergedIterator.hasNext()) { + final Entry next = mergedIterator.next(); + + settableColumnSelectorFactory.set(next.values); + grouper.aggregate(next.key); // grouper always returns ok or throws an exception + settableColumnSelectorFactory.set(null); + } + } + catch (IOException e) { + throw Throwables.propagate(e); + } + + grouper.finish(); + }); + + return new Pair<>(grouper.iterator(), future); + } + + private static class SettableColumnSelectorFactory implements ColumnSelectorFactory + { + private static final int UNKNOWN_COLUMN_INDEX = -1; + private final Object2IntMap columnIndexMap; + + private Object[] values; + + SettableColumnSelectorFactory(AggregatorFactory[] aggregatorFactories) + { + columnIndexMap = new Object2IntArrayMap<>(aggregatorFactories.length); + columnIndexMap.defaultReturnValue(UNKNOWN_COLUMN_INDEX); + for (int i = 0; i < aggregatorFactories.length; i++) { + columnIndexMap.put(aggregatorFactories[i].getName(), i); + } + } + + public void set(Object[] values) + { + this.values = values; + } + + private int checkAndGetColumnIndex(String columnName) + { + final int columnIndex = columnIndexMap.getInt(columnName); + Preconditions.checkState( + columnIndex != UNKNOWN_COLUMN_INDEX, + "Cannot find a proper column index for column[%s]", + columnName + ); + return columnIndex; + } + + @Override + public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) + { + throw new UnsupportedOperationException(); + } + + @Override + public FloatColumnSelector makeFloatColumnSelector(String columnName) + { + return new FloatColumnSelector() + { + @Override + public float getFloat() + { + return ((Number) values[checkAndGetColumnIndex(columnName)]).floatValue(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // do nothing + } + }; + } + + @Override + public LongColumnSelector makeLongColumnSelector(String columnName) + { + return new LongColumnSelector() + { + @Override + public long getLong() + { + return ((Number) values[checkAndGetColumnIndex(columnName)]).longValue(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // do nothing + } + }; + } + + @Override + public DoubleColumnSelector makeDoubleColumnSelector(String columnName) + { + return new DoubleColumnSelector() + { + @Override + public double getDouble() + { + return ((Number) values[checkAndGetColumnIndex(columnName)]).doubleValue(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // do nothing + } + }; + } + + @Override + public ObjectColumnSelector makeObjectColumnSelector(String columnName) + { + return new ObjectColumnSelector() + { + @Override + public Class classOfObject() + { + return Object.class; + } + + @Override + public Object get() + { + return values[checkAndGetColumnIndex(columnName)]; + } + }; + } + + @Override + public ColumnCapabilities getColumnCapabilities(String column) + { + throw new UnsupportedOperationException(); + } + } +} From 3cdce754f252710d28f2c8fbacdd2fb7962bab23 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 2 Sep 2017 20:02:19 +0900 Subject: [PATCH 14/44] Add ParallelCombinerTest --- .../epinephelinae/ConcurrentGrouper.java | 4 +- .../epinephelinae/ParallelCombiner.java | 14 +- .../epinephelinae/ConcurrentGrouperTest.java | 39 ++--- .../epinephelinae/ParallelCombinerTest.java | 140 ++++++++++++++++++ 4 files changed, 171 insertions(+), 26 deletions(-) create mode 100644 processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java 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 d1a67629662e..46edd1f917da 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 @@ -139,8 +139,8 @@ public ConcurrentGrouper( getCombiningFactories(aggregatorFactories), combineKeySerdeFactory, executor, - keyObjComparator, - concurrencyHint + concurrencyHint, + sortHasNonGroupingFields ); this.maxDictionarySizeForCombiner = combineKeySerdeFactory.getMaxDictionarySize(); } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java index bc6db2f5dee3..23e4e5dd55ca 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java @@ -76,15 +76,15 @@ public ParallelCombiner( AggregatorFactory[] combiningFactories, KeySerdeFactory combineKeySerdeFactory, ListeningExecutorService executor, - Comparator> keyObjComparator, - int concurrencyHint + int concurrencyHint, + boolean sortHasNonGroupingFields ) { this.combineBufferSupplier = combineBufferSupplier; this.combiningFactories = combiningFactories; this.combineKeySerdeFactory = combineKeySerdeFactory; this.executor = executor; - this.keyObjComparator = keyObjComparator; + this.keyObjComparator = combineKeySerdeFactory.objectComparator(sortHasNonGroupingFields);; this.concurrencyHint = concurrencyHint; } @@ -98,11 +98,11 @@ public ParallelCombiner( * @return an iterator of the root grouper of the combining tree */ public CloseableIterator> combine( - List>> sortedIterators, + List>> sortedIterators, List mergedDictionary ) { - // CombineBuffer is initialized when this method is called + // CombineBuffer is initialized when this method is called and closed after the result iterator is done final ResourceHolder combineBufferHolder = combineBufferSupplier.get(); final ByteBuffer combineBuffer = combineBufferHolder.get(); final int minimumRequiredBufferCapacity = StreamingMergeSortedGrouper.requiredBufferCapacity( @@ -258,7 +258,7 @@ private static int computeRequiredBufferNum(int numLeafNodes, int combineDegree) * tasks */ private Pair>, List> buildCombineTree( - List>> sortedIterators, + List>> sortedIterators, Supplier bufferSupplier, AggregatorFactory[] combiningFactories, int combineDegree, @@ -302,7 +302,7 @@ private Pair>, List> buildCombineTree( } private Pair>, Future> runCombiner( - List>> iterators, + List>> iterators, ByteBuffer combineBuffer, AggregatorFactory[] combiningFactories, List dictionary diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java index 5a79ef4a6012..6259dd111d4d 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java @@ -26,8 +26,8 @@ import com.google.common.util.concurrent.MoreExecutors; import io.druid.collections.ResourceHolder; import io.druid.jackson.DefaultObjectMapper; -import io.druid.java.util.common.parsers.CloseableIterator; import io.druid.java.util.common.IAE; +import io.druid.java.util.common.parsers.CloseableIterator; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.dimension.DimensionSpec; @@ -61,19 +61,19 @@ public class ConcurrentGrouperTest { - private static final ExecutorService service = Executors.newFixedThreadPool(8); - private static final TestResourceHolder testResourceHolder = new TestResourceHolder(); + private static final ExecutorService SERVICE = Executors.newFixedThreadPool(8); + private static final TestResourceHolder TEST_RESOURCE_HOLDER = new TestResourceHolder(256); @AfterClass public static void teardown() { - service.shutdown(); + SERVICE.shutdown(); } @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - private static final Supplier bufferSupplier = new Supplier() + private static final Supplier BUFFER_SUPPLIER = new Supplier() { private final AtomicBoolean called = new AtomicBoolean(false); private ByteBuffer buffer; @@ -89,7 +89,7 @@ public ByteBuffer get() } }; - private static final Supplier> combineBufferSupplier = new Supplier>() + private static final Supplier> COMBINE_BUFFER_SUPPLIER = new Supplier>() { private final AtomicBoolean called = new AtomicBoolean(false); @@ -97,17 +97,22 @@ public ByteBuffer get() public ResourceHolder get() { if (called.compareAndSet(false, true)) { - return testResourceHolder; + return TEST_RESOURCE_HOLDER; } else { throw new IAE("should be called once"); } } }; - private static class TestResourceHolder implements ResourceHolder + static class TestResourceHolder implements ResourceHolder { private boolean closed; - private ByteBuffer buffer = ByteBuffer.allocate(256); + private ByteBuffer buffer; + + TestResourceHolder(int bufferSize) + { + buffer = ByteBuffer.allocate(bufferSize); + } @Override public ByteBuffer get() @@ -122,7 +127,7 @@ public void close() } } - private static final KeySerdeFactory keySerdeFactory = new KeySerdeFactory() + static final KeySerdeFactory KEY_SERDE_FACTORY = new KeySerdeFactory() { @Override public long getMaxDictionarySize() @@ -260,10 +265,10 @@ public DoubleColumnSelector makeDoubleColumnSelector(String columnName) public void testAggregate() throws InterruptedException, ExecutionException, IOException { final ConcurrentGrouper grouper = new ConcurrentGrouper<>( - bufferSupplier, - combineBufferSupplier, - keySerdeFactory, - keySerdeFactory, + BUFFER_SUPPLIER, + COMBINE_BUFFER_SUPPLIER, + KEY_SERDE_FACTORY, + KEY_SERDE_FACTORY, null_factory, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, 24, @@ -274,7 +279,7 @@ public void testAggregate() throws InterruptedException, ExecutionException, IOE 8, null, false, - MoreExecutors.listeningDecorator(service), + MoreExecutors.listeningDecorator(SERVICE), 0, false, 0 @@ -286,7 +291,7 @@ public void testAggregate() throws InterruptedException, ExecutionException, IOE Future[] futures = new Future[8]; for (int i = 0; i < 8; i++) { - futures[i] = service.submit(new Runnable() + futures[i] = SERVICE.submit(new Runnable() { @Override public void run() @@ -306,7 +311,7 @@ public void run() final List> actual = Lists.newArrayList(iterator); iterator.close(); - Assert.assertTrue(testResourceHolder.closed); + Assert.assertTrue(TEST_RESOURCE_HOLDER.closed); final List> expected = new ArrayList<>(); for (long i = 0; i < numRows; i++) { diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java new file mode 100644 index 000000000000..ae5325c47ede --- /dev/null +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java @@ -0,0 +1,140 @@ +/* + * 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.Supplier; +import com.google.common.util.concurrent.MoreExecutors; +import io.druid.collections.ResourceHolder; +import io.druid.concurrent.Execs; +import io.druid.java.util.common.IAE; +import io.druid.java.util.common.parsers.CloseableIterator; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.groupby.epinephelinae.ConcurrentGrouperTest.TestResourceHolder; +import io.druid.query.groupby.epinephelinae.Grouper.Entry; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicBoolean; + +public class ParallelCombinerTest +{ + private static final int THREAD_NUM = 8; + private static final ExecutorService SERVICE = Execs.multiThreaded(THREAD_NUM, "parallel-combiner-test-%d"); + private static final TestResourceHolder TEST_RESOURCE_HOLDER = new TestResourceHolder(512); + + private static final Supplier> COMBINE_BUFFER_SUPPLIER = + new Supplier>() + { + private final AtomicBoolean called = new AtomicBoolean(false); + + @Override + public ResourceHolder get() + { + if (called.compareAndSet(false, true)) { + return TEST_RESOURCE_HOLDER; + } else { + throw new IAE("should be called once"); + } + } + }; + + private static final class TestIterator implements CloseableIterator> + { + private final Iterator> innerIterator; + private boolean closed; + + TestIterator(Iterator> innerIterator) + { + this.innerIterator = innerIterator; + } + + @Override + public boolean hasNext() + { + return innerIterator.hasNext(); + } + + @Override + public Entry next() + { + return innerIterator.next(); + } + + public boolean isClosed() + { + return closed; + } + + @Override + public void close() throws IOException + { + if (!closed) { + closed = true; + } + } + } + + @AfterClass + public static void teardown() + { + SERVICE.shutdown(); + } + + @Test + public void testCombine() throws IOException + { + final ParallelCombiner combiner = new ParallelCombiner<>( + COMBINE_BUFFER_SUPPLIER, + new AggregatorFactory[]{new CountAggregatorFactory("cnt").getCombiningFactory()}, + ConcurrentGrouperTest.KEY_SERDE_FACTORY, + MoreExecutors.listeningDecorator(SERVICE), + THREAD_NUM, + false + ); + + final int numRows = 1000; + final List> baseIterator = new ArrayList<>(numRows); + for (long i = 0; i < numRows; i++) { + baseIterator.add(new Entry<>(i, new Object[]{i * 10})); + } + + final List iterators = new ArrayList<>(8); + for (int i = 0; i < 8; i++) { + iterators.add(new TestIterator(baseIterator.iterator())); + } + + try (final CloseableIterator> iterator = combiner.combine(iterators, new ArrayList<>())) { + long expectedKey = 0; + while (iterator.hasNext()) { + Assert.assertEquals(new Entry<>(expectedKey, new Object[]{expectedKey++ * 80}), iterator.next()); + } + } + + iterators.forEach(it -> Assert.assertTrue(it.isClosed())); + } +} From 2a619b946a235e063efa956345d1f28f6cd3bcc7 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 2 Sep 2017 20:15:58 +0900 Subject: [PATCH 15/44] Handle InterruptedException --- .../query/groupby/epinephelinae/ParallelCombiner.java | 11 ++++++++--- .../epinephelinae/StreamingMergeSortedGrouper.java | 8 ++++---- .../groupby/epinephelinae/ParallelCombinerTest.java | 2 +- 3 files changed, 13 insertions(+), 8 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java index 23e4e5dd55ca..ad3326d7ba40 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java @@ -53,6 +53,7 @@ import java.util.Collections; import java.util.Comparator; import java.util.List; +import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; @@ -147,10 +148,14 @@ private static void checkCombineFutures(List combineFutures) { for (Future future : combineFutures) { try { - // futures should be done before reaching here and throw exceptions if they failed - future.get(); + if (!future.isDone()) { + // Cancel futures if close() for the iterator is called early due to some reason (e.g., test failure) + future.cancel(true); + } else { + future.get(); + } } - catch (InterruptedException e) { + catch (InterruptedException | CancellationException e) { throw new QueryInterruptedException(e); } catch (ExecutionException e) { diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java index c8e0c8807626..2e58ea87f4eb 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java @@ -239,7 +239,7 @@ private void increaseWriteIndex() { if (curWriteIndex == maxSlotNum - 1) { final long startLoopAt = System.currentTimeMillis(); - while (nextReadIndex == -1 || nextReadIndex == 0) { + while ((nextReadIndex == -1 || nextReadIndex == 0) && !Thread.interrupted()) { if (System.currentTimeMillis() - startLoopAt > timeoutMs) { throw new RuntimeException(new TimeoutException()); } @@ -248,7 +248,7 @@ private void increaseWriteIndex() } else { final int nextWriteIndex = curWriteIndex + 1; final long startLoopAt = System.currentTimeMillis(); - while (nextWriteIndex == nextReadIndex) { + while ((nextWriteIndex == nextReadIndex) && !Thread.interrupted()) { if (System.currentTimeMillis() - startLoopAt > timeoutMs) { throw new RuntimeException(new TimeoutException()); } @@ -301,7 +301,7 @@ public CloseableIterator> iterator() { // Waits for some data to be ready final long startLoopAt = System.currentTimeMillis(); - while ((curWriteIndex == -1 || curWriteIndex == 0) && !finished) { + while ((curWriteIndex == -1 || curWriteIndex == 0) && !finished && !Thread.interrupted()) { if (System.currentTimeMillis() - startLoopAt > timeoutMs) { throw new RuntimeException(new TimeoutException()); } @@ -343,7 +343,7 @@ public Entry next() if (!finished) { final long startLoopAt = System.currentTimeMillis(); - while (toBeUpdated == curWriteIndex && !finished) { + while (toBeUpdated == curWriteIndex && !finished && !Thread.interrupted()) { if (System.currentTimeMillis() - startLoopAt > timeoutMs) { throw new RuntimeException(new TimeoutException()); } diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java index ae5325c47ede..5c1ec778ca8d 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java @@ -102,7 +102,7 @@ public void close() throws IOException @AfterClass public static void teardown() { - SERVICE.shutdown(); + SERVICE.shutdownNow(); } @Test From 3f14db44e5fc619c6b219ca33ed6864de1243850 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 2 Sep 2017 20:28:10 +0900 Subject: [PATCH 16/44] use AbstractPrioritizedCallable --- .../epinephelinae/ConcurrentGrouper.java | 3 +- .../epinephelinae/ParallelCombiner.java | 58 +++++++++++-------- .../epinephelinae/ParallelCombinerTest.java | 3 +- 3 files changed, 39 insertions(+), 25 deletions(-) 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 46edd1f917da..6e47ebaa2e2d 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 @@ -139,8 +139,9 @@ public ConcurrentGrouper( getCombiningFactories(aggregatorFactories), combineKeySerdeFactory, executor, + sortHasNonGroupingFields, concurrencyHint, - sortHasNonGroupingFields + priority ); this.maxDictionarySizeForCombiner = combineKeySerdeFactory.getMaxDictionarySize(); } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java index ad3326d7ba40..e14de0bc997c 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java @@ -31,6 +31,7 @@ import io.druid.java.util.common.Pair; import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.parsers.CloseableIterator; +import io.druid.query.AbstractPrioritizedCallable; import io.druid.query.QueryInterruptedException; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.dimension.DimensionSpec; @@ -71,14 +72,16 @@ public class ParallelCombiner private final ListeningExecutorService executor; private final Comparator> keyObjComparator; private final int concurrencyHint; + private final int priority; public ParallelCombiner( Supplier> combineBufferSupplier, AggregatorFactory[] combiningFactories, KeySerdeFactory combineKeySerdeFactory, ListeningExecutorService executor, + boolean sortHasNonGroupingFields, int concurrencyHint, - boolean sortHasNonGroupingFields + int priority ) { this.combineBufferSupplier = combineBufferSupplier; @@ -87,6 +90,7 @@ public ParallelCombiner( this.executor = executor; this.keyObjComparator = combineKeySerdeFactory.objectComparator(sortHasNonGroupingFields);; this.concurrencyHint = concurrencyHint; + this.priority = priority; } /** @@ -322,29 +326,37 @@ private Pair>, Future> runCombiner( combiningFactories ); - final ListenableFuture future = executor.submit(() -> { - grouper.init(); - - try ( - CloseableIterator> mergedIterator = CloseableIterators.mergeSorted( - iterators, - keyObjComparator - ) - ) { - while (mergedIterator.hasNext()) { - final Entry next = mergedIterator.next(); - - settableColumnSelectorFactory.set(next.values); - grouper.aggregate(next.key); // grouper always returns ok or throws an exception - settableColumnSelectorFactory.set(null); + final ListenableFuture future = executor.submit( + new AbstractPrioritizedCallable(priority) + { + @Override + public Void call() throws Exception + { + grouper.init(); + + try ( + CloseableIterator> mergedIterator = CloseableIterators.mergeSorted( + iterators, + keyObjComparator + ) + ) { + while (mergedIterator.hasNext()) { + final Entry next = mergedIterator.next(); + + settableColumnSelectorFactory.set(next.values); + grouper.aggregate(next.key); // grouper always returns ok or throws an exception + settableColumnSelectorFactory.set(null); + } + } + catch (IOException e) { + throw Throwables.propagate(e); + } + + grouper.finish(); + return null; + } } - } - catch (IOException e) { - throw Throwables.propagate(e); - } - - grouper.finish(); - }); + ); return new Pair<>(grouper.iterator(), future); } diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java index 5c1ec778ca8d..0fe75d822cc9 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java @@ -113,8 +113,9 @@ public void testCombine() throws IOException new AggregatorFactory[]{new CountAggregatorFactory("cnt").getCombiningFactory()}, ConcurrentGrouperTest.KEY_SERDE_FACTORY, MoreExecutors.listeningDecorator(SERVICE), + false, THREAD_NUM, - false + 0 ); final int numRows = 1000; From dee9633b599652b6ee72661dc250ad06e8de6e70 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 7 Sep 2017 21:35:50 +0900 Subject: [PATCH 17/44] Address comments --- .../CloseableGrouperIterator.java | 3 +- .../epinephelinae/ConcurrentGrouper.java | 8 +- .../epinephelinae/ParallelCombiner.java | 142 ++++++++++-------- .../StreamingMergeSortedGrouper.java | 7 +- .../epinephelinae/ParallelCombinerTest.java | 7 +- 5 files changed, 90 insertions(+), 77 deletions(-) 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 index 6accdfd7752b..c8002c79afc2 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/CloseableGrouperIterator.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/CloseableGrouperIterator.java @@ -19,7 +19,6 @@ package io.druid.query.groupby.epinephelinae; -import com.google.common.base.Throwables; import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.parsers.CloseableIterator; import io.druid.query.groupby.epinephelinae.Grouper.Entry; @@ -74,7 +73,7 @@ public void close() closer.close(); } catch (IOException e) { - throw Throwables.propagate(e); + throw new RuntimeException(e); } } } 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 6e47ebaa2e2d..37906ca8a73b 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 @@ -40,6 +40,7 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Arrays; import java.util.Comparator; import java.util.HashSet; import java.util.List; @@ -331,12 +332,11 @@ private List tryMergeDictionary() final List dictionary = grouper.getDictionary(); for (String key : dictionary) { - if (!mergedDictionary.contains(key)) { + if (mergedDictionary.add(key)) { totalDictionarySize += RowBasedGrouperHelper.estimateStringKeySize(key); if (totalDictionarySize > maxDictionarySizeForCombiner) { return null; } - mergedDictionary.add(key); } } } @@ -361,9 +361,7 @@ private int grouperNumberForKeyHash(int keyHash) private AggregatorFactory[] getCombiningFactories(AggregatorFactory[] aggregatorFactories) { final AggregatorFactory[] combiningFactories = new AggregatorFactory[aggregatorFactories.length]; - for (int i = 0; i < aggregatorFactories.length; i++) { - combiningFactories[i] = aggregatorFactories[i].getCombiningFactory(); - } + Arrays.setAll(combiningFactories, i -> aggregatorFactories[i].getCombiningFactory()); return combiningFactories; } } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java index e14de0bc997c..02c6a1caff9a 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java @@ -23,6 +23,7 @@ import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.base.Throwables; +import com.google.common.collect.Iterables; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import io.druid.collections.ResourceHolder; @@ -51,7 +52,6 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.concurrent.CancellationException; @@ -88,7 +88,7 @@ public ParallelCombiner( this.combiningFactories = combiningFactories; this.combineKeySerdeFactory = combineKeySerdeFactory; this.executor = executor; - this.keyObjComparator = combineKeySerdeFactory.objectComparator(sortHasNonGroupingFields);; + this.keyObjComparator = combineKeySerdeFactory.objectComparator(sortHasNonGroupingFields); this.concurrencyHint = concurrencyHint; this.priority = priority; } @@ -130,7 +130,7 @@ public CloseableIterator> combine( final Supplier bufferSupplier = createCombineBufferSupplier(combineBuffer, numBuffers, sliceSize); - final Pair>, List> combineIteratorAndFutures = buildCombineTree( + final Pair>>, List> combineIteratorAndFutures = buildCombineTree( sortedIterators, bufferSupplier, combiningFactories, @@ -138,7 +138,7 @@ public CloseableIterator> combine( mergedDictionary ); - final CloseableIterator> combineIterator = combineIteratorAndFutures.lhs; + final CloseableIterator> combineIterator = Iterables.getOnlyElement(combineIteratorAndFutures.lhs); final List combineFutures = combineIteratorAndFutures.rhs; final Closer closer = Closer.create(); @@ -192,12 +192,12 @@ public ByteBuffer get() /** * Find a minimum size of the buffer slice and corresponding combining degree and number of slices. Note that each - * node in the combining tree is executed by different threads. This method assumes that higher degree of parallelism - * can exploit better performance and find such a shape of the combining tree. + * node in the combining tree is executed by different threads. This method assumes that using more threads can + * exploit better performance and find such a shape of the combining tree. * * @param combineBuffer entire buffer used for combining tree * @param requiredMinimumBufferCapacity minimum buffer capacity for {@link StreamingMergeSortedGrouper} - * @param concurrencyHint available degree of parallelism + * @param numAvailableThreads number of available threads * @param numLeafNodes number of leaf nodes of combining tree * * @return a pair of degree and number of buffers if found. @@ -205,13 +205,13 @@ public ByteBuffer get() private static Pair findCombineDegreeAndNumBuffers( ByteBuffer combineBuffer, int requiredMinimumBufferCapacity, - int concurrencyHint, + int numAvailableThreads, int numLeafNodes ) { for (int degree = MINIMUM_COMBINE_DEGREE; degree <= numLeafNodes; degree++) { final int requiredBufferNum = computeRequiredBufferNum(numLeafNodes, degree); - if (requiredBufferNum <= concurrencyHint) { + if (requiredBufferNum <= numAvailableThreads) { final int expectedSliceSize = combineBuffer.capacity() / requiredBufferNum; if (expectedSliceSize >= requiredMinimumBufferCapacity) { return Pair.of(degree, requiredBufferNum); @@ -223,90 +223,101 @@ private static Pair findCombineDegreeAndNumBuffers( } /** - * Recursively compute the number of required buffers for a combining tree in a top-down manner. Since each node of + * Recursively compute the number of required buffers for a combining tree in a bottom-up manner. Since each node of * the combining tree represents a combining task and each combining task requires one buffer, the number of required * buffers is the number of nodes of the combining tree. * - * @param numLeafNodes number of leaf nodes + * @param numChildNodes number of child nodes * @param combineDegree combine degree * * @return minimum number of buffers required for combining tree * - * @see {@link #buildCombineTree(List, Supplier, AggregatorFactory[], int, List)} + * @see #buildCombineTree(List, Supplier, AggregatorFactory[], int, List) */ - private static int computeRequiredBufferNum(int numLeafNodes, int combineDegree) + private static int computeRequiredBufferNum(int numChildNodes, int combineDegree) { - if (numLeafNodes > combineDegree) { - final int numLeafNodesPerChild = (numLeafNodes + combineDegree - 1) / combineDegree; // ceiling - int sum = 1; // count for the current node - for (int i = 0; i < combineDegree; i++) { - // further compute for child nodes - sum += computeRequiredBufferNum( - Math.min(numLeafNodesPerChild, numLeafNodes - i * numLeafNodesPerChild), - combineDegree - ); - } - - return sum; + // numChildrenForLastNode used to determine that the last node is needed for the current level. + // Please see buildCombineTree() for more details. + final int numChildrenForLastNode = numChildNodes % combineDegree; + final int numCurLevelNodes = numChildNodes / combineDegree + (numChildrenForLastNode > 1 ? 1 : 0); + final int numChildOfParentNodes = numCurLevelNodes + (numChildrenForLastNode == 1 ? 1 : 0); + + if (numChildOfParentNodes == 1) { + return numCurLevelNodes; } else { - return 1; + return numCurLevelNodes + + computeRequiredBufferNum(numChildOfParentNodes, combineDegree); } } /** - * Recursively build a combining tree in a top-down manner. Each node of the tree is a task that combines input + * Recursively build a combining tree in a bottom-up manner. Each node of the tree is a task that combines input * iterators asynchronously. * - * @param sortedIterators sorted iterators - * @param bufferSupplier combining buffer supplier - * @param combiningFactories array of combining aggregator factories - * @param combineDegree combining degree - * @param dictionary merged dictionary + * @param childIterators all iterators of the child level + * @param bufferSupplier combining buffer supplier + * @param combiningFactories array of combining aggregator factories + * @param combineDegree combining degree + * @param dictionary merged dictionary * - * @return a pair of an iterator of the root of the combining tree and a list of futures of all executed combining - * tasks + * @return a pair of a list of iterators of the current level in the combining tree and a list of futures of all + * executed combining tasks */ - private Pair>, List> buildCombineTree( - List>> sortedIterators, + private Pair>>, List> buildCombineTree( + List>> childIterators, Supplier bufferSupplier, AggregatorFactory[] combiningFactories, int combineDegree, List dictionary ) { - final int numIterators = sortedIterators.size(); - if (numIterators > combineDegree) { - final List>> childIterators = new ArrayList<>(combineDegree); - final List combineFutures = new ArrayList<>(combineDegree + 1); - - final int iteratorsPerChild = (numIterators + combineDegree - 1) / combineDegree; // ceiling - for (int i = 0; i < combineDegree; i++) { - final Pair>, List> childIteratorAndFutures = buildCombineTree( - sortedIterators.subList(i * iteratorsPerChild, Math.min(numIterators, (i + 1) * iteratorsPerChild)), - bufferSupplier, + final int numChildLevelIterators = childIterators.size(); + final List>> childIteratorsOfNextLevel = new ArrayList<>(); + final List combineFutures = new ArrayList<>(); + + // The below algorithm creates the combining nodes of the current level. It first checks that the number of children + // to be combined together is 1. If it is, the intermediate combining node for that child is not needed. Instead, it + // can be directly connected to a node of the parent level. Here is an example of generated tree when + // numLeafNodes = 6 and combineDegree = 2. + // o + // / \ + // o \ + // / \ \ + // o o o + // / \ / \ / \ + // o o o o o o + + int i; + for (i = 0; i < numChildLevelIterators; i += combineDegree) { + if (i < numChildLevelIterators - 1) { + final List>> subIterators = childIterators.subList( + i, + Math.min(i + combineDegree, numChildLevelIterators) + ); + final Pair>, Future> iteratorAndFuture = runCombiner( + subIterators, + bufferSupplier.get(), combiningFactories, - combineDegree, dictionary ); - childIterators.add(childIteratorAndFutures.lhs); - combineFutures.addAll(childIteratorAndFutures.rhs); + + childIteratorsOfNextLevel.add(iteratorAndFuture.lhs); + combineFutures.add(iteratorAndFuture.rhs); + } else { + // If there remains one child, it can be directly connected to a node of the parent level. + childIteratorsOfNextLevel.add(childIterators.get(i)); } - final Pair>, Future> iteratorAndFuture = runCombiner( - childIterators, - bufferSupplier.get(), - combiningFactories, - dictionary - ); - combineFutures.add(iteratorAndFuture.rhs); - return new Pair<>(iteratorAndFuture.lhs, combineFutures); + } + + if (childIteratorsOfNextLevel.size() == 1) { + // This is the root + return Pair.of(childIteratorsOfNextLevel, combineFutures); } else { - final Pair>, Future> iteratorAndFuture = runCombiner( - sortedIterators, - bufferSupplier.get(), - combiningFactories, - dictionary - ); - return new Pair<>(iteratorAndFuture.lhs, Collections.singletonList(iteratorAndFuture.rhs)); + // Build the parent level iterators + final Pair>>, List> parentIteratorsAndFutures = + buildCombineTree(childIteratorsOfNextLevel, bufferSupplier, combiningFactories, combineDegree, dictionary); + combineFutures.addAll(parentIteratorsAndFutures.rhs); + return Pair.of(parentIteratorsAndFutures.lhs, combineFutures); } } @@ -325,6 +336,7 @@ private Pair>, Future> runCombiner( settableColumnSelectorFactory, combiningFactories ); + grouper.init(); final ListenableFuture future = executor.submit( new AbstractPrioritizedCallable(priority) @@ -332,8 +344,6 @@ private Pair>, Future> runCombiner( @Override public Void call() throws Exception { - grouper.init(); - try ( CloseableIterator> mergedIterator = CloseableIterators.mergeSorted( iterators, diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java index 2e58ea87f4eb..5a2d429151ed 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java @@ -22,9 +22,10 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Supplier; -import io.druid.java.util.common.parsers.CloseableIterator; import io.druid.java.util.common.IAE; +import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; +import io.druid.java.util.common.parsers.CloseableIterator; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.BufferAggregator; import io.druid.segment.ColumnSelectorFactory; @@ -296,6 +297,10 @@ public void finish() */ public CloseableIterator> iterator() { + if (!initialized) { + throw new ISE("Grouper should be initialized first"); + } + return new CloseableIterator>() { { diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java index 0fe75d822cc9..5262dc4498af 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java @@ -124,15 +124,16 @@ public void testCombine() throws IOException baseIterator.add(new Entry<>(i, new Object[]{i * 10})); } - final List iterators = new ArrayList<>(8); - for (int i = 0; i < 8; i++) { + final int leafNum = 8; + final List iterators = new ArrayList<>(leafNum); + for (int i = 0; i < leafNum; i++) { iterators.add(new TestIterator(baseIterator.iterator())); } try (final CloseableIterator> iterator = combiner.combine(iterators, new ArrayList<>())) { long expectedKey = 0; while (iterator.hasNext()) { - Assert.assertEquals(new Entry<>(expectedKey, new Object[]{expectedKey++ * 80}), iterator.next()); + Assert.assertEquals(new Entry<>(expectedKey, new Object[]{expectedKey++ * leafNum * 10}), iterator.next()); } } From 0f5c3a8b82415b34fa765dc375d87ae8fe4daa3b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 8 Sep 2017 17:17:54 +0900 Subject: [PATCH 18/44] [maven-release-plugin] prepare release druid-0.11.0-sg --- api/pom.xml | 2 +- aws-common/pom.xml | 2 +- benchmarks/pom.xml | 2 +- bytebuffer-collections/pom.xml | 5 ++--- common/pom.xml | 2 +- distribution/pom.xml | 9 ++++----- examples/pom.xml | 2 +- extendedset/pom.xml | 5 ++--- extensions-contrib/ambari-metrics-emitter/pom.xml | 5 ++--- extensions-contrib/azure-extensions/pom.xml | 5 ++--- extensions-contrib/cassandra-storage/pom.xml | 2 +- extensions-contrib/cloudfiles-extensions/pom.xml | 5 ++--- extensions-contrib/distinctcount/pom.xml | 5 ++--- extensions-contrib/druid-rocketmq/pom.xml | 6 ++---- extensions-contrib/google-extensions/pom.xml | 2 +- extensions-contrib/graphite-emitter/pom.xml | 5 ++--- extensions-contrib/kafka-eight-simpleConsumer/pom.xml | 5 ++--- extensions-contrib/kafka-emitter/pom.xml | 5 ++--- extensions-contrib/orc-extensions/pom.xml | 6 ++---- extensions-contrib/parquet-extensions/pom.xml | 6 ++---- extensions-contrib/rabbitmq/pom.xml | 2 +- extensions-contrib/redis-cache/pom.xml | 5 ++--- extensions-contrib/scan-query/pom.xml | 6 ++---- extensions-contrib/sqlserver-metadata-storage/pom.xml | 2 +- extensions-contrib/statsd-emitter/pom.xml | 6 ++---- extensions-contrib/thrift-extensions/pom.xml | 6 ++---- extensions-contrib/time-min-max/pom.xml | 6 ++---- extensions-contrib/virtual-columns/pom.xml | 2 +- extensions-core/avro-extensions/pom.xml | 5 ++--- extensions-core/caffeine-cache/pom.xml | 2 +- extensions-core/datasketches/pom.xml | 5 ++--- extensions-core/druid-kerberos/pom.xml | 5 ++--- extensions-core/hdfs-storage/pom.xml | 2 +- extensions-core/histogram/pom.xml | 2 +- extensions-core/kafka-eight/pom.xml | 2 +- extensions-core/kafka-extraction-namespace/pom.xml | 5 ++--- extensions-core/kafka-indexing-service/pom.xml | 2 +- extensions-core/lookups-cached-global/pom.xml | 5 ++--- extensions-core/lookups-cached-single/pom.xml | 5 ++--- extensions-core/mysql-metadata-storage/pom.xml | 2 +- extensions-core/postgresql-metadata-storage/pom.xml | 2 +- extensions-core/protobuf-extensions/pom.xml | 6 ++---- extensions-core/s3-extensions/pom.xml | 5 ++--- extensions-core/simple-client-sslcontext/pom.xml | 6 ++---- extensions-core/stats/pom.xml | 2 +- hll/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- integration-tests/pom.xml | 2 +- java-util/pom.xml | 2 +- pom.xml | 7 +++---- processing/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- sql/pom.xml | 5 ++--- 55 files changed, 88 insertions(+), 126 deletions(-) diff --git a/api/pom.xml b/api/pom.xml index 5a921f8edda6..c68ad3cd60b0 100644 --- a/api/pom.xml +++ b/api/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg diff --git a/aws-common/pom.xml b/aws-common/pom.xml index a765e4dc87e8..1d88f2418a4c 100644 --- a/aws-common/pom.xml +++ b/aws-common/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg diff --git a/benchmarks/pom.xml b/benchmarks/pom.xml index 34c7abf7ed3c..89291799ebcf 100644 --- a/benchmarks/pom.xml +++ b/benchmarks/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg diff --git a/bytebuffer-collections/pom.xml b/bytebuffer-collections/pom.xml index b77ee65e9a32..82aeb3ab8781 100755 --- a/bytebuffer-collections/pom.xml +++ b/bytebuffer-collections/pom.xml @@ -18,14 +18,13 @@ ~ under the License. --> - + 4.0.0 io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg bytebuffer-collections diff --git a/common/pom.xml b/common/pom.xml index b24e68c63ed3..9bc11446c073 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg diff --git a/distribution/pom.xml b/distribution/pom.xml index 4dca3b03f514..7b1215fd921d 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -16,8 +16,7 @@ ~ limitations under the License. --> - + 4.0.0 pom @@ -29,7 +28,7 @@ druid io.druid - 0.11.0-SNAPSHOT + 0.11.0-sg @@ -60,7 +59,7 @@ java -classpath - + -Ddruid.extensions.loadList=[] -Ddruid.extensions.directory=${project.build.directory}/extensions @@ -190,7 +189,7 @@ java -classpath - + -Ddruid.extensions.loadList=[] -Ddruid.extensions.directory=${project.build.directory}/extensions diff --git a/examples/pom.xml b/examples/pom.xml index fbb75bbae5c9..cab671f380d8 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg diff --git a/extendedset/pom.xml b/extendedset/pom.xml index f5cc2b595e70..e6b97cc05413 100755 --- a/extendedset/pom.xml +++ b/extendedset/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 extendedset @@ -32,7 +31,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg diff --git a/extensions-contrib/ambari-metrics-emitter/pom.xml b/extensions-contrib/ambari-metrics-emitter/pom.xml index 251355c7239f..d87cbcbc6fbf 100644 --- a/extensions-contrib/ambari-metrics-emitter/pom.xml +++ b/extensions-contrib/ambari-metrics-emitter/pom.xml @@ -18,14 +18,13 @@ ~ under the License. --> - + 4.0.0 io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/azure-extensions/pom.xml b/extensions-contrib/azure-extensions/pom.xml index f259a976b1b4..d34a5c4cb8b1 100644 --- a/extensions-contrib/azure-extensions/pom.xml +++ b/extensions-contrib/azure-extensions/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions.contrib @@ -30,7 +29,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/cassandra-storage/pom.xml b/extensions-contrib/cassandra-storage/pom.xml index d22e13902e8a..b0f53e6460be 100644 --- a/extensions-contrib/cassandra-storage/pom.xml +++ b/extensions-contrib/cassandra-storage/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/cloudfiles-extensions/pom.xml b/extensions-contrib/cloudfiles-extensions/pom.xml index 1958236ccd2a..b7725405e0fa 100644 --- a/extensions-contrib/cloudfiles-extensions/pom.xml +++ b/extensions-contrib/cloudfiles-extensions/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions.contrib @@ -30,7 +29,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/distinctcount/pom.xml b/extensions-contrib/distinctcount/pom.xml index 5bae557dd374..91019b6b2ec8 100644 --- a/extensions-contrib/distinctcount/pom.xml +++ b/extensions-contrib/distinctcount/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions.contrib @@ -30,7 +29,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/druid-rocketmq/pom.xml b/extensions-contrib/druid-rocketmq/pom.xml index f0c0508363df..b8af8823348e 100644 --- a/extensions-contrib/druid-rocketmq/pom.xml +++ b/extensions-contrib/druid-rocketmq/pom.xml @@ -17,14 +17,12 @@ ~ specific language governing permissions and limitations ~ under the License. --> - + 4.0.0 druid io.druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/google-extensions/pom.xml b/extensions-contrib/google-extensions/pom.xml index d990a3751077..f1d0133240a0 100644 --- a/extensions-contrib/google-extensions/pom.xml +++ b/extensions-contrib/google-extensions/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/graphite-emitter/pom.xml b/extensions-contrib/graphite-emitter/pom.xml index d25e9159c625..aa666f86e700 100644 --- a/extensions-contrib/graphite-emitter/pom.xml +++ b/extensions-contrib/graphite-emitter/pom.xml @@ -18,14 +18,13 @@ ~ under the License. --> - + 4.0.0 io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/kafka-eight-simpleConsumer/pom.xml b/extensions-contrib/kafka-eight-simpleConsumer/pom.xml index d52856ebed05..699de948bd4b 100644 --- a/extensions-contrib/kafka-eight-simpleConsumer/pom.xml +++ b/extensions-contrib/kafka-eight-simpleConsumer/pom.xml @@ -16,8 +16,7 @@ ~ limitations under the License. --> - + 4.0.0 io.druid.extensions.contrib druid-kafka-eight-simple-consumer @@ -27,7 +26,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/kafka-emitter/pom.xml b/extensions-contrib/kafka-emitter/pom.xml index 5d7fd6b18ad3..d7db9b61a004 100644 --- a/extensions-contrib/kafka-emitter/pom.xml +++ b/extensions-contrib/kafka-emitter/pom.xml @@ -18,14 +18,13 @@ ~ under the License. --> - + 4.0.0 io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/orc-extensions/pom.xml b/extensions-contrib/orc-extensions/pom.xml index e6c330e87d74..32e1c5359b01 100644 --- a/extensions-contrib/orc-extensions/pom.xml +++ b/extensions-contrib/orc-extensions/pom.xml @@ -15,9 +15,7 @@ ~ See the License for the specific language governing permissions and ~ limitations under the License. --> - + io.druid.extensions.contrib druid-orc-extensions druid-orc-extensions @@ -26,7 +24,7 @@ druid io.druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml 4.0.0 diff --git a/extensions-contrib/parquet-extensions/pom.xml b/extensions-contrib/parquet-extensions/pom.xml index ca4d0fcf4d59..326c96e153f1 100644 --- a/extensions-contrib/parquet-extensions/pom.xml +++ b/extensions-contrib/parquet-extensions/pom.xml @@ -1,7 +1,5 @@ - + io.druid.extensions.contrib druid-parquet-extensions druid-parquet-extensions @@ -10,7 +8,7 @@ druid io.druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml 4.0.0 diff --git a/extensions-contrib/rabbitmq/pom.xml b/extensions-contrib/rabbitmq/pom.xml index 016a7a54ce2c..ae301eb51ca0 100644 --- a/extensions-contrib/rabbitmq/pom.xml +++ b/extensions-contrib/rabbitmq/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/redis-cache/pom.xml b/extensions-contrib/redis-cache/pom.xml index 8f77f1138851..7dfe086197fd 100644 --- a/extensions-contrib/redis-cache/pom.xml +++ b/extensions-contrib/redis-cache/pom.xml @@ -19,8 +19,7 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions.contrib @@ -30,7 +29,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/scan-query/pom.xml b/extensions-contrib/scan-query/pom.xml index 328be800e5d7..b94c9c52596b 100644 --- a/extensions-contrib/scan-query/pom.xml +++ b/extensions-contrib/scan-query/pom.xml @@ -18,14 +18,12 @@ ~ under the License. --> - + io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml 4.0.0 diff --git a/extensions-contrib/sqlserver-metadata-storage/pom.xml b/extensions-contrib/sqlserver-metadata-storage/pom.xml index 012cedbbf20b..29fe2b9e066d 100644 --- a/extensions-contrib/sqlserver-metadata-storage/pom.xml +++ b/extensions-contrib/sqlserver-metadata-storage/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/statsd-emitter/pom.xml b/extensions-contrib/statsd-emitter/pom.xml index eb5daee7cbd3..a8bcfcca487c 100644 --- a/extensions-contrib/statsd-emitter/pom.xml +++ b/extensions-contrib/statsd-emitter/pom.xml @@ -17,13 +17,11 @@ ~ specific language governing permissions and limitations ~ under the License. --> - + druid io.druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml 4.0.0 diff --git a/extensions-contrib/thrift-extensions/pom.xml b/extensions-contrib/thrift-extensions/pom.xml index bc689162fb25..f4eebff0b542 100644 --- a/extensions-contrib/thrift-extensions/pom.xml +++ b/extensions-contrib/thrift-extensions/pom.xml @@ -1,7 +1,5 @@ - + io.druid.extensions.contrib druid-thrift-extensions @@ -11,7 +9,7 @@ druid io.druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml 4.0.0 diff --git a/extensions-contrib/time-min-max/pom.xml b/extensions-contrib/time-min-max/pom.xml index fc3b15067f40..37efbb81f726 100644 --- a/extensions-contrib/time-min-max/pom.xml +++ b/extensions-contrib/time-min-max/pom.xml @@ -17,13 +17,11 @@ ~ specific language governing permissions and limitations ~ under the License. --> - + druid io.druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml 4.0.0 diff --git a/extensions-contrib/virtual-columns/pom.xml b/extensions-contrib/virtual-columns/pom.xml index 6d8dac5abfdc..75900c20e317 100644 --- a/extensions-contrib/virtual-columns/pom.xml +++ b/extensions-contrib/virtual-columns/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/avro-extensions/pom.xml b/extensions-core/avro-extensions/pom.xml index 9ff968338ecc..ca2551647c74 100644 --- a/extensions-core/avro-extensions/pom.xml +++ b/extensions-core/avro-extensions/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions @@ -30,7 +29,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/caffeine-cache/pom.xml b/extensions-core/caffeine-cache/pom.xml index e4ff268a34e6..aeb26351fac1 100644 --- a/extensions-core/caffeine-cache/pom.xml +++ b/extensions-core/caffeine-cache/pom.xml @@ -30,7 +30,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/datasketches/pom.xml b/extensions-core/datasketches/pom.xml index 9f7ebfe5d847..a368ba7c91e1 100644 --- a/extensions-core/datasketches/pom.xml +++ b/extensions-core/datasketches/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions @@ -30,7 +29,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/druid-kerberos/pom.xml b/extensions-core/druid-kerberos/pom.xml index 8d2dcd50a6c9..d2e774b08139 100644 --- a/extensions-core/druid-kerberos/pom.xml +++ b/extensions-core/druid-kerberos/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions @@ -30,7 +29,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/hdfs-storage/pom.xml b/extensions-core/hdfs-storage/pom.xml index 4dd68dbecae5..bafde5112530 100644 --- a/extensions-core/hdfs-storage/pom.xml +++ b/extensions-core/hdfs-storage/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/histogram/pom.xml b/extensions-core/histogram/pom.xml index 8eb5ae08db29..100aebf2ef5e 100644 --- a/extensions-core/histogram/pom.xml +++ b/extensions-core/histogram/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/kafka-eight/pom.xml b/extensions-core/kafka-eight/pom.xml index 7e5f43cf0c29..bf9aed2f6b7d 100644 --- a/extensions-core/kafka-eight/pom.xml +++ b/extensions-core/kafka-eight/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/kafka-extraction-namespace/pom.xml b/extensions-core/kafka-extraction-namespace/pom.xml index b1dd4f3450d6..075478fdf6b9 100644 --- a/extensions-core/kafka-extraction-namespace/pom.xml +++ b/extensions-core/kafka-extraction-namespace/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions druid-kafka-extraction-namespace @@ -29,7 +28,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/kafka-indexing-service/pom.xml b/extensions-core/kafka-indexing-service/pom.xml index 9c078961a7fa..c84851ab183c 100644 --- a/extensions-core/kafka-indexing-service/pom.xml +++ b/extensions-core/kafka-indexing-service/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/lookups-cached-global/pom.xml b/extensions-core/lookups-cached-global/pom.xml index c06aef7a6deb..46750c366b2c 100644 --- a/extensions-core/lookups-cached-global/pom.xml +++ b/extensions-core/lookups-cached-global/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions druid-lookups-cached-global @@ -29,7 +28,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/lookups-cached-single/pom.xml b/extensions-core/lookups-cached-single/pom.xml index e62a1590e86c..317d48c262f1 100644 --- a/extensions-core/lookups-cached-single/pom.xml +++ b/extensions-core/lookups-cached-single/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions druid-lookups-cached-single @@ -29,7 +28,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/mysql-metadata-storage/pom.xml b/extensions-core/mysql-metadata-storage/pom.xml index e411e9953614..cd5ea290cc84 100644 --- a/extensions-core/mysql-metadata-storage/pom.xml +++ b/extensions-core/mysql-metadata-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/postgresql-metadata-storage/pom.xml b/extensions-core/postgresql-metadata-storage/pom.xml index 3910e39d52ac..34fdddd95c03 100644 --- a/extensions-core/postgresql-metadata-storage/pom.xml +++ b/extensions-core/postgresql-metadata-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/protobuf-extensions/pom.xml b/extensions-core/protobuf-extensions/pom.xml index 551ded9511d0..5904debeacc3 100644 --- a/extensions-core/protobuf-extensions/pom.xml +++ b/extensions-core/protobuf-extensions/pom.xml @@ -16,9 +16,7 @@ ~ limitations under the License. --> - + 4.0.0 io.druid.extensions @@ -29,7 +27,7 @@ druid io.druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/s3-extensions/pom.xml b/extensions-core/s3-extensions/pom.xml index b12dd46cb349..05149133840b 100644 --- a/extensions-core/s3-extensions/pom.xml +++ b/extensions-core/s3-extensions/pom.xml @@ -16,8 +16,7 @@ ~ limitations under the License. --> - + 4.0.0 io.druid.extensions @@ -28,7 +27,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/simple-client-sslcontext/pom.xml b/extensions-core/simple-client-sslcontext/pom.xml index be4333b13fce..11f5b0d6080f 100644 --- a/extensions-core/simple-client-sslcontext/pom.xml +++ b/extensions-core/simple-client-sslcontext/pom.xml @@ -1,11 +1,9 @@ - + druid io.druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml 4.0.0 diff --git a/extensions-core/stats/pom.xml b/extensions-core/stats/pom.xml index 44ce7c53a7dd..b0feb252bb38 100644 --- a/extensions-core/stats/pom.xml +++ b/extensions-core/stats/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/hll/pom.xml b/hll/pom.xml index e837d3c148be..01bcdc6ebffa 100644 --- a/hll/pom.xml +++ b/hll/pom.xml @@ -24,7 +24,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg druid-hll diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 9fd6d8007e26..c9439e24c7f0 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 2ab98ac87143..676137e70caa 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index 1864d651e513..d63460f887ba 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg diff --git a/java-util/pom.xml b/java-util/pom.xml index 2b023a8902fc..9ab5a2fad64e 100644 --- a/java-util/pom.xml +++ b/java-util/pom.xml @@ -24,7 +24,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg java-util diff --git a/pom.xml b/pom.xml index e04d669e4219..1b3ce6bd4887 100644 --- a/pom.xml +++ b/pom.xml @@ -16,8 +16,7 @@ ~ limitations under the License. --> - + 4.0.0 @@ -27,7 +26,7 @@ druid - 0.11.0-SNAPSHOT + 0.11.0-sg pom ${project.groupId}:${project.artifactId} @@ -54,7 +53,7 @@ scm:git:ssh://git@github.com/druid-io/druid.git scm:git:ssh://git@github.com/druid-io/druid.git https://github.com/druid-io/druid.git - 0.11.0-SNAPSHOT + druid-0.11.0-sg diff --git a/processing/pom.xml b/processing/pom.xml index 9479b623c48b..b87c1f20b07f 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg diff --git a/server/pom.xml b/server/pom.xml index 5a5dca163b1c..94a7520f8cb9 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg diff --git a/services/pom.xml b/services/pom.xml index 506b4a29c274..e9f4649b7f2c 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -25,7 +25,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg @@ -116,7 +116,7 @@ - + diff --git a/sql/pom.xml b/sql/pom.xml index 5311c33c700a..011896ed4c31 100644 --- a/sql/pom.xml +++ b/sql/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 druid-sql @@ -29,7 +28,7 @@ io.druid druid - 0.11.0-SNAPSHOT + 0.11.0-sg From 5c6b31e488c413073e2b1d4ce128bdff0649b41a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 8 Sep 2017 17:17:54 +0900 Subject: [PATCH 19/44] [maven-release-plugin] prepare for next development iteration --- api/pom.xml | 2 +- aws-common/pom.xml | 2 +- benchmarks/pom.xml | 2 +- bytebuffer-collections/pom.xml | 2 +- common/pom.xml | 2 +- distribution/pom.xml | 2 +- examples/pom.xml | 2 +- extendedset/pom.xml | 2 +- extensions-contrib/ambari-metrics-emitter/pom.xml | 2 +- extensions-contrib/azure-extensions/pom.xml | 2 +- extensions-contrib/cassandra-storage/pom.xml | 2 +- extensions-contrib/cloudfiles-extensions/pom.xml | 2 +- extensions-contrib/distinctcount/pom.xml | 2 +- extensions-contrib/druid-rocketmq/pom.xml | 2 +- extensions-contrib/google-extensions/pom.xml | 2 +- extensions-contrib/graphite-emitter/pom.xml | 2 +- extensions-contrib/kafka-eight-simpleConsumer/pom.xml | 2 +- extensions-contrib/kafka-emitter/pom.xml | 2 +- extensions-contrib/orc-extensions/pom.xml | 2 +- extensions-contrib/parquet-extensions/pom.xml | 2 +- extensions-contrib/rabbitmq/pom.xml | 2 +- extensions-contrib/redis-cache/pom.xml | 2 +- extensions-contrib/scan-query/pom.xml | 2 +- extensions-contrib/sqlserver-metadata-storage/pom.xml | 2 +- extensions-contrib/statsd-emitter/pom.xml | 2 +- extensions-contrib/thrift-extensions/pom.xml | 2 +- extensions-contrib/time-min-max/pom.xml | 2 +- extensions-contrib/virtual-columns/pom.xml | 2 +- extensions-core/avro-extensions/pom.xml | 2 +- extensions-core/caffeine-cache/pom.xml | 2 +- extensions-core/datasketches/pom.xml | 2 +- extensions-core/druid-kerberos/pom.xml | 2 +- extensions-core/hdfs-storage/pom.xml | 2 +- extensions-core/histogram/pom.xml | 2 +- extensions-core/kafka-eight/pom.xml | 2 +- extensions-core/kafka-extraction-namespace/pom.xml | 2 +- extensions-core/kafka-indexing-service/pom.xml | 2 +- extensions-core/lookups-cached-global/pom.xml | 2 +- extensions-core/lookups-cached-single/pom.xml | 2 +- extensions-core/mysql-metadata-storage/pom.xml | 2 +- extensions-core/postgresql-metadata-storage/pom.xml | 2 +- extensions-core/protobuf-extensions/pom.xml | 2 +- extensions-core/s3-extensions/pom.xml | 2 +- extensions-core/simple-client-sslcontext/pom.xml | 2 +- extensions-core/stats/pom.xml | 2 +- hll/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- integration-tests/pom.xml | 2 +- java-util/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- sql/pom.xml | 2 +- 55 files changed, 56 insertions(+), 56 deletions(-) diff --git a/api/pom.xml b/api/pom.xml index c68ad3cd60b0..989d27d95534 100644 --- a/api/pom.xml +++ b/api/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT diff --git a/aws-common/pom.xml b/aws-common/pom.xml index 1d88f2418a4c..caeb19e36b89 100644 --- a/aws-common/pom.xml +++ b/aws-common/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT diff --git a/benchmarks/pom.xml b/benchmarks/pom.xml index 89291799ebcf..fcb7843c8dc4 100644 --- a/benchmarks/pom.xml +++ b/benchmarks/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT diff --git a/bytebuffer-collections/pom.xml b/bytebuffer-collections/pom.xml index 82aeb3ab8781..102d88589fe0 100755 --- a/bytebuffer-collections/pom.xml +++ b/bytebuffer-collections/pom.xml @@ -24,7 +24,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT bytebuffer-collections diff --git a/common/pom.xml b/common/pom.xml index 9bc11446c073..7f0b8b95c715 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT diff --git a/distribution/pom.xml b/distribution/pom.xml index 7b1215fd921d..6b816ac62c94 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -28,7 +28,7 @@ druid io.druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index cab671f380d8..de10f4ed48cb 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT diff --git a/extendedset/pom.xml b/extendedset/pom.xml index e6b97cc05413..93dac5109499 100755 --- a/extendedset/pom.xml +++ b/extendedset/pom.xml @@ -31,7 +31,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT diff --git a/extensions-contrib/ambari-metrics-emitter/pom.xml b/extensions-contrib/ambari-metrics-emitter/pom.xml index d87cbcbc6fbf..65f4455a480c 100644 --- a/extensions-contrib/ambari-metrics-emitter/pom.xml +++ b/extensions-contrib/ambari-metrics-emitter/pom.xml @@ -24,7 +24,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/azure-extensions/pom.xml b/extensions-contrib/azure-extensions/pom.xml index d34a5c4cb8b1..82f4bcd51e72 100644 --- a/extensions-contrib/azure-extensions/pom.xml +++ b/extensions-contrib/azure-extensions/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/cassandra-storage/pom.xml b/extensions-contrib/cassandra-storage/pom.xml index b0f53e6460be..edd035be8d19 100644 --- a/extensions-contrib/cassandra-storage/pom.xml +++ b/extensions-contrib/cassandra-storage/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/cloudfiles-extensions/pom.xml b/extensions-contrib/cloudfiles-extensions/pom.xml index b7725405e0fa..75401641b9f9 100644 --- a/extensions-contrib/cloudfiles-extensions/pom.xml +++ b/extensions-contrib/cloudfiles-extensions/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/distinctcount/pom.xml b/extensions-contrib/distinctcount/pom.xml index 91019b6b2ec8..c35149ef11f5 100644 --- a/extensions-contrib/distinctcount/pom.xml +++ b/extensions-contrib/distinctcount/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/druid-rocketmq/pom.xml b/extensions-contrib/druid-rocketmq/pom.xml index b8af8823348e..577ed9377a84 100644 --- a/extensions-contrib/druid-rocketmq/pom.xml +++ b/extensions-contrib/druid-rocketmq/pom.xml @@ -22,7 +22,7 @@ druid io.druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/google-extensions/pom.xml b/extensions-contrib/google-extensions/pom.xml index f1d0133240a0..10a4627d85a7 100644 --- a/extensions-contrib/google-extensions/pom.xml +++ b/extensions-contrib/google-extensions/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/graphite-emitter/pom.xml b/extensions-contrib/graphite-emitter/pom.xml index aa666f86e700..1b412a4eaba7 100644 --- a/extensions-contrib/graphite-emitter/pom.xml +++ b/extensions-contrib/graphite-emitter/pom.xml @@ -24,7 +24,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/kafka-eight-simpleConsumer/pom.xml b/extensions-contrib/kafka-eight-simpleConsumer/pom.xml index 699de948bd4b..3d396b022216 100644 --- a/extensions-contrib/kafka-eight-simpleConsumer/pom.xml +++ b/extensions-contrib/kafka-eight-simpleConsumer/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/kafka-emitter/pom.xml b/extensions-contrib/kafka-emitter/pom.xml index d7db9b61a004..fec6da4a5aa5 100644 --- a/extensions-contrib/kafka-emitter/pom.xml +++ b/extensions-contrib/kafka-emitter/pom.xml @@ -24,7 +24,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/orc-extensions/pom.xml b/extensions-contrib/orc-extensions/pom.xml index 32e1c5359b01..c3163a0aafa3 100644 --- a/extensions-contrib/orc-extensions/pom.xml +++ b/extensions-contrib/orc-extensions/pom.xml @@ -24,7 +24,7 @@ druid io.druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/parquet-extensions/pom.xml b/extensions-contrib/parquet-extensions/pom.xml index 326c96e153f1..8be40e086a56 100644 --- a/extensions-contrib/parquet-extensions/pom.xml +++ b/extensions-contrib/parquet-extensions/pom.xml @@ -8,7 +8,7 @@ druid io.druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/rabbitmq/pom.xml b/extensions-contrib/rabbitmq/pom.xml index ae301eb51ca0..576b247f772f 100644 --- a/extensions-contrib/rabbitmq/pom.xml +++ b/extensions-contrib/rabbitmq/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/redis-cache/pom.xml b/extensions-contrib/redis-cache/pom.xml index 7dfe086197fd..b657fdca1297 100644 --- a/extensions-contrib/redis-cache/pom.xml +++ b/extensions-contrib/redis-cache/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/scan-query/pom.xml b/extensions-contrib/scan-query/pom.xml index b94c9c52596b..ce7bdebd22e5 100644 --- a/extensions-contrib/scan-query/pom.xml +++ b/extensions-contrib/scan-query/pom.xml @@ -23,7 +23,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/sqlserver-metadata-storage/pom.xml b/extensions-contrib/sqlserver-metadata-storage/pom.xml index 29fe2b9e066d..9aa8013ff910 100644 --- a/extensions-contrib/sqlserver-metadata-storage/pom.xml +++ b/extensions-contrib/sqlserver-metadata-storage/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/statsd-emitter/pom.xml b/extensions-contrib/statsd-emitter/pom.xml index a8bcfcca487c..bf6d1addb1d7 100644 --- a/extensions-contrib/statsd-emitter/pom.xml +++ b/extensions-contrib/statsd-emitter/pom.xml @@ -21,7 +21,7 @@ druid io.druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/thrift-extensions/pom.xml b/extensions-contrib/thrift-extensions/pom.xml index f4eebff0b542..cdeae5a32b5d 100644 --- a/extensions-contrib/thrift-extensions/pom.xml +++ b/extensions-contrib/thrift-extensions/pom.xml @@ -9,7 +9,7 @@ druid io.druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/time-min-max/pom.xml b/extensions-contrib/time-min-max/pom.xml index 37efbb81f726..8931502b7aca 100644 --- a/extensions-contrib/time-min-max/pom.xml +++ b/extensions-contrib/time-min-max/pom.xml @@ -21,7 +21,7 @@ druid io.druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/virtual-columns/pom.xml b/extensions-contrib/virtual-columns/pom.xml index 75900c20e317..fb81f14c7ba1 100644 --- a/extensions-contrib/virtual-columns/pom.xml +++ b/extensions-contrib/virtual-columns/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/avro-extensions/pom.xml b/extensions-core/avro-extensions/pom.xml index ca2551647c74..e469f1a61c4c 100644 --- a/extensions-core/avro-extensions/pom.xml +++ b/extensions-core/avro-extensions/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/caffeine-cache/pom.xml b/extensions-core/caffeine-cache/pom.xml index aeb26351fac1..4be81d6b4b2f 100644 --- a/extensions-core/caffeine-cache/pom.xml +++ b/extensions-core/caffeine-cache/pom.xml @@ -30,7 +30,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/datasketches/pom.xml b/extensions-core/datasketches/pom.xml index a368ba7c91e1..9699be14ea90 100644 --- a/extensions-core/datasketches/pom.xml +++ b/extensions-core/datasketches/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/druid-kerberos/pom.xml b/extensions-core/druid-kerberos/pom.xml index d2e774b08139..dfa8e25748f3 100644 --- a/extensions-core/druid-kerberos/pom.xml +++ b/extensions-core/druid-kerberos/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/hdfs-storage/pom.xml b/extensions-core/hdfs-storage/pom.xml index bafde5112530..39e433658756 100644 --- a/extensions-core/hdfs-storage/pom.xml +++ b/extensions-core/hdfs-storage/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/histogram/pom.xml b/extensions-core/histogram/pom.xml index 100aebf2ef5e..d67ed9619691 100644 --- a/extensions-core/histogram/pom.xml +++ b/extensions-core/histogram/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/kafka-eight/pom.xml b/extensions-core/kafka-eight/pom.xml index bf9aed2f6b7d..03a1ce6bcde7 100644 --- a/extensions-core/kafka-eight/pom.xml +++ b/extensions-core/kafka-eight/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/kafka-extraction-namespace/pom.xml b/extensions-core/kafka-extraction-namespace/pom.xml index 075478fdf6b9..5f4392e536f6 100644 --- a/extensions-core/kafka-extraction-namespace/pom.xml +++ b/extensions-core/kafka-extraction-namespace/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/kafka-indexing-service/pom.xml b/extensions-core/kafka-indexing-service/pom.xml index c84851ab183c..653cbae4293e 100644 --- a/extensions-core/kafka-indexing-service/pom.xml +++ b/extensions-core/kafka-indexing-service/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/lookups-cached-global/pom.xml b/extensions-core/lookups-cached-global/pom.xml index 46750c366b2c..ad20b8ff1225 100644 --- a/extensions-core/lookups-cached-global/pom.xml +++ b/extensions-core/lookups-cached-global/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/lookups-cached-single/pom.xml b/extensions-core/lookups-cached-single/pom.xml index 317d48c262f1..dd4fc9f714fc 100644 --- a/extensions-core/lookups-cached-single/pom.xml +++ b/extensions-core/lookups-cached-single/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/mysql-metadata-storage/pom.xml b/extensions-core/mysql-metadata-storage/pom.xml index cd5ea290cc84..262c34357ec2 100644 --- a/extensions-core/mysql-metadata-storage/pom.xml +++ b/extensions-core/mysql-metadata-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/postgresql-metadata-storage/pom.xml b/extensions-core/postgresql-metadata-storage/pom.xml index 34fdddd95c03..0bc9755ff2f1 100644 --- a/extensions-core/postgresql-metadata-storage/pom.xml +++ b/extensions-core/postgresql-metadata-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/protobuf-extensions/pom.xml b/extensions-core/protobuf-extensions/pom.xml index 5904debeacc3..330dca1c7f72 100644 --- a/extensions-core/protobuf-extensions/pom.xml +++ b/extensions-core/protobuf-extensions/pom.xml @@ -27,7 +27,7 @@ druid io.druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/s3-extensions/pom.xml b/extensions-core/s3-extensions/pom.xml index 05149133840b..e092f1592f0c 100644 --- a/extensions-core/s3-extensions/pom.xml +++ b/extensions-core/s3-extensions/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/simple-client-sslcontext/pom.xml b/extensions-core/simple-client-sslcontext/pom.xml index 11f5b0d6080f..319fd7fa4fdc 100644 --- a/extensions-core/simple-client-sslcontext/pom.xml +++ b/extensions-core/simple-client-sslcontext/pom.xml @@ -3,7 +3,7 @@ druid io.druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-core/stats/pom.xml b/extensions-core/stats/pom.xml index b0feb252bb38..583528e065df 100644 --- a/extensions-core/stats/pom.xml +++ b/extensions-core/stats/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT ../../pom.xml diff --git a/hll/pom.xml b/hll/pom.xml index 01bcdc6ebffa..a2e4b8b75646 100644 --- a/hll/pom.xml +++ b/hll/pom.xml @@ -24,7 +24,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT druid-hll diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index c9439e24c7f0..d29f330a7bbd 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 676137e70caa..17adaf1597a5 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index d63460f887ba..2f6227e3fb66 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT diff --git a/java-util/pom.xml b/java-util/pom.xml index 9ab5a2fad64e..299511a66035 100644 --- a/java-util/pom.xml +++ b/java-util/pom.xml @@ -24,7 +24,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT java-util diff --git a/pom.xml b/pom.xml index 1b3ce6bd4887..fa2af3f7d9d0 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT pom ${project.groupId}:${project.artifactId} @@ -53,7 +53,7 @@ scm:git:ssh://git@github.com/druid-io/druid.git scm:git:ssh://git@github.com/druid-io/druid.git https://github.com/druid-io/druid.git - druid-0.11.0-sg + 0.11.0-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index b87c1f20b07f..6c2170293e30 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 94a7520f8cb9..00b55e545d58 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index e9f4649b7f2c..857e6c1b9fe0 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -25,7 +25,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT diff --git a/sql/pom.xml b/sql/pom.xml index 011896ed4c31..a0255379f150 100644 --- a/sql/pom.xml +++ b/sql/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-sg2-SNAPSHOT From 9258453fc95666eb448f03cb949c51be62e4a910 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 9 Sep 2017 12:24:23 +0900 Subject: [PATCH 20/44] Address comments --- .../epinephelinae/ParallelCombiner.java | 6 ++++-- .../epinephelinae/RowBasedGrouperHelper.java | 20 ++++++++++--------- 2 files changed, 15 insertions(+), 11 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java index 02c6a1caff9a..814bc36199a8 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java @@ -286,9 +286,11 @@ private Pair>>, List> buildCombine // o o o // / \ / \ / \ // o o o o o o + // + // We can expect that the aggregates can be combined as early as possible because the tree is built in a bottom-up + // manner. - int i; - for (i = 0; i < numChildLevelIterators; i += combineDegree) { + for (int i = 0; i < numChildLevelIterators; i += combineDegree) { if (i < numChildLevelIterators - 1) { final List>> subIterators = childIterators.subList( i, diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index a199c3cdfd48..c206b3e90a69 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -66,6 +66,8 @@ import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ValueType; import io.druid.segment.data.IndexedInts; +import it.unimi.dsi.fastutil.ints.IntArrays; +import it.unimi.dsi.fastutil.ints.IntComparator; import it.unimi.dsi.fastutil.objects.Object2IntMap; import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import org.joda.time.DateTime; @@ -81,6 +83,8 @@ import java.util.Map; import java.util.Set; import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; // this class contains shared code between GroupByMergingQueryRunnerV2 and GroupByRowProcessor public class RowBasedGrouperHelper @@ -939,7 +943,7 @@ static long estimateStringKeySize(String key) private static class RowBasedKeySerde implements Grouper.KeySerde { private static final int DICTIONARY_INITIAL_CAPACITY = 10000; - private static final int UNKNOWN_DICTIONARY_KEY = -1; + private static final int UNKNOWN_DICTIONARY_ID = -1; private final boolean includeTimestamp; private final boolean sortByDimsFirst; @@ -962,6 +966,7 @@ private static class RowBasedKeySerde implements Grouper.KeySerde its position if it were sorted by dictionary value + // This is mutable only when runtime dictionary generation is enabled. private int[] sortableIds = null; RowBasedKeySerde( @@ -985,7 +990,7 @@ private static class RowBasedKeySerde implements Grouper.KeySerde(DICTIONARY_INITIAL_CAPACITY) : new Object2IntOpenHashMap<>(dictionary.size()); - this.reverseDictionary.defaultReturnValue(UNKNOWN_DICTIONARY_KEY); + this.reverseDictionary.defaultReturnValue(UNKNOWN_DICTIONARY_ID); this.maxDictionarySize = maxDictionarySize; this.serdeHelpers = makeSerdeHelpers(limitSpec != null, enableRuntimeDictionaryGeneration); this.keySize = (includeTimestamp ? Longs.BYTES : 0) + getTotalKeySize(); @@ -1107,7 +1112,6 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, { final int cmp = compareDimsInBuffersForNullFudgeTimestamp( serdeHelpers, - dimCount, lhsBuffer, rhsBuffer, lhsPosition, @@ -1134,7 +1138,6 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, return compareDimsInBuffersForNullFudgeTimestamp( serdeHelpers, - dimCount, lhsBuffer, rhsBuffer, lhsPosition, @@ -1499,7 +1502,7 @@ public boolean putToKeyBuffer(RowBasedKey key, int idx) private int addToDictionary(final String s) { int idx = reverseDictionary.getInt(s); - if (idx == UNKNOWN_DICTIONARY_KEY) { + if (idx == UNKNOWN_DICTIONARY_ID) { final long additionalEstimatedSize = estimateStringKeySize(s); if (currentEstimatedSize + additionalEstimatedSize > maxDictionarySize) { return -1; @@ -1531,7 +1534,7 @@ public boolean putToKeyBuffer(RowBasedKey key, int idx) final String stringKey = (String) key.getKey()[idx]; final int dictIndex = reverseDictionary.getInt(stringKey); - if (dictIndex == UNKNOWN_DICTIONARY_KEY) { + if (dictIndex == UNKNOWN_DICTIONARY_ID) { throw new ISE("Cannot find key[%s] from dictionary", stringKey); } keyBuffer.putInt(dictIndex); @@ -1698,15 +1701,14 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, private static int compareDimsInBuffersForNullFudgeTimestamp( List serdeHelpers, - int dimCount, ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition ) { - for (int i = 0; i < dimCount; i++) { - final int cmp = serdeHelpers.get(i).compare( + for (RowBasedKeySerdeHelper serdeHelper : serdeHelpers) { + final int cmp = serdeHelper.compare( lhsBuffer, rhsBuffer, lhsPosition + Longs.BYTES, From de73645ffe5a3011fa412cc10c11640292a746cc Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 9 Sep 2017 12:38:46 +0900 Subject: [PATCH 21/44] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit 5c6b31e488c413073e2b1d4ce128bdff0649b41a. --- api/pom.xml | 2 +- aws-common/pom.xml | 2 +- benchmarks/pom.xml | 2 +- bytebuffer-collections/pom.xml | 2 +- common/pom.xml | 2 +- distribution/pom.xml | 2 +- examples/pom.xml | 2 +- extendedset/pom.xml | 2 +- extensions-contrib/ambari-metrics-emitter/pom.xml | 2 +- extensions-contrib/azure-extensions/pom.xml | 2 +- extensions-contrib/cassandra-storage/pom.xml | 2 +- extensions-contrib/cloudfiles-extensions/pom.xml | 2 +- extensions-contrib/distinctcount/pom.xml | 2 +- extensions-contrib/druid-rocketmq/pom.xml | 2 +- extensions-contrib/google-extensions/pom.xml | 2 +- extensions-contrib/graphite-emitter/pom.xml | 2 +- extensions-contrib/kafka-eight-simpleConsumer/pom.xml | 2 +- extensions-contrib/kafka-emitter/pom.xml | 2 +- extensions-contrib/orc-extensions/pom.xml | 2 +- extensions-contrib/parquet-extensions/pom.xml | 2 +- extensions-contrib/rabbitmq/pom.xml | 2 +- extensions-contrib/redis-cache/pom.xml | 2 +- extensions-contrib/scan-query/pom.xml | 2 +- extensions-contrib/sqlserver-metadata-storage/pom.xml | 2 +- extensions-contrib/statsd-emitter/pom.xml | 2 +- extensions-contrib/thrift-extensions/pom.xml | 2 +- extensions-contrib/time-min-max/pom.xml | 2 +- extensions-contrib/virtual-columns/pom.xml | 2 +- extensions-core/avro-extensions/pom.xml | 2 +- extensions-core/caffeine-cache/pom.xml | 2 +- extensions-core/datasketches/pom.xml | 2 +- extensions-core/druid-kerberos/pom.xml | 2 +- extensions-core/hdfs-storage/pom.xml | 2 +- extensions-core/histogram/pom.xml | 2 +- extensions-core/kafka-eight/pom.xml | 2 +- extensions-core/kafka-extraction-namespace/pom.xml | 2 +- extensions-core/kafka-indexing-service/pom.xml | 2 +- extensions-core/lookups-cached-global/pom.xml | 2 +- extensions-core/lookups-cached-single/pom.xml | 2 +- extensions-core/mysql-metadata-storage/pom.xml | 2 +- extensions-core/postgresql-metadata-storage/pom.xml | 2 +- extensions-core/protobuf-extensions/pom.xml | 2 +- extensions-core/s3-extensions/pom.xml | 2 +- extensions-core/simple-client-sslcontext/pom.xml | 2 +- extensions-core/stats/pom.xml | 2 +- hll/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- integration-tests/pom.xml | 2 +- java-util/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- sql/pom.xml | 2 +- 55 files changed, 56 insertions(+), 56 deletions(-) diff --git a/api/pom.xml b/api/pom.xml index 989d27d95534..c68ad3cd60b0 100644 --- a/api/pom.xml +++ b/api/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg diff --git a/aws-common/pom.xml b/aws-common/pom.xml index caeb19e36b89..1d88f2418a4c 100644 --- a/aws-common/pom.xml +++ b/aws-common/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg diff --git a/benchmarks/pom.xml b/benchmarks/pom.xml index fcb7843c8dc4..89291799ebcf 100644 --- a/benchmarks/pom.xml +++ b/benchmarks/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg diff --git a/bytebuffer-collections/pom.xml b/bytebuffer-collections/pom.xml index 102d88589fe0..82aeb3ab8781 100755 --- a/bytebuffer-collections/pom.xml +++ b/bytebuffer-collections/pom.xml @@ -24,7 +24,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg bytebuffer-collections diff --git a/common/pom.xml b/common/pom.xml index 7f0b8b95c715..9bc11446c073 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg diff --git a/distribution/pom.xml b/distribution/pom.xml index 6b816ac62c94..7b1215fd921d 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -28,7 +28,7 @@ druid io.druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg diff --git a/examples/pom.xml b/examples/pom.xml index de10f4ed48cb..cab671f380d8 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg diff --git a/extendedset/pom.xml b/extendedset/pom.xml index 93dac5109499..e6b97cc05413 100755 --- a/extendedset/pom.xml +++ b/extendedset/pom.xml @@ -31,7 +31,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg diff --git a/extensions-contrib/ambari-metrics-emitter/pom.xml b/extensions-contrib/ambari-metrics-emitter/pom.xml index 65f4455a480c..d87cbcbc6fbf 100644 --- a/extensions-contrib/ambari-metrics-emitter/pom.xml +++ b/extensions-contrib/ambari-metrics-emitter/pom.xml @@ -24,7 +24,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/azure-extensions/pom.xml b/extensions-contrib/azure-extensions/pom.xml index 82f4bcd51e72..d34a5c4cb8b1 100644 --- a/extensions-contrib/azure-extensions/pom.xml +++ b/extensions-contrib/azure-extensions/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/cassandra-storage/pom.xml b/extensions-contrib/cassandra-storage/pom.xml index edd035be8d19..b0f53e6460be 100644 --- a/extensions-contrib/cassandra-storage/pom.xml +++ b/extensions-contrib/cassandra-storage/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/cloudfiles-extensions/pom.xml b/extensions-contrib/cloudfiles-extensions/pom.xml index 75401641b9f9..b7725405e0fa 100644 --- a/extensions-contrib/cloudfiles-extensions/pom.xml +++ b/extensions-contrib/cloudfiles-extensions/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/distinctcount/pom.xml b/extensions-contrib/distinctcount/pom.xml index c35149ef11f5..91019b6b2ec8 100644 --- a/extensions-contrib/distinctcount/pom.xml +++ b/extensions-contrib/distinctcount/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/druid-rocketmq/pom.xml b/extensions-contrib/druid-rocketmq/pom.xml index 577ed9377a84..b8af8823348e 100644 --- a/extensions-contrib/druid-rocketmq/pom.xml +++ b/extensions-contrib/druid-rocketmq/pom.xml @@ -22,7 +22,7 @@ druid io.druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/google-extensions/pom.xml b/extensions-contrib/google-extensions/pom.xml index 10a4627d85a7..f1d0133240a0 100644 --- a/extensions-contrib/google-extensions/pom.xml +++ b/extensions-contrib/google-extensions/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/graphite-emitter/pom.xml b/extensions-contrib/graphite-emitter/pom.xml index 1b412a4eaba7..aa666f86e700 100644 --- a/extensions-contrib/graphite-emitter/pom.xml +++ b/extensions-contrib/graphite-emitter/pom.xml @@ -24,7 +24,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/kafka-eight-simpleConsumer/pom.xml b/extensions-contrib/kafka-eight-simpleConsumer/pom.xml index 3d396b022216..699de948bd4b 100644 --- a/extensions-contrib/kafka-eight-simpleConsumer/pom.xml +++ b/extensions-contrib/kafka-eight-simpleConsumer/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/kafka-emitter/pom.xml b/extensions-contrib/kafka-emitter/pom.xml index fec6da4a5aa5..d7db9b61a004 100644 --- a/extensions-contrib/kafka-emitter/pom.xml +++ b/extensions-contrib/kafka-emitter/pom.xml @@ -24,7 +24,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/orc-extensions/pom.xml b/extensions-contrib/orc-extensions/pom.xml index c3163a0aafa3..32e1c5359b01 100644 --- a/extensions-contrib/orc-extensions/pom.xml +++ b/extensions-contrib/orc-extensions/pom.xml @@ -24,7 +24,7 @@ druid io.druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml 4.0.0 diff --git a/extensions-contrib/parquet-extensions/pom.xml b/extensions-contrib/parquet-extensions/pom.xml index 8be40e086a56..326c96e153f1 100644 --- a/extensions-contrib/parquet-extensions/pom.xml +++ b/extensions-contrib/parquet-extensions/pom.xml @@ -8,7 +8,7 @@ druid io.druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml 4.0.0 diff --git a/extensions-contrib/rabbitmq/pom.xml b/extensions-contrib/rabbitmq/pom.xml index 576b247f772f..ae301eb51ca0 100644 --- a/extensions-contrib/rabbitmq/pom.xml +++ b/extensions-contrib/rabbitmq/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/redis-cache/pom.xml b/extensions-contrib/redis-cache/pom.xml index b657fdca1297..7dfe086197fd 100644 --- a/extensions-contrib/redis-cache/pom.xml +++ b/extensions-contrib/redis-cache/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/scan-query/pom.xml b/extensions-contrib/scan-query/pom.xml index ce7bdebd22e5..b94c9c52596b 100644 --- a/extensions-contrib/scan-query/pom.xml +++ b/extensions-contrib/scan-query/pom.xml @@ -23,7 +23,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml 4.0.0 diff --git a/extensions-contrib/sqlserver-metadata-storage/pom.xml b/extensions-contrib/sqlserver-metadata-storage/pom.xml index 9aa8013ff910..29fe2b9e066d 100644 --- a/extensions-contrib/sqlserver-metadata-storage/pom.xml +++ b/extensions-contrib/sqlserver-metadata-storage/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-contrib/statsd-emitter/pom.xml b/extensions-contrib/statsd-emitter/pom.xml index bf6d1addb1d7..a8bcfcca487c 100644 --- a/extensions-contrib/statsd-emitter/pom.xml +++ b/extensions-contrib/statsd-emitter/pom.xml @@ -21,7 +21,7 @@ druid io.druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml 4.0.0 diff --git a/extensions-contrib/thrift-extensions/pom.xml b/extensions-contrib/thrift-extensions/pom.xml index cdeae5a32b5d..f4eebff0b542 100644 --- a/extensions-contrib/thrift-extensions/pom.xml +++ b/extensions-contrib/thrift-extensions/pom.xml @@ -9,7 +9,7 @@ druid io.druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml 4.0.0 diff --git a/extensions-contrib/time-min-max/pom.xml b/extensions-contrib/time-min-max/pom.xml index 8931502b7aca..37efbb81f726 100644 --- a/extensions-contrib/time-min-max/pom.xml +++ b/extensions-contrib/time-min-max/pom.xml @@ -21,7 +21,7 @@ druid io.druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml 4.0.0 diff --git a/extensions-contrib/virtual-columns/pom.xml b/extensions-contrib/virtual-columns/pom.xml index fb81f14c7ba1..75900c20e317 100644 --- a/extensions-contrib/virtual-columns/pom.xml +++ b/extensions-contrib/virtual-columns/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/avro-extensions/pom.xml b/extensions-core/avro-extensions/pom.xml index e469f1a61c4c..ca2551647c74 100644 --- a/extensions-core/avro-extensions/pom.xml +++ b/extensions-core/avro-extensions/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/caffeine-cache/pom.xml b/extensions-core/caffeine-cache/pom.xml index 4be81d6b4b2f..aeb26351fac1 100644 --- a/extensions-core/caffeine-cache/pom.xml +++ b/extensions-core/caffeine-cache/pom.xml @@ -30,7 +30,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/datasketches/pom.xml b/extensions-core/datasketches/pom.xml index 9699be14ea90..a368ba7c91e1 100644 --- a/extensions-core/datasketches/pom.xml +++ b/extensions-core/datasketches/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/druid-kerberos/pom.xml b/extensions-core/druid-kerberos/pom.xml index dfa8e25748f3..d2e774b08139 100644 --- a/extensions-core/druid-kerberos/pom.xml +++ b/extensions-core/druid-kerberos/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/hdfs-storage/pom.xml b/extensions-core/hdfs-storage/pom.xml index 39e433658756..bafde5112530 100644 --- a/extensions-core/hdfs-storage/pom.xml +++ b/extensions-core/hdfs-storage/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/histogram/pom.xml b/extensions-core/histogram/pom.xml index d67ed9619691..100aebf2ef5e 100644 --- a/extensions-core/histogram/pom.xml +++ b/extensions-core/histogram/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/kafka-eight/pom.xml b/extensions-core/kafka-eight/pom.xml index 03a1ce6bcde7..bf9aed2f6b7d 100644 --- a/extensions-core/kafka-eight/pom.xml +++ b/extensions-core/kafka-eight/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/kafka-extraction-namespace/pom.xml b/extensions-core/kafka-extraction-namespace/pom.xml index 5f4392e536f6..075478fdf6b9 100644 --- a/extensions-core/kafka-extraction-namespace/pom.xml +++ b/extensions-core/kafka-extraction-namespace/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/kafka-indexing-service/pom.xml b/extensions-core/kafka-indexing-service/pom.xml index 653cbae4293e..c84851ab183c 100644 --- a/extensions-core/kafka-indexing-service/pom.xml +++ b/extensions-core/kafka-indexing-service/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/lookups-cached-global/pom.xml b/extensions-core/lookups-cached-global/pom.xml index ad20b8ff1225..46750c366b2c 100644 --- a/extensions-core/lookups-cached-global/pom.xml +++ b/extensions-core/lookups-cached-global/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/lookups-cached-single/pom.xml b/extensions-core/lookups-cached-single/pom.xml index dd4fc9f714fc..317d48c262f1 100644 --- a/extensions-core/lookups-cached-single/pom.xml +++ b/extensions-core/lookups-cached-single/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/mysql-metadata-storage/pom.xml b/extensions-core/mysql-metadata-storage/pom.xml index 262c34357ec2..cd5ea290cc84 100644 --- a/extensions-core/mysql-metadata-storage/pom.xml +++ b/extensions-core/mysql-metadata-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/postgresql-metadata-storage/pom.xml b/extensions-core/postgresql-metadata-storage/pom.xml index 0bc9755ff2f1..34fdddd95c03 100644 --- a/extensions-core/postgresql-metadata-storage/pom.xml +++ b/extensions-core/postgresql-metadata-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/protobuf-extensions/pom.xml b/extensions-core/protobuf-extensions/pom.xml index 330dca1c7f72..5904debeacc3 100644 --- a/extensions-core/protobuf-extensions/pom.xml +++ b/extensions-core/protobuf-extensions/pom.xml @@ -27,7 +27,7 @@ druid io.druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/s3-extensions/pom.xml b/extensions-core/s3-extensions/pom.xml index e092f1592f0c..05149133840b 100644 --- a/extensions-core/s3-extensions/pom.xml +++ b/extensions-core/s3-extensions/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/extensions-core/simple-client-sslcontext/pom.xml b/extensions-core/simple-client-sslcontext/pom.xml index 319fd7fa4fdc..11f5b0d6080f 100644 --- a/extensions-core/simple-client-sslcontext/pom.xml +++ b/extensions-core/simple-client-sslcontext/pom.xml @@ -3,7 +3,7 @@ druid io.druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml 4.0.0 diff --git a/extensions-core/stats/pom.xml b/extensions-core/stats/pom.xml index 583528e065df..b0feb252bb38 100644 --- a/extensions-core/stats/pom.xml +++ b/extensions-core/stats/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg ../../pom.xml diff --git a/hll/pom.xml b/hll/pom.xml index a2e4b8b75646..01bcdc6ebffa 100644 --- a/hll/pom.xml +++ b/hll/pom.xml @@ -24,7 +24,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg druid-hll diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index d29f330a7bbd..c9439e24c7f0 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 17adaf1597a5..676137e70caa 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index 2f6227e3fb66..d63460f887ba 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg diff --git a/java-util/pom.xml b/java-util/pom.xml index 299511a66035..9ab5a2fad64e 100644 --- a/java-util/pom.xml +++ b/java-util/pom.xml @@ -24,7 +24,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg java-util diff --git a/pom.xml b/pom.xml index fa2af3f7d9d0..1b3ce6bd4887 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg pom ${project.groupId}:${project.artifactId} @@ -53,7 +53,7 @@ scm:git:ssh://git@github.com/druid-io/druid.git scm:git:ssh://git@github.com/druid-io/druid.git https://github.com/druid-io/druid.git - 0.11.0-SNAPSHOT + druid-0.11.0-sg diff --git a/processing/pom.xml b/processing/pom.xml index 6c2170293e30..b87c1f20b07f 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg diff --git a/server/pom.xml b/server/pom.xml index 00b55e545d58..94a7520f8cb9 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg diff --git a/services/pom.xml b/services/pom.xml index 857e6c1b9fe0..e9f4649b7f2c 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -25,7 +25,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg diff --git a/sql/pom.xml b/sql/pom.xml index a0255379f150..011896ed4c31 100644 --- a/sql/pom.xml +++ b/sql/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.11.0-sg2-SNAPSHOT + 0.11.0-sg From 5a8787994c0ed8ff79a799c514dd5999e9a08035 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 9 Sep 2017 12:38:54 +0900 Subject: [PATCH 22/44] Revert "[maven-release-plugin] prepare release druid-0.11.0-sg" This reverts commit 0f5c3a8b82415b34fa765dc375d87ae8fe4daa3b. --- api/pom.xml | 2 +- aws-common/pom.xml | 2 +- benchmarks/pom.xml | 2 +- bytebuffer-collections/pom.xml | 5 +++-- common/pom.xml | 2 +- distribution/pom.xml | 9 +++++---- examples/pom.xml | 2 +- extendedset/pom.xml | 5 +++-- extensions-contrib/ambari-metrics-emitter/pom.xml | 5 +++-- extensions-contrib/azure-extensions/pom.xml | 5 +++-- extensions-contrib/cassandra-storage/pom.xml | 2 +- extensions-contrib/cloudfiles-extensions/pom.xml | 5 +++-- extensions-contrib/distinctcount/pom.xml | 5 +++-- extensions-contrib/druid-rocketmq/pom.xml | 6 ++++-- extensions-contrib/google-extensions/pom.xml | 2 +- extensions-contrib/graphite-emitter/pom.xml | 5 +++-- extensions-contrib/kafka-eight-simpleConsumer/pom.xml | 5 +++-- extensions-contrib/kafka-emitter/pom.xml | 5 +++-- extensions-contrib/orc-extensions/pom.xml | 6 ++++-- extensions-contrib/parquet-extensions/pom.xml | 6 ++++-- extensions-contrib/rabbitmq/pom.xml | 2 +- extensions-contrib/redis-cache/pom.xml | 5 +++-- extensions-contrib/scan-query/pom.xml | 6 ++++-- extensions-contrib/sqlserver-metadata-storage/pom.xml | 2 +- extensions-contrib/statsd-emitter/pom.xml | 6 ++++-- extensions-contrib/thrift-extensions/pom.xml | 6 ++++-- extensions-contrib/time-min-max/pom.xml | 6 ++++-- extensions-contrib/virtual-columns/pom.xml | 2 +- extensions-core/avro-extensions/pom.xml | 5 +++-- extensions-core/caffeine-cache/pom.xml | 2 +- extensions-core/datasketches/pom.xml | 5 +++-- extensions-core/druid-kerberos/pom.xml | 5 +++-- extensions-core/hdfs-storage/pom.xml | 2 +- extensions-core/histogram/pom.xml | 2 +- extensions-core/kafka-eight/pom.xml | 2 +- extensions-core/kafka-extraction-namespace/pom.xml | 5 +++-- extensions-core/kafka-indexing-service/pom.xml | 2 +- extensions-core/lookups-cached-global/pom.xml | 5 +++-- extensions-core/lookups-cached-single/pom.xml | 5 +++-- extensions-core/mysql-metadata-storage/pom.xml | 2 +- extensions-core/postgresql-metadata-storage/pom.xml | 2 +- extensions-core/protobuf-extensions/pom.xml | 6 ++++-- extensions-core/s3-extensions/pom.xml | 5 +++-- extensions-core/simple-client-sslcontext/pom.xml | 6 ++++-- extensions-core/stats/pom.xml | 2 +- hll/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- integration-tests/pom.xml | 2 +- java-util/pom.xml | 2 +- pom.xml | 7 ++++--- processing/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- sql/pom.xml | 5 +++-- 55 files changed, 126 insertions(+), 88 deletions(-) diff --git a/api/pom.xml b/api/pom.xml index c68ad3cd60b0..5a921f8edda6 100644 --- a/api/pom.xml +++ b/api/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT diff --git a/aws-common/pom.xml b/aws-common/pom.xml index 1d88f2418a4c..a765e4dc87e8 100644 --- a/aws-common/pom.xml +++ b/aws-common/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT diff --git a/benchmarks/pom.xml b/benchmarks/pom.xml index 89291799ebcf..34c7abf7ed3c 100644 --- a/benchmarks/pom.xml +++ b/benchmarks/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT diff --git a/bytebuffer-collections/pom.xml b/bytebuffer-collections/pom.xml index 82aeb3ab8781..b77ee65e9a32 100755 --- a/bytebuffer-collections/pom.xml +++ b/bytebuffer-collections/pom.xml @@ -18,13 +18,14 @@ ~ under the License. --> - + 4.0.0 io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT bytebuffer-collections diff --git a/common/pom.xml b/common/pom.xml index 9bc11446c073..b24e68c63ed3 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT diff --git a/distribution/pom.xml b/distribution/pom.xml index 7b1215fd921d..4dca3b03f514 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -16,7 +16,8 @@ ~ limitations under the License. --> - + 4.0.0 pom @@ -28,7 +29,7 @@ druid io.druid - 0.11.0-sg + 0.11.0-SNAPSHOT @@ -59,7 +60,7 @@ java -classpath - + -Ddruid.extensions.loadList=[] -Ddruid.extensions.directory=${project.build.directory}/extensions @@ -189,7 +190,7 @@ java -classpath - + -Ddruid.extensions.loadList=[] -Ddruid.extensions.directory=${project.build.directory}/extensions diff --git a/examples/pom.xml b/examples/pom.xml index cab671f380d8..fbb75bbae5c9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT diff --git a/extendedset/pom.xml b/extendedset/pom.xml index e6b97cc05413..f5cc2b595e70 100755 --- a/extendedset/pom.xml +++ b/extendedset/pom.xml @@ -18,7 +18,8 @@ ~ under the License. --> - + 4.0.0 extendedset @@ -31,7 +32,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT diff --git a/extensions-contrib/ambari-metrics-emitter/pom.xml b/extensions-contrib/ambari-metrics-emitter/pom.xml index d87cbcbc6fbf..251355c7239f 100644 --- a/extensions-contrib/ambari-metrics-emitter/pom.xml +++ b/extensions-contrib/ambari-metrics-emitter/pom.xml @@ -18,13 +18,14 @@ ~ under the License. --> - + 4.0.0 io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/azure-extensions/pom.xml b/extensions-contrib/azure-extensions/pom.xml index d34a5c4cb8b1..f259a976b1b4 100644 --- a/extensions-contrib/azure-extensions/pom.xml +++ b/extensions-contrib/azure-extensions/pom.xml @@ -18,7 +18,8 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions.contrib @@ -29,7 +30,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/cassandra-storage/pom.xml b/extensions-contrib/cassandra-storage/pom.xml index b0f53e6460be..d22e13902e8a 100644 --- a/extensions-contrib/cassandra-storage/pom.xml +++ b/extensions-contrib/cassandra-storage/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/cloudfiles-extensions/pom.xml b/extensions-contrib/cloudfiles-extensions/pom.xml index b7725405e0fa..1958236ccd2a 100644 --- a/extensions-contrib/cloudfiles-extensions/pom.xml +++ b/extensions-contrib/cloudfiles-extensions/pom.xml @@ -18,7 +18,8 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions.contrib @@ -29,7 +30,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/distinctcount/pom.xml b/extensions-contrib/distinctcount/pom.xml index 91019b6b2ec8..5bae557dd374 100644 --- a/extensions-contrib/distinctcount/pom.xml +++ b/extensions-contrib/distinctcount/pom.xml @@ -18,7 +18,8 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions.contrib @@ -29,7 +30,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/druid-rocketmq/pom.xml b/extensions-contrib/druid-rocketmq/pom.xml index b8af8823348e..f0c0508363df 100644 --- a/extensions-contrib/druid-rocketmq/pom.xml +++ b/extensions-contrib/druid-rocketmq/pom.xml @@ -17,12 +17,14 @@ ~ specific language governing permissions and limitations ~ under the License. --> - + 4.0.0 druid io.druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/google-extensions/pom.xml b/extensions-contrib/google-extensions/pom.xml index f1d0133240a0..d990a3751077 100644 --- a/extensions-contrib/google-extensions/pom.xml +++ b/extensions-contrib/google-extensions/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/graphite-emitter/pom.xml b/extensions-contrib/graphite-emitter/pom.xml index aa666f86e700..d25e9159c625 100644 --- a/extensions-contrib/graphite-emitter/pom.xml +++ b/extensions-contrib/graphite-emitter/pom.xml @@ -18,13 +18,14 @@ ~ under the License. --> - + 4.0.0 io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/kafka-eight-simpleConsumer/pom.xml b/extensions-contrib/kafka-eight-simpleConsumer/pom.xml index 699de948bd4b..d52856ebed05 100644 --- a/extensions-contrib/kafka-eight-simpleConsumer/pom.xml +++ b/extensions-contrib/kafka-eight-simpleConsumer/pom.xml @@ -16,7 +16,8 @@ ~ limitations under the License. --> - + 4.0.0 io.druid.extensions.contrib druid-kafka-eight-simple-consumer @@ -26,7 +27,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/kafka-emitter/pom.xml b/extensions-contrib/kafka-emitter/pom.xml index d7db9b61a004..5d7fd6b18ad3 100644 --- a/extensions-contrib/kafka-emitter/pom.xml +++ b/extensions-contrib/kafka-emitter/pom.xml @@ -18,13 +18,14 @@ ~ under the License. --> - + 4.0.0 io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/orc-extensions/pom.xml b/extensions-contrib/orc-extensions/pom.xml index 32e1c5359b01..e6c330e87d74 100644 --- a/extensions-contrib/orc-extensions/pom.xml +++ b/extensions-contrib/orc-extensions/pom.xml @@ -15,7 +15,9 @@ ~ See the License for the specific language governing permissions and ~ limitations under the License. --> - + io.druid.extensions.contrib druid-orc-extensions druid-orc-extensions @@ -24,7 +26,7 @@ druid io.druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/parquet-extensions/pom.xml b/extensions-contrib/parquet-extensions/pom.xml index 326c96e153f1..ca4d0fcf4d59 100644 --- a/extensions-contrib/parquet-extensions/pom.xml +++ b/extensions-contrib/parquet-extensions/pom.xml @@ -1,5 +1,7 @@ - + io.druid.extensions.contrib druid-parquet-extensions druid-parquet-extensions @@ -8,7 +10,7 @@ druid io.druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/rabbitmq/pom.xml b/extensions-contrib/rabbitmq/pom.xml index ae301eb51ca0..016a7a54ce2c 100644 --- a/extensions-contrib/rabbitmq/pom.xml +++ b/extensions-contrib/rabbitmq/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/redis-cache/pom.xml b/extensions-contrib/redis-cache/pom.xml index 7dfe086197fd..8f77f1138851 100644 --- a/extensions-contrib/redis-cache/pom.xml +++ b/extensions-contrib/redis-cache/pom.xml @@ -19,7 +19,8 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions.contrib @@ -29,7 +30,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/scan-query/pom.xml b/extensions-contrib/scan-query/pom.xml index b94c9c52596b..328be800e5d7 100644 --- a/extensions-contrib/scan-query/pom.xml +++ b/extensions-contrib/scan-query/pom.xml @@ -18,12 +18,14 @@ ~ under the License. --> - + io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/sqlserver-metadata-storage/pom.xml b/extensions-contrib/sqlserver-metadata-storage/pom.xml index 29fe2b9e066d..012cedbbf20b 100644 --- a/extensions-contrib/sqlserver-metadata-storage/pom.xml +++ b/extensions-contrib/sqlserver-metadata-storage/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/statsd-emitter/pom.xml b/extensions-contrib/statsd-emitter/pom.xml index a8bcfcca487c..eb5daee7cbd3 100644 --- a/extensions-contrib/statsd-emitter/pom.xml +++ b/extensions-contrib/statsd-emitter/pom.xml @@ -17,11 +17,13 @@ ~ specific language governing permissions and limitations ~ under the License. --> - + druid io.druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/thrift-extensions/pom.xml b/extensions-contrib/thrift-extensions/pom.xml index f4eebff0b542..bc689162fb25 100644 --- a/extensions-contrib/thrift-extensions/pom.xml +++ b/extensions-contrib/thrift-extensions/pom.xml @@ -1,5 +1,7 @@ - + io.druid.extensions.contrib druid-thrift-extensions @@ -9,7 +11,7 @@ druid io.druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/time-min-max/pom.xml b/extensions-contrib/time-min-max/pom.xml index 37efbb81f726..fc3b15067f40 100644 --- a/extensions-contrib/time-min-max/pom.xml +++ b/extensions-contrib/time-min-max/pom.xml @@ -17,11 +17,13 @@ ~ specific language governing permissions and limitations ~ under the License. --> - + druid io.druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/virtual-columns/pom.xml b/extensions-contrib/virtual-columns/pom.xml index 75900c20e317..6d8dac5abfdc 100644 --- a/extensions-contrib/virtual-columns/pom.xml +++ b/extensions-contrib/virtual-columns/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/avro-extensions/pom.xml b/extensions-core/avro-extensions/pom.xml index ca2551647c74..9ff968338ecc 100644 --- a/extensions-core/avro-extensions/pom.xml +++ b/extensions-core/avro-extensions/pom.xml @@ -18,7 +18,8 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions @@ -29,7 +30,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/caffeine-cache/pom.xml b/extensions-core/caffeine-cache/pom.xml index aeb26351fac1..e4ff268a34e6 100644 --- a/extensions-core/caffeine-cache/pom.xml +++ b/extensions-core/caffeine-cache/pom.xml @@ -30,7 +30,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/datasketches/pom.xml b/extensions-core/datasketches/pom.xml index a368ba7c91e1..9f7ebfe5d847 100644 --- a/extensions-core/datasketches/pom.xml +++ b/extensions-core/datasketches/pom.xml @@ -18,7 +18,8 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions @@ -29,7 +30,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/druid-kerberos/pom.xml b/extensions-core/druid-kerberos/pom.xml index d2e774b08139..8d2dcd50a6c9 100644 --- a/extensions-core/druid-kerberos/pom.xml +++ b/extensions-core/druid-kerberos/pom.xml @@ -18,7 +18,8 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions @@ -29,7 +30,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/hdfs-storage/pom.xml b/extensions-core/hdfs-storage/pom.xml index bafde5112530..4dd68dbecae5 100644 --- a/extensions-core/hdfs-storage/pom.xml +++ b/extensions-core/hdfs-storage/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/histogram/pom.xml b/extensions-core/histogram/pom.xml index 100aebf2ef5e..8eb5ae08db29 100644 --- a/extensions-core/histogram/pom.xml +++ b/extensions-core/histogram/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/kafka-eight/pom.xml b/extensions-core/kafka-eight/pom.xml index bf9aed2f6b7d..7e5f43cf0c29 100644 --- a/extensions-core/kafka-eight/pom.xml +++ b/extensions-core/kafka-eight/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/kafka-extraction-namespace/pom.xml b/extensions-core/kafka-extraction-namespace/pom.xml index 075478fdf6b9..b1dd4f3450d6 100644 --- a/extensions-core/kafka-extraction-namespace/pom.xml +++ b/extensions-core/kafka-extraction-namespace/pom.xml @@ -18,7 +18,8 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions druid-kafka-extraction-namespace @@ -28,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/kafka-indexing-service/pom.xml b/extensions-core/kafka-indexing-service/pom.xml index c84851ab183c..9c078961a7fa 100644 --- a/extensions-core/kafka-indexing-service/pom.xml +++ b/extensions-core/kafka-indexing-service/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/lookups-cached-global/pom.xml b/extensions-core/lookups-cached-global/pom.xml index 46750c366b2c..c06aef7a6deb 100644 --- a/extensions-core/lookups-cached-global/pom.xml +++ b/extensions-core/lookups-cached-global/pom.xml @@ -18,7 +18,8 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions druid-lookups-cached-global @@ -28,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/lookups-cached-single/pom.xml b/extensions-core/lookups-cached-single/pom.xml index 317d48c262f1..e62a1590e86c 100644 --- a/extensions-core/lookups-cached-single/pom.xml +++ b/extensions-core/lookups-cached-single/pom.xml @@ -18,7 +18,8 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions druid-lookups-cached-single @@ -28,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/mysql-metadata-storage/pom.xml b/extensions-core/mysql-metadata-storage/pom.xml index cd5ea290cc84..e411e9953614 100644 --- a/extensions-core/mysql-metadata-storage/pom.xml +++ b/extensions-core/mysql-metadata-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/postgresql-metadata-storage/pom.xml b/extensions-core/postgresql-metadata-storage/pom.xml index 34fdddd95c03..3910e39d52ac 100644 --- a/extensions-core/postgresql-metadata-storage/pom.xml +++ b/extensions-core/postgresql-metadata-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/protobuf-extensions/pom.xml b/extensions-core/protobuf-extensions/pom.xml index 5904debeacc3..551ded9511d0 100644 --- a/extensions-core/protobuf-extensions/pom.xml +++ b/extensions-core/protobuf-extensions/pom.xml @@ -16,7 +16,9 @@ ~ limitations under the License. --> - + 4.0.0 io.druid.extensions @@ -27,7 +29,7 @@ druid io.druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/s3-extensions/pom.xml b/extensions-core/s3-extensions/pom.xml index 05149133840b..b12dd46cb349 100644 --- a/extensions-core/s3-extensions/pom.xml +++ b/extensions-core/s3-extensions/pom.xml @@ -16,7 +16,8 @@ ~ limitations under the License. --> - + 4.0.0 io.druid.extensions @@ -27,7 +28,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/simple-client-sslcontext/pom.xml b/extensions-core/simple-client-sslcontext/pom.xml index 11f5b0d6080f..be4333b13fce 100644 --- a/extensions-core/simple-client-sslcontext/pom.xml +++ b/extensions-core/simple-client-sslcontext/pom.xml @@ -1,9 +1,11 @@ - + druid io.druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-core/stats/pom.xml b/extensions-core/stats/pom.xml index b0feb252bb38..44ce7c53a7dd 100644 --- a/extensions-core/stats/pom.xml +++ b/extensions-core/stats/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT ../../pom.xml diff --git a/hll/pom.xml b/hll/pom.xml index 01bcdc6ebffa..e837d3c148be 100644 --- a/hll/pom.xml +++ b/hll/pom.xml @@ -24,7 +24,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT druid-hll diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index c9439e24c7f0..9fd6d8007e26 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 676137e70caa..2ab98ac87143 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index d63460f887ba..1864d651e513 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT diff --git a/java-util/pom.xml b/java-util/pom.xml index 9ab5a2fad64e..2b023a8902fc 100644 --- a/java-util/pom.xml +++ b/java-util/pom.xml @@ -24,7 +24,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT java-util diff --git a/pom.xml b/pom.xml index 1b3ce6bd4887..e04d669e4219 100644 --- a/pom.xml +++ b/pom.xml @@ -16,7 +16,8 @@ ~ limitations under the License. --> - + 4.0.0 @@ -26,7 +27,7 @@ druid - 0.11.0-sg + 0.11.0-SNAPSHOT pom ${project.groupId}:${project.artifactId} @@ -53,7 +54,7 @@ scm:git:ssh://git@github.com/druid-io/druid.git scm:git:ssh://git@github.com/druid-io/druid.git https://github.com/druid-io/druid.git - druid-0.11.0-sg + 0.11.0-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index b87c1f20b07f..9479b623c48b 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 94a7520f8cb9..5a5dca163b1c 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index e9f4649b7f2c..506b4a29c274 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -25,7 +25,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT @@ -116,7 +116,7 @@ - + diff --git a/sql/pom.xml b/sql/pom.xml index 011896ed4c31..5311c33c700a 100644 --- a/sql/pom.xml +++ b/sql/pom.xml @@ -18,7 +18,8 @@ ~ under the License. --> - + 4.0.0 druid-sql @@ -28,7 +29,7 @@ io.druid druid - 0.11.0-sg + 0.11.0-SNAPSHOT From e0699c22ce1577b38134ce7f8ca36c9bdbefcf1d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 9 Sep 2017 12:41:19 +0900 Subject: [PATCH 23/44] Fix build failure --- .../query/groupby/epinephelinae/RowBasedGrouperHelper.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index c206b3e90a69..566b9454ab1a 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -66,8 +66,6 @@ import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ValueType; import io.druid.segment.data.IndexedInts; -import it.unimi.dsi.fastutil.ints.IntArrays; -import it.unimi.dsi.fastutil.ints.IntComparator; import it.unimi.dsi.fastutil.objects.Object2IntMap; import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import org.joda.time.DateTime; @@ -83,8 +81,6 @@ import java.util.Map; import java.util.Set; import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.IntStream; // this class contains shared code between GroupByMergingQueryRunnerV2 and GroupByRowProcessor public class RowBasedGrouperHelper From 324201c0691a63f4e17f6e301097a9086b0e3540 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sun, 10 Sep 2017 10:55:38 +0900 Subject: [PATCH 24/44] Change list to array --- .../epinephelinae/RowBasedGrouperHelper.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index 566b9454ab1a..75f7c80f3d6f 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -947,7 +947,7 @@ private static class RowBasedKeySerde implements Grouper.KeySerde serdeHelpers; + private final RowBasedKeySerdeHelper[] serdeHelpers; private final DefaultLimitSpec limitSpec; private final List valueTypes; @@ -1057,7 +1057,7 @@ public ByteBuffer toByteBuffer(RowBasedKey key) dimStart = 0; } for (int i = dimStart; i < key.getKey().length; i++) { - if (!serdeHelpers.get(i - dimStart).putToKeyBuffer(key, i)) { + if (!serdeHelpers[i - dimStart].putToKeyBuffer(key, i)) { return null; } } @@ -1086,7 +1086,7 @@ public RowBasedKey fromByteBuffer(ByteBuffer buffer, int position) for (int i = dimStart; i < key.length; i++) { // Writes value from buffer to key[i] - serdeHelpers.get(i - dimStart).getFromByteBuffer(buffer, dimsPosition, i, key); + serdeHelpers[i - dimStart].getFromByteBuffer(buffer, dimsPosition, i, key); } return new RowBasedKey(key); @@ -1149,7 +1149,7 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) { for (int i = 0; i < dimCount; i++) { - final int cmp = serdeHelpers.get(i).compare( + final int cmp = serdeHelpers[i].compare( lhsBuffer, rhsBuffer, lhsPosition, @@ -1185,7 +1185,7 @@ public Grouper.BufferComparator bufferComparatorWithAggregators( needsReverse = orderSpec.getDirection() != OrderByColumnSpec.Direction.ASCENDING; int dimIndex = OrderByColumnSpec.getDimIndexForOrderBy(orderSpec, dimensions); if (dimIndex >= 0) { - RowBasedKeySerdeHelper serdeHelper = serdeHelpers.get(dimIndex); + RowBasedKeySerdeHelper serdeHelper = serdeHelpers[dimIndex]; orderByHelpers.add(serdeHelper); orderByIndices.add(dimIndex); needsReverses.add(needsReverse); @@ -1214,7 +1214,7 @@ public Grouper.BufferComparator bufferComparatorWithAggregators( for (int i = 0; i < dimCount; i++) { if (!orderByIndices.contains(i)) { - otherDimHelpers.add(serdeHelpers.get(i)); + otherDimHelpers.add(serdeHelpers[i]); needsReverses.add(false); // default to Ascending order if dim is not in an orderby spec } } @@ -1328,7 +1328,7 @@ private int getTotalKeySize() return size; } - private List makeSerdeHelpers( + private RowBasedKeySerdeHelper[] makeSerdeHelpers( boolean pushLimitDown, boolean enableRuntimeDictionaryGeneration ) @@ -1357,7 +1357,7 @@ private List makeSerdeHelpers( helpers.add(helper); } - return helpers; + return helpers.toArray(new RowBasedKeySerdeHelper[helpers.size()]); } private RowBasedKeySerdeHelper makeSerdeHelper( @@ -1696,7 +1696,7 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, } private static int compareDimsInBuffersForNullFudgeTimestamp( - List serdeHelpers, + RowBasedKeySerdeHelper[] serdeHelpers, ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, From b748da59631fb3c5a881a6eefb61826ed39cdf24 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sun, 10 Sep 2017 15:12:28 +0900 Subject: [PATCH 25/44] rename sortableIds --- .../epinephelinae/RowBasedGrouperHelper.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index 75f7c80f3d6f..d9e061d39843 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -961,9 +961,9 @@ private static class RowBasedKeySerde implements Grouper.KeySerde its position if it were sorted by dictionary value + // dictionary id -> rank of the sorted dictionary // This is mutable only when runtime dictionary generation is enabled. - private int[] sortableIds = null; + private int[] rankOfDictionaryIds = null; RowBasedKeySerde( final boolean includeTimestamp, @@ -1020,9 +1020,9 @@ private void initializeSortableIds() } Arrays.sort(dictAndIds, Comparator.comparing(pair -> pair.lhs)); - sortableIds = new int[dictionarySize]; + rankOfDictionaryIds = new int[dictionarySize]; for (int i = 0; i < dictionarySize; i++) { - sortableIds[dictAndIds[i].rhs] = i; + rankOfDictionaryIds[dictAndIds[i].rhs] = i; } } @@ -1095,7 +1095,7 @@ public RowBasedKey fromByteBuffer(ByteBuffer buffer, int position) @Override public Grouper.BufferComparator bufferComparator() { - if (sortableIds == null) { + if (rankOfDictionaryIds == null) { initializeSortableIds(); } @@ -1314,7 +1314,7 @@ public void reset() if (enableRuntimeDictionaryGeneration) { dictionary.clear(); reverseDictionary.clear(); - sortableIds = null; + rankOfDictionaryIds = null; currentEstimatedSize = 0; } } @@ -1431,8 +1431,8 @@ protected AbstractStringRowBasedKeySerdeHelper( this.keyBufferPosition = keyBufferPosition; if (!pushLimitDown) { bufferComparator = (lhsBuffer, rhsBuffer, lhsPosition, rhsPosition) -> Ints.compare( - sortableIds[lhsBuffer.getInt(lhsPosition + keyBufferPosition)], - sortableIds[rhsBuffer.getInt(rhsPosition + keyBufferPosition)] + rankOfDictionaryIds[lhsBuffer.getInt(lhsPosition + keyBufferPosition)], + rankOfDictionaryIds[rhsBuffer.getInt(rhsPosition + keyBufferPosition)] ); } else { final StringComparator realComparator = stringComparator == null ? From 91ed8a251b834181c2938386d2c12feb92739af1 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 12 Sep 2017 11:47:51 -0700 Subject: [PATCH 26/44] Address comments --- .../epinephelinae/ConcurrentGrouper.java | 2 +- .../epinephelinae/ParallelCombiner.java | 4 +- .../epinephelinae/SpillingGrouper.java | 45 ++++++++++--------- .../StreamingMergeSortedGrouper.java | 23 ++++------ 4 files changed, 37 insertions(+), 37 deletions(-) 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 37906ca8a73b..4ad28823a1b6 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 @@ -329,7 +329,7 @@ private List tryMergeDictionary() long totalDictionarySize = 0L; for (SpillingGrouper grouper : groupers) { - final List dictionary = grouper.getDictionary(); + final List dictionary = grouper.mergeAndGetDictionary(); for (String key : dictionary) { if (mergedDictionary.add(key)) { diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java index 814bc36199a8..087d6c277cf6 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java @@ -234,7 +234,7 @@ private static Pair findCombineDegreeAndNumBuffers( * * @see #buildCombineTree(List, Supplier, AggregatorFactory[], int, List) */ - private static int computeRequiredBufferNum(int numChildNodes, int combineDegree) + static int computeRequiredBufferNum(int numChildNodes, int combineDegree) { // numChildrenForLastNode used to determine that the last node is needed for the current level. // Please see buildCombineTree() for more details. @@ -338,7 +338,7 @@ private Pair>, Future> runCombiner( settableColumnSelectorFactory, combiningFactories ); - grouper.init(); + grouper.init(); // init() must be called before iterator(), so cannot be called inside the below callable. final ListenableFuture future = executor.submit( new AbstractPrioritizedCallable(priority) 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 2b3f71368970..8e48705a37af 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 @@ -27,9 +27,9 @@ import com.google.common.base.Throwables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; -import io.druid.java.util.common.parsers.CloseableIterator; import io.druid.java.util.common.CloseableIterators; -import io.druid.java.util.common.guava.CloseQuietly; +import io.druid.java.util.common.io.Closer; +import io.druid.java.util.common.parsers.CloseableIterator; import io.druid.query.BaseQuery; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.groupby.orderby.DefaultLimitSpec; @@ -37,7 +37,6 @@ 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; @@ -69,7 +68,6 @@ public class SpillingGrouper implements Grouper private final List files = Lists.newArrayList(); private final List dictionaryFiles = Lists.newArrayList(); - private final List closeables = Lists.newArrayList(); private final boolean sortHasNonGroupingFields; private boolean spillingAllowed = false; @@ -172,18 +170,25 @@ public void close() deleteFiles(); } - public List getDictionary() + /** + * Returns a dictionary of string keys added to this grouper. Note that the dictionary of keySerde is spilled on + * local storage whenever the inner grouper is spilled. If there are spilled dictionaries, this method loads them + * from disk and returns a merged dictionary. + * + * @return a dictionary which is a list of unique strings + */ + public List mergeAndGetDictionary() { final Set mergedDictionary = new HashSet<>(); mergedDictionary.addAll(keySerde.getDictionary()); for (File dictFile : dictionaryFiles) { - try { - final MappingIterator dictIterator = spillMapper.readValues( - spillMapper.getFactory().createParser(new LZ4BlockInputStream(new FileInputStream(dictFile))), - spillMapper.getTypeFactory().constructType(String.class) - ); - + try ( + final MappingIterator dictIterator = spillMapper.readValues( + spillMapper.getFactory().createParser(new LZ4BlockInputStream(new FileInputStream(dictFile))), + spillMapper.getTypeFactory().constructType(String.class) + ) + ) { while (dictIterator.hasNext()) { mergedDictionary.add(dictIterator.next()); } @@ -208,6 +213,7 @@ public CloseableIterator> iterator(final boolean sorted) iterators.add(grouper.iterator(sorted)); + final Closer closer = Closer.create(); for (final File file : files) { final MappingIterator> fileIterator = read(file, keySerde.keyClazz()); iterators.add( @@ -233,16 +239,19 @@ public Entry apply(Entry entry) ) ) ); - closeables.add(fileIterator); + closer.register(fileIterator); } + final Iterator> baseIterator; if (sortHasNonGroupingFields) { - return CloseableIterators.mergeSorted(iterators, defaultOrderKeyObjComparator); + baseIterator = CloseableIterators.mergeSorted(iterators, defaultOrderKeyObjComparator); } else { - return sorted ? - CloseableIterators.mergeSorted(iterators, keyObjComparator) : - CloseableIterators.concat(iterators); + baseIterator = sorted ? + CloseableIterators.mergeSorted(iterators, keyObjComparator) : + CloseableIterators.concat(iterators); } + + return CloseableIterators.wrap(baseIterator, closer); } private void spill() throws IOException @@ -287,10 +296,6 @@ private MappingIterator> read(final File file, final Class implements Grouper private int maxSlotNum; private boolean initialized; - // Below variables can be read/written by differernt threads - /** - * Indicate that this grouper consumed the last input or not. + * Indicate that this grouper consumed the last input or not. Always set by the writing thread and ready by the + * reading thread. */ private volatile boolean finished; /** - * Currently writing position. This is always moved ahead of nextReadIndex. + * Currently writing position. This is always moved ahead of nextReadIndex. + * Also, it is always incremented by the writing thread and read by both the writing and the reading threads. */ private volatile int curWriteIndex; /** - * Next read position. This can be moved to a position only when write for the position is finished. + * Next read position. This can be moved to a position only when write for the position is finished. + * Also, it is always incremented by the reading thread and read by both the writing and the reading threads. */ private volatile int nextReadIndex; @@ -189,9 +190,9 @@ public AggregateResult aggregate(KeyType key) return AggregateResult.ok(); } - catch (Throwable t) { - finish(); - throw t; + catch (RuntimeException e) { + finished = true; + throw e; } } @@ -204,12 +205,6 @@ private boolean keyEquals(ByteBuffer curKeyBuffer, ByteBuffer buffer, int buffer } } - for (; i + Integer.BYTES <= keySize; i += Integer.BYTES) { - if (curKeyBuffer.getInt(i) != buffer.getInt(bufferOffset + i)) { - return false; - } - } - for (; i < keySize; i++) { if (curKeyBuffer.get(i) != buffer.get(bufferOffset + i)) { return false; From a9a43b894598756dd2f3f25305556ace5073f1c8 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 12 Sep 2017 15:08:22 -0700 Subject: [PATCH 27/44] change to foreach loop --- .../java/io/druid/java/util/common/parsers/JSONPathParser.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/parsers/JSONPathParser.java b/java-util/src/main/java/io/druid/java/util/common/parsers/JSONPathParser.java index 58c2728d20ac..2a84fef3e76f 100644 --- a/java-util/src/main/java/io/druid/java/util/common/parsers/JSONPathParser.java +++ b/java-util/src/main/java/io/druid/java/util/common/parsers/JSONPathParser.java @@ -195,8 +195,7 @@ private Object valueConversionFunction(JsonNode val) if (val.isArray()) { List newList = new ArrayList<>(); - for (Iterator it = val.iterator(); it.hasNext(); ) { - JsonNode entry = it.next(); + for (JsonNode entry : val) { newList.add(valueConversionFunction(entry)); } return newList; From e7144bb44617b24860a77ee305de36157277da9b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 13 Sep 2017 10:55:57 -0700 Subject: [PATCH 28/44] Fix comment --- .../query/groupby/epinephelinae/RowBasedGrouperHelper.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index 7ef69b88d76c..5c98c18c88ef 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -966,7 +966,8 @@ private static class RowBasedKeySerde implements Grouper.KeySerde rank of the sorted dictionary - // This is mutable only when runtime dictionary generation is enabled. + // This is initialized in the constructor and bufferComparator() with static dictionary and dynamic dictionary, + // respectively. private int[] rankOfDictionaryIds = null; RowBasedKeySerde( From 3343392f8eb16163f9b984e3e3fcf598b3df7267 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 13 Sep 2017 12:10:51 -0700 Subject: [PATCH 29/44] Revert keyEquals() --- .../groupby/epinephelinae/StreamingMergeSortedGrouper.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java index 902e2f004d04..f6ed202358d7 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java @@ -205,6 +205,12 @@ private boolean keyEquals(ByteBuffer curKeyBuffer, ByteBuffer buffer, int buffer } } + for (; i + Integer.BYTES <= keySize; i += Integer.BYTES) { + if (curKeyBuffer.getInt(i) != buffer.getInt(bufferOffset + i)) { + return false; + } + } + for (; i < keySize; i++) { if (curKeyBuffer.get(i) != buffer.get(bufferOffset + i)) { return false; From 4628c5e5c2dbdd97a52a7ac2f48c024417268c3d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 13 Sep 2017 12:14:16 -0700 Subject: [PATCH 30/44] Remove loop --- .../groupby/epinephelinae/StreamingMergeSortedGrouper.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java index f6ed202358d7..ac890cd6a88a 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java @@ -205,10 +205,11 @@ private boolean keyEquals(ByteBuffer curKeyBuffer, ByteBuffer buffer, int buffer } } - for (; i + Integer.BYTES <= keySize; i += Integer.BYTES) { + if (i + Integer.BYTES <= keySize) { if (curKeyBuffer.getInt(i) != buffer.getInt(bufferOffset + i)) { return false; } + i += Integer.BYTES; } for (; i < keySize; i++) { From 4369c3b1cea2844fcad36d0bac59e3f2e454c392 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 13 Sep 2017 14:47:03 -0700 Subject: [PATCH 31/44] Address comments --- .../epinephelinae/RowBasedGrouperHelper.java | 111 +++++++++++++----- .../epinephelinae/RowBasedKeySerdeHelper.java | 5 +- .../StreamingMergeSortedGrouper.java | 59 +++++----- 3 files changed, 115 insertions(+), 60 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index 5c98c18c88ef..aa06797dbf9d 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -66,6 +66,8 @@ import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ValueType; import io.druid.segment.data.IndexedInts; +import it.unimi.dsi.fastutil.ints.IntArrays; +import it.unimi.dsi.fastutil.ints.IntComparator; import it.unimi.dsi.fastutil.objects.Object2IntMap; import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import org.joda.time.DateTime; @@ -75,12 +77,14 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; +import java.util.BitSet; import java.util.Comparator; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.Function; +import java.util.stream.IntStream; // this class contains shared code between GroupByMergingQueryRunnerV2 and GroupByRowProcessor public class RowBasedGrouperHelper @@ -952,6 +956,7 @@ private static class RowBasedKeySerde implements Grouper.KeySerde valueTypes; @@ -994,6 +999,8 @@ private static class RowBasedKeySerde implements Grouper.KeySerde serdeHelpers[i].getBufferComparator()); this.keySize = (includeTimestamp ? Longs.BYTES : 0) + getTotalKeySize(); this.keyBuffer = ByteBuffer.allocate(keySize); @@ -1012,25 +1019,62 @@ private static class RowBasedKeySerde implements Grouper.KeySerde[] dictAndIds = new Pair[dictionarySize]; - for (int id = 0; id < dictionarySize; id++) { - dictAndIds[id] = new Pair<>(dictionary.get(id), id); - } - Arrays.sort(dictAndIds, Comparator.comparing(pair -> pair.lhs)); + rankOfDictionaryIds = IntStream.range(0, dictionarySize).toArray(); + IntArrays.quickSort( + rankOfDictionaryIds, + new IntComparator() + { + @Override + public int compare(int i1, int i2) + { + return dictionary.get(i1).compareTo(dictionary.get(i2)); + } + + @Override + public int compare(Integer o1, Integer o2) + { + return compare(o1.intValue(), o2.intValue()); + } + } + ); - rankOfDictionaryIds = new int[dictionarySize]; - for (int i = 0; i < dictionarySize; i++) { - rankOfDictionaryIds[dictAndIds[i].rhs] = i; + inverse(rankOfDictionaryIds); + } + + private static void inverse(int[] a) { + final BitSet visited = new BitSet(a.length); + for (int i = 0; i < a.length; i++) { + if (!visited.get(i)) { + inverseLoop(a, i, visited); + } } } + private static void inverseLoop(int[] a, int startValue, BitSet visited) { + final int startIndex = a[startValue]; + + int nextIndex = startIndex; + int nextValue = startValue; + + do { + final int curIndex = nextIndex; + final int curValue = nextValue; + + nextValue = nextIndex; + nextIndex = a[curIndex]; + + visited.set(curIndex); + a[curIndex] = curValue; + } while (nextIndex != startIndex); + } + @Override public int keySize() { @@ -1101,7 +1145,7 @@ public RowBasedKey fromByteBuffer(ByteBuffer buffer, int position) public Grouper.BufferComparator bufferComparator() { if (rankOfDictionaryIds == null) { - initializeSortableIds(); + initializeRankOfDictionaryIds(); } if (includeTimestamp) { @@ -1112,7 +1156,7 @@ public Grouper.BufferComparator bufferComparator() public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) { final int cmp = compareDimsInBuffersForNullFudgeTimestamp( - serdeHelpers, + serdeHelperComparators, lhsBuffer, rhsBuffer, lhsPosition, @@ -1138,7 +1182,7 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, } return compareDimsInBuffersForNullFudgeTimestamp( - serdeHelpers, + serdeHelperComparators, lhsBuffer, rhsBuffer, lhsPosition, @@ -1154,7 +1198,7 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) { for (int i = 0; i < dimCount; i++) { - final int cmp = serdeHelpers[i].compare( + final int cmp = serdeHelperComparators[i].compare( lhsBuffer, rhsBuffer, lhsPosition, @@ -1227,6 +1271,9 @@ public Grouper.BufferComparator bufferComparatorWithAggregators( adjustedSerdeHelpers = orderByHelpers; adjustedSerdeHelpers.addAll(otherDimHelpers); + final BufferComparator[] adjustedSerdeHelperComparators = new BufferComparator[adjustedSerdeHelpers.size()]; + Arrays.setAll(adjustedSerdeHelperComparators, i -> adjustedSerdeHelpers.get(i).getBufferComparator()); + final int fieldCount = dimCount + aggCount; if (includeTimestamp) { @@ -1237,7 +1284,7 @@ public Grouper.BufferComparator bufferComparatorWithAggregators( public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) { final int cmp = compareDimsInBuffersForNullFudgeTimestampForPushDown( - adjustedSerdeHelpers, + adjustedSerdeHelperComparators, needsReverses, fieldCount, lhsBuffer, @@ -1265,7 +1312,7 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, } int cmp = compareDimsInBuffersForNullFudgeTimestampForPushDown( - adjustedSerdeHelpers, + adjustedSerdeHelperComparators, needsReverses, fieldCount, lhsBuffer, @@ -1287,14 +1334,14 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, for (int i = 0; i < fieldCount; i++) { final int cmp; if (needsReverses.get(i)) { - cmp = adjustedSerdeHelpers.get(i).compare( + cmp = adjustedSerdeHelperComparators[i].compare( rhsBuffer, lhsBuffer, rhsPosition, lhsPosition ); } else { - cmp = adjustedSerdeHelpers.get(i).compare( + cmp = adjustedSerdeHelperComparators[i].compare( lhsBuffer, rhsBuffer, lhsPosition, @@ -1464,9 +1511,9 @@ public void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int dimValId } @Override - public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) + public BufferComparator getBufferComparator() { - return bufferComparator.compare(lhsBuffer, rhsBuffer, lhsPosition, rhsPosition); + return bufferComparator; } } @@ -1590,9 +1637,9 @@ public void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int dimValId } @Override - public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) + public BufferComparator getBufferComparator() { - return bufferComparator.compare(lhsBuffer, rhsBuffer, lhsPosition, rhsPosition); + return bufferComparator; } } @@ -1641,9 +1688,9 @@ public void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int dimValId } @Override - public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) + public BufferComparator getBufferComparator() { - return bufferComparator.compare(lhsBuffer, rhsBuffer, lhsPosition, rhsPosition); + return bufferComparator; } } @@ -1693,23 +1740,23 @@ public void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int dimValId } @Override - public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) + public BufferComparator getBufferComparator() { - return bufferComparator.compare(lhsBuffer, rhsBuffer, lhsPosition, rhsPosition); + return bufferComparator; } } } private static int compareDimsInBuffersForNullFudgeTimestamp( - RowBasedKeySerdeHelper[] serdeHelpers, + BufferComparator[] serdeHelperComparators, ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition ) { - for (RowBasedKeySerdeHelper serdeHelper : serdeHelpers) { - final int cmp = serdeHelper.compare( + for (BufferComparator comparator : serdeHelperComparators) { + final int cmp = comparator.compare( lhsBuffer, rhsBuffer, lhsPosition + Longs.BYTES, @@ -1724,7 +1771,7 @@ private static int compareDimsInBuffersForNullFudgeTimestamp( } private static int compareDimsInBuffersForNullFudgeTimestampForPushDown( - List serdeHelpers, + BufferComparator[] serdeHelperComparators, List needsReverses, int dimCount, ByteBuffer lhsBuffer, @@ -1736,14 +1783,14 @@ private static int compareDimsInBuffersForNullFudgeTimestampForPushDown( for (int i = 0; i < dimCount; i++) { final int cmp; if (needsReverses.get(i)) { - cmp = serdeHelpers.get(i).compare( + cmp = serdeHelperComparators[i].compare( rhsBuffer, lhsBuffer, rhsPosition + Longs.BYTES, lhsPosition + Longs.BYTES ); } else { - cmp = serdeHelpers.get(i).compare( + cmp = serdeHelperComparators[i].compare( lhsBuffer, rhsBuffer, lhsPosition + Longs.BYTES, diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedKeySerdeHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedKeySerdeHelper.java index 500d7924715d..476c116e6846 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedKeySerdeHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedKeySerdeHelper.java @@ -19,6 +19,7 @@ package io.druid.query.groupby.epinephelinae; +import io.druid.query.groupby.epinephelinae.Grouper.BufferComparator; import io.druid.query.groupby.epinephelinae.RowBasedGrouperHelper.RowBasedKey; import java.nio.ByteBuffer; @@ -69,5 +70,7 @@ interface RowBasedKeySerdeHelper * * @return Negative number if lhs < rhs, positive if lhs > rhs, 0 if lhs == rhs */ - int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition); +// int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition); + + BufferComparator getBufferComparator(); } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java index ac890cd6a88a..dcc956a00ac7 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java @@ -45,8 +45,8 @@ public class StreamingMergeSortedGrouper implements Grouper { private static final Logger LOG = new Logger(StreamingMergeSortedGrouper.class); - // Timeout for waiting for a slot to be available for read/write. This is required to prevent for the processing - // thread from being blocked if its iterator is not consumed due to some failure. + // Timeout for waiting for a slot to be available for read/write. This is required to prevent the processing + // thread from being blocked if the iterator of this grouper is not consumed due to some failures. private static final long DEFAULT_TIMEOUT_MS = 5000L; private final Supplier bufferSupplier; @@ -59,17 +59,17 @@ public class StreamingMergeSortedGrouper implements Grouper // Below variables are initialized when init() is called private ByteBuffer buffer; - private int maxSlotNum; + private int maxNumSlots; private boolean initialized; /** - * Indicate that this grouper consumed the last input or not. Always set by the writing thread and ready by the + * Indicate that this grouper consumed the last input or not. Always set by the writing thread and read by the * reading thread. */ private volatile boolean finished; /** - * Currently writing position. This is always moved ahead of nextReadIndex. + * Current write position. This is always moved ahead of nextReadIndex. * Also, it is always incremented by the writing thread and read by both the writing and the reading threads. */ private volatile int curWriteIndex; @@ -139,9 +139,9 @@ public void init() { if (!initialized) { buffer = bufferSupplier.get(); - maxSlotNum = buffer.capacity() / recordSize; + maxNumSlots = buffer.capacity() / recordSize; Preconditions.checkState( - maxSlotNum > 2, + maxNumSlots > 2, "Buffer[%s] should be large enough to store at least three records[%s]", buffer.capacity(), recordSize @@ -240,21 +240,23 @@ private void initNewSlot(ByteBuffer newKey) */ private void increaseWriteIndex() { - if (curWriteIndex == maxSlotNum - 1) { + if (curWriteIndex == maxNumSlots - 1) { final long startLoopAt = System.currentTimeMillis(); - while ((nextReadIndex == -1 || nextReadIndex == 0) && !Thread.interrupted()) { + while ((nextReadIndex == -1 || nextReadIndex == 0) && !Thread.currentThread().isInterrupted()) { if (System.currentTimeMillis() - startLoopAt > timeoutMs) { throw new RuntimeException(new TimeoutException()); } + Thread.yield(); } curWriteIndex = 0; } else { final int nextWriteIndex = curWriteIndex + 1; final long startLoopAt = System.currentTimeMillis(); - while ((nextWriteIndex == nextReadIndex) && !Thread.interrupted()) { + while ((nextWriteIndex == nextReadIndex) && !Thread.currentThread().isInterrupted()) { if (System.currentTimeMillis() - startLoopAt > timeoutMs) { throw new RuntimeException(new TimeoutException()); } + Thread.yield(); } curWriteIndex = nextWriteIndex; } @@ -307,13 +309,7 @@ public CloseableIterator> iterator() { { // Waits for some data to be ready - final long startLoopAt = System.currentTimeMillis(); - while ((curWriteIndex == -1 || curWriteIndex == 0) && !finished && !Thread.interrupted()) { - if (System.currentTimeMillis() - startLoopAt > timeoutMs) { - throw new RuntimeException(new TimeoutException()); - } - } - nextReadIndex = 0; + increaseReadIndex(0); } @Override @@ -327,7 +323,7 @@ private int remaining() if (curWriteIndex >= nextReadIndex) { return curWriteIndex - nextReadIndex; } else { - return (maxSlotNum - nextReadIndex) + curWriteIndex; + return (maxNumSlots - nextReadIndex) + curWriteIndex; } } @@ -346,20 +342,29 @@ public Entry next() values[i] = aggregators[i].get(buffer, recordOffset + aggregatorOffsets[i]); } - final int toBeUpdated = nextReadIndex == maxSlotNum - 1 ? 0 : nextReadIndex + 1; + final int increaseTo = nextReadIndex == maxNumSlots - 1 ? 0 : nextReadIndex + 1; + increaseReadIndex(increaseTo); + + return new Entry<>(key, values); + } - if (!finished) { - final long startLoopAt = System.currentTimeMillis(); - while (toBeUpdated == curWriteIndex && !finished && !Thread.interrupted()) { - if (System.currentTimeMillis() - startLoopAt > timeoutMs) { - throw new RuntimeException(new TimeoutException()); - } + private void increaseReadIndex(int increaseTo) + { + final long startLoopAt = System.currentTimeMillis(); + while ((!isReady() || increaseTo == curWriteIndex) && !finished && !Thread.currentThread().isInterrupted()) { + if (System.currentTimeMillis() - startLoopAt > timeoutMs) { + throw new RuntimeException(new TimeoutException()); } + + Thread.yield(); } - nextReadIndex = toBeUpdated; + nextReadIndex = increaseTo; + } - return new Entry<>(key, values); + private boolean isReady() + { + return curWriteIndex != -1; } @Override From 0beacc355bba13195f99887ce98af9f46ff6c402 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 14 Sep 2017 14:29:18 -0700 Subject: [PATCH 32/44] Fix build fail --- .../epinephelinae/RowBasedGrouperHelper.java | 20 ++++++++++++++----- .../epinephelinae/RowBasedKeySerdeHelper.java | 12 +---------- 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index aa06797dbf9d..3652e83d0063 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -1048,7 +1048,16 @@ public int compare(Integer o1, Integer o2) inverse(rankOfDictionaryIds); } - private static void inverse(int[] a) { + /** + * Inverses the values of the given array with their indexes. + * For example, the result for [2, 0, 1] is [1, 2, 0] because + * + * a[0]: 2 => a[2]: 0 + * a[1]: 0 => a[0]: 1 + * a[2]: 1 => a[1]: 2 + */ + private static void inverse(int[] a) + { final BitSet visited = new BitSet(a.length); for (int i = 0; i < a.length; i++) { if (!visited.get(i)) { @@ -1057,7 +1066,8 @@ private static void inverse(int[] a) { } } - private static void inverseLoop(int[] a, int startValue, BitSet visited) { + private static void inverseLoop(int[] a, int startValue, BitSet visited) + { final int startIndex = a[startValue]; int nextIndex = startIndex; @@ -1470,11 +1480,11 @@ private static boolean isPrimitiveComparable(boolean pushLimitDown, @Nullable St private abstract class AbstractStringRowBasedKeySerdeHelper implements RowBasedKeySerdeHelper { - protected final int keyBufferPosition; + final int keyBufferPosition; - protected final BufferComparator bufferComparator; + final BufferComparator bufferComparator; - protected AbstractStringRowBasedKeySerdeHelper( + AbstractStringRowBasedKeySerdeHelper( int keyBufferPosition, boolean pushLimitDown, @Nullable StringComparator stringComparator diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedKeySerdeHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedKeySerdeHelper.java index 476c116e6846..c7e3437ded97 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedKeySerdeHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedKeySerdeHelper.java @@ -60,17 +60,7 @@ interface RowBasedKeySerdeHelper void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int dimValIdx, Comparable[] dimValues); /** - * Compare the values at lhsBuffer[lhsPosition] and rhsBuffer[rhsPosition] using the natural ordering - * for this SerdeHelper's value type. - * - * @param lhsBuffer ByteBuffer containing an array of grouping keys for a row - * @param rhsBuffer ByteBuffer containing an array of grouping keys for a row - * @param lhsPosition Position of value within lhsBuffer - * @param rhsPosition Position of value within rhsBuffer - * - * @return Negative number if lhs < rhs, positive if lhs > rhs, 0 if lhs == rhs + * Return a {@link BufferComparator} to compare keys stored in ByteBuffer. */ -// int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition); - BufferComparator getBufferComparator(); } From 08d1ed623266eeb0a95e651412e6b24666ce6fb7 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 20 Sep 2017 06:54:49 +0900 Subject: [PATCH 33/44] Address comments --- .../io/druid/common/utils/IntArrayUtils.java | 64 +++++++++++++++++ .../druid/common/utils/IntArrayUtilsTest.java | 54 +++++++++++++++ .../epinephelinae/ConcurrentGrouper.java | 4 +- .../epinephelinae/ParallelCombiner.java | 8 ++- .../epinephelinae/RowBasedGrouperHelper.java | 41 +---------- .../StreamingMergeSortedGrouper.java | 69 +++++++++++-------- .../epinephelinae/ParallelCombinerTest.java | 3 +- .../StreamingMergeSortedGrouperTest.java | 23 ++++--- 8 files changed, 185 insertions(+), 81 deletions(-) create mode 100644 common/src/main/java/io/druid/common/utils/IntArrayUtils.java create mode 100644 common/src/test/java/io/druid/common/utils/IntArrayUtilsTest.java diff --git a/common/src/main/java/io/druid/common/utils/IntArrayUtils.java b/common/src/main/java/io/druid/common/utils/IntArrayUtils.java new file mode 100644 index 000000000000..83cf5ecfd228 --- /dev/null +++ b/common/src/main/java/io/druid/common/utils/IntArrayUtils.java @@ -0,0 +1,64 @@ +/* + * 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.common.utils; + +public class IntArrayUtils +{ + /** + * Inverses the values of the given array with their indexes. + * For example, the result for [2, 0, 1] is [1, 2, 0] because + * + * a[0]: 2 => a[2]: 0 + * a[1]: 0 => a[0]: 1 + * a[2]: 1 => a[1]: 2 + */ + public static void inverse(int[] a) + { + for (int i = 0; i < a.length; i++) { + if (a[i] >= 0) { + inverseLoop(a, i); + } + } + + for (int i = 0; i < a.length; i++) { + a[i] = ~a[i]; + } + } + + private static void inverseLoop(int[] a, int startValue) + { + final int startIndex = a[startValue]; + + int nextIndex = startIndex; + int nextValue = startValue; + + do { + final int curIndex = nextIndex; + final int curValue = nextValue; + + nextValue = curIndex; + nextIndex = a[curIndex]; + + a[curIndex] = ~curValue; + } while (nextIndex != startIndex); + } + + private IntArrayUtils() {} +} diff --git a/common/src/test/java/io/druid/common/utils/IntArrayUtilsTest.java b/common/src/test/java/io/druid/common/utils/IntArrayUtilsTest.java new file mode 100644 index 000000000000..dc7f22839f60 --- /dev/null +++ b/common/src/test/java/io/druid/common/utils/IntArrayUtilsTest.java @@ -0,0 +1,54 @@ +/* + * 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.common.utils; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.List; +import java.util.Random; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +public class IntArrayUtilsTest +{ + @Test + public void testInverse() + { + final int numVals = 10000; + final Random random = new Random(System.currentTimeMillis()); + final int[] inverted = new int[numVals]; + final int[] original = new int[numVals]; + + final List ints = IntStream.range(0, numVals).boxed().collect(Collectors.toList()); + Collections.shuffle(ints, random); + + for (int i = 0; i < numVals; i++) { + inverted[i] = ints.get(i); + original[i] = inverted[i]; + } + IntArrayUtils.inverse(inverted); + + for (int i = 0; i < numVals; i++) { + Assert.assertEquals(i, inverted[original[i]]); + } + } +} 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 dfc861df10e0..cb225c71ab9d 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 @@ -32,6 +32,7 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.parsers.CloseableIterator; import io.druid.query.AbstractPrioritizedCallable; +import io.druid.query.QueryContexts; import io.druid.query.QueryInterruptedException; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.groupby.orderby.DefaultLimitSpec; @@ -142,7 +143,8 @@ public ConcurrentGrouper( executor, sortHasNonGroupingFields, concurrencyHint, - priority + priority, + queryTimeoutAt ); this.maxDictionarySizeForCombiner = combineKeySerdeFactory.getMaxDictionarySize(); } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java index 087d6c277cf6..518b31e1bc6f 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java @@ -73,6 +73,7 @@ public class ParallelCombiner private final Comparator> keyObjComparator; private final int concurrencyHint; private final int priority; + private final long queryTimeoutAt; public ParallelCombiner( Supplier> combineBufferSupplier, @@ -81,7 +82,8 @@ public ParallelCombiner( ListeningExecutorService executor, boolean sortHasNonGroupingFields, int concurrencyHint, - int priority + int priority, + long queryTimeoutAt ) { this.combineBufferSupplier = combineBufferSupplier; @@ -91,6 +93,7 @@ public ParallelCombiner( this.keyObjComparator = combineKeySerdeFactory.objectComparator(sortHasNonGroupingFields); this.concurrencyHint = concurrencyHint; this.priority = priority; + this.queryTimeoutAt = queryTimeoutAt; } /** @@ -336,7 +339,8 @@ private Pair>, Future> runCombiner( Suppliers.ofInstance(combineBuffer), combineKeySerdeFactory.factorizeWithDictionary(dictionary), settableColumnSelectorFactory, - combiningFactories + combiningFactories, + queryTimeoutAt ); grouper.init(); // init() must be called before iterator(), so cannot be called inside the below callable. diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index 3652e83d0063..ca5e0588fb56 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -34,6 +34,7 @@ import com.google.common.primitives.Longs; import com.google.common.util.concurrent.ListeningExecutorService; import io.druid.collections.ResourceHolder; +import io.druid.common.utils.IntArrayUtils; import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; import io.druid.java.util.common.IAE; @@ -77,7 +78,6 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; -import java.util.BitSet; import java.util.Comparator; import java.util.HashSet; import java.util.List; @@ -1045,44 +1045,7 @@ public int compare(Integer o1, Integer o2) } ); - inverse(rankOfDictionaryIds); - } - - /** - * Inverses the values of the given array with their indexes. - * For example, the result for [2, 0, 1] is [1, 2, 0] because - * - * a[0]: 2 => a[2]: 0 - * a[1]: 0 => a[0]: 1 - * a[2]: 1 => a[1]: 2 - */ - private static void inverse(int[] a) - { - final BitSet visited = new BitSet(a.length); - for (int i = 0; i < a.length; i++) { - if (!visited.get(i)) { - inverseLoop(a, i, visited); - } - } - } - - private static void inverseLoop(int[] a, int startValue, BitSet visited) - { - final int startIndex = a[startValue]; - - int nextIndex = startIndex; - int nextValue = startValue; - - do { - final int curIndex = nextIndex; - final int curValue = nextValue; - - nextValue = nextIndex; - nextIndex = a[curIndex]; - - visited.set(curIndex); - a[curIndex] = curValue; - } while (nextIndex != startIndex); + IntArrayUtils.inverse(rankOfDictionaryIds); } @Override diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java index dcc956a00ac7..1c385c36c209 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java @@ -19,13 +19,13 @@ package io.druid.query.groupby.epinephelinae; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; import io.druid.java.util.common.parsers.CloseableIterator; +import io.druid.query.QueryContexts; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.BufferAggregator; import io.druid.segment.ColumnSelectorFactory; @@ -33,6 +33,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.NoSuchElementException; +import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; /** @@ -45,9 +46,11 @@ public class StreamingMergeSortedGrouper implements Grouper { private static final Logger LOG = new Logger(StreamingMergeSortedGrouper.class); - // Timeout for waiting for a slot to be available for read/write. This is required to prevent the processing - // thread from being blocked if the iterator of this grouper is not consumed due to some failures. - private static final long DEFAULT_TIMEOUT_MS = 5000L; + private static final long DEFAULT_TIMEOUT_NS = 5_000_000_000L; // 5 seconds + + // Threashold time for spin locks in increaseWriteIndex() and increaseReadIndex(). Thread.yield() is called for the + // waiting thread after this threadhold time. + private static final long SPIN_FOR_TIMEOUT_THRESHOLD_NS = 1000L; private final Supplier bufferSupplier; private final KeySerde keySerde; @@ -55,7 +58,10 @@ public class StreamingMergeSortedGrouper implements Grouper private final int[] aggregatorOffsets; private final int keySize; private final int recordSize; // size of (key + all aggregates) - private final long timeoutMs; + + // Timeout for waiting for a slot to be available for read/write. This is required to prevent the processing + // thread from being blocked if the iterator of this grouper is not consumed due to some failures. + private final long queryTimeoutAtNs; // Below variables are initialized when init() is called private ByteBuffer buffer; @@ -99,23 +105,12 @@ public static int requiredBufferCapacity( return recordSize * 3; } - public StreamingMergeSortedGrouper( - final Supplier bufferSupplier, - final KeySerde keySerde, - final ColumnSelectorFactory columnSelectorFactory, - final AggregatorFactory[] aggregatorFactories - ) - { - this(bufferSupplier, keySerde, columnSelectorFactory, aggregatorFactories, DEFAULT_TIMEOUT_MS); - } - - @VisibleForTesting StreamingMergeSortedGrouper( final Supplier bufferSupplier, final KeySerde keySerde, final ColumnSelectorFactory columnSelectorFactory, final AggregatorFactory[] aggregatorFactories, - final long timeoutMs + final long queryTimeoutAtMs ) { this.bufferSupplier = bufferSupplier; @@ -131,7 +126,11 @@ public StreamingMergeSortedGrouper( offset += aggregatorFactories[i].getMaxIntermediateSize(); } this.recordSize = offset; - this.timeoutMs = timeoutMs; + final long timeoutNs = queryTimeoutAtMs == QueryContexts.NO_TIMEOUT ? + DEFAULT_TIMEOUT_NS : + TimeUnit.MILLISECONDS.toNanos(queryTimeoutAtMs - System.currentTimeMillis()); + + this.queryTimeoutAtNs = System.nanoTime() + timeoutNs; } @Override @@ -240,23 +239,32 @@ private void initNewSlot(ByteBuffer newKey) */ private void increaseWriteIndex() { + final long startAt = System.nanoTime(); + final long spinTimeoutAt = startAt + SPIN_FOR_TIMEOUT_THRESHOLD_NS; + long now = startAt; + if (curWriteIndex == maxNumSlots - 1) { - final long startLoopAt = System.currentTimeMillis(); + // Should wait for the reading thread to start reading only if the writing thread should overwrite the first slot. while ((nextReadIndex == -1 || nextReadIndex == 0) && !Thread.currentThread().isInterrupted()) { - if (System.currentTimeMillis() - startLoopAt > timeoutMs) { + if (now >= queryTimeoutAtNs) { throw new RuntimeException(new TimeoutException()); } - Thread.yield(); + if (now >= spinTimeoutAt) { + Thread.yield(); + } + now = System.nanoTime(); } curWriteIndex = 0; } else { final int nextWriteIndex = curWriteIndex + 1; - final long startLoopAt = System.currentTimeMillis(); while ((nextWriteIndex == nextReadIndex) && !Thread.currentThread().isInterrupted()) { - if (System.currentTimeMillis() - startLoopAt > timeoutMs) { + if (now >= queryTimeoutAtNs) { throw new RuntimeException(new TimeoutException()); } - Thread.yield(); + if (now >= spinTimeoutAt) { + Thread.yield(); + } + now = System.nanoTime(); } curWriteIndex = nextWriteIndex; } @@ -350,13 +358,18 @@ public Entry next() private void increaseReadIndex(int increaseTo) { - final long startLoopAt = System.currentTimeMillis(); + final long startAt = System.nanoTime(); + final long spinTimeoutAt = startAt + SPIN_FOR_TIMEOUT_THRESHOLD_NS; + long now = startAt; + while ((!isReady() || increaseTo == curWriteIndex) && !finished && !Thread.currentThread().isInterrupted()) { - if (System.currentTimeMillis() - startLoopAt > timeoutMs) { + if (now >= queryTimeoutAtNs) { throw new RuntimeException(new TimeoutException()); } - - Thread.yield(); + if (now >= spinTimeoutAt) { + Thread.yield(); + } + now = System.nanoTime(); } nextReadIndex = increaseTo; diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java index 5262dc4498af..6d1cd4de1328 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java @@ -115,7 +115,8 @@ public void testCombine() throws IOException MoreExecutors.listeningDecorator(SERVICE), false, THREAD_NUM, - 0 + 0, + 0 // use default timeout ); final int numRows = 1000; diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouperTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouperTest.java index 1bcdb3c66642..0c52819d930a 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouperTest.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouperTest.java @@ -41,7 +41,9 @@ import java.util.ArrayList; import java.util.Comparator; import java.util.List; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; import java.util.concurrent.TimeoutException; public class StreamingMergeSortedGrouperTest @@ -90,18 +92,18 @@ public void testEmptyIterator() } @Test(timeout = 5000L) - public void testStreamingAggregateWithLargeBuffer() + public void testStreamingAggregateWithLargeBuffer() throws ExecutionException, InterruptedException { testStreamingAggregate(1024); } @Test(timeout = 5000L) - public void testStreamingAggregateWithMinimumBuffer() + public void testStreamingAggregateWithMinimumBuffer() throws ExecutionException, InterruptedException { testStreamingAggregate(60); } - private void testStreamingAggregate(int bufferSize) + private void testStreamingAggregate(int bufferSize) throws ExecutionException, InterruptedException { final ExecutorService exec = Execs.multiThreaded(2, "merge-sorted-grouper-test-%d"); final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory(); @@ -113,7 +115,7 @@ private void testStreamingAggregate(int bufferSize) } try { - exec.submit(() -> { + final Future future = exec.submit(() -> { columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 10L))); for (int i = 0; i < 1024; i++) { @@ -126,12 +128,13 @@ private void testStreamingAggregate(int bufferSize) }); final List> unsortedEntries = Lists.newArrayList(grouper.iterator(true)); + final List> actual = Ordering.from((Comparator>) (o1, o2) -> Ints.compare(o1.getKey(), o2.getKey())) + .sortedCopy(unsortedEntries); - Assert.assertEquals( - expected, - Ordering.from((Comparator>) (o1, o2) -> Ints.compare(o1.getKey(), o2.getKey())) - .sortedCopy(unsortedEntries) - ); + if (!actual.equals(expected)) { + future.get(); // Check there is an exception occured + Assert.fail(); + } } finally { exec.shutdownNow(); @@ -177,7 +180,7 @@ private StreamingMergeSortedGrouper newGrouper( new LongSumAggregatorFactory("valueSum", "value"), new CountAggregatorFactory("count") }, - 1000 + System.currentTimeMillis() + 1000L ); grouper.init(); return grouper; From ac5a024b10e86120c7d49217abb499a2a9363ec3 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 20 Sep 2017 07:01:56 +0900 Subject: [PATCH 34/44] Remove unused imports --- .../io/druid/query/groupby/epinephelinae/ConcurrentGrouper.java | 1 - 1 file changed, 1 deletion(-) 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 cb225c71ab9d..23b3d56829cb 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 @@ -32,7 +32,6 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.parsers.CloseableIterator; import io.druid.query.AbstractPrioritizedCallable; -import io.druid.query.QueryContexts; import io.druid.query.QueryInterruptedException; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.groupby.orderby.DefaultLimitSpec; From f168b7425d4dbfbdd2a66806738b3e44c35302a6 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 20 Sep 2017 07:47:45 +0900 Subject: [PATCH 35/44] Fix method name --- .../io/druid/query/groupby/epinephelinae/ParallelCombiner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java index 518b31e1bc6f..56083e1d4edb 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java @@ -484,7 +484,7 @@ public Class classOfObject() } @Override - public Object get() + public Object getObject() { return values[checkAndGetColumnIndex(columnName)]; } From be123c1117833103e5ff33124429b277122683d3 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 20 Sep 2017 15:43:00 +0900 Subject: [PATCH 36/44] Split intermediate and leaf combine degrees --- docs/content/querying/groupbyquery.md | 8 +- .../query/groupby/GroupByQueryConfig.java | 14 ++++ .../epinephelinae/ConcurrentGrouper.java | 6 +- .../epinephelinae/ParallelCombiner.java | 82 +++++++++++++------ .../epinephelinae/RowBasedGrouperHelper.java | 3 +- .../StreamingMergeSortedGrouper.java | 10 +-- .../epinephelinae/ConcurrentGrouperTest.java | 3 +- .../epinephelinae/ParallelCombinerTest.java | 5 +- 8 files changed, 94 insertions(+), 37 deletions(-) diff --git a/docs/content/querying/groupbyquery.md b/docs/content/querying/groupbyquery.md index fc323217ec06..7598735f7b2b 100644 --- a/docs/content/querying/groupbyquery.md +++ b/docs/content/querying/groupbyquery.md @@ -123,8 +123,7 @@ the query context. If neither the context field nor the property is set, the "v2 - "v2", the default, 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 +concurrent facts map combined with an on-heap string dictionary. This may optionally involve spilling to disk. Data nodes sort and merge results before sending to the broker. Here, if data is spilled on disk, data nodes create a combining tree which merges results using multiple threads. Each node of the tree combines the input rows from child nodes which are run by different threads. If data is not spilled, data nodes simply merges results using a single thread. Finally, data nodes send the merged 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. @@ -220,6 +219,8 @@ When using the "v2" strategy, the following runtime properties apply: |`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.|100000000| |`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)| |`druid.query.groupBy.singleThreaded`|Merge results using a single thread.|false| +|`druid.query.groupBy.forceHashAggregation`|Force to use hash-based aggregation.|false| +|`druid.query.groupBy.intermediateCombineDegree`|The number of intermediate nodes combined together in the combining tree. Higher degrees will need a less numer of threads which might be helpful to improve the query performance by reducing the overhead of too many threads if the server has sufficiently powerful cpu cores.|8| 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 @@ -249,7 +250,8 @@ 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.| +|`forceHashAggregation`|Overrides the value of `druid.query.groupBy.forceHashAggregation` for this query.| +|`intermediateCombineDegree`|Overrides the value of `druid.query.groupBy.intermediateCombineDegree` for this query.| When using the "v1" strategy, the following query context parameters apply: 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 5ed063db1f44..d7bf6fea7174 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java @@ -38,6 +38,7 @@ public class GroupByQueryConfig 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"; + private static final String CTX_KEY_INTERMEDIATE_COMBINE_DEGREE = "intermediateCombineDegree"; @JsonProperty private String defaultStrategy = GroupByStrategySelector.STRATEGY_V2; @@ -75,6 +76,9 @@ public class GroupByQueryConfig @JsonProperty private boolean forceHashAggregation = false; + @JsonProperty + private int intermediateCombineDegree = 8; + public String getDefaultStrategy() { return defaultStrategy; @@ -145,6 +149,11 @@ public boolean isForceHashAggregation() return forceHashAggregation; } + public int getIntermediateCombineDegree() + { + return intermediateCombineDegree; + } + public GroupByQueryConfig withOverrides(final GroupByQuery query) { final GroupByQueryConfig newConfig = new GroupByQueryConfig(); @@ -180,6 +189,10 @@ public GroupByQueryConfig withOverrides(final GroupByQuery query) ); newConfig.forcePushDownLimit = query.getContextBoolean(CTX_KEY_FORCE_LIMIT_PUSH_DOWN, isForcePushDownLimit()); newConfig.forceHashAggregation = query.getContextBoolean(CTX_KEY_FORCE_HASH_AGGREGATION, isForceHashAggregation()); + newConfig.intermediateCombineDegree = query.getContextValue( + CTX_KEY_INTERMEDIATE_COMBINE_DEGREE, + getIntermediateCombineDegree() + ); return newConfig; } @@ -198,6 +211,7 @@ public String toString() ", maxOnDiskStorage=" + maxOnDiskStorage + ", forcePushDownLimit=" + forcePushDownLimit + ", forceHashAggregation=" + forceHashAggregation + + ", intermediateCombineDegree=" + intermediateCombineDegree + '}'; } } 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 23b3d56829cb..359af382a6a7 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 @@ -109,7 +109,8 @@ public ConcurrentGrouper( final ListeningExecutorService executor, final int priority, final boolean hasQueryTimeout, - final long queryTimeoutAt + final long queryTimeoutAt, + final int intermediateCombineDegree ) { Preconditions.checkArgument(concurrencyHint > 0, "concurrencyHint > 0"); @@ -143,7 +144,8 @@ public ConcurrentGrouper( sortHasNonGroupingFields, concurrencyHint, priority, - queryTimeoutAt + queryTimeoutAt, + intermediateCombineDegree ); this.maxDictionarySizeForCombiner = combineKeySerdeFactory.getMaxDictionarySize(); } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java index 56083e1d4edb..4aad81a06abd 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java @@ -64,7 +64,27 @@ */ public class ParallelCombiner { - private static final int MINIMUM_COMBINE_DEGREE = 2; + // The combining tree created by this class can have two different degrees for intermediate nodes. + // The "leaf combine degree (LCD)" is the number of leaf nodes combined together, while the "intermediate combine + // degree (ICD)" is the number of non-leaf nodes combined together. The below picture shows an example where LCD = 2 + // and ICD = 4. + // + // o <- non-leaf node + // / / \ \ <- ICD = 4 + // o o o o <- non-leaf nodes + // / \ / \ / \ / \ <- LCD = 2 + // o o o o o o o o <- leaf nodes + // + // The reason why we need two different degrees is to optimize the number of non-leaf nodes which are run by + // different threads at the same time. Note that the leaf nodes are sorted iterators of SpillingGroupers which + // generally returns multiple rows of the same grouping key which in turn should be combined, while the non-leaf nodes + // are iterators of StreamingMergeSortedGroupers and always returns a single row per grouping key. Generally, the + // performance will get better as LCD becomes low while ICD is some value larger than LCD because the amount of work + // each thread has to do can be properly tuned. The optimal values for LCD and ICD may vary with query and data. Here, + // we use a simple heuristic to avoid complex optimization. That is, ICD is fixed as a user-configurable value and the + // minimum LCD satisfying the memory restriction is searched. See findLeafCombineDegreeAndNumBuffers() for more + // details. + private static final int MINIMUM_LEAF_COMBINE_DEGREE = 2; private final Supplier> combineBufferSupplier; private final AggregatorFactory[] combiningFactories; @@ -75,6 +95,10 @@ public class ParallelCombiner private final int priority; private final long queryTimeoutAt; + // The default value is 8 which comes from an experiment. A non-leaf node will combine up to intermediateCombineDegree + // rows for the same grouping key. + private final int intermediateCombineDegree; + public ParallelCombiner( Supplier> combineBufferSupplier, AggregatorFactory[] combiningFactories, @@ -83,7 +107,8 @@ public ParallelCombiner( boolean sortHasNonGroupingFields, int concurrencyHint, int priority, - long queryTimeoutAt + long queryTimeoutAt, + int intermediateCombineDegree ) { this.combineBufferSupplier = combineBufferSupplier; @@ -93,6 +118,8 @@ public ParallelCombiner( this.keyObjComparator = combineKeySerdeFactory.objectComparator(sortHasNonGroupingFields); this.concurrencyHint = concurrencyHint; this.priority = priority; + this.intermediateCombineDegree = intermediateCombineDegree; + this.queryTimeoutAt = queryTimeoutAt; } @@ -118,16 +145,16 @@ public CloseableIterator> combine( combiningFactories ); // We want to maximize the parallelism while the size of buffer slice is greater than the minimum buffer size - // required by StreamingMergeSortedGrouper. Here, we find the degree of the cominbing tree and the required number - // of buffers maximizing the degree of parallelism. - final Pair degreeAndNumBuffers = findCombineDegreeAndNumBuffers( + // required by StreamingMergeSortedGrouper. Here, we find the leafCombineDegree of the cominbing tree and the + // required number of buffers maximizing the parallelism. + final Pair degreeAndNumBuffers = findLeafCombineDegreeAndNumBuffers( combineBuffer, minimumRequiredBufferCapacity, concurrencyHint, sortedIterators.size() ); - final int combineDegree = degreeAndNumBuffers.lhs; + final int leafCombineDegree = degreeAndNumBuffers.lhs; final int numBuffers = degreeAndNumBuffers.rhs; final int sliceSize = combineBuffer.capacity() / numBuffers; @@ -137,7 +164,7 @@ public CloseableIterator> combine( sortedIterators, bufferSupplier, combiningFactories, - combineDegree, + leafCombineDegree, mergedDictionary ); @@ -194,35 +221,36 @@ public ByteBuffer get() } /** - * Find a minimum size of the buffer slice and corresponding combining degree and number of slices. Note that each - * node in the combining tree is executed by different threads. This method assumes that using more threads can - * exploit better performance and find such a shape of the combining tree. + * Find a minimum size of the buffer slice and corresponding leafCombineDegree and number of slices. Note that each + * node in the combining tree is executed by different threads. This method assumes that combining the leaf nodes + * requires threads as many as possible, while combining intermediate nodes is not. See the comment on + * {@link #MINIMUM_LEAF_COMBINE_DEGREE} for more details. * * @param combineBuffer entire buffer used for combining tree * @param requiredMinimumBufferCapacity minimum buffer capacity for {@link StreamingMergeSortedGrouper} * @param numAvailableThreads number of available threads * @param numLeafNodes number of leaf nodes of combining tree * - * @return a pair of degree and number of buffers if found. + * @return a pair of leafCombineDegree and number of buffers if found. */ - private static Pair findCombineDegreeAndNumBuffers( + private Pair findLeafCombineDegreeAndNumBuffers( ByteBuffer combineBuffer, int requiredMinimumBufferCapacity, int numAvailableThreads, int numLeafNodes ) { - for (int degree = MINIMUM_COMBINE_DEGREE; degree <= numLeafNodes; degree++) { - final int requiredBufferNum = computeRequiredBufferNum(numLeafNodes, degree); + for (int leafCombineDegree = MINIMUM_LEAF_COMBINE_DEGREE; leafCombineDegree <= numLeafNodes; leafCombineDegree++) { + final int requiredBufferNum = computeRequiredBufferNum(numLeafNodes, leafCombineDegree); if (requiredBufferNum <= numAvailableThreads) { final int expectedSliceSize = combineBuffer.capacity() / requiredBufferNum; if (expectedSliceSize >= requiredMinimumBufferCapacity) { - return Pair.of(degree, requiredBufferNum); + return Pair.of(leafCombineDegree, requiredBufferNum); } } } - throw new ISE("Cannot find a proper combine degree"); + throw new ISE("Cannot find a proper leaf combine degree. Try increasing druid.processing.buffer.sizeBytes."); } /** @@ -231,13 +259,13 @@ private static Pair findCombineDegreeAndNumBuffers( * buffers is the number of nodes of the combining tree. * * @param numChildNodes number of child nodes - * @param combineDegree combine degree + * @param combineDegree combine degree for the current level * * @return minimum number of buffers required for combining tree * * @see #buildCombineTree(List, Supplier, AggregatorFactory[], int, List) */ - static int computeRequiredBufferNum(int numChildNodes, int combineDegree) + private int computeRequiredBufferNum(int numChildNodes, int combineDegree) { // numChildrenForLastNode used to determine that the last node is needed for the current level. // Please see buildCombineTree() for more details. @@ -249,7 +277,7 @@ static int computeRequiredBufferNum(int numChildNodes, int combineDegree) return numCurLevelNodes; } else { return numCurLevelNodes + - computeRequiredBufferNum(numChildOfParentNodes, combineDegree); + computeRequiredBufferNum(numChildOfParentNodes, intermediateCombineDegree); } } @@ -257,10 +285,10 @@ static int computeRequiredBufferNum(int numChildNodes, int combineDegree) * Recursively build a combining tree in a bottom-up manner. Each node of the tree is a task that combines input * iterators asynchronously. * - * @param childIterators all iterators of the child level + * @param childIterators all iterators of the child level * @param bufferSupplier combining buffer supplier * @param combiningFactories array of combining aggregator factories - * @param combineDegree combining degree + * @param combineDegree combining degree for the current level * @param dictionary merged dictionary * * @return a pair of a list of iterators of the current level in the combining tree and a list of futures of all @@ -281,7 +309,9 @@ private Pair>>, List> buildCombine // The below algorithm creates the combining nodes of the current level. It first checks that the number of children // to be combined together is 1. If it is, the intermediate combining node for that child is not needed. Instead, it // can be directly connected to a node of the parent level. Here is an example of generated tree when - // numLeafNodes = 6 and combineDegree = 2. + // numLeafNodes = 6 and leafCombineDegree = intermediateCombineDegree = 2. See the description of + // MINIMUM_LEAF_COMBINE_DEGREE for more details about leafCombineDegree and intermediateCombineDegree. + // // o // / \ // o \ @@ -320,7 +350,13 @@ private Pair>>, List> buildCombine } else { // Build the parent level iterators final Pair>>, List> parentIteratorsAndFutures = - buildCombineTree(childIteratorsOfNextLevel, bufferSupplier, combiningFactories, combineDegree, dictionary); + buildCombineTree( + childIteratorsOfNextLevel, + bufferSupplier, + combiningFactories, + intermediateCombineDegree, + dictionary + ); combineFutures.addAll(parentIteratorsAndFutures.rhs); return Pair.of(parentIteratorsAndFutures.lhs, combineFutures); } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index ca5e0588fb56..38578bd99d00 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -235,7 +235,8 @@ public static Pair, Accumulator> crea grouperSorter, priority, hasQueryTimeout, - queryTimeoutAt + queryTimeoutAt, + querySpecificConfig.getIntermediateCombineDegree() ); } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java index 1c385c36c209..e153949f3a26 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java @@ -317,7 +317,7 @@ public CloseableIterator> iterator() { { // Waits for some data to be ready - increaseReadIndex(0); + increaseReadIndexTo(0); } @Override @@ -351,18 +351,18 @@ public Entry next() } final int increaseTo = nextReadIndex == maxNumSlots - 1 ? 0 : nextReadIndex + 1; - increaseReadIndex(increaseTo); + increaseReadIndexTo(increaseTo); return new Entry<>(key, values); } - private void increaseReadIndex(int increaseTo) + private void increaseReadIndexTo(int target) { final long startAt = System.nanoTime(); final long spinTimeoutAt = startAt + SPIN_FOR_TIMEOUT_THRESHOLD_NS; long now = startAt; - while ((!isReady() || increaseTo == curWriteIndex) && !finished && !Thread.currentThread().isInterrupted()) { + while ((!isReady() || target == curWriteIndex) && !finished && !Thread.currentThread().isInterrupted()) { if (now >= queryTimeoutAtNs) { throw new RuntimeException(new TimeoutException()); } @@ -372,7 +372,7 @@ private void increaseReadIndex(int increaseTo) now = System.nanoTime(); } - nextReadIndex = increaseTo; + nextReadIndex = target; } private boolean isReady() diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java index 4998db82b1ef..05190f1398c7 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java @@ -283,7 +283,8 @@ public void testAggregate() throws InterruptedException, ExecutionException, IOE MoreExecutors.listeningDecorator(SERVICE), 0, false, - 0 + 0, + 4 ); grouper.init(); diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java index 6d1cd4de1328..2389842527be 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java @@ -115,8 +115,9 @@ public void testCombine() throws IOException MoreExecutors.listeningDecorator(SERVICE), false, THREAD_NUM, - 0, - 0 // use default timeout + 0, // default priority + 0, // default timeout + 4 ); final int numRows = 1000; From 919ecbde0b1487000f57a35fde96d5d80027e7e3 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 21 Sep 2017 10:30:45 +0900 Subject: [PATCH 37/44] Add comments to StreamingMergeSortedGrouper --- .../StreamingMergeSortedGrouper.java | 124 ++++++++++++++---- 1 file changed, 100 insertions(+), 24 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java index e153949f3a26..e3af65eaeb01 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java @@ -38,18 +38,23 @@ /** * A streaming grouper which can aggregate sorted inputs. This grouper can aggregate while its iterator is being - * consumed. Also, the aggregation thread and iterating thread can be different. + * consumed. The aggregation thread and the iterating thread can be different. * - * This grouper is backed by a circular array off-heap buffer. Reading iterator is able to read from an array slot only - * if the write for that slot is finished. + * This grouper is backed by an off-heap circular array. The reading thread is able to read data from an array slot + * only when aggregation for the grouping key correspoing to that slot is finished. Since the reading and writing + * threads cannot access the same array slot at the same time, they can read/write data without contention. + * + * This class uses the spinlock for waiting for at least one slot to become available when the array is empty or full. + * If the array is empty, the reading thread waits for the aggregation for an array slot is finished. If the array is + * full, the writing thread waits for the reading thread to read at least one aggregate from the array. */ public class StreamingMergeSortedGrouper implements Grouper { private static final Logger LOG = new Logger(StreamingMergeSortedGrouper.class); - private static final long DEFAULT_TIMEOUT_NS = 5_000_000_000L; // 5 seconds + private static final long DEFAULT_TIMEOUT_NS = 5_000_000_000L; // 5 seconds. default timeout for spinlock - // Threashold time for spin locks in increaseWriteIndex() and increaseReadIndex(). Thread.yield() is called for the - // waiting thread after this threadhold time. + // Threashold time for spinlocks in increaseWriteIndex() and increaseReadIndex(). The waiting thread calls + // Thread.yield() after this threadhold time. private static final long SPIN_FOR_TIMEOUT_THRESHOLD_NS = 1000L; private final Supplier bufferSupplier; @@ -59,37 +64,51 @@ public class StreamingMergeSortedGrouper implements Grouper private final int keySize; private final int recordSize; // size of (key + all aggregates) - // Timeout for waiting for a slot to be available for read/write. This is required to prevent the processing - // thread from being blocked if the iterator of this grouper is not consumed due to some failures. + // Timeout for spinlock. This is required to prevent the writing thread from being blocked if the iterator of this + // grouper is not consumed due to some failures which potentially makes the whole system being paused. private final long queryTimeoutAtNs; - // Below variables are initialized when init() is called + // Below variables are initialized when init() is called. private ByteBuffer buffer; private int maxNumSlots; private boolean initialized; /** - * Indicate that this grouper consumed the last input or not. Always set by the writing thread and read by the + * Indicate that this grouper consumed the last input or not. The writing thread must set this value to true by + * calling {@link #finish()} when it's done. This variable is always set by the writing thread and read by the * reading thread. */ private volatile boolean finished; /** - * Current write position. This is always moved ahead of nextReadIndex. - * Also, it is always incremented by the writing thread and read by both the writing and the reading threads. + * Current write index of the array. This points to the array slot where the aggregation is currently performed. Its + * initial value is -1 which means any data are not written yet. Since it's assumed that the input is sorted by the + * grouping key, this variable is moved to the next slot whenever a new grouping key is found. Once it reaches the + * last slot of the array, it moves to the first slot. + * + * This is always moved ahead of {@link #nextReadIndex}. If the array is full, this variable + * cannot be moved until {@link #nextReadIndex} is moved. See {@link #increaseWriteIndex()} for more details. This + * variable is always incremented by the writing thread and read by both the writing and the reading threads. */ private volatile int curWriteIndex; /** - * Next read position. This can be moved to a position only when write for the position is finished. - * Also, it is always incremented by the reading thread and read by both the writing and the reading threads. + * Next read index of the array. This points to the array slot which the reading thread will read next. Its initial + * value is -1 which means any data are not read yet. This variable can point an array slot only when the aggregation + * for that slot is finished. Once it reaches the last slot of the array, it moves to the first slot. + * + * This always follows {@link #curWriteIndex}. If the array is empty, this variable cannot be moved until the + * aggregation for at least one grouping key is finished which in turn {@link #curWriteIndex} is moved. See + * {@link #iterator()} for more details. This variable is always incremented by the reading thread and read by both + * the writing and the reading threads. */ private volatile int nextReadIndex; /** - * Returns the minimum buffer capacity required to use this grouper. This grouper keeps track read/write indexes - * and they cannot point the same position at the same time. Since the read/write indexes circularly, the required - * minimum buffer capacity is 3 * record size. + * Returns the minimum buffer capacity required for this grouper. This grouper keeps track read/write indexes + * and they cannot point the same array slot at the same time. Since the read/write indexes move circularly, one + * extra slot is needed in addition to the read/write slots. Finally, the required minimum buffer capacity is + * 3 * record size. * * @return required minimum buffer capacity */ @@ -126,6 +145,9 @@ public static int requiredBufferCapacity( offset += aggregatorFactories[i].getMaxIntermediateSize(); } this.recordSize = offset; + + // queryTimeoutAtMs comes from System.currentTimeMillis(), but we should use System.nanoTime() to check timeout in + // this class. See increaseWriteIndex() and increaseReadIndex(). final long timeoutNs = queryTimeoutAtMs == QueryContexts.NO_TIMEOUT ? DEFAULT_TIMEOUT_NS : TimeUnit.MILLISECONDS.toNanos(queryTimeoutAtMs - System.currentTimeMillis()); @@ -179,6 +201,8 @@ public AggregateResult aggregate(KeyType key) final int prevRecordOffset = curWriteIndex * recordSize; if (curWriteIndex == -1 || !keyEquals(keyBuffer, buffer, prevRecordOffset)) { + // Initialize a new slot for the new key. This may be potentially blocked if the array is full until at least + // one slot becomes available. initNewSlot(keyBuffer); } @@ -195,8 +219,18 @@ public AggregateResult aggregate(KeyType key) } } + /** + * Checks two keys contained in the given buffers are same. + * + * @param curKeyBuffer the buffer for the given key from {@link #aggregate(Object)} + * @param buffer the whole array buffer + * @param bufferOffset the key offset of the buffer + * + * @return true if the two buffers are same. + */ private boolean keyEquals(ByteBuffer curKeyBuffer, ByteBuffer buffer, int bufferOffset) { + // Since this method is frequently called per each input row, the compare performance matters. int i = 0; for (; i + Long.BYTES <= keySize; i += Long.BYTES) { if (curKeyBuffer.getLong(i) != buffer.getLong(bufferOffset + i)) { @@ -205,6 +239,7 @@ private boolean keyEquals(ByteBuffer curKeyBuffer, ByteBuffer buffer, int buffer } if (i + Integer.BYTES <= keySize) { + // This can be called at most once because we already compared using getLong() in the above. if (curKeyBuffer.getInt(i) != buffer.getInt(bufferOffset + i)) { return false; } @@ -220,8 +255,13 @@ private boolean keyEquals(ByteBuffer curKeyBuffer, ByteBuffer buffer, int buffer return true; } + /** + * Initialize a new slot for a new grouping key. This may be potentially blocked if the array is full until at least + * one slot becomes available. + */ private void initNewSlot(ByteBuffer newKey) { + // Wait if the array is full and increase curWriteIndex increaseWriteIndex(); final int recordOffset = recordSize * curWriteIndex; @@ -234,8 +274,7 @@ private void initNewSlot(ByteBuffer newKey) } /** - * Wait for {@link #nextReadIndex} to be moved if necessary and move {@link #curWriteIndex}. {@link #nextReadIndex} - * is checked in while loops instead of waiting using a lock to avoid frequent thread park. + * Wait for {@link #nextReadIndex} to be moved if necessary and move {@link #curWriteIndex}. */ private void increaseWriteIndex() { @@ -243,12 +282,23 @@ private void increaseWriteIndex() final long spinTimeoutAt = startAt + SPIN_FOR_TIMEOUT_THRESHOLD_NS; long now = startAt; + // In the below, we check that the array is full and wait for at least one slot to become available. This is done + // in a while loop instead of using a lock to avoid frequent thread park. + // + // nextReadIndex is a volatile variable and the changes on it should be continuously checked until they are seen. + // See the following links. + // * http://docs.oracle.com/javase/specs/jls/se7/html/jls-8.html#jls-8.3.1.4 + // * http://docs.oracle.com/javase/specs/jls/se7/html/jls-17.html#jls-17.4.5 + // * https://stackoverflow.com/questions/11761552/detailed-semantics-of-volatile-regarding-timeliness-of-visibility + if (curWriteIndex == maxNumSlots - 1) { - // Should wait for the reading thread to start reading only if the writing thread should overwrite the first slot. + // We additionally check that nextReadIndex is -1 here because the writing thread should wait for the reading + // thread to start reading only when the writing thread tries to overwrite the first slot for the first time. while ((nextReadIndex == -1 || nextReadIndex == 0) && !Thread.currentThread().isInterrupted()) { if (now >= queryTimeoutAtNs) { throw new RuntimeException(new TimeoutException()); } + // Thread.yield() should not be called from the very beginning if (now >= spinTimeoutAt) { Thread.yield(); } @@ -261,6 +311,7 @@ private void increaseWriteIndex() if (now >= queryTimeoutAtNs) { throw new RuntimeException(new TimeoutException()); } + // Thread.yield() should not be called from the very beginning if (now >= spinTimeoutAt) { Thread.yield(); } @@ -301,9 +352,9 @@ public void finish() } /** - * Return a sorted iterator. This method can be called safely while writing and iterating thread and writing thread - * can be different. The result iterator always returns sorted results. This method should be called only one time - * per grouper. + * Return a sorted iterator. This method can be called safely while writing, and the iterating thread and the writing + * thread can be different. The result iterator always returns sorted results. This method should be called only one + * time per grouper. * * @return a sorted iterator */ @@ -316,16 +367,24 @@ public CloseableIterator> iterator() return new CloseableIterator>() { { - // Waits for some data to be ready + // Wait for some data to be ready and initialize nextReadIndex. increaseReadIndexTo(0); } @Override public boolean hasNext() { + // Once finished becomes true, curWriteIndex isn't changed anymore and thus remainig() can be computed safely + // because nextReadIndex is changed only by the reading thread. return !finished || remaining() > 0; } + /** + * Calculate the number of remaining items in the array. Must be called only when + * {@link StreamingMergeSortedGrouper#finished} is true. + * + * @return the number of remaining items + */ private int remaining() { if (curWriteIndex >= nextReadIndex) { @@ -351,21 +410,38 @@ public Entry next() } final int increaseTo = nextReadIndex == maxNumSlots - 1 ? 0 : nextReadIndex + 1; + // Wait if the array is empty until at least one slot becomes available for read, and then increase + // nextReadIndex. increaseReadIndexTo(increaseTo); return new Entry<>(key, values); } + /** + * Wait for {@link StreamingMergeSortedGrouper#curWriteIndex} to be moved if necessary and move + * {@link StreamingMergeSortedGrouper#nextReadIndex}. + * + * @param target the target index {@link StreamingMergeSortedGrouper#nextReadIndex} will move to + */ private void increaseReadIndexTo(int target) { final long startAt = System.nanoTime(); final long spinTimeoutAt = startAt + SPIN_FOR_TIMEOUT_THRESHOLD_NS; long now = startAt; + // In the below, we check that the array is empty and wait for at least one slot to become available. This is + // done in a while loop instead of using a lock to avoid frequent thread park. + // + // curWriteIndex is a volatile variable and the changes on it should be continuously checked until they are seen. + // See the following links. + // * http://docs.oracle.com/javase/specs/jls/se7/html/jls-8.html#jls-8.3.1.4 + // * http://docs.oracle.com/javase/specs/jls/se7/html/jls-17.html#jls-17.4.5 + // * https://stackoverflow.com/questions/11761552/detailed-semantics-of-volatile-regarding-timeliness-of-visibility while ((!isReady() || target == curWriteIndex) && !finished && !Thread.currentThread().isInterrupted()) { if (now >= queryTimeoutAtNs) { throw new RuntimeException(new TimeoutException()); } + // Thread.yield() should not be called from the very beginning if (now >= spinTimeoutAt) { Thread.yield(); } From aa1249adb3436ab995217c9e989871c57c29bf03 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 21 Sep 2017 15:32:17 +0900 Subject: [PATCH 38/44] Add more comments and fix overflow --- .../StreamingMergeSortedGrouper.java | 133 ++++++++++-------- 1 file changed, 74 insertions(+), 59 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java index e3af65eaeb01..c4d4c71c5a08 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java @@ -35,6 +35,7 @@ import java.util.NoSuchElementException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.function.Predicate; /** * A streaming grouper which can aggregate sorted inputs. This grouper can aggregate while its iterator is being @@ -51,10 +52,10 @@ public class StreamingMergeSortedGrouper implements Grouper { private static final Logger LOG = new Logger(StreamingMergeSortedGrouper.class); - private static final long DEFAULT_TIMEOUT_NS = 5_000_000_000L; // 5 seconds. default timeout for spinlock + private static final long DEFAULT_TIMEOUT_NS = TimeUnit.SECONDS.toNanos(5); // default timeout for spinlock // Threashold time for spinlocks in increaseWriteIndex() and increaseReadIndex(). The waiting thread calls - // Thread.yield() after this threadhold time. + // Thread.yield() after this threadhold time elapses. private static final long SPIN_FOR_TIMEOUT_THRESHOLD_NS = 1000L; private final Supplier bufferSupplier; @@ -64,8 +65,10 @@ public class StreamingMergeSortedGrouper implements Grouper private final int keySize; private final int recordSize; // size of (key + all aggregates) - // Timeout for spinlock. This is required to prevent the writing thread from being blocked if the iterator of this - // grouper is not consumed due to some failures which potentially makes the whole system being paused. + // Timeout for the current query. + // The query must fail with a timeout exception if System.nanoTime() >= queryTimeoutAtNs. This is used in the + // spinlocks to prevent the writing thread from being blocked if the iterator of this grouper is not consumed due to + // some failures which potentially makes the whole system being paused. private final long queryTimeoutAtNs; // Below variables are initialized when init() is called. @@ -278,15 +281,10 @@ private void initNewSlot(ByteBuffer newKey) */ private void increaseWriteIndex() { - final long startAt = System.nanoTime(); - final long spinTimeoutAt = startAt + SPIN_FOR_TIMEOUT_THRESHOLD_NS; - long now = startAt; - - // In the below, we check that the array is full and wait for at least one slot to become available. This is done - // in a while loop instead of using a lock to avoid frequent thread park. + // In the below, we check that the array is full and wait for at least one slot to become available. // - // nextReadIndex is a volatile variable and the changes on it should be continuously checked until they are seen. - // See the following links. + // nextReadIndex is a volatile variable and the changes on it are continuously checked until they are seen in + // waitFor(). See the following links. // * http://docs.oracle.com/javase/specs/jls/se7/html/jls-8.html#jls-8.3.1.4 // * http://docs.oracle.com/javase/specs/jls/se7/html/jls-17.html#jls-17.4.5 // * https://stackoverflow.com/questions/11761552/detailed-semantics-of-volatile-regarding-timeliness-of-visibility @@ -294,29 +292,22 @@ private void increaseWriteIndex() if (curWriteIndex == maxNumSlots - 1) { // We additionally check that nextReadIndex is -1 here because the writing thread should wait for the reading // thread to start reading only when the writing thread tries to overwrite the first slot for the first time. - while ((nextReadIndex == -1 || nextReadIndex == 0) && !Thread.currentThread().isInterrupted()) { - if (now >= queryTimeoutAtNs) { - throw new RuntimeException(new TimeoutException()); - } - // Thread.yield() should not be called from the very beginning - if (now >= spinTimeoutAt) { - Thread.yield(); - } - now = System.nanoTime(); - } + waitFor( + notUsed -> (nextReadIndex == -1 || nextReadIndex == 0) && !Thread.currentThread().isInterrupted(), + queryTimeoutAtNs + ); + + // Changes on nextReadIndex happens-before changing curWriteIndex. curWriteIndex = 0; } else { final int nextWriteIndex = curWriteIndex + 1; - while ((nextWriteIndex == nextReadIndex) && !Thread.currentThread().isInterrupted()) { - if (now >= queryTimeoutAtNs) { - throw new RuntimeException(new TimeoutException()); - } - // Thread.yield() should not be called from the very beginning - if (now >= spinTimeoutAt) { - Thread.yield(); - } - now = System.nanoTime(); - } + + waitFor( + notUsed -> (nextWriteIndex == nextReadIndex) && !Thread.currentThread().isInterrupted(), + queryTimeoutAtNs + ); + + // Changes on nextReadIndex happens-before changing curWriteIndex. curWriteIndex = nextWriteIndex; } } @@ -348,6 +339,8 @@ public void close() public void finish() { increaseWriteIndex(); + // Once finished is set, curWirteIndex must not be changed. This guarantees that the remaining number of items in + // the array is always decreased as the reading thread proceeds. See hasNext() and remaining() below. finished = true; } @@ -374,8 +367,13 @@ public CloseableIterator> iterator() @Override public boolean hasNext() { - // Once finished becomes true, curWriteIndex isn't changed anymore and thus remainig() can be computed safely - // because nextReadIndex is changed only by the reading thread. + // If setting finished happens-before the below check, curWriteIndex isn't changed anymore and thus remainig() + // can be computed safely because nextReadIndex is changed only by the reading thread. + // Otherwise, hasNext() always returns true. + // + // The below line can be executed between increasing curWriteIndex and setting finished in + // StreamingMergeSortedGrouper.finish(), but it is also a valid case because there should be at least one slot + // which is not read yet before finished is set. return !finished || remaining() > 0; } @@ -401,6 +399,10 @@ public Entry next() throw new NoSuchElementException(); } + // Here, nextReadIndex should be valid which means: + // - a valid array index which should be >= 0 and < maxNumSlots + // - an index of the array slot where the aggregation for the corresponding grouping key is done + // - an index of the array slot which is not read yet final int recordOffset = recordSize * nextReadIndex; final KeyType key = keySerde.fromByteBuffer(buffer, recordOffset); @@ -409,10 +411,10 @@ public Entry next() values[i] = aggregators[i].get(buffer, recordOffset + aggregatorOffsets[i]); } - final int increaseTo = nextReadIndex == maxNumSlots - 1 ? 0 : nextReadIndex + 1; + final int targetIndex = nextReadIndex == maxNumSlots - 1 ? 0 : nextReadIndex + 1; // Wait if the array is empty until at least one slot becomes available for read, and then increase // nextReadIndex. - increaseReadIndexTo(increaseTo); + increaseReadIndexTo(targetIndex); return new Entry<>(key, values); } @@ -425,37 +427,23 @@ public Entry next() */ private void increaseReadIndexTo(int target) { - final long startAt = System.nanoTime(); - final long spinTimeoutAt = startAt + SPIN_FOR_TIMEOUT_THRESHOLD_NS; - long now = startAt; - - // In the below, we check that the array is empty and wait for at least one slot to become available. This is - // done in a while loop instead of using a lock to avoid frequent thread park. + // Check that the array is empty and wait for at least one slot to become available. // - // curWriteIndex is a volatile variable and the changes on it should be continuously checked until they are seen. - // See the following links. + // curWriteIndex is a volatile variable and the changes on it are continuously checked until they are seen in + // waitFor(). See the following links. // * http://docs.oracle.com/javase/specs/jls/se7/html/jls-8.html#jls-8.3.1.4 // * http://docs.oracle.com/javase/specs/jls/se7/html/jls-17.html#jls-17.4.5 // * https://stackoverflow.com/questions/11761552/detailed-semantics-of-volatile-regarding-timeliness-of-visibility - while ((!isReady() || target == curWriteIndex) && !finished && !Thread.currentThread().isInterrupted()) { - if (now >= queryTimeoutAtNs) { - throw new RuntimeException(new TimeoutException()); - } - // Thread.yield() should not be called from the very beginning - if (now >= spinTimeoutAt) { - Thread.yield(); - } - now = System.nanoTime(); - } + waitFor( + notUsed -> (curWriteIndex == -1 || target == curWriteIndex) && + !finished && !Thread.currentThread().isInterrupted(), + queryTimeoutAtNs + ); + // Changes on curWriteIndex happens-before changing nextReadIndex. nextReadIndex = target; } - private boolean isReady() - { - return curWriteIndex != -1; - } - @Override public void close() throws IOException { @@ -464,6 +452,33 @@ public void close() throws IOException }; } + /** + * Wait while the given predicate is true. Throws an exception the current nano time is greater than the given + * timeout. + */ + private static void waitFor(Predicate predicate, long queryTimeoutAtNs) + { + final long startAt = System.nanoTime(); + final long spinTimeoutAt = startAt + SPIN_FOR_TIMEOUT_THRESHOLD_NS; + long timeoutNs = queryTimeoutAtNs - startAt; + long spinTimeoutNs = SPIN_FOR_TIMEOUT_THRESHOLD_NS; + + // In the below, we check that the predicate is true and wait for it becomes false. + // This is done in a while loop instead of using a lock to avoid frequent thread park. + while (predicate.test(null)) { + if (timeoutNs <= 0L) { + throw new RuntimeException(new TimeoutException()); + } + // Thread.yield() should not be called from the very beginning + if (spinTimeoutNs <= 0L) { + Thread.yield(); + } + long now = System.nanoTime(); + timeoutNs = queryTimeoutAtNs - now; + spinTimeoutNs = spinTimeoutAt - now; + } + } + /** * Return a sorted iterator. This method can be called safely while writing and iterating thread and writing thread * can be different. The result iterator always returns sorted results. This method should be called only one time From 7a67475708e516ff49cbf5138c7f340655d5ec4b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 7 Oct 2017 21:42:59 +0900 Subject: [PATCH 39/44] Address comments --- .../epinephelinae/SpillingGrouper.java | 2 +- .../StreamingMergeSortedGrouper.java | 106 +++++++++++------- .../epinephelinae/ConcurrentGrouperTest.java | 60 ++++++---- 3 files changed, 104 insertions(+), 64 deletions(-) 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 2d320258d000..4ef2f26681d9 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 @@ -29,8 +29,8 @@ import com.google.common.collect.Lists; import io.druid.java.util.common.CloseableIterators; import io.druid.java.util.common.io.Closer; -import io.druid.java.util.common.parsers.CloseableIterator; import io.druid.java.util.common.logger.Logger; +import io.druid.java.util.common.parsers.CloseableIterator; import io.druid.query.BaseQuery; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.groupby.orderby.DefaultLimitSpec; diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java index c4d4c71c5a08..6413f4529d05 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java @@ -35,7 +35,6 @@ import java.util.NoSuchElementException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.function.Predicate; /** * A streaming grouper which can aggregate sorted inputs. This grouper can aggregate while its iterator is being @@ -70,6 +69,7 @@ public class StreamingMergeSortedGrouper implements Grouper // spinlocks to prevent the writing thread from being blocked if the iterator of this grouper is not consumed due to // some failures which potentially makes the whole system being paused. private final long queryTimeoutAtNs; + private final boolean hasQueryTimeout; // Below variables are initialized when init() is called. private ByteBuffer buffer; @@ -151,9 +151,10 @@ public static int requiredBufferCapacity( // queryTimeoutAtMs comes from System.currentTimeMillis(), but we should use System.nanoTime() to check timeout in // this class. See increaseWriteIndex() and increaseReadIndex(). - final long timeoutNs = queryTimeoutAtMs == QueryContexts.NO_TIMEOUT ? - DEFAULT_TIMEOUT_NS : - TimeUnit.MILLISECONDS.toNanos(queryTimeoutAtMs - System.currentTimeMillis()); + this.hasQueryTimeout = queryTimeoutAtMs != QueryContexts.NO_TIMEOUT; + final long timeoutNs = hasQueryTimeout ? + TimeUnit.MILLISECONDS.toNanos(queryTimeoutAtMs - System.currentTimeMillis()) : + QueryContexts.NO_TIMEOUT; this.queryTimeoutAtNs = System.nanoTime() + timeoutNs; } @@ -281,10 +282,16 @@ private void initNewSlot(ByteBuffer newKey) */ private void increaseWriteIndex() { + final long startAtNs = System.nanoTime(); + final long queryTimeoutAtNs = getQueryTimeoutAtNs(startAtNs); + final long spinTimeoutAtNs = startAtNs + SPIN_FOR_TIMEOUT_THRESHOLD_NS; + long timeoutNs = queryTimeoutAtNs - startAtNs; + long spinTimeoutNs = SPIN_FOR_TIMEOUT_THRESHOLD_NS; + // In the below, we check that the array is full and wait for at least one slot to become available. // // nextReadIndex is a volatile variable and the changes on it are continuously checked until they are seen in - // waitFor(). See the following links. + // the while loop. See the following links. // * http://docs.oracle.com/javase/specs/jls/se7/html/jls-8.html#jls-8.3.1.4 // * http://docs.oracle.com/javase/specs/jls/se7/html/jls-17.html#jls-17.4.5 // * https://stackoverflow.com/questions/11761552/detailed-semantics-of-volatile-regarding-timeliness-of-visibility @@ -292,20 +299,39 @@ private void increaseWriteIndex() if (curWriteIndex == maxNumSlots - 1) { // We additionally check that nextReadIndex is -1 here because the writing thread should wait for the reading // thread to start reading only when the writing thread tries to overwrite the first slot for the first time. - waitFor( - notUsed -> (nextReadIndex == -1 || nextReadIndex == 0) && !Thread.currentThread().isInterrupted(), - queryTimeoutAtNs - ); + + // The below condition is checked in a while loop instead of using a lock to avoid frequent thread park. + while ((nextReadIndex == -1 || nextReadIndex == 0) && !Thread.currentThread().isInterrupted()) { + if (timeoutNs <= 0L) { + throw new RuntimeException(new TimeoutException()); + } + // Thread.yield() should not be called from the very beginning + if (spinTimeoutNs <= 0L) { + Thread.yield(); + } + long now = System.nanoTime(); + timeoutNs = queryTimeoutAtNs - now; + spinTimeoutNs = spinTimeoutAtNs - now; + } // Changes on nextReadIndex happens-before changing curWriteIndex. curWriteIndex = 0; } else { final int nextWriteIndex = curWriteIndex + 1; - waitFor( - notUsed -> (nextWriteIndex == nextReadIndex) && !Thread.currentThread().isInterrupted(), - queryTimeoutAtNs - ); + // The below condition is checked in a while loop instead of using a lock to avoid frequent thread park. + while ((nextWriteIndex == nextReadIndex) && !Thread.currentThread().isInterrupted()) { + if (timeoutNs <= 0L) { + throw new RuntimeException(new TimeoutException()); + } + // Thread.yield() should not be called from the very beginning + if (spinTimeoutNs <= 0L) { + Thread.yield(); + } + long now = System.nanoTime(); + timeoutNs = queryTimeoutAtNs - now; + spinTimeoutNs = spinTimeoutAtNs - now; + } // Changes on nextReadIndex happens-before changing curWriteIndex. curWriteIndex = nextWriteIndex; @@ -430,15 +456,31 @@ private void increaseReadIndexTo(int target) // Check that the array is empty and wait for at least one slot to become available. // // curWriteIndex is a volatile variable and the changes on it are continuously checked until they are seen in - // waitFor(). See the following links. + // the while loop. See the following links. // * http://docs.oracle.com/javase/specs/jls/se7/html/jls-8.html#jls-8.3.1.4 // * http://docs.oracle.com/javase/specs/jls/se7/html/jls-17.html#jls-17.4.5 // * https://stackoverflow.com/questions/11761552/detailed-semantics-of-volatile-regarding-timeliness-of-visibility - waitFor( - notUsed -> (curWriteIndex == -1 || target == curWriteIndex) && - !finished && !Thread.currentThread().isInterrupted(), - queryTimeoutAtNs - ); + + final long startAtNs = System.nanoTime(); + final long queryTimeoutAtNs = getQueryTimeoutAtNs(startAtNs); + final long spinTimeoutAtNs = startAtNs + SPIN_FOR_TIMEOUT_THRESHOLD_NS; + long timeoutNs = queryTimeoutAtNs - startAtNs; + long spinTimeoutNs = SPIN_FOR_TIMEOUT_THRESHOLD_NS; + + // The below condition is checked in a while loop instead of using a lock to avoid frequent thread park. + while ((curWriteIndex == -1 || target == curWriteIndex) && + !finished && !Thread.currentThread().isInterrupted()) { + if (timeoutNs <= 0L) { + throw new RuntimeException(new TimeoutException()); + } + // Thread.yield() should not be called from the very beginning + if (spinTimeoutNs <= 0L) { + Thread.yield(); + } + long now = System.nanoTime(); + timeoutNs = queryTimeoutAtNs - now; + spinTimeoutNs = spinTimeoutAtNs - now; + } // Changes on curWriteIndex happens-before changing nextReadIndex. nextReadIndex = target; @@ -452,31 +494,9 @@ public void close() throws IOException }; } - /** - * Wait while the given predicate is true. Throws an exception the current nano time is greater than the given - * timeout. - */ - private static void waitFor(Predicate predicate, long queryTimeoutAtNs) + private long getQueryTimeoutAtNs(long startAtNs) { - final long startAt = System.nanoTime(); - final long spinTimeoutAt = startAt + SPIN_FOR_TIMEOUT_THRESHOLD_NS; - long timeoutNs = queryTimeoutAtNs - startAt; - long spinTimeoutNs = SPIN_FOR_TIMEOUT_THRESHOLD_NS; - - // In the below, we check that the predicate is true and wait for it becomes false. - // This is done in a while loop instead of using a lock to avoid frequent thread park. - while (predicate.test(null)) { - if (timeoutNs <= 0L) { - throw new RuntimeException(new TimeoutException()); - } - // Thread.yield() should not be called from the very beginning - if (spinTimeoutNs <= 0L) { - Thread.yield(); - } - long now = System.nanoTime(); - timeoutNs = queryTimeoutAtNs - now; - spinTimeoutNs = spinTimeoutAt - now; - } + return hasQueryTimeout ? queryTimeoutAtNs : startAtNs + DEFAULT_TIMEOUT_NS; } /** diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java index 05190f1398c7..08636d90d772 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java @@ -47,10 +47,14 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collection; import java.util.Comparator; import java.util.List; import java.util.concurrent.ExecutionException; @@ -59,11 +63,20 @@ import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicBoolean; +@RunWith(Parameterized.class) public class ConcurrentGrouperTest { private static final ExecutorService SERVICE = Executors.newFixedThreadPool(8); private static final TestResourceHolder TEST_RESOURCE_HOLDER = new TestResourceHolder(256); - private static final int BYTE_BUFFER_SIZE = 256; + + @Parameters(name = "bufferSize={0}") + public static Collection constructorFeeder() + { + return ImmutableList.of( + new Object[]{1024 * 32}, + new Object[]{1024 * 1024} + ); + } @AfterClass public static void teardown() @@ -74,22 +87,6 @@ public static void teardown() @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - private static final Supplier BUFFER_SUPPLIER = new Supplier() - { - private final AtomicBoolean called = new AtomicBoolean(false); - private ByteBuffer buffer; - - @Override - public ByteBuffer get() - { - if (called.compareAndSet(false, true)) { - buffer = ByteBuffer.allocate(BYTE_BUFFER_SIZE); - } - - return buffer; - } - }; - private static final Supplier> COMBINE_BUFFER_SUPPLIER = new Supplier>() { private final AtomicBoolean called = new AtomicBoolean(false); @@ -107,6 +104,7 @@ public ResourceHolder get() static class TestResourceHolder implements ResourceHolder { + private boolean taken; private boolean closed; private ByteBuffer buffer; @@ -118,6 +116,7 @@ static class TestResourceHolder implements ResourceHolder @Override public ByteBuffer get() { + taken = true; return buffer; } @@ -262,17 +261,38 @@ public DoubleColumnSelector makeDoubleColumnSelector(String columnName) } }; + private Supplier bufferSupplier; + + public ConcurrentGrouperTest(int bufferSize) + { + bufferSupplier = new Supplier() + { + private final AtomicBoolean called = new AtomicBoolean(false); + private ByteBuffer buffer; + + @Override + public ByteBuffer get() + { + if (called.compareAndSet(false, true)) { + buffer = ByteBuffer.allocate(bufferSize); + } + + return buffer; + } + }; + } + @Test() public void testAggregate() throws InterruptedException, ExecutionException, IOException { final ConcurrentGrouper grouper = new ConcurrentGrouper<>( - BUFFER_SUPPLIER, + bufferSupplier, COMBINE_BUFFER_SUPPLIER, KEY_SERDE_FACTORY, KEY_SERDE_FACTORY, null_factory, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, - 24, + 1024, 0.7f, 1, new LimitedTemporaryStorage(temporaryFolder.newFolder(), 1024 * 1024), @@ -313,7 +333,7 @@ public void run() final List> actual = Lists.newArrayList(iterator); iterator.close(); - Assert.assertTrue(TEST_RESOURCE_HOLDER.closed); + Assert.assertTrue(!TEST_RESOURCE_HOLDER.taken || TEST_RESOURCE_HOLDER.closed); final List> expected = new ArrayList<>(); for (long i = 0; i < numRows; i++) { From 766cbceece06e8d87d9e2d603ed23acaab3862de Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 7 Oct 2017 21:53:13 +0900 Subject: [PATCH 40/44] ConcurrentGrouperTest cleanup --- .../epinephelinae/ConcurrentGrouperTest.java | 203 +++++++++--------- .../epinephelinae/ParallelCombinerTest.java | 5 +- 2 files changed, 107 insertions(+), 101 deletions(-) diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java index 08636d90d772..c118b0ab2264 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java @@ -68,6 +68,14 @@ public class ConcurrentGrouperTest { private static final ExecutorService SERVICE = Executors.newFixedThreadPool(8); private static final TestResourceHolder TEST_RESOURCE_HOLDER = new TestResourceHolder(256); + private static final KeySerdeFactory KEY_SERDE_FACTORY = new TestKeySerdeFactory(); + private static final Supplier> COMBINE_BUFFER_SUPPLIER = new TestBufferSupplier(); + private static final ColumnSelectorFactory NULL_FACTORY = new TestColumnSelectorFactory(); + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private Supplier bufferSupplier; @Parameters(name = "bufferSize={0}") public static Collection constructorFeeder() @@ -84,23 +92,87 @@ public static void teardown() SERVICE.shutdown(); } - @Rule - public TemporaryFolder temporaryFolder = new TemporaryFolder(); - - private static final Supplier> COMBINE_BUFFER_SUPPLIER = new Supplier>() + public ConcurrentGrouperTest(int bufferSize) { - private final AtomicBoolean called = new AtomicBoolean(false); - - @Override - public ResourceHolder get() + bufferSupplier = new Supplier() { - if (called.compareAndSet(false, true)) { - return TEST_RESOURCE_HOLDER; - } else { - throw new IAE("should be called once"); + private final AtomicBoolean called = new AtomicBoolean(false); + private ByteBuffer buffer; + + @Override + public ByteBuffer get() + { + if (called.compareAndSet(false, true)) { + buffer = ByteBuffer.allocate(bufferSize); + } + + return buffer; } + }; + } + + @Test() + public void testAggregate() throws InterruptedException, ExecutionException, IOException + { + final ConcurrentGrouper grouper = new ConcurrentGrouper<>( + bufferSupplier, + COMBINE_BUFFER_SUPPLIER, + KEY_SERDE_FACTORY, + KEY_SERDE_FACTORY, + NULL_FACTORY, + new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, + 1024, + 0.7f, + 1, + new LimitedTemporaryStorage(temporaryFolder.newFolder(), 1024 * 1024), + new DefaultObjectMapper(), + 8, + null, + false, + MoreExecutors.listeningDecorator(SERVICE), + 0, + false, + 0, + 4 + ); + grouper.init(); + + final int numRows = 1000; + + Future[] futures = new Future[8]; + + for (int i = 0; i < 8; i++) { + futures[i] = SERVICE.submit(new Runnable() + { + @Override + public void run() + { + for (long i = 0; i < numRows; i++) { + grouper.aggregate(i); + } + } + }); } - }; + + for (Future eachFuture : futures) { + eachFuture.get(); + } + + final CloseableIterator> iterator = grouper.iterator(true); + final List> actual = Lists.newArrayList(iterator); + iterator.close(); + + Assert.assertTrue(!TEST_RESOURCE_HOLDER.taken || TEST_RESOURCE_HOLDER.closed); + + final List> expected = new ArrayList<>(); + for (long i = 0; i < numRows; i++) { + expected.add(new Entry<>(i, new Object[]{8L})); + } + + Assert.assertEquals(expected, actual); + + grouper.close(); + } static class TestResourceHolder implements ResourceHolder { @@ -127,7 +199,7 @@ public void close() } } - static final KeySerdeFactory KEY_SERDE_FACTORY = new KeySerdeFactory() + static class TestKeySerdeFactory implements KeySerdeFactory { @Override public long getMaxDictionarySize() @@ -220,9 +292,24 @@ public int compare(Grouper.Entry o1, Grouper.Entry o2) } }; } - }; + } - private static final ColumnSelectorFactory null_factory = new ColumnSelectorFactory() + private static class TestBufferSupplier implements Supplier> + { + private final AtomicBoolean called = new AtomicBoolean(false); + + @Override + public ResourceHolder get() + { + if (called.compareAndSet(false, true)) { + return TEST_RESOURCE_HOLDER; + } else { + throw new IAE("should be called once"); + } + } + } + + private static class TestColumnSelectorFactory implements ColumnSelectorFactory { @Override public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) @@ -259,89 +346,5 @@ public DoubleColumnSelector makeDoubleColumnSelector(String columnName) { return null; } - }; - - private Supplier bufferSupplier; - - public ConcurrentGrouperTest(int bufferSize) - { - bufferSupplier = new Supplier() - { - private final AtomicBoolean called = new AtomicBoolean(false); - private ByteBuffer buffer; - - @Override - public ByteBuffer get() - { - if (called.compareAndSet(false, true)) { - buffer = ByteBuffer.allocate(bufferSize); - } - - return buffer; - } - }; - } - - @Test() - public void testAggregate() throws InterruptedException, ExecutionException, IOException - { - final ConcurrentGrouper grouper = new ConcurrentGrouper<>( - bufferSupplier, - COMBINE_BUFFER_SUPPLIER, - KEY_SERDE_FACTORY, - KEY_SERDE_FACTORY, - null_factory, - new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, - 1024, - 0.7f, - 1, - new LimitedTemporaryStorage(temporaryFolder.newFolder(), 1024 * 1024), - new DefaultObjectMapper(), - 8, - null, - false, - MoreExecutors.listeningDecorator(SERVICE), - 0, - false, - 0, - 4 - ); - grouper.init(); - - final int numRows = 1000; - - Future[] futures = new Future[8]; - - for (int i = 0; i < 8; i++) { - futures[i] = SERVICE.submit(new Runnable() - { - @Override - public void run() - { - for (long i = 0; i < numRows; i++) { - grouper.aggregate(i); - } - } - }); - } - - for (Future eachFuture : futures) { - eachFuture.get(); - } - - final CloseableIterator> iterator = grouper.iterator(true); - final List> actual = Lists.newArrayList(iterator); - iterator.close(); - - Assert.assertTrue(!TEST_RESOURCE_HOLDER.taken || TEST_RESOURCE_HOLDER.closed); - - final List> expected = new ArrayList<>(); - for (long i = 0; i < numRows; i++) { - expected.add(new Entry<>(i, new Object[]{8L})); - } - - Assert.assertEquals(expected, actual); - - grouper.close(); } } diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java index 2389842527be..eeba1d72efe9 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ParallelCombinerTest.java @@ -27,8 +27,10 @@ import io.druid.java.util.common.parsers.CloseableIterator; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.groupby.epinephelinae.ConcurrentGrouperTest.TestKeySerdeFactory; import io.druid.query.groupby.epinephelinae.ConcurrentGrouperTest.TestResourceHolder; import io.druid.query.groupby.epinephelinae.Grouper.Entry; +import io.druid.query.groupby.epinephelinae.Grouper.KeySerdeFactory; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Test; @@ -46,6 +48,7 @@ public class ParallelCombinerTest private static final int THREAD_NUM = 8; private static final ExecutorService SERVICE = Execs.multiThreaded(THREAD_NUM, "parallel-combiner-test-%d"); private static final TestResourceHolder TEST_RESOURCE_HOLDER = new TestResourceHolder(512); + private static final KeySerdeFactory KEY_SERDE_FACTORY = new TestKeySerdeFactory(); private static final Supplier> COMBINE_BUFFER_SUPPLIER = new Supplier>() @@ -111,7 +114,7 @@ public void testCombine() throws IOException final ParallelCombiner combiner = new ParallelCombiner<>( COMBINE_BUFFER_SUPPLIER, new AggregatorFactory[]{new CountAggregatorFactory("cnt").getCombiningFactory()}, - ConcurrentGrouperTest.KEY_SERDE_FACTORY, + KEY_SERDE_FACTORY, MoreExecutors.listeningDecorator(SERVICE), false, THREAD_NUM, From 39cf1f2ed02f803fd641dd9f7fc181261a65bf80 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 12 Oct 2017 21:41:18 +0900 Subject: [PATCH 41/44] add thread number configuration for parallel combining --- docs/content/querying/groupbyquery.md | 4 +- .../query/groupby/GroupByQueryConfig.java | 14 ++++ .../epinephelinae/ConcurrentGrouper.java | 80 +++++++++++++++---- .../epinephelinae/RowBasedGrouperHelper.java | 7 +- .../query/groupby/GroupByQueryRunnerTest.java | 23 +++++- .../epinephelinae/ConcurrentGrouperTest.java | 3 +- 6 files changed, 108 insertions(+), 23 deletions(-) diff --git a/docs/content/querying/groupbyquery.md b/docs/content/querying/groupbyquery.md index 7598735f7b2b..62a21facf04f 100644 --- a/docs/content/querying/groupbyquery.md +++ b/docs/content/querying/groupbyquery.md @@ -220,7 +220,8 @@ When using the "v2" strategy, the following runtime properties apply: |`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)| |`druid.query.groupBy.singleThreaded`|Merge results using a single thread.|false| |`druid.query.groupBy.forceHashAggregation`|Force to use hash-based aggregation.|false| -|`druid.query.groupBy.intermediateCombineDegree`|The number of intermediate nodes combined together in the combining tree. Higher degrees will need a less numer of threads which might be helpful to improve the query performance by reducing the overhead of too many threads if the server has sufficiently powerful cpu cores.|8| +|`druid.query.groupBy.intermediateCombineDegree`|Number of intermediate nodes combined together in the combining tree. Higher degrees will need a less numer of threads which might be helpful to improve the query performance by reducing the overhead of too many threads if the server has sufficiently powerful cpu cores.|8| +|`druid.query.groupBy.numParallelCombineThreads`|Hint for the number of parallel combining threads. This should be larger than 1 to turn on the parallel combining feature. The actual number of threads used for parallel combining is min(`druid.query.groupBy.numParallelCombineThreads`, `druid.processing.numThreads`).|1| 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 @@ -252,6 +253,7 @@ When using the "v2" strategy, the following query context parameters apply: |`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`|Overrides the value of `druid.query.groupBy.forceHashAggregation` for this query.| |`intermediateCombineDegree`|Overrides the value of `druid.query.groupBy.intermediateCombineDegree` for this query.| +|`numParallelCombineThreads`|Overrides the value of `druid.query.groupBy.numParallelCombineThreads` for this query.| When using the "v1" strategy, the following query context parameters apply: 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 d7bf6fea7174..9c744dc24aa1 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java @@ -39,6 +39,7 @@ public class GroupByQueryConfig private static final String CTX_KEY_MAX_MERGING_DICTIONARY_SIZE = "maxMergingDictionarySize"; private static final String CTX_KEY_FORCE_HASH_AGGREGATION = "forceHashAggregation"; private static final String CTX_KEY_INTERMEDIATE_COMBINE_DEGREE = "intermediateCombineDegree"; + private static final String CTX_KEY_NUM_PARALLEL_COMBINE_THREADS = "numParallelCombineThreads"; @JsonProperty private String defaultStrategy = GroupByStrategySelector.STRATEGY_V2; @@ -79,6 +80,9 @@ public class GroupByQueryConfig @JsonProperty private int intermediateCombineDegree = 8; + @JsonProperty + private int numParallelCombineThreads = 1; + public String getDefaultStrategy() { return defaultStrategy; @@ -154,6 +158,11 @@ public int getIntermediateCombineDegree() return intermediateCombineDegree; } + public int getNumParallelCombineThreads() + { + return numParallelCombineThreads; + } + public GroupByQueryConfig withOverrides(final GroupByQuery query) { final GroupByQueryConfig newConfig = new GroupByQueryConfig(); @@ -193,6 +202,10 @@ public GroupByQueryConfig withOverrides(final GroupByQuery query) CTX_KEY_INTERMEDIATE_COMBINE_DEGREE, getIntermediateCombineDegree() ); + newConfig.numParallelCombineThreads = query.getContextValue( + CTX_KEY_NUM_PARALLEL_COMBINE_THREADS, + getNumParallelCombineThreads() + ); return newConfig; } @@ -212,6 +225,7 @@ public String toString() ", forcePushDownLimit=" + forcePushDownLimit + ", forceHashAggregation=" + forceHashAggregation + ", intermediateCombineDegree=" + intermediateCombineDegree + + ", numParallelCombineThreads=" + numParallelCombineThreads + '}'; } } 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 359af382a6a7..fe3754aa6916 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 @@ -34,6 +34,7 @@ import io.druid.query.AbstractPrioritizedCallable; import io.druid.query.QueryInterruptedException; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.groupby.GroupByQueryConfig; import io.druid.query.groupby.orderby.DefaultLimitSpec; import io.druid.segment.ColumnSelectorFactory; @@ -86,12 +87,56 @@ public class ConcurrentGrouper implements Grouper private final int priority; private final boolean hasQueryTimeout; private final long queryTimeoutAt; - private final ParallelCombiner parallelCombiner; private final long maxDictionarySizeForCombiner; + @Nullable + private final ParallelCombiner parallelCombiner; private volatile boolean initialized = false; public ConcurrentGrouper( + final GroupByQueryConfig groupByQueryConfig, + final Supplier bufferSupplier, + final Supplier> combineBufferSupplier, + final KeySerdeFactory keySerdeFactory, + final KeySerdeFactory combineKeySerdeFactory, + final ColumnSelectorFactory columnSelectorFactory, + final AggregatorFactory[] aggregatorFactories, + final LimitedTemporaryStorage temporaryStorage, + final ObjectMapper spillMapper, + final int concurrencyHint, + final DefaultLimitSpec limitSpec, + final boolean sortHasNonGroupingFields, + final ListeningExecutorService executor, + final int priority, + final boolean hasQueryTimeout, + final long queryTimeoutAt + ) + { + this( + bufferSupplier, + combineBufferSupplier, + keySerdeFactory, + combineKeySerdeFactory, + columnSelectorFactory, + aggregatorFactories, + groupByQueryConfig.getBufferGrouperMaxSize(), + groupByQueryConfig.getBufferGrouperMaxLoadFactor(), + groupByQueryConfig.getBufferGrouperInitialBuckets(), + temporaryStorage, + spillMapper, + concurrencyHint, + limitSpec, + sortHasNonGroupingFields, + executor, + priority, + hasQueryTimeout, + queryTimeoutAt, + groupByQueryConfig.getIntermediateCombineDegree(), + groupByQueryConfig.getNumParallelCombineThreads() + ); + } + + ConcurrentGrouper( final Supplier bufferSupplier, final Supplier> combineBufferSupplier, final KeySerdeFactory keySerdeFactory, @@ -110,7 +155,8 @@ public ConcurrentGrouper( final int priority, final boolean hasQueryTimeout, final long queryTimeoutAt, - final int intermediateCombineDegree + final int intermediateCombineDegree, + final int numParallelCombineThreads ) { Preconditions.checkArgument(concurrencyHint > 0, "concurrencyHint > 0"); @@ -135,19 +181,23 @@ public ConcurrentGrouper( this.priority = priority; this.hasQueryTimeout = hasQueryTimeout; this.queryTimeoutAt = queryTimeoutAt; - - this.parallelCombiner = new ParallelCombiner<>( - combineBufferSupplier, - getCombiningFactories(aggregatorFactories), - combineKeySerdeFactory, - executor, - sortHasNonGroupingFields, - concurrencyHint, - priority, - queryTimeoutAt, - intermediateCombineDegree - ); this.maxDictionarySizeForCombiner = combineKeySerdeFactory.getMaxDictionarySize(); + + if (numParallelCombineThreads > 1) { + this.parallelCombiner = new ParallelCombiner<>( + combineBufferSupplier, + getCombiningFactories(aggregatorFactories), + combineKeySerdeFactory, + executor, + sortHasNonGroupingFields, + Math.min(numParallelCombineThreads, concurrencyHint), + priority, + queryTimeoutAt, + intermediateCombineDegree + ); + } else { + this.parallelCombiner = null; + } } @Override @@ -258,7 +308,7 @@ public CloseableIterator> iterator(final boolean sorted) // Parallel combine is used only when data is spilled. This is because ConcurrentGrouper uses two different modes // depending on data is spilled or not. If data is not spilled, all inputs are completely aggregated and no more // aggregation is required. - if (sorted && spilling && isParallelizable()) { + if (sorted && spilling && parallelCombiner != null) { // First try to merge dictionaries generated by all underlying groupers. If it is merged successfully, the same // merged dictionary is used for all combining threads final List dictionary = tryMergeDictionary(); diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index 98c1914f7d89..df23a97abce3 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -218,15 +218,13 @@ public static Pair, Accumulator> crea ); grouper = new ConcurrentGrouper<>( + querySpecificConfig, bufferSupplier, combineBufferSupplier, keySerdeFactory, combineKeySerdeFactory, columnSelectorFactory, aggregatorFactories, - querySpecificConfig.getBufferGrouperMaxSize(), - querySpecificConfig.getBufferGrouperMaxLoadFactor(), - querySpecificConfig.getBufferGrouperInitialBuckets(), temporaryStorage, spillMapper, concurrencyHint, @@ -235,8 +233,7 @@ public static Pair, Accumulator> crea grouperSorter, priority, hasQueryTimeout, - queryTimeoutAt, - querySpecificConfig.getIntermediateCombineDegree() + queryTimeoutAt ); } 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 07a9e902eb82..6623be5a3553 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -297,6 +297,26 @@ public String toString() return "v2SmallDictionary"; } }; + final GroupByQueryConfig v2ParallelCombineConfig = new GroupByQueryConfig() + { + @Override + public String getDefaultStrategy() + { + return GroupByStrategySelector.STRATEGY_V2; + } + + @Override + public int getNumParallelCombineThreads() + { + return DEFAULT_PROCESSING_CONFIG.getNumThreads(); + } + + @Override + public String toString() + { + return "v2ParallelCombine"; + } + }; v1Config.setMaxIntermediateRows(10000); v1SingleThreadedConfig.setMaxIntermediateRows(10000); @@ -306,7 +326,8 @@ public String toString() v1SingleThreadedConfig, v2Config, v2SmallBufferConfig, - v2SmallDictionaryConfig + v2SmallDictionaryConfig, + v2ParallelCombineConfig ); } diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java index c118b0ab2264..f85713d672e6 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java @@ -133,7 +133,8 @@ public void testAggregate() throws InterruptedException, ExecutionException, IOE 0, false, 0, - 4 + 4, + 8 ); grouper.init(); From 0723960b76e6a5105637427716c4101636ef558a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 13 Oct 2017 16:32:23 +0900 Subject: [PATCH 42/44] improve doc --- docs/content/querying/groupbyquery.md | 138 +++++++++++++++----------- 1 file changed, 80 insertions(+), 58 deletions(-) diff --git a/docs/content/querying/groupbyquery.md b/docs/content/querying/groupbyquery.md index 62a21facf04f..68128fa71413 100644 --- a/docs/content/querying/groupbyquery.md +++ b/docs/content/querying/groupbyquery.md @@ -49,9 +49,9 @@ An example groupBy query object is shown below: ], "intervals": [ "2012-01-01T00:00:00.000/2012-01-03T00:00:00.000" ], "having": { - "type": "greaterThan", - "aggregation": "total_usage", - "value": 100 + "type": "greaterThan", + "aggregation": "total_usage", + "value": 100 } } ``` @@ -123,7 +123,8 @@ the query context. If neither the context field nor the property is set, the "v2 - "v2", the default, 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 sort and merge results before sending to the broker. Here, if data is spilled on disk, data nodes create a combining tree which merges results using multiple threads. Each node of the tree combines the input rows from child nodes which are run by different threads. If data is not spilled, data nodes simply merges results using a single thread. Finally, data nodes send the merged results to the broker, which merges result streams using an N-way merge. The broker materializes +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. @@ -179,7 +180,10 @@ disk space. With groupBy v2, cluster operators should make sure that the off-heap hash tables and on-heap merging dictionaries will not exceed available memory for the maximum possible concurrent query load (given by -druid.processing.numMergeBuffers). +druid.processing.numMergeBuffers). Especially the amount of direct memory needed by Druid is at least +`druid.processing.buffer.sizeBytes * (druid.processing.numMergeBuffers + druid.processing.numThreads + 1)`. Operators can +ensure at least this amount of direct memory is available by providing `-XX:MaxDirectMemorySize=` at the command +line. When using groupBy v1, all aggregation is done on-heap, and resource limits are done through the parameter druid.query.groupBy.maxResults. This is a cap on the maximum number of results in a result set. Queries that exceed @@ -187,6 +191,31 @@ this limit will fail with a "Resource limit exceeded" error indicating they exce operators should make sure that the on-heap aggregations will not exceed available JVM heap space for the expected concurrent query load. +#### Performance tuning for groupBy v2 + +##### Limit pushdown optimization + +Druid pushes down the `limit` spec in groupBy queries to the segments on historicals wherever possible to early prune unnecessary intermediate results and minimize the amount of data transferred to brokers. By default, this technique is applied only when all fields in the `orderBy` spec is a subset of the grouping keys. This is because the `limitPushDown` doesn't guarantee the exact results if the `orderBy` spec includes any fields that are not in the grouping keys. However, you can enable this technique even in such cases if you can sacrifice some accuracy for fast query processing like in topN queries. See `forceLimitPushDown` in [advanced groupBy v2 configurations](#groupby-v2-configurations). + + +##### Optimizing hash table + +The groupBy v2 engine uses an open addressing hash table for aggregation. The hash table is initalized with a given initial bucket number and gradually grows on buffer full. On hash collisions, the linear probing technique is used. + +The default number of initial buckets is 1024 and the default max load factor of the hash table is 0.7. If you can see too many collisions in the hash table, you can adjust these numbers. See `bufferGrouperInitialBuckets` and `bufferGrouperMaxLoadFactor` in [Advanced groupBy v2 configurations](#groupby-v2-configurations). + + +##### Parallel combine + +Once a historical finishes aggregation using the hash table, it sorts data before sending to the broker for N-way merge aggregation in the broker. By default, historicals use all their available processing threads (configured by `druid.processing.numThreads`) for aggregation, but use a single thread for sorting which is an http thread to send data to brokers. + +This is to prevent some heavy groupBy queries from blocking other queries. In Druid, the processing threads are shared between all submitted queries and they are _not interruptible_. It means, if a heavy query takes all available processing threads, all other queries might be blocked until the heavy query is finished. GroupBy queries usually take longer time than timeseries or topN queries, they should release processing threads as soon as possible. + +However, you might care about the performance of some really heavy groupBy queries. Usually, the performance bottleneck of heavy groupBy queries is the sorting. In such cases, you can use processing threads for sorting as well. This is called _parallel combine_. To enable parallel combine, see `numParallelCombineThreads` in [Advanced groupBy v2 configurations](#groupby-v2-configurations). Note that parallel combine can be enabled for only when data is actually spilled (see [Memory tuning and resource limits](#memory-tuning-and-resource-limits)). + +Once parallel combine is enabled, the groupBy v2 engine creates a combining tree. Each intermediate node of the tree is a thread merging aggregates from the child nodes. The leaf node threads read and merge aggregates from hash tables including spilled ones. Usually, leaf nodes are slower than intermediate nodes because they need to read data from disk. As a result, a less number of threads are used for intermediate nodes by default. You can change the degree of intermeidate nodes. See `intermediateCombineDegree` in [Advanced groupBy v2 configurations](#groupby-v2-configurations). + + #### Alternatives There are some situations where other query types may be a better choice than groupBy. @@ -207,60 +236,53 @@ indexing mechanism, and runs the outer query on these materialized results. "v2" inner query's results stream with off-heap fact map and on-heap string dictionary that can spill to disk. Both strategy perform the outer query on the broker in a single-threaded fashion. -#### Server configuration +#### Configurations -When using the "v2" strategy, the following runtime properties apply: +This section describes the configurations for groupBy queries. You can set system-wide configurations by adding them to runtime properties or query-specific configurations by adding them to query contexts. All runtime properties are prefixed by `druid.query.groupBy`. -|Property|Description|Default| -|--------|-----------|-------| -|`druid.query.groupBy.defaultStrategy`|Default groupBy query strategy.|v2| -|`druid.query.groupBy.bufferGrouperInitialBuckets`|Initial number of buckets in the off-heap hash table used for grouping results. Set to 0 to use a reasonable default.|0| -|`druid.query.groupBy.bufferGrouperMaxLoadFactor`|Maximum load factor of the off-heap hash table used for grouping results. When the load factor exceeds this size, the table will be grown or spilled to disk. Set to 0 to use a reasonable default.|0| -|`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.|100000000| -|`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)| -|`druid.query.groupBy.singleThreaded`|Merge results using a single thread.|false| -|`druid.query.groupBy.forceHashAggregation`|Force to use hash-based aggregation.|false| -|`druid.query.groupBy.intermediateCombineDegree`|Number of intermediate nodes combined together in the combining tree. Higher degrees will need a less numer of threads which might be helpful to improve the query performance by reducing the overhead of too many threads if the server has sufficiently powerful cpu cores.|8| -|`druid.query.groupBy.numParallelCombineThreads`|Hint for the number of parallel combining threads. This should be larger than 1 to turn on the parallel combining feature. The actual number of threads used for parallel combining is min(`druid.query.groupBy.numParallelCombineThreads`, `druid.processing.numThreads`).|1| +#### Commonly tuned configurations -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. +##### Configurations for groupBy v2 + +|Server runtime property (prefixed by `druid.query.groupBy.`)|Query context parameter|Description|Default| +|-----------------------|-----------------------|-----------|-------| +|`maxMergingDictionarySize`|`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.|100000000| +|`maxOnDiskStorage`|`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)| + + +##### Configurations for groupBy v1 + +|Server runtime property (prefixed by `druid.query.groupBy.`)|Query context parameter|Description|Default| +|-----------------------|-----------------------|-----------|-------| +|`maxIntermediateRows`|`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| +|`maxResults`|`maxResults`|Maximum number of results. Queries that exceed this limit will fail.|500000| + + +#### Advanced configurations + +##### Common configuragions for all groupBy strategies + +|Server runtime property (prefixed by `druid.query.groupBy.`)|Query context parameter|Description|Default| +|-----------------------|-----------------------|-----------|-------| +|`defaultStrategy`|`groupByStrategy`|Default groupBy query strategy.|v2| +|`singleThreaded`|`groupByIsSingleThreaded`|Merge results using a single thread.|false| + + +##### GroupBy v2 configurations + +|Server runtime property (prefixed by `druid.query.groupBy.`)|Query context parameter|Description|Default| +|-----------------------|-----------------------|-----------|-------| +|`bufferGrouperInitialBuckets`|`bufferGrouperInitialBuckets`|Initial number of buckets in the off-heap hash table used for grouping results. Set to 0 to use a reasonable default.|0| +|`bufferGrouperMaxLoadFactor`|`bufferGrouperMaxLoadFactor`|Maximum load factor of the off-heap hash table used for grouping results. When the load factor exceeds this size, the table will be grown or spilled to disk. Set to 0 to use a reasonable default.|0| +|`forceHashAggregation`|`forceHashAggregation`|Force to use hash-based aggregation.|false| +|`intermediateCombineDegree`|`intermediateCombineDegree`|Number of intermediate nodes combined together in the combining tree. Higher degrees will need a less numer of threads which might be helpful to improve the query performance by reducing the overhead of too many threads if the server has sufficiently powerful cpu cores.|8| +|`numParallelCombineThreads`|`numParallelCombineThreads`|Hint for the number of parallel combining threads. This should be larger than 1 to turn on the parallel combining feature. The actual number of threads used for parallel combining is min(`druid.query.groupBy.numParallelCombineThreads`, `druid.processing.numThreads`).|1 (disabled)| +|None|`sortByDimsFirst`|Sort the results first by dimension values and then by timestamp.|false| +|None|`forceLimitPushDown`|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.|false| + + +##### GroupBy v1 configurations -When using the "v1" strategy, the following runtime properties apply: - -|Property|Description|Default| -|--------|-----------|-------| -|`druid.query.groupBy.defaultStrategy`|Default groupBy query strategy.|v2| -|`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| - -#### Query context - -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.| -|`groupByIsSingleThreaded`|Overrides the value of `druid.query.groupBy.singleThreaded` for this query.| -|`bufferGrouperInitialBuckets`|Overrides the value of `druid.query.groupBy.bufferGrouperInitialBuckets` for this query.| -|`bufferGrouperMaxLoadFactor`|Overrides the value of `druid.query.groupBy.bufferGrouperMaxLoadFactor` for this query.| -|`maxMergingDictionarySize`|Can be used to lower the value of `druid.query.groupBy.maxMergingDictionarySize` for this query.| -|`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`|Overrides the value of `druid.query.groupBy.forceHashAggregation` for this query.| -|`intermediateCombineDegree`|Overrides the value of `druid.query.groupBy.intermediateCombineDegree` for this query.| -|`numParallelCombineThreads`|Overrides the value of `druid.query.groupBy.numParallelCombineThreads` for this query.| - -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.| +|Server runtime property|Query context parameter|Description|Default| +|-----------------------|-----------------------|-----------|-------| +|None|`useOffheap`|Set to true to store aggregations off-heap when merging results.|false| From 78df53da673e9c31528b7ed562e4366f768a0bc5 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 14 Oct 2017 11:02:48 +0900 Subject: [PATCH 43/44] address comments --- docs/content/operations/performance-faq.md | 1 + docs/content/querying/groupbyquery.md | 95 ++++++++++++------- .../epinephelinae/ConcurrentGrouper.java | 6 ++ .../query/groupby/epinephelinae/Grouper.java | 2 +- .../epinephelinae/ParallelCombiner.java | 10 +- .../StreamingMergeSortedGrouper.java | 2 +- 6 files changed, 80 insertions(+), 36 deletions(-) diff --git a/docs/content/operations/performance-faq.md b/docs/content/operations/performance-faq.md index 44456367be31..edeaf77d8aee 100644 --- a/docs/content/operations/performance-faq.md +++ b/docs/content/operations/performance-faq.md @@ -34,6 +34,7 @@ A useful formula for estimating direct memory usage follows: `druid.processing.buffer.sizeBytes * (druid.processing.numMergeBuffers + druid.processing.numThreads + 1)` The `+1` is a fuzzy parameter meant to account for the decompression and dictionary merging buffers and may need to be adjusted based on the characteristics of the data being ingested/queried. +Operators can ensure at least this amount of direct memory is available by providing `-XX:MaxDirectMemorySize=` at the command line. ## What is the intermediate computation buffer? The intermediate computation buffer 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. The default size is 1073741824 bytes (1GB). diff --git a/docs/content/querying/groupbyquery.md b/docs/content/querying/groupbyquery.md index 68128fa71413..d67671fdd188 100644 --- a/docs/content/querying/groupbyquery.md +++ b/docs/content/querying/groupbyquery.md @@ -180,10 +180,7 @@ disk space. With groupBy v2, cluster operators should make sure that the off-heap hash tables and on-heap merging dictionaries will not exceed available memory for the maximum possible concurrent query load (given by -druid.processing.numMergeBuffers). Especially the amount of direct memory needed by Druid is at least -`druid.processing.buffer.sizeBytes * (druid.processing.numMergeBuffers + druid.processing.numThreads + 1)`. Operators can -ensure at least this amount of direct memory is available by providing `-XX:MaxDirectMemorySize=` at the command -line. +druid.processing.numMergeBuffers). See [How much direct memory does Druid use?](../operations/performance-faq.html) for more details. When using groupBy v1, all aggregation is done on-heap, and resource limits are done through the parameter druid.query.groupBy.maxResults. This is a cap on the maximum number of results in a result set. Queries that exceed @@ -207,13 +204,13 @@ The default number of initial buckets is 1024 and the default max load factor of ##### Parallel combine -Once a historical finishes aggregation using the hash table, it sorts data before sending to the broker for N-way merge aggregation in the broker. By default, historicals use all their available processing threads (configured by `druid.processing.numThreads`) for aggregation, but use a single thread for sorting which is an http thread to send data to brokers. +Once a historical finishes aggregation using the hash table, it sorts aggregates and merge them before sending to the broker for N-way merge aggregation in the broker. By default, historicals use all their available processing threads (configured by `druid.processing.numThreads`) for aggregation, but use a single thread for sorting and merging aggregates which is an http thread to send data to brokers. This is to prevent some heavy groupBy queries from blocking other queries. In Druid, the processing threads are shared between all submitted queries and they are _not interruptible_. It means, if a heavy query takes all available processing threads, all other queries might be blocked until the heavy query is finished. GroupBy queries usually take longer time than timeseries or topN queries, they should release processing threads as soon as possible. -However, you might care about the performance of some really heavy groupBy queries. Usually, the performance bottleneck of heavy groupBy queries is the sorting. In such cases, you can use processing threads for sorting as well. This is called _parallel combine_. To enable parallel combine, see `numParallelCombineThreads` in [Advanced groupBy v2 configurations](#groupby-v2-configurations). Note that parallel combine can be enabled for only when data is actually spilled (see [Memory tuning and resource limits](#memory-tuning-and-resource-limits)). +However, you might care about the performance of some really heavy groupBy queries. Usually, the performance bottleneck of heavy groupBy queries is merging sorted aggregates. In such cases, you can use processing threads for it as well. This is called _parallel combine_. To enable parallel combine, see `numParallelCombineThreads` in [Advanced groupBy v2 configurations](#groupby-v2-configurations). Note that parallel combine can be enabled only when data is actually spilled (see [Memory tuning and resource limits](#memory-tuning-and-resource-limits)). -Once parallel combine is enabled, the groupBy v2 engine creates a combining tree. Each intermediate node of the tree is a thread merging aggregates from the child nodes. The leaf node threads read and merge aggregates from hash tables including spilled ones. Usually, leaf nodes are slower than intermediate nodes because they need to read data from disk. As a result, a less number of threads are used for intermediate nodes by default. You can change the degree of intermeidate nodes. See `intermediateCombineDegree` in [Advanced groupBy v2 configurations](#groupby-v2-configurations). +Once parallel combine is enabled, the groupBy v2 engine can create a combining tree for merging sorted aggregates. Each intermediate node of the tree is a thread merging aggregates from the child nodes. The leaf node threads read and merge aggregates from hash tables including spilled ones. Usually, leaf nodes are slower than intermediate nodes because they need to read data from disk. As a result, less threads are used for intermediate nodes by default. You can change the degree of intermeidate nodes. See `intermediateCombineDegree` in [Advanced groupBy v2 configurations](#groupby-v2-configurations). #### Alternatives @@ -244,45 +241,79 @@ This section describes the configurations for groupBy queries. You can set syste ##### Configurations for groupBy v2 -|Server runtime property (prefixed by `druid.query.groupBy.`)|Query context parameter|Description|Default| -|-----------------------|-----------------------|-----------|-------| -|`maxMergingDictionarySize`|`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.|100000000| -|`maxOnDiskStorage`|`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)| +Supported runtime properties: +|Property|Description|Default| +|--------|-----------|-------| +|`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.|100000000| +|`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)| -##### Configurations for groupBy v1 +Supported query contexts: -|Server runtime property (prefixed by `druid.query.groupBy.`)|Query context parameter|Description|Default| -|-----------------------|-----------------------|-----------|-------| -|`maxIntermediateRows`|`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| -|`maxResults`|`maxResults`|Maximum number of results. Queries that exceed this limit will fail.|500000| +|Key|Description| +|---|-----------| +|`maxMergingDictionarySize`|Can be used to lower the value of `druid.query.groupBy.maxMergingDictionarySize` for this query.| +|`maxOnDiskStorage`|Can be used to lower the value of `druid.query.groupBy.maxOnDiskStorage` for this query.| #### Advanced configurations ##### Common configuragions for all groupBy strategies -|Server runtime property (prefixed by `druid.query.groupBy.`)|Query context parameter|Description|Default| -|-----------------------|-----------------------|-----------|-------| -|`defaultStrategy`|`groupByStrategy`|Default groupBy query strategy.|v2| -|`singleThreaded`|`groupByIsSingleThreaded`|Merge results using a single thread.|false| +Supported runtime properties: + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.query.groupBy.defaultStrategy`|Default groupBy query strategy.|v2| +|`druid.query.groupBy.singleThreaded`|Merge results using a single thread.|false| + +Supported query contexts: + +|Key|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.| ##### GroupBy v2 configurations -|Server runtime property (prefixed by `druid.query.groupBy.`)|Query context parameter|Description|Default| -|-----------------------|-----------------------|-----------|-------| -|`bufferGrouperInitialBuckets`|`bufferGrouperInitialBuckets`|Initial number of buckets in the off-heap hash table used for grouping results. Set to 0 to use a reasonable default.|0| -|`bufferGrouperMaxLoadFactor`|`bufferGrouperMaxLoadFactor`|Maximum load factor of the off-heap hash table used for grouping results. When the load factor exceeds this size, the table will be grown or spilled to disk. Set to 0 to use a reasonable default.|0| -|`forceHashAggregation`|`forceHashAggregation`|Force to use hash-based aggregation.|false| -|`intermediateCombineDegree`|`intermediateCombineDegree`|Number of intermediate nodes combined together in the combining tree. Higher degrees will need a less numer of threads which might be helpful to improve the query performance by reducing the overhead of too many threads if the server has sufficiently powerful cpu cores.|8| -|`numParallelCombineThreads`|`numParallelCombineThreads`|Hint for the number of parallel combining threads. This should be larger than 1 to turn on the parallel combining feature. The actual number of threads used for parallel combining is min(`druid.query.groupBy.numParallelCombineThreads`, `druid.processing.numThreads`).|1 (disabled)| -|None|`sortByDimsFirst`|Sort the results first by dimension values and then by timestamp.|false| -|None|`forceLimitPushDown`|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.|false| +Supported runtime properties: + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.query.groupBy.bufferGrouperInitialBuckets`|Initial number of buckets in the off-heap hash table used for grouping results. Set to 0 to use a reasonable default (1024).|0| +|`druid.query.groupBy.bufferGrouperMaxLoadFactor`|Maximum load factor of the off-heap hash table used for grouping results. When the load factor exceeds this size, the table will be grown or spilled to disk. Set to 0 to use a reasonable default (0.7).|0| +|`druid.query.groupBy.forceHashAggregation`|Force to use hash-based aggregation.|false| +|`druid.query.groupBy.intermediateCombineDegree`|Number of intermediate nodes combined together in the combining tree. Higher degrees will need less threads which might be helpful to improve the query performance by reducing the overhead of too many threads if the server has sufficiently powerful cpu cores.|8| +|`druid.query.groupBy.numParallelCombineThreads`|Hint for the number of parallel combining threads. This should be larger than 1 to turn on the parallel combining feature. The actual number of threads used for parallel combining is min(`druid.query.groupBy.numParallelCombineThreads`, `druid.processing.numThreads`).|1 (disabled)| + +Supported query contexts: + +|Key|Description|Default| +|---|-----------|-------| +|`bufferGrouperInitialBuckets`|Overrides the value of `druid.query.groupBy.bufferGrouperInitialBuckets` for this query.|None| +|`bufferGrouperMaxLoadFactor`|Overrides the value of `druid.query.groupBy.bufferGrouperMaxLoadFactor` for this query.|None| +|`forceHashAggregation`|Overrides the value of `druid.query.groupBy.forceHashAggregation`|None| +|`intermediateCombineDegree`|Overrides the value of `druid.query.groupBy.intermediateCombineDegree`|None| +|`numParallelCombineThreads`|Overrides the value of `druid.query.groupBy.numParallelCombineThreads`|None| +|`sortByDimsFirst`|Sort the results first by dimension values and then by timestamp.|false| +|`forceLimitPushDown`|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.|false| ##### GroupBy v1 configurations -|Server runtime property|Query context parameter|Description|Default| -|-----------------------|-----------------------|-----------|-------| -|None|`useOffheap`|Set to true to store aggregations off-heap when merging results.|false| +Supported runtime properties: + +|Property|Description|Default| +|--------|-----------|-------| +|`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| + +Supported query contexts: + +|Key|Description|Default| +|---|-----------|-------| +|`maxIntermediateRows`|Can be used to lower the value of `druid.query.groupBy.maxIntermediateRows` for this query.|None| +|`maxResults`|Can be used to lower the value of `druid.query.groupBy.maxResults` for this query.|None| +|`useOffheap`|Set to true to store aggregations off-heap when merging results.|false| + 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 fe3754aa6916..56786b42f849 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 @@ -160,6 +160,12 @@ public ConcurrentGrouper( ) { Preconditions.checkArgument(concurrencyHint > 0, "concurrencyHint > 0"); + Preconditions.checkArgument( + concurrencyHint >= numParallelCombineThreads, + "numParallelCombineThreads[%d] cannot larger than concurrencyHint[%d]", + numParallelCombineThreads, + concurrencyHint + ); this.groupers = new ArrayList<>(concurrencyHint); this.threadLocalGrouper = ThreadLocal.withInitial(() -> groupers.get(threadNumber.getAndIncrement())); 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 4912e5743a8c..7e1a071fb9d5 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 @@ -103,7 +103,7 @@ default ToIntFunction hashFunction() *

* Some implementations allow writes even after this method is called. After you are done with the iterator * returned by this method, you should either call {@link #close()} (if you are done with the Grouper) or - * {@link #reset()} (if you want to reuse it). Some implmenetations allow calling {@link #iterator(boolean)} again if + * {@link #reset()} (if you want to reuse it). Some implementations allow calling {@link #iterator(boolean)} again if * you want another iterator. But, this method must not be called by multiple threads concurrently. *

* If "sorted" is true then the iterator will return sorted results. It will use KeyType's natural ordering on diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java index 4aad81a06abd..8bec94bbf6b3 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java @@ -214,7 +214,7 @@ public ByteBuffer get() if (i < numBuffers) { return Groupers.getSlice(combineBuffer, sliceSize, i++); } else { - throw new ISE("Requested number of buffer slices exceeds the planned one"); + throw new ISE("Requested number[%d] of buffer slices exceeds the planned one[%d]", i++, numBuffers); } } }; @@ -250,7 +250,13 @@ private Pair findLeafCombineDegreeAndNumBuffers( } } - throw new ISE("Cannot find a proper leaf combine degree. Try increasing druid.processing.buffer.sizeBytes."); + throw new ISE( + "Cannot find a proper leaf combine degree for the combining tree. " + + "Each node of the combining tree requires a buffer of [%d] bytes. " + + "Try increasing druid.processing.buffer.sizeBytes for larger buffer or " + + "druid.query.groupBy.intermediateCombineDegree for a smaller tree", + requiredMinimumBufferCapacity + ); } /** diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java index 6413f4529d05..f2faca0d2f69 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/StreamingMergeSortedGrouper.java @@ -365,7 +365,7 @@ public void close() public void finish() { increaseWriteIndex(); - // Once finished is set, curWirteIndex must not be changed. This guarantees that the remaining number of items in + // Once finished is set, curWriteIndex must not be changed. This guarantees that the remaining number of items in // the array is always decreased as the reading thread proceeds. See hasNext() and remaining() below. finished = true; } From 101391bdbea4cefb2c2804e889bc5340d82aade0 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 14 Oct 2017 15:48:07 +0900 Subject: [PATCH 44/44] fix build --- .../io/druid/query/groupby/epinephelinae/ParallelCombiner.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java index f40464ba9a43..d043a9070d76 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ParallelCombiner.java @@ -42,9 +42,6 @@ import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.ColumnValueSelector; import io.druid.segment.DimensionSelector; -import io.druid.segment.DoubleColumnSelector; -import io.druid.segment.FloatColumnSelector; -import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; import io.druid.segment.column.ColumnCapabilities; import it.unimi.dsi.fastutil.objects.Object2IntArrayMap;