diff --git a/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java b/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java index b889e1ea2771..532af26dee2d 100644 --- a/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java +++ b/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java @@ -83,7 +83,7 @@ public BufferAggregator factorizeBuffered(ColumnSelectorFactory columnFactory) { DimensionSelector selector = makeDimensionSelector(columnFactory); if (selector == null) { - return new EmptyDistinctCountBufferAggregator(); + return EmptyDistinctCountBufferAggregator.instance(); } else { return new DistinctCountBufferAggregator(makeDimensionSelector(columnFactory)); } diff --git a/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/DistinctCountBufferAggregator.java b/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/DistinctCountBufferAggregator.java index 809cd0ac796f..b60b832f4ce6 100644 --- a/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/DistinctCountBufferAggregator.java +++ b/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/DistinctCountBufferAggregator.java @@ -22,16 +22,17 @@ import io.druid.collections.bitmap.MutableBitmap; import io.druid.collections.bitmap.WrappedRoaringBitmap; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.DimensionSelector; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; import java.nio.ByteBuffer; -import java.util.HashMap; -import java.util.Map; public class DistinctCountBufferAggregator implements BufferAggregator { private final DimensionSelector selector; - private final Map mutableBitmapCollection = new HashMap<>(); + private final Int2ObjectMap mutableBitmapCollection = new Int2ObjectOpenHashMap<>(); public DistinctCountBufferAggregator( DimensionSelector selector @@ -89,4 +90,10 @@ public void close() { mutableBitmapCollection.clear(); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } } diff --git a/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/EmptyDistinctCountBufferAggregator.java b/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/EmptyDistinctCountBufferAggregator.java index 0238a8e39064..62e95a2c0936 100644 --- a/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/EmptyDistinctCountBufferAggregator.java +++ b/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/EmptyDistinctCountBufferAggregator.java @@ -20,13 +20,24 @@ package io.druid.query.aggregation.distinctcount; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import java.nio.ByteBuffer; -public class EmptyDistinctCountBufferAggregator implements BufferAggregator +/** + * The difference from {@link io.druid.query.aggregation.NoopBufferAggregator} is that + * EmptyDistinctCountBufferAggregator returns 0 instead of null from {@link #get(ByteBuffer, int)}. + */ +public final class EmptyDistinctCountBufferAggregator implements BufferAggregator { + private static final EmptyDistinctCountBufferAggregator INSTANCE = new EmptyDistinctCountBufferAggregator(); + + static EmptyDistinctCountBufferAggregator instance() + { + return INSTANCE; + } - public EmptyDistinctCountBufferAggregator() + private EmptyDistinctCountBufferAggregator() { } @@ -62,4 +73,9 @@ public long getLong(ByteBuffer buf, int position) public void close() { } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } } diff --git a/extensions-contrib/time-min-max/src/main/java/io/druid/query/aggregation/TimestampBufferAggregator.java b/extensions-contrib/time-min-max/src/main/java/io/druid/query/aggregation/TimestampBufferAggregator.java index e195005e1f29..6ee0bb815aaf 100644 --- a/extensions-contrib/time-min-max/src/main/java/io/druid/query/aggregation/TimestampBufferAggregator.java +++ b/extensions-contrib/time-min-max/src/main/java/io/druid/query/aggregation/TimestampBufferAggregator.java @@ -20,6 +20,7 @@ package io.druid.query.aggregation; import io.druid.data.input.impl.TimestampSpec; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ObjectColumnSelector; import java.nio.ByteBuffer; @@ -79,7 +80,14 @@ public long getLong(ByteBuffer buf, int position) } @Override - public void close() { + public void close() + { + } + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + inspector.visit("comparator", comparator); } } diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchBufferAggregator.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchBufferAggregator.java index 263f24c201e8..2014314675c7 100644 --- a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchBufferAggregator.java +++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchBufferAggregator.java @@ -20,12 +20,20 @@ package io.druid.query.aggregation.datasketches.theta; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import java.nio.ByteBuffer; -public class EmptySketchBufferAggregator implements BufferAggregator +public final class EmptySketchBufferAggregator implements BufferAggregator { - public EmptySketchBufferAggregator() + private static final EmptySketchBufferAggregator INSTANCE = new EmptySketchBufferAggregator(); + + public static EmptySketchBufferAggregator instance() + { + return INSTANCE; + } + + private EmptySketchBufferAggregator() { } @@ -61,4 +69,9 @@ public long getLong(ByteBuffer buf, int position) public void close() { } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } } diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java index 6419b390b76a..1f25a1d6e1b4 100644 --- a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java @@ -73,7 +73,7 @@ public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName); if (selector == null) { - return new EmptySketchBufferAggregator(); + return EmptySketchBufferAggregator.instance(); } else { return new SketchBufferAggregator(selector, size, getMaxIntermediateSize()); } diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchBufferAggregator.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchBufferAggregator.java index 03dc9d6372ca..c36f908d339a 100644 --- a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchBufferAggregator.java +++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchBufferAggregator.java @@ -26,6 +26,7 @@ import com.yahoo.sketches.theta.SetOperation; import com.yahoo.sketches.theta.Union; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ObjectColumnSelector; import java.nio.ByteBuffer; @@ -113,4 +114,9 @@ public void close() unions.clear(); } + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } } diff --git a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramBufferAggregator.java b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramBufferAggregator.java index 297fde021d09..a139ec5c9717 100644 --- a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramBufferAggregator.java +++ b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramBufferAggregator.java @@ -20,6 +20,7 @@ package io.druid.query.aggregation.histogram; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.FloatColumnSelector; import java.nio.ByteBuffer; @@ -99,4 +100,10 @@ public void close() { // no resources to cleanup } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } } diff --git a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingBufferAggregator.java b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingBufferAggregator.java index bd52b603badb..3a9713ba08ea 100644 --- a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingBufferAggregator.java +++ b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingBufferAggregator.java @@ -20,6 +20,7 @@ package io.druid.query.aggregation.histogram; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ObjectColumnSelector; import java.nio.ByteBuffer; @@ -102,4 +103,10 @@ public void close() { // no resources to cleanup } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } } diff --git a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorFactory.java b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorFactory.java index 37fa1f08419a..b871ce985950 100644 --- a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorFactory.java +++ b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorFactory.java @@ -23,14 +23,14 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; - import io.druid.java.util.common.IAE; import io.druid.java.util.common.StringUtils; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactoryNotMergeableException; -import io.druid.query.aggregation.Aggregators; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.aggregation.NoopAggregator; +import io.druid.query.aggregation.NoopBufferAggregator; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.ObjectColumnSelector; import org.apache.commons.codec.binary.Base64; @@ -95,7 +95,7 @@ public Aggregator factorize(ColumnSelectorFactory metricFactory) { ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName); if (selector == null) { - return Aggregators.noopAggregator(); + return NoopAggregator.instance(); } if ("float".equalsIgnoreCase(inputType)) { @@ -115,7 +115,7 @@ public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName); if (selector == null) { - return Aggregators.noopBufferAggregator(); + return NoopBufferAggregator.instance(); } if ("float".equalsIgnoreCase(inputType)) { return new VarianceBufferAggregator.FloatVarianceAggregator( diff --git a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceBufferAggregator.java b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceBufferAggregator.java index 77017de52312..95825e4b2637 100644 --- a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceBufferAggregator.java +++ b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceBufferAggregator.java @@ -22,6 +22,7 @@ import com.google.common.primitives.Doubles; import com.google.common.primitives.Longs; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.FloatColumnSelector; import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; @@ -102,6 +103,12 @@ public void aggregate(ByteBuffer buf, int position) buf.putDouble(position + NVARIANCE_OFFSET, variance); } } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } } public static final class LongVarianceAggregator extends VarianceBufferAggregator @@ -128,6 +135,12 @@ public void aggregate(ByteBuffer buf, int position) buf.putDouble(position + NVARIANCE_OFFSET, variance); } } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } } public static final class ObjectVarianceAggregator extends VarianceBufferAggregator @@ -167,5 +180,11 @@ public void aggregate(ByteBuffer buf, int position) buf.putDouble(position + SUM_OFFSET, sum); buf.putDouble(position + NVARIANCE_OFFSET, nvariance); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } } } diff --git a/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceAggregatorCollectorTest.java b/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceAggregatorCollectorTest.java index 5d4fff92f47f..3204b041cb4c 100644 --- a/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceAggregatorCollectorTest.java +++ b/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceAggregatorCollectorTest.java @@ -20,10 +20,9 @@ package io.druid.query.aggregation.variance; import com.google.common.collect.Lists; - import io.druid.java.util.common.Pair; -import io.druid.segment.FloatColumnSelector; import io.druid.segment.ObjectColumnSelector; +import io.druid.segment.TestFloatColumnSelector; import org.junit.Assert; import org.junit.Test; @@ -142,7 +141,7 @@ public void testVariance() } } - private static class FloatHandOver implements FloatColumnSelector + private static class FloatHandOver extends TestFloatColumnSelector { float v; diff --git a/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceTopNQueryTest.java b/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceTopNQueryTest.java index 7985bce9d82b..f3a9441b8a34 100644 --- a/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceTopNQueryTest.java +++ b/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceTopNQueryTest.java @@ -54,7 +54,7 @@ public class VarianceTopNQueryTest @Parameterized.Parameters(name="{0}") public static Iterable constructorFeeder() throws IOException { - return TopNQueryRunnerTest.constructorFeeder(); + return QueryRunnerTestHelper.transformToConstructionFeeder(TopNQueryRunnerTest.queryRunners()); } private final QueryRunner runner; diff --git a/pom.xml b/pom.xml index cb7cdf700e4d..ecfa8850e247 100644 --- a/pom.xml +++ b/pom.xml @@ -638,6 +638,16 @@ RoaringBitmap 0.5.18 + + org.ow2.asm + asm + 5.2 + + + org.ow2.asm + asm-commons + 5.2 + org.apache.calcite calcite-core @@ -815,10 +825,11 @@ 1.0 - sun.nio.ch.DirectBuffer sun.misc.Cleaner + sun.misc.Unsafe diff --git a/processing/pom.xml b/processing/pom.xml index 0f70c720fec0..ad9a805e6d15 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -93,6 +93,14 @@ commons-lang commons-lang + + org.ow2.asm + asm + + + org.ow2.asm + asm-commons + diff --git a/processing/src/main/java/io/druid/query/BaseQuery.java b/processing/src/main/java/io/druid/query/BaseQuery.java index f272d1599edb..22d0fb1a4ba5 100644 --- a/processing/src/main/java/io/druid/query/BaseQuery.java +++ b/processing/src/main/java/io/druid/query/BaseQuery.java @@ -97,6 +97,13 @@ private static boolean parseBoolean(Query query, String key, boolean defa } } + public static void checkInterrupted() + { + if (Thread.interrupted()) { + throw new QueryInterruptedException(new InterruptedException()); + } + } + public static final String QUERYID = "queryId"; private final DataSource dataSource; private final boolean descending; diff --git a/processing/src/main/java/io/druid/query/aggregation/Aggregators.java b/processing/src/main/java/io/druid/query/aggregation/Aggregators.java deleted file mode 100644 index 068784d8d6e1..000000000000 --- a/processing/src/main/java/io/druid/query/aggregation/Aggregators.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * 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.aggregation; - -import java.nio.ByteBuffer; - -/** - */ -public class Aggregators -{ - public static Aggregator noopAggregator() - { - return new Aggregator() - { - @Override - public void aggregate() - { - - } - - @Override - public void reset() - { - - } - - @Override - public Object get() - { - return null; - } - - @Override - public float getFloat() - { - return 0; - } - - @Override - public void close() - { - - } - - @Override - public long getLong() - { - return 0; - } - }; - } - - public static BufferAggregator noopBufferAggregator() - { - return new BufferAggregator() - { - @Override - public void init(ByteBuffer buf, int position) - { - - } - - @Override - public void aggregate(ByteBuffer buf, int position) - { - - } - - @Override - public Object get(ByteBuffer buf, int position) - { - return null; - } - - @Override - public float getFloat(ByteBuffer buf, int position) - { - return 0; - } - - - @Override - public long getLong(ByteBuffer buf, int position) - { - return 0L; - } - - @Override - public void close() - { - - } - }; - } -} diff --git a/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java index bbdf9100bbab..3b565dcce2d2 100644 --- a/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java @@ -19,6 +19,10 @@ package io.druid.query.aggregation; +import io.druid.query.monomorphicprocessing.CalledFromHotLoop; +import io.druid.query.monomorphicprocessing.HotLoopCallee; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; + import java.nio.ByteBuffer; /** @@ -29,7 +33,7 @@ * Thus, an Aggregator can be thought of as a closure over some other thing that is stateful and changes between calls * to aggregate(...). */ -public interface BufferAggregator +public interface BufferAggregator extends HotLoopCallee { /** * Initializes the buffer location @@ -44,6 +48,7 @@ public interface BufferAggregator * @param buf byte buffer to initialize * @param position offset within the byte buffer for initialization */ + @CalledFromHotLoop void init(ByteBuffer buf, int position); /** @@ -57,6 +62,7 @@ public interface BufferAggregator * @param buf byte buffer storing the byte array representation of the aggregate * @param position offset within the byte buffer at which the current aggregate value is stored */ + @CalledFromHotLoop void aggregate(ByteBuffer buf, int position); /** @@ -110,4 +116,14 @@ public interface BufferAggregator * Release any resources used by the aggregator */ void close(); + + /** + * {@inheritDoc} + * + *

The default implementation inspects nothing. Classes that implement {@code BufferAggregator} are encouraged to + * override this method, following the specification of {@link HotLoopCallee#inspectRuntimeShape}. + */ + default void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/CountBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/CountBufferAggregator.java index ec5804e4100a..daedab3f585f 100644 --- a/processing/src/main/java/io/druid/query/aggregation/CountBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/CountBufferAggregator.java @@ -19,6 +19,8 @@ package io.druid.query.aggregation; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; + import java.nio.ByteBuffer; /** @@ -62,4 +64,9 @@ public void close() { // no resources to cleanup } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/DoubleBufferAggregator.java new file mode 100644 index 000000000000..0e868674816c --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleBufferAggregator.java @@ -0,0 +1,65 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.aggregation; + +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.FloatColumnSelector; + +import java.nio.ByteBuffer; + +public abstract class DoubleBufferAggregator implements BufferAggregator +{ + protected final FloatColumnSelector selector; + + DoubleBufferAggregator(FloatColumnSelector selector) + { + this.selector = selector; + } + + @Override + public Object get(ByteBuffer buf, int position) + { + return buf.getDouble(position); + } + + @Override + public float getFloat(ByteBuffer buf, int position) + { + return (float) buf.getDouble(position); + } + + @Override + public long getLong(ByteBuffer buf, int position) + { + return (long) buf.getDouble(position); + } + + @Override + public void close() + { + // no resources to cleanup + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleMaxBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxBufferAggregator.java index 3a79da6eefb7..7a08e5a0434a 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleMaxBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxBufferAggregator.java @@ -25,13 +25,12 @@ /** */ -public class DoubleMaxBufferAggregator implements BufferAggregator +public class DoubleMaxBufferAggregator extends DoubleBufferAggregator { - private final FloatColumnSelector selector; - public DoubleMaxBufferAggregator(FloatColumnSelector selector) + DoubleMaxBufferAggregator(FloatColumnSelector selector) { - this.selector = selector; + super(selector); } @Override @@ -45,28 +44,4 @@ public void aggregate(ByteBuffer buf, int position) { buf.putDouble(position, Math.max(buf.getDouble(position), (double) selector.get())); } - - @Override - public Object get(ByteBuffer buf, int position) - { - return buf.getDouble(position); - } - - @Override - public float getFloat(ByteBuffer buf, int position) - { - return (float) buf.getDouble(position); - } - - @Override - public long getLong(ByteBuffer buf, int position) - { - return (long) buf.getDouble(position); - } - - @Override - public void close() - { - // no resources to cleanup - } } diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleMinBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/DoubleMinBufferAggregator.java index f4c3f475002e..b11712818d29 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleMinBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMinBufferAggregator.java @@ -25,13 +25,12 @@ /** */ -public class DoubleMinBufferAggregator implements BufferAggregator +public class DoubleMinBufferAggregator extends DoubleBufferAggregator { - private final FloatColumnSelector selector; - public DoubleMinBufferAggregator(FloatColumnSelector selector) + DoubleMinBufferAggregator(FloatColumnSelector selector) { - this.selector = selector; + super(selector); } @Override @@ -45,29 +44,4 @@ public void aggregate(ByteBuffer buf, int position) { buf.putDouble(position, Math.min(buf.getDouble(position), (double) selector.get())); } - - @Override - public Object get(ByteBuffer buf, int position) - { - return buf.getDouble(position); - } - - @Override - public float getFloat(ByteBuffer buf, int position) - { - return (float) buf.getDouble(position); - } - - - @Override - public long getLong(ByteBuffer buf, int position) - { - return (long) buf.getDouble(position); - } - - @Override - public void close() - { - // no resources to cleanup - } } diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleSumBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/DoubleSumBufferAggregator.java index e67a64d7c686..2c065ce49a7f 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleSumBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleSumBufferAggregator.java @@ -25,15 +25,12 @@ /** */ -public class DoubleSumBufferAggregator implements BufferAggregator +public class DoubleSumBufferAggregator extends DoubleBufferAggregator { - private final FloatColumnSelector selector; - public DoubleSumBufferAggregator( - FloatColumnSelector selector - ) + DoubleSumBufferAggregator(FloatColumnSelector selector) { - this.selector = selector; + super(selector); } @Override @@ -47,29 +44,4 @@ public void aggregate(ByteBuffer buf, int position) { buf.putDouble(position, buf.getDouble(position) + (double) selector.get()); } - - @Override - public Object get(ByteBuffer buf, int position) - { - return buf.getDouble(position); - } - - @Override - public float getFloat(ByteBuffer buf, int position) - { - return (float) buf.getDouble(position); - } - - - @Override - public long getLong(ByteBuffer buf, int position) - { - return (long) buf.getDouble(position); - } - - @Override - public void close() - { - // no resources to cleanup - } } diff --git a/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java index bd53197804e3..f1d615ddc0a6 100644 --- a/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java @@ -20,6 +20,7 @@ package io.druid.query.aggregation; import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import java.nio.ByteBuffer; @@ -71,4 +72,11 @@ public void close() { delegate.close(); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("matcher", matcher); + inspector.visit("delegate", delegate); + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/HistogramBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/HistogramBufferAggregator.java index 8ad856c33abb..4a6dd9a85087 100644 --- a/processing/src/main/java/io/druid/query/aggregation/HistogramBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/HistogramBufferAggregator.java @@ -21,6 +21,7 @@ import com.google.common.primitives.Floats; import com.google.common.primitives.Longs; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.FloatColumnSelector; import java.nio.ByteBuffer; @@ -105,4 +106,10 @@ public void close() { // no resources to cleanup } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/JavaScriptBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/JavaScriptBufferAggregator.java index c2acca6fb349..b0c822e2d058 100644 --- a/processing/src/main/java/io/druid/query/aggregation/JavaScriptBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/JavaScriptBufferAggregator.java @@ -20,6 +20,7 @@ package io.druid.query.aggregation; import com.google.common.collect.Lists; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ObjectColumnSelector; import java.nio.ByteBuffer; @@ -74,4 +75,11 @@ public long getLong(ByteBuffer buf, int position) public void close() { script.close(); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selectorList", selectorList); + inspector.visit("script", script); + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/LongBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/LongBufferAggregator.java new file mode 100644 index 000000000000..8f1092366aa7 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/LongBufferAggregator.java @@ -0,0 +1,65 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.aggregation; + +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.LongColumnSelector; + +import java.nio.ByteBuffer; + +public abstract class LongBufferAggregator implements BufferAggregator +{ + protected final LongColumnSelector selector; + + LongBufferAggregator(LongColumnSelector selector) + { + this.selector = selector; + } + + @Override + public Object get(ByteBuffer buf, int position) + { + return buf.getLong(position); + } + + @Override + public float getFloat(ByteBuffer buf, int position) + { + return (float) buf.getLong(position); + } + + @Override + public long getLong(ByteBuffer buf, int position) + { + return buf.getLong(position); + } + + @Override + public void close() + { + // no resources to cleanup + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/LongMaxBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/LongMaxBufferAggregator.java index 291245a6faed..d918ea76fd41 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongMaxBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongMaxBufferAggregator.java @@ -25,13 +25,12 @@ /** */ -public class LongMaxBufferAggregator implements BufferAggregator +public class LongMaxBufferAggregator extends LongBufferAggregator { - private final LongColumnSelector selector; - public LongMaxBufferAggregator(LongColumnSelector selector) + LongMaxBufferAggregator(LongColumnSelector selector) { - this.selector = selector; + super(selector); } @Override @@ -45,28 +44,4 @@ public void aggregate(ByteBuffer buf, int position) { buf.putLong(position, Math.max(buf.getLong(position), selector.get())); } - - @Override - public Object get(ByteBuffer buf, int position) - { - return buf.getLong(position); - } - - @Override - public float getFloat(ByteBuffer buf, int position) - { - return (float) buf.getLong(position); - } - - @Override - public long getLong(ByteBuffer buf, int position) - { - return buf.getLong(position); - } - - @Override - public void close() - { - // no resources to cleanup - } } diff --git a/processing/src/main/java/io/druid/query/aggregation/LongMinBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/LongMinBufferAggregator.java index 29c5a29e6459..3b0e206e9844 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongMinBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongMinBufferAggregator.java @@ -25,13 +25,12 @@ /** */ -public class LongMinBufferAggregator implements BufferAggregator +public class LongMinBufferAggregator extends LongBufferAggregator { - private final LongColumnSelector selector; - public LongMinBufferAggregator(LongColumnSelector selector) + LongMinBufferAggregator(LongColumnSelector selector) { - this.selector = selector; + super(selector); } @Override @@ -45,29 +44,4 @@ public void aggregate(ByteBuffer buf, int position) { buf.putLong(position, Math.min(buf.getLong(position), selector.get())); } - - @Override - public Object get(ByteBuffer buf, int position) - { - return buf.getLong(position); - } - - @Override - public float getFloat(ByteBuffer buf, int position) - { - return (float) buf.getLong(position); - } - - - @Override - public long getLong(ByteBuffer buf, int position) - { - return buf.getLong(position); - } - - @Override - public void close() - { - // no resources to cleanup - } } diff --git a/processing/src/main/java/io/druid/query/aggregation/LongSumBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/LongSumBufferAggregator.java index 208155dbe8cd..ad029471136a 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongSumBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongSumBufferAggregator.java @@ -25,15 +25,12 @@ /** */ -public class LongSumBufferAggregator implements BufferAggregator +public class LongSumBufferAggregator extends LongBufferAggregator { - private final LongColumnSelector selector; - public LongSumBufferAggregator( - LongColumnSelector selector - ) + LongSumBufferAggregator(LongColumnSelector selector) { - this.selector = selector; + super(selector); } @Override @@ -47,29 +44,4 @@ public void aggregate(ByteBuffer buf, int position) { buf.putLong(position, buf.getLong(position) + selector.get()); } - - @Override - public Object get(ByteBuffer buf, int position) - { - return buf.getLong(position); - } - - @Override - public float getFloat(ByteBuffer buf, int position) - { - return (float) buf.getLong(position); - } - - - @Override - public long getLong(ByteBuffer buf, int position) - { - return buf.getLong(position); - } - - @Override - public void close() - { - // no resources to cleanup - } } diff --git a/processing/src/main/java/io/druid/segment/IndexedIntsOffset.java b/processing/src/main/java/io/druid/query/aggregation/NoopAggregator.java similarity index 58% rename from processing/src/main/java/io/druid/segment/IndexedIntsOffset.java rename to processing/src/main/java/io/druid/query/aggregation/NoopAggregator.java index af0cc824cce4..ffb9874511e8 100644 --- a/processing/src/main/java/io/druid/segment/IndexedIntsOffset.java +++ b/processing/src/main/java/io/druid/query/aggregation/NoopAggregator.java @@ -17,47 +17,51 @@ * under the License. */ -package io.druid.segment; +package io.druid.query.aggregation; -import io.druid.segment.data.IndexedInts; -import io.druid.segment.data.Offset; - -/** -*/ -class IndexedIntsOffset implements Offset +public final class NoopAggregator implements Aggregator { - int currRow; - private final IndexedInts invertedIndex; + private static final NoopAggregator INSTANCE = new NoopAggregator(); + + public static NoopAggregator instance() + { + return INSTANCE; + } + + private NoopAggregator() + { + } + + @Override + public void aggregate() + { + } - public IndexedIntsOffset(IndexedInts invertedIndex) + @Override + public void reset() { - this.invertedIndex = invertedIndex; - currRow = 0; } @Override - public void increment() + public Object get() { - ++currRow; + return null; } @Override - public boolean withinBounds() + public float getFloat() { - return currRow < invertedIndex.size(); + return 0; } @Override - public Offset clone() + public void close() { - final IndexedIntsOffset retVal = new IndexedIntsOffset(invertedIndex); - retVal.currRow = currRow; - return retVal; } @Override - public int getOffset() + public long getLong() { - return invertedIndex.get(currRow); + return 0; } } diff --git a/processing/src/main/java/io/druid/query/aggregation/NoopBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/NoopBufferAggregator.java new file mode 100644 index 000000000000..fd4d6bd51d7c --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/NoopBufferAggregator.java @@ -0,0 +1,77 @@ +/* + * 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.aggregation; + +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; + +import java.nio.ByteBuffer; + +public final class NoopBufferAggregator implements BufferAggregator +{ + private static final NoopBufferAggregator INSTANCE = new NoopBufferAggregator(); + + public static NoopBufferAggregator instance() + { + return INSTANCE; + } + + private NoopBufferAggregator() + { + } + + @Override + public void init(ByteBuffer buf, int position) + { + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + } + + @Override + public Object get(ByteBuffer buf, int position) + { + return null; + } + + @Override + public float getFloat(ByteBuffer buf, int position) + { + return 0; + } + + + @Override + public long getLong(ByteBuffer buf, int position) + { + return 0L; + } + + @Override + public void close() + { + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregator.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregator.java index 62b9dd3c147f..ae96ff13736b 100644 --- a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregator.java @@ -19,12 +19,13 @@ package io.druid.query.aggregation.cardinality; +import com.google.common.annotations.VisibleForTesting; import com.google.common.hash.HashFunction; import com.google.common.hash.Hasher; import com.google.common.hash.Hashing; import io.druid.hll.HyperLogLogCollector; -import io.druid.query.aggregation.Aggregator; import io.druid.query.ColumnSelectorPlus; +import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.cardinality.types.CardinalityAggregatorColumnSelectorStrategy; import java.util.List; @@ -32,48 +33,59 @@ public class CardinalityAggregator implements Aggregator { private final String name; - private final List> selectorPlusList; + private final ColumnSelectorPlus[] selectorPluses; private final boolean byRow; public static final HashFunction hashFn = Hashing.murmur3_128(); - protected static void hashRow( - List> selectorPlusList, + static void hashRow( + ColumnSelectorPlus[] selectorPluses, HyperLogLogCollector collector ) { final Hasher hasher = hashFn.newHasher(); - for (int k = 0; k < selectorPlusList.size(); ++k) { + for (int k = 0; k < selectorPluses.length; ++k) { if (k != 0) { hasher.putByte((byte) 0); } - ColumnSelectorPlus selectorPlus = selectorPlusList.get(k); + ColumnSelectorPlus selectorPlus = selectorPluses[k]; selectorPlus.getColumnSelectorStrategy().hashRow(selectorPlus.getSelector(), hasher); } collector.add(hasher.hash().asBytes()); } - protected static void hashValues( - List> selectorPlusList, + static void hashValues( + ColumnSelectorPlus[] selectorPluses, HyperLogLogCollector collector ) { - for (final ColumnSelectorPlus selectorPlus : selectorPlusList) { + for (final ColumnSelectorPlus selectorPlus : selectorPluses) { selectorPlus.getColumnSelectorStrategy().hashValues(selectorPlus.getSelector(), collector); } } private HyperLogLogCollector collector; - public CardinalityAggregator( + @VisibleForTesting + @SuppressWarnings("unchecked") + CardinalityAggregator( String name, List> selectorPlusList, boolean byRow ) + { + this(name, selectorPlusList.toArray(new ColumnSelectorPlus[] {}), byRow); + } + + CardinalityAggregator( + String name, + ColumnSelectorPlus[] selectorPluses, + boolean byRow + ) { this.name = name; - this.selectorPlusList = selectorPlusList; + this.selectorPluses = selectorPluses; this.collector = HyperLogLogCollector.makeLatestCollector(); this.byRow = byRow; } @@ -82,9 +94,9 @@ public CardinalityAggregator( public void aggregate() { if (byRow) { - hashRow(selectorPlusList, collector); + hashRow(selectorPluses, collector); } else { - hashValues(selectorPlusList, collector); + hashValues(selectorPluses, collector); } } @@ -115,7 +127,7 @@ public long getLong() @Override public Aggregator clone() { - return new CardinalityAggregator(name, selectorPlusList, byRow); + return new CardinalityAggregator(name, selectorPluses, byRow); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java index eb623ecd6ade..a4b9be2e99cc 100644 --- a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java @@ -31,8 +31,9 @@ import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactoryNotMergeableException; -import io.druid.query.aggregation.Aggregators; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.aggregation.NoopAggregator; +import io.druid.query.aggregation.NoopBufferAggregator; import io.druid.query.aggregation.cardinality.types.CardinalityAggregatorColumnSelectorStrategy; import io.druid.query.aggregation.cardinality.types.CardinalityAggregatorColumnSelectorStrategyFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; @@ -44,7 +45,6 @@ import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.Comparator; import java.util.List; @@ -137,36 +137,34 @@ public CardinalityAggregatorFactory( @Override public Aggregator factorize(final ColumnSelectorFactory columnFactory) { - List> selectorPlusList = - Arrays.asList(DimensionHandlerUtils.createColumnSelectorPluses( + ColumnSelectorPlus[] selectorPluses = + DimensionHandlerUtils.createColumnSelectorPluses( STRATEGY_FACTORY, fields, columnFactory - )); + ); - if (selectorPlusList.isEmpty()) { - return Aggregators.noopAggregator(); + if (selectorPluses.length == 0) { + return NoopAggregator.instance(); } - - return new CardinalityAggregator(name, selectorPlusList, byRow); + return new CardinalityAggregator(name, selectorPluses, byRow); } @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory columnFactory) { - List> selectorPlusList = - Arrays.asList(DimensionHandlerUtils.createColumnSelectorPluses( + ColumnSelectorPlus[] selectorPluses = + DimensionHandlerUtils.createColumnSelectorPluses( STRATEGY_FACTORY, fields, columnFactory - )); + ); - if (selectorPlusList.isEmpty()) { - return Aggregators.noopBufferAggregator(); + if (selectorPluses.length == 0) { + return NoopBufferAggregator.instance(); } - - return new CardinalityBufferAggregator(selectorPlusList, byRow); + return new CardinalityBufferAggregator(selectorPluses, byRow); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityBufferAggregator.java index cbe522806161..6a2546cb8993 100644 --- a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityBufferAggregator.java @@ -20,26 +20,26 @@ package io.druid.query.aggregation.cardinality; import io.druid.hll.HyperLogLogCollector; -import io.druid.query.aggregation.BufferAggregator; import io.druid.query.ColumnSelectorPlus; +import io.druid.query.aggregation.BufferAggregator; import io.druid.query.aggregation.cardinality.types.CardinalityAggregatorColumnSelectorStrategy; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import java.nio.ByteBuffer; -import java.util.List; public class CardinalityBufferAggregator implements BufferAggregator { - private final List> selectorPlusList; + private final ColumnSelectorPlus[] selectorPluses; private final boolean byRow; private static final byte[] EMPTY_BYTES = HyperLogLogCollector.makeEmptyVersionedByteArray(); - public CardinalityBufferAggregator( - List> selectorPlusList, + CardinalityBufferAggregator( + ColumnSelectorPlus[] selectorPluses, boolean byRow ) { - this.selectorPlusList = selectorPlusList; + this.selectorPluses = selectorPluses; this.byRow = byRow; } @@ -63,9 +63,9 @@ public void aggregate(ByteBuffer buf, int position) try { final HyperLogLogCollector collector = HyperLogLogCollector.makeCollector(buf); if (byRow) { - CardinalityAggregator.hashRow(selectorPlusList, collector); + CardinalityAggregator.hashRow(selectorPluses, collector); } else { - CardinalityAggregator.hashValues(selectorPlusList, collector); + CardinalityAggregator.hashValues(selectorPluses, collector); } } finally { @@ -102,4 +102,10 @@ public void close() { // no resources to cleanup } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selectorPluses", selectorPluses); + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/first/DoubleFirstAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/first/DoubleFirstAggregatorFactory.java index f63c398adb37..3bdaba5ef64d 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/DoubleFirstAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/DoubleFirstAggregatorFactory.java @@ -30,6 +30,7 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactoryNotMergeableException; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.ObjectColumnSelector; import io.druid.segment.column.Column; @@ -148,6 +149,12 @@ public void aggregate(ByteBuffer buf, int position) buf.putDouble(position + Longs.BYTES, pair.rhs); } } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } }; } }; diff --git a/processing/src/main/java/io/druid/query/aggregation/first/DoubleFirstBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/first/DoubleFirstBufferAggregator.java index 8349dad65f1d..4baac526da7e 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/DoubleFirstBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/DoubleFirstBufferAggregator.java @@ -22,6 +22,7 @@ import com.google.common.primitives.Longs; import io.druid.collections.SerializablePair; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.FloatColumnSelector; import io.druid.segment.LongColumnSelector; @@ -79,4 +80,11 @@ public void close() { // no resources to cleanup } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("timeSelector", timeSelector); + inspector.visit("valueSelector", valueSelector); + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/first/LongFirstAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/first/LongFirstAggregatorFactory.java index e1d1f1871075..605fdcf85bc1 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/LongFirstAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/LongFirstAggregatorFactory.java @@ -29,6 +29,7 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactoryNotMergeableException; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.ObjectColumnSelector; import io.druid.segment.column.Column; @@ -138,6 +139,12 @@ public void aggregate(ByteBuffer buf, int position) buf.putLong(position + Longs.BYTES, pair.rhs); } } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } }; } }; diff --git a/processing/src/main/java/io/druid/query/aggregation/first/LongFirstBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/first/LongFirstBufferAggregator.java index e7f750660f42..5808ff148a95 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/LongFirstBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/LongFirstBufferAggregator.java @@ -22,6 +22,7 @@ import com.google.common.primitives.Longs; import io.druid.collections.SerializablePair; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.LongColumnSelector; import java.nio.ByteBuffer; @@ -78,4 +79,11 @@ public void close() { // no resources to cleanup } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("timeSelector", timeSelector); + inspector.visit("valueSelector", valueSelector); + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java index 444aae09f91c..b938807544ec 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java @@ -28,8 +28,9 @@ import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactoryNotMergeableException; -import io.druid.query.aggregation.Aggregators; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.aggregation.NoopAggregator; +import io.druid.query.aggregation.NoopBufferAggregator; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.ObjectColumnSelector; import org.apache.commons.codec.binary.Base64; @@ -85,7 +86,7 @@ public Aggregator factorize(ColumnSelectorFactory metricFactory) ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName); if (selector == null) { - return Aggregators.noopAggregator(); + return NoopAggregator.instance(); } final Class classOfObject = selector.classOfObject(); @@ -104,7 +105,7 @@ public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName); if (selector == null) { - return Aggregators.noopBufferAggregator(); + return NoopBufferAggregator.instance(); } final Class classOfObject = selector.classOfObject(); diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java index b700dc936cbd..5379e65ff673 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java @@ -21,6 +21,7 @@ import io.druid.hll.HyperLogLogCollector; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ObjectColumnSelector; import java.nio.ByteBuffer; @@ -102,4 +103,10 @@ public void close() { // no resources to cleanup } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/last/DoubleLastAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/last/DoubleLastAggregatorFactory.java index bacdfc66b195..8c74a9f7312a 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/DoubleLastAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/DoubleLastAggregatorFactory.java @@ -32,6 +32,7 @@ import io.druid.query.aggregation.BufferAggregator; import io.druid.query.aggregation.first.DoubleFirstAggregatorFactory; import io.druid.query.aggregation.first.LongFirstAggregatorFactory; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.ObjectColumnSelector; import io.druid.segment.column.Column; @@ -131,6 +132,12 @@ public void aggregate(ByteBuffer buf, int position) buf.putDouble(position + Longs.BYTES, pair.rhs); } } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } }; } }; diff --git a/processing/src/main/java/io/druid/query/aggregation/last/DoubleLastBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/last/DoubleLastBufferAggregator.java index e02118e4ae00..84f86d3caaea 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/DoubleLastBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/DoubleLastBufferAggregator.java @@ -22,6 +22,7 @@ import com.google.common.primitives.Longs; import io.druid.collections.SerializablePair; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.FloatColumnSelector; import io.druid.segment.LongColumnSelector; @@ -79,4 +80,11 @@ public void close() { // no resources to cleanup } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("timeSelector", timeSelector); + inspector.visit("valueSelector", valueSelector); + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/last/LongLastAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/last/LongLastAggregatorFactory.java index e4d46815e60b..6450150ab5ae 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/LongLastAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/LongLastAggregatorFactory.java @@ -31,6 +31,7 @@ import io.druid.query.aggregation.BufferAggregator; import io.druid.query.aggregation.first.DoubleFirstAggregatorFactory; import io.druid.query.aggregation.first.LongFirstAggregatorFactory; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.ObjectColumnSelector; import io.druid.segment.column.Column; @@ -131,6 +132,12 @@ public void aggregate(ByteBuffer buf, int position) buf.putLong(position + Longs.BYTES, pair.rhs); } } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } }; } }; diff --git a/processing/src/main/java/io/druid/query/aggregation/last/LongLastBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/last/LongLastBufferAggregator.java index 8bc09828eeb4..82892e2c503e 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/LongLastBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/LongLastBufferAggregator.java @@ -22,6 +22,7 @@ import com.google.common.primitives.Longs; import io.druid.collections.SerializablePair; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.LongColumnSelector; import java.nio.ByteBuffer; @@ -78,4 +79,11 @@ public void close() { // no resources to cleanup } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("timeSelector", timeSelector); + inspector.visit("valueSelector", valueSelector); + } } diff --git a/processing/src/main/java/io/druid/query/dimension/ForwardingFilteredDimensionSelector.java b/processing/src/main/java/io/druid/query/dimension/ForwardingFilteredDimensionSelector.java index ffb9e56f5d4f..52c489cce6eb 100644 --- a/processing/src/main/java/io/druid/query/dimension/ForwardingFilteredDimensionSelector.java +++ b/processing/src/main/java/io/druid/query/dimension/ForwardingFilteredDimensionSelector.java @@ -24,6 +24,7 @@ import com.google.common.base.Predicates; import io.druid.java.util.common.IAE; import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.DimensionSelector; import io.druid.segment.DimensionSelectorUtils; import io.druid.segment.IdLookup; @@ -173,4 +174,11 @@ public int lookupId(String name) { return forwardMapping.get(baseIdLookup.lookupId(name)); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + inspector.visit("forwardMapping", forwardMapping); + } } diff --git a/processing/src/main/java/io/druid/query/dimension/PredicateFilteredDimensionSelector.java b/processing/src/main/java/io/druid/query/dimension/PredicateFilteredDimensionSelector.java index a46accc04914..44b9fc347471 100644 --- a/processing/src/main/java/io/druid/query/dimension/PredicateFilteredDimensionSelector.java +++ b/processing/src/main/java/io/druid/query/dimension/PredicateFilteredDimensionSelector.java @@ -21,6 +21,7 @@ import com.google.common.base.Predicate; import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.DimensionSelector; import io.druid.segment.IdLookup; import io.druid.segment.data.ArrayBasedIndexedInts; @@ -132,4 +133,11 @@ public IdLookup idLookup() { return selector.idLookup(); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + inspector.visit("predicate", predicate); + } } diff --git a/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java b/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java index 959aeeea5996..d8d15070a8bd 100644 --- a/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java +++ b/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java @@ -27,6 +27,7 @@ import io.druid.query.dimension.DimensionSpec; import io.druid.query.extraction.ExtractionFn; import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.DimensionSelector; import io.druid.segment.FloatColumnSelector; @@ -164,6 +165,13 @@ public IdLookup idLookup() { return null; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("row", row); + inspector.visit("extractionFn", extractionFn); + } }; } else { return new DimensionSelector() @@ -289,6 +297,13 @@ public IdLookup idLookup() { return null; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("row", row); + inspector.visit("extractionFn", extractionFn); + } }; } } @@ -296,17 +311,26 @@ public IdLookup idLookup() @Override public FloatColumnSelector makeFloatColumnSelector(final String columnName) { + abstract class RowBasedFloatColumnSelector implements FloatColumnSelector + { + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("row", row); + } + } if (columnName.equals(Column.TIME_COLUMN_NAME)) { - return new FloatColumnSelector() + class TimeFloatColumnSelector extends RowBasedFloatColumnSelector { @Override public float get() { return (float) row.get().getTimestampFromEpoch(); } - }; + } + return new TimeFloatColumnSelector(); } else { - return new FloatColumnSelector() + return new RowBasedFloatColumnSelector() { @Override public float get() @@ -320,17 +344,26 @@ public float get() @Override public LongColumnSelector makeLongColumnSelector(final String columnName) { + abstract class RowBasedLongColumnSelector implements LongColumnSelector + { + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("row", row); + } + } if (columnName.equals(Column.TIME_COLUMN_NAME)) { - return new LongColumnSelector() + class TimeLongColumnSelector extends RowBasedLongColumnSelector { @Override public long get() { return row.get().getTimestampFromEpoch(); } - }; + } + return new TimeLongColumnSelector(); } else { - return new LongColumnSelector() + return new RowBasedLongColumnSelector() { @Override public long get() 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 1b5cf9a2831c..623513ed8aa7 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 @@ -38,7 +38,7 @@ import io.druid.java.util.common.Pair; import io.druid.java.util.common.granularity.AllGranularity; import io.druid.java.util.common.guava.Accumulator; -import io.druid.query.QueryInterruptedException; +import io.druid.query.BaseQuery; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.dimension.DimensionSpec; import io.druid.query.groupby.GroupByQuery; @@ -152,9 +152,7 @@ public Grouper accumulate( final Row row ) { - if (Thread.interrupted()) { - throw new QueryInterruptedException(new InterruptedException()); - } + BaseQuery.checkInterrupted(); if (theGrouper == null) { // Pass-through null returns without doing more work. 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 c5a4317d58f9..47962d23c9f3 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 @@ -28,7 +28,7 @@ import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import io.druid.java.util.common.guava.CloseQuietly; -import io.druid.query.QueryInterruptedException; +import io.druid.query.BaseQuery; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.ColumnSelectorFactory; import net.jpountz.lz4.LZ4BlockInputStream; @@ -200,9 +200,7 @@ private void spill() throws IOException outFile = out.getFile(); final Iterator> it = grouper.iterator(true); while (it.hasNext()) { - if (Thread.interrupted()) { - throw new QueryInterruptedException(new InterruptedException()); - } + BaseQuery.checkInterrupted(); jsonGenerator.writeObject(it.next()); } diff --git a/processing/src/main/java/io/druid/query/monomorphicprocessing/CalledFromHotLoop.java b/processing/src/main/java/io/druid/query/monomorphicprocessing/CalledFromHotLoop.java new file mode 100644 index 000000000000..4f8650c36c1f --- /dev/null +++ b/processing/src/main/java/io/druid/query/monomorphicprocessing/CalledFromHotLoop.java @@ -0,0 +1,43 @@ +/* + * 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.monomorphicprocessing; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Annotation for methods, which are called from hot loops during query processing. Those methods and their downstream + * callees should be optimized as much as possible. + * + *

Currently CalledFromHotLoop is a source-level annotation, used only for documentation. + * + * @see HotLoopCallee + */ +@Documented +@Inherited +@Retention(RetentionPolicy.SOURCE) +@Target(ElementType.METHOD) +public @interface CalledFromHotLoop +{ +} diff --git a/processing/src/main/java/io/druid/query/monomorphicprocessing/HotLoopCallee.java b/processing/src/main/java/io/druid/query/monomorphicprocessing/HotLoopCallee.java new file mode 100644 index 000000000000..af82900fc803 --- /dev/null +++ b/processing/src/main/java/io/druid/query/monomorphicprocessing/HotLoopCallee.java @@ -0,0 +1,43 @@ +/* + * 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.monomorphicprocessing; + +/** + * Marker interface for abstractions, which are called from hot loops during query processing. Some of the methods of + * interfaces extending HotLoopCallee should be annotated with {@link CalledFromHotLoop}. + */ +public interface HotLoopCallee +{ + /** + * Implementations of this method should call {@code inspector.visit()} with all fields of this class, which meet two + * conditions: + * 1. They are used in methods of this class, annotated with {@link CalledFromHotLoop} + * 2. They are either: + * a. Nullable objects + * b. Instances of HotLoopCallee + * c. Objects, which don't always have a specific class in runtime. For example, a field of type {@link + * java.util.Set} could be {@link java.util.HashSet} or {@link java.util.TreeSet} in runtime, depending on how + * this instance (the instance on which inspectRuntimeShape() is called) is configured. + * d. ByteBuffer or similar objects, where byte order matters + * e. boolean flags, affecting branch taking + * f. Arrays of objects, meeting conditions any of conditions a-e. + */ + void inspectRuntimeShape(RuntimeShapeInspector inspector); +} diff --git a/processing/src/main/java/io/druid/query/monomorphicprocessing/RuntimeShapeInspector.java b/processing/src/main/java/io/druid/query/monomorphicprocessing/RuntimeShapeInspector.java new file mode 100644 index 000000000000..44d9fbeb9192 --- /dev/null +++ b/processing/src/main/java/io/druid/query/monomorphicprocessing/RuntimeShapeInspector.java @@ -0,0 +1,44 @@ +/* + * 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.monomorphicprocessing; + +import javax.annotation.Nullable; + +/** + * @see HotLoopCallee#inspectRuntimeShape(RuntimeShapeInspector) + */ +public interface RuntimeShapeInspector +{ + void visit(String fieldName, @Nullable HotLoopCallee value); + + void visit(String fieldName, @Nullable Object value); + + void visit(String fieldName, T[] values); + + void visit(String flagName, boolean flagValue); + + /** + * To be called from {@link HotLoopCallee#inspectRuntimeShape(RuntimeShapeInspector)} with something, that is + * important to ensure monomorphism and predictable branch taking in hot loops, but doesn't apply to other visit() + * methods in RuntimeShapeInspector. For example, {@link io.druid.segment.BitmapOffset#inspectRuntimeShape} reports + * bitmap population via this method, to ensure predictable branch taking inside Bitmap's iterators. + */ + void visit(String key, String runtimeShape); +} diff --git a/processing/src/main/java/io/druid/query/monomorphicprocessing/SpecializationService.java b/processing/src/main/java/io/druid/query/monomorphicprocessing/SpecializationService.java new file mode 100644 index 000000000000..e2d00b05031f --- /dev/null +++ b/processing/src/main/java/io/druid/query/monomorphicprocessing/SpecializationService.java @@ -0,0 +1,401 @@ +/* + * 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.monomorphicprocessing; + +import com.google.common.collect.ImmutableMap; +import com.google.common.io.ByteStreams; +import io.druid.concurrent.Execs; +import io.druid.java.util.common.logger.Logger; +import org.objectweb.asm.ClassReader; +import org.objectweb.asm.ClassVisitor; +import org.objectweb.asm.ClassWriter; +import org.objectweb.asm.commons.ClassRemapper; +import org.objectweb.asm.commons.SimpleRemapper; +import sun.misc.Unsafe; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.io.InputStream; +import java.lang.reflect.Field; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Manages class specialization during query processing. + * Usage: + * + * String runtimeShape = stringRuntimeShape.of(bufferAggregator); + * SpecializationState specializationState = SpecializationService.getSpecializationState( + * ProcessingAlgorithmImpl.class, + * runtimeShape + * ); + * ProcessingAlgorithm algorithm = specializationState.getSpecializedOrDefault(new ProcessingAlgorithmImpl()); + * long loopIterations = new ProcessingAlgorithmImpl().run(bufferAggregator, ...); + * specializationState.accountLoopIterations(loopIterations); + * + * ProcessingAlgorithmImpl.class, passed as prototypeClass to {@link #getSpecializationState} methods must have public + * no-arg constructor and must be stateless (no fields). + * + * @see SpecializationState + */ +public final class SpecializationService +{ + private static final Logger LOG = new Logger(SpecializationService.class); + + private static final Unsafe UNSAFE; + static { + try { + Field theUnsafe = Unsafe.class.getDeclaredField("theUnsafe"); + theUnsafe.setAccessible(true); + UNSAFE = (Unsafe) theUnsafe.get(null); + } + catch (Exception e) { + throw new RuntimeException("Cannot access Unsafe methods", e); + } + } + + /** + * If true, specialization is not actually done, an instance of prototypeClass is used as a "specialized" instance. + * Useful for analysis of generated assembly with JITWatch (https://github.com/AdoptOpenJDK/jitwatch), because + * JITWatch shows only classes present in the loaded JAR (prototypeClass should be), not classes generated during + * runtime. + */ + private static final boolean fakeSpecialize = Boolean.getBoolean("fakeSpecialize"); + + /** + * Number of loop iterations, accounted via {@link SpecializationState#accountLoopIterations(long)} in + * {@link WindowedLoopIterationCounter} during the last hour window, after which WindowedLoopIterationCounter decides + * to specialize class for the specific runtimeShape. The default value is chosen to be so that the specialized + * class will likely be compiled with C2 HotSpot compiler with the default values of *BackEdgeThreshold options. + */ + private static final int triggerSpecializationIterationsThreshold = + Integer.getInteger("triggerSpecializationIterationsThreshold", 10_000); + + /** + * The maximum number of specializations, that this service is allowed to make. It's not unlimited because each + * specialization takes some JVM memory (machine code cache, byte code, etc.) + */ + private static final int maxSpecializations = Integer.getInteger("maxSpecializations", 1000); + + private static final ExecutorService classSpecializationExecutor = Execs.singleThreaded("class-specialization-%d"); + + private static final AtomicLong specializedClassCounter = new AtomicLong(); + + private static final ClassValue perPrototypeClassState = + new ClassValue() + { + @Override + protected PerPrototypeClassState computeValue(Class type) + { + return new PerPrototypeClassState<>(type); + } + }; + + /** + * @param type of query processing algorithm + * @see SpecializationService class-level javadoc for details + */ + public static SpecializationState getSpecializationState( + Class prototypeClass, + String runtimeShape + ) + { + return getSpecializationState(prototypeClass, runtimeShape, ImmutableMap., Class>of()); + } + + /** + * @param classRemapping classes, that should be replaced in the bytecode of the given prototypeClass when specialized + * @see #getSpecializationState(Class, String) + */ + @SuppressWarnings("unchecked") + public static SpecializationState getSpecializationState( + Class prototypeClass, + String runtimeShape, + ImmutableMap, Class> classRemapping + ) + { + return perPrototypeClassState.get(prototypeClass).getSpecializationState(runtimeShape, classRemapping); + } + + static class PerPrototypeClassState + { + private final Class prototypeClass; + private final ConcurrentMap> specializationStates = + new ConcurrentHashMap<>(); + private final String prototypeClassBytecodeName; + private final String specializedClassNamePrefix; + + private byte[] prototypeClassBytecode; + + PerPrototypeClassState(Class prototypeClass) + { + this.prototypeClass = prototypeClass; + String prototypeClassName = prototypeClass.getName(); + prototypeClassBytecodeName = classBytecodeName(prototypeClassName); + specializedClassNamePrefix = prototypeClassName + "$Copy"; + } + + SpecializationState getSpecializationState(String runtimeShape, ImmutableMap, Class> classRemapping) + { + SpecializationId specializationId = new SpecializationId(runtimeShape, classRemapping); + SpecializationState alreadyExistingState = specializationStates.get(specializationId); + if (alreadyExistingState != null) { + return alreadyExistingState; + } + return specializationStates.computeIfAbsent(specializationId, id -> new WindowedLoopIterationCounter<>(this, id)); + } + + T specialize(ImmutableMap, Class> classRemapping) + { + String specializedClassName = specializedClassNamePrefix + specializedClassCounter.get(); + ClassWriter specializedClassWriter = new ClassWriter(0); + SimpleRemapper remapper = new SimpleRemapper(createRemapping(classRemapping, specializedClassName)); + ClassVisitor classTransformer = new ClassRemapper(specializedClassWriter, remapper); + try { + ClassReader prototypeClassReader = new ClassReader(getPrototypeClassBytecode()); + prototypeClassReader.accept(classTransformer, 0); + byte[] specializedClassBytecode = specializedClassWriter.toByteArray(); + Class specializedClass = defineClass(specializedClassName, specializedClassBytecode); + specializedClassCounter.incrementAndGet(); + return specializedClass.newInstance(); + } + catch (InstantiationException | IllegalAccessException | IOException e) { + throw new RuntimeException(e); + } + } + + private HashMap createRemapping( + ImmutableMap, Class> classRemapping, + String specializedClassName + ) + { + HashMap remapping = new HashMap<>(); + remapping.put(prototypeClassBytecodeName, classBytecodeName(specializedClassName)); + for (Map.Entry, Class> classRemappingEntry : classRemapping.entrySet()) { + Class sourceClass = classRemappingEntry.getKey(); + Class remappingClass = classRemappingEntry.getValue(); + remapping.put(classBytecodeName(sourceClass.getName()), classBytecodeName(remappingClass.getName())); + } + return remapping; + } + + @SuppressWarnings("unchecked") + private Class defineClass(String specializedClassName, byte[] specializedClassBytecode) + { + return (Class) UNSAFE.defineClass( + specializedClassName, + specializedClassBytecode, + 0, + specializedClassBytecode.length, + prototypeClass.getClassLoader(), + prototypeClass.getProtectionDomain() + ); + } + + /** + * No synchronization, because {@link #specialize} is called only from {@link #classSpecializationExecutor}, i. e. + * from a single thread. + */ + byte[] getPrototypeClassBytecode() throws IOException + { + if (prototypeClassBytecode == null) { + ClassLoader cl = prototypeClass.getClassLoader(); + try (InputStream prototypeClassBytecodeStream = + cl.getResourceAsStream(prototypeClassBytecodeName + ".class")) { + prototypeClassBytecode = ByteStreams.toByteArray(prototypeClassBytecodeStream); + } + } + return prototypeClassBytecode; + } + + private static String classBytecodeName(String className) + { + return className.replace('.', '/'); + } + } + + private static class SpecializationId + { + private final String runtimeShape; + private final ImmutableMap, Class> classRemapping; + private final int hashCode; + + private SpecializationId(String runtimeShape, ImmutableMap, Class> classRemapping) + { + this.runtimeShape = runtimeShape; + this.classRemapping = classRemapping; + this.hashCode = runtimeShape.hashCode() * 1000003 + classRemapping.hashCode(); + } + + @Override + public boolean equals(Object obj) + { + if (!(obj instanceof SpecializationId)) { + return false; + } + SpecializationId other = (SpecializationId) obj; + return runtimeShape.equals(other.runtimeShape) && classRemapping.equals(other.classRemapping); + } + + @Override + public int hashCode() + { + return hashCode; + } + } + + /** + * Accumulates the number of iterations during the last hour. (Window size = 1 hour) + */ + static class WindowedLoopIterationCounter extends SpecializationState implements Runnable + { + private final PerPrototypeClassState perPrototypeClassState; + private final SpecializationId specializationId; + /** A map with the number of iterations per each minute during the last hour */ + private final ConcurrentMap perMinuteIterations = new ConcurrentHashMap<>(); + private final AtomicBoolean specializationScheduled = new AtomicBoolean(false); + + WindowedLoopIterationCounter( + PerPrototypeClassState perPrototypeClassState, + SpecializationId specializationId + ) + { + this.perPrototypeClassState = perPrototypeClassState; + this.specializationId = specializationId; + } + + @Nullable + @Override + public T getSpecialized() + { + return null; + } + + @Override + public void accountLoopIterations(long loopIterations) + { + if (specializationScheduled.get()) { + return; + } + if (loopIterations > triggerSpecializationIterationsThreshold || + addAndGetTotalIterationsOverTheLastHour(loopIterations) > triggerSpecializationIterationsThreshold) { + if (specializationScheduled.compareAndSet(false, true)) { + classSpecializationExecutor.submit(this); + } + } + } + + private long addAndGetTotalIterationsOverTheLastHour(long newIterations) + { + long currentMillis = System.currentTimeMillis(); + long currentMinute = TimeUnit.MILLISECONDS.toMinutes(currentMillis); + long minuteOneHourAgo = currentMinute - TimeUnit.HOURS.toMinutes(1); + long totalIterations = 0; + boolean currentMinutePresent = false; + for (Iterator> it = perMinuteIterations.entrySet().iterator(); it.hasNext(); ) { + Map.Entry minuteStats = it.next(); + long minute = minuteStats.getKey(); + if (minute < minuteOneHourAgo) { + it.remove(); + } else if (minute == currentMinute) { + totalIterations += minuteStats.getValue().addAndGet(newIterations); + currentMinutePresent = true; + } else { + totalIterations += minuteStats.getValue().get(); + } + } + if (!currentMinutePresent) { + perMinuteIterations.computeIfAbsent(currentMinute, AtomicLong::new).addAndGet(newIterations); + totalIterations += newIterations; + } + return totalIterations; + } + + @Override + public void run() + { + try { + T specialized; + if (specializedClassCounter.get() > maxSpecializations) { + // Don't specialize, just instantiate the prototype class and emit a warning. + // The "better" approach is probably to implement some kind of cache eviction from + // PerPrototypeClassState.specializationStates. But it might be that nobody ever hits even the current + // maxSpecializations limit, so implementing cache eviction is an unnecessary complexity. + specialized = perPrototypeClassState.prototypeClass.newInstance(); + LOG.warn( + "SpecializationService couldn't make more than [%d] specializations. " + + "Not doing specialization for runtime shape[%s] and class remapping[%s], using the prototype class[%s]", + maxSpecializations, + specializationId.runtimeShape, + specializationId.classRemapping, + perPrototypeClassState.prototypeClass + ); + } else if (fakeSpecialize) { + specialized = perPrototypeClassState.prototypeClass.newInstance(); + } else { + specialized = perPrototypeClassState.specialize(specializationId.classRemapping); + } + perPrototypeClassState.specializationStates.put(specializationId, new Specialized<>(specialized)); + } + catch (Exception e) { + LOG.error( + e, + "Error specializing prototype class[%s] for runtime shape[%s] and class remapping[%s]", + perPrototypeClassState.prototypeClass, + specializationId.runtimeShape, + specializationId.classRemapping + ); + } + } + } + + static class Specialized extends SpecializationState + { + private final T specialized; + + Specialized(T specialized) + { + this.specialized = specialized; + } + + @Override + public T getSpecialized() + { + return specialized; + } + + @Override + public void accountLoopIterations(long loopIterations) + { + // do nothing + } + } + + private SpecializationService() + { + } +} diff --git a/processing/src/main/java/io/druid/query/monomorphicprocessing/SpecializationState.java b/processing/src/main/java/io/druid/query/monomorphicprocessing/SpecializationState.java new file mode 100644 index 000000000000..bc12128d01ce --- /dev/null +++ b/processing/src/main/java/io/druid/query/monomorphicprocessing/SpecializationState.java @@ -0,0 +1,52 @@ +/* + * 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.monomorphicprocessing; + +import javax.annotation.Nullable; + +/** + * @param type of query processing algorithm + * @see SpecializationService + */ +public abstract class SpecializationState +{ + /** + * Returns an instance of specialized version of query processing algorithm, if available, null otherwise. + */ + @Nullable + public abstract T getSpecialized(); + + /** + * Returns an instance of specialized version of query processing algorithm, if available, defaultInstance otherwise. + */ + public final T getSpecializedOrDefault(T defaultInstance) + { + T specialized = getSpecialized(); + return specialized != null ? specialized : defaultInstance; + } + + /** + * Accounts the number of loop iterations, made when processing queries without specialized algorithm, i. e. after + * {@link #getSpecialized()} returned null. If sufficiently many loop iterations were made, {@link + * SpecializationService} decides that the algorithm is worth to be specialized, and {@link #getSpecialized()} will + * return non-null during later queries. + */ + public abstract void accountLoopIterations(long loopIterations); +} diff --git a/processing/src/main/java/io/druid/query/monomorphicprocessing/StringRuntimeShape.java b/processing/src/main/java/io/druid/query/monomorphicprocessing/StringRuntimeShape.java new file mode 100644 index 000000000000..7a209e7cd6fb --- /dev/null +++ b/processing/src/main/java/io/druid/query/monomorphicprocessing/StringRuntimeShape.java @@ -0,0 +1,193 @@ +/* + * 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.monomorphicprocessing; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; + +/** + * Class to be used to obtain String representation of runtime shape of one or several {@link HotLoopCallee}s. + * Example: + * + * String offsetShape = StringRuntimeShape.of(offset); + * String dimensionSelectorShape = StringRuntimeShape.of(dimensionSelector); + */ +public final class StringRuntimeShape +{ + + public static String of(HotLoopCallee hotLoopCallee) + { + return new Inspector().runtimeShapeOf(hotLoopCallee); + } + + public static String of(HotLoopCallee... hotLoopCallees) + { + return new Inspector().runtimeShapeOf(hotLoopCallees); + } + + private StringRuntimeShape() + { + } + + private static class Inspector implements RuntimeShapeInspector + { + private final StringBuilder sb = new StringBuilder(); + private int indent = 0; + + String runtimeShapeOf(HotLoopCallee hotLoopCallee) + { + visit(hotLoopCallee); + return sb.toString(); + } + + String runtimeShapeOf(HotLoopCallee[] hotLoopCallees) + { + for (HotLoopCallee hotLoopCallee : hotLoopCallees) { + visit(hotLoopCallee); + sb.append(",\n"); + } + sb.setLength(sb.length() - 2); + return sb.toString(); + } + + private void indent() + { + for (int i = 0; i < indent; i++) { + sb.append(' '); + } + } + + private void incrementIndent() + { + indent += 2; + } + + private void decrementIndent() + { + indent -= 2; + } + + private void visit(@Nullable Object value) + { + if (value == null) { + sb.append("null"); + return; + } + sb.append(value.getClass().getName()); + if (value instanceof HotLoopCallee) { + appendHotLoopCalleeShape((HotLoopCallee) value); + } else if (value instanceof ByteBuffer) { + // ByteBuffers are treated specially because the byte order is an important part of the runtime shape. + appendByteBufferShape((ByteBuffer) value); + } + } + + private void appendHotLoopCalleeShape(HotLoopCallee value) + { + sb.append(" {\n"); + int lengthBeforeInspection = sb.length(); + incrementIndent(); + value.inspectRuntimeShape(this); + decrementIndent(); + if (sb.length() == lengthBeforeInspection) { + // remove " {\n" + sb.setLength(lengthBeforeInspection - 3); + } else { + removeLastComma(); + indent(); + sb.append('}'); + } + } + + private void appendByteBufferShape(ByteBuffer byteBuffer) + { + sb.append(" {order: "); + sb.append(byteBuffer.order().toString()); + sb.append('}'); + } + + private void removeLastComma() + { + assert sb.charAt(sb.length() - 2) == ',' && sb.charAt(sb.length() - 1) == '\n'; + sb.setCharAt(sb.length() - 2, '\n'); + sb.setLength(sb.length() - 1); + } + + @Override + public void visit(String fieldName, @Nullable HotLoopCallee value) + { + visit(fieldName, (Object) value); + } + + @Override + public void visit(String fieldName, @Nullable Object value) + { + indent(); + sb.append(fieldName); + sb.append(": "); + visit(value); + sb.append(",\n"); + } + + @Override + public void visit(String fieldName, T[] values) + { + indent(); + sb.append(fieldName); + sb.append(": [\n"); + int lengthBeforeInspection = sb.length(); + incrementIndent(); + for (T value : values) { + indent(); + visit(value); + sb.append(",\n"); + } + decrementIndent(); + if (sb.length() == lengthBeforeInspection) { + sb.setCharAt(lengthBeforeInspection - 1, ']'); + } else { + removeLastComma(); + indent(); + sb.append(']'); + } + sb.append(",\n"); + } + + @Override + public void visit(String flagName, boolean flagValue) + { + indent(); + sb.append(flagName); + sb.append(": "); + sb.append(flagValue); + sb.append(",\n"); + } + + @Override + public void visit(String key, String runtimeShape) + { + indent(); + sb.append(key); + sb.append(": "); + sb.append(runtimeShape); + sb.append(",\n"); + } + } +} diff --git a/processing/src/main/java/io/druid/query/topn/Generic1AggPooledTopNScanner.java b/processing/src/main/java/io/druid/query/topn/Generic1AggPooledTopNScanner.java new file mode 100644 index 000000000000..296a4841b5e7 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/Generic1AggPooledTopNScanner.java @@ -0,0 +1,43 @@ +/* + * 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.topn; + +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.Cursor; +import io.druid.segment.DimensionSelector; + +import java.nio.ByteBuffer; + +public interface Generic1AggPooledTopNScanner +{ + /** + * @param aggregatorSize number of bytes required by aggregator for a single aggregation + * @param positions a cache for positions in resultsBuffer, where specific (indexed) dimension values are aggregated + * @return number of scanned rows, i. e. number of steps made with the given cursor + */ + long scanAndAggregate( + DimensionSelector dimensionSelector, + BufferAggregator aggregator, + int aggregatorSize, + Cursor cursor, + int[] positions, + ByteBuffer resultsBuffer + ); +} diff --git a/processing/src/main/java/io/druid/query/topn/Generic1AggPooledTopNScannerPrototype.java b/processing/src/main/java/io/druid/query/topn/Generic1AggPooledTopNScannerPrototype.java new file mode 100644 index 000000000000..e5269fd1706f --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/Generic1AggPooledTopNScannerPrototype.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.query.topn; + +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.Cursor; +import io.druid.segment.DimensionSelector; +import io.druid.segment.data.IndexedInts; + +import java.nio.ByteBuffer; + +public final class Generic1AggPooledTopNScannerPrototype implements Generic1AggPooledTopNScanner +{ + @Override + public long scanAndAggregate( + DimensionSelector dimensionSelector, + BufferAggregator aggregator, + int aggregatorSize, + Cursor cursor, + int[] positions, + ByteBuffer resultsBuffer + ) + { + long scannedRows = 0; + int positionToAllocate = 0; + while (!cursor.isDoneOrInterrupted()) { + final IndexedInts dimValues = dimensionSelector.getRow(); + final int dimSize = dimValues.size(); + for (int i = 0; i < dimSize; i++) { + int dimIndex = dimValues.get(i); + int position = positions[dimIndex]; + if (position >= 0) { + aggregator.aggregate(resultsBuffer, position); + } else if (position == TopNAlgorithm.INIT_POSITION_VALUE) { + positions[dimIndex] = positionToAllocate; + position = positionToAllocate; + aggregator.init(resultsBuffer, position); + aggregator.aggregate(resultsBuffer, position); + positionToAllocate += aggregatorSize; + } + } + scannedRows++; + cursor.advanceUninterruptibly(); + } + return scannedRows; + } +} diff --git a/processing/src/main/java/io/druid/query/topn/Generic2AggPooledTopNScanner.java b/processing/src/main/java/io/druid/query/topn/Generic2AggPooledTopNScanner.java new file mode 100644 index 000000000000..f465a35a2252 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/Generic2AggPooledTopNScanner.java @@ -0,0 +1,46 @@ +/* + * 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.topn; + +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.Cursor; +import io.druid.segment.DimensionSelector; + +import java.nio.ByteBuffer; + +public interface Generic2AggPooledTopNScanner +{ + /** + * @param aggregator1Size number of bytes required by aggregator1 for a single aggregation + * @param aggregator2Size number of bytes required by aggregator2 for a single aggregation + * @param positions a cache for positions in resultsBuffer, where specific (indexed) dimension values are aggregated + * @return number of scanned rows, i. e. number of steps made with the given cursor + */ + long scanAndAggregate( + DimensionSelector dimensionSelector, + BufferAggregator aggregator1, + int aggregator1Size, + BufferAggregator aggregator2, + int aggregator2Size, + Cursor cursor, + int[] positions, + ByteBuffer resultsBuffer + ); +} diff --git a/processing/src/main/java/io/druid/query/topn/Generic2AggPooledTopNScannerPrototype.java b/processing/src/main/java/io/druid/query/topn/Generic2AggPooledTopNScannerPrototype.java new file mode 100644 index 000000000000..d6a039e0f540 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/Generic2AggPooledTopNScannerPrototype.java @@ -0,0 +1,71 @@ +/* + * 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.topn; + +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.Cursor; +import io.druid.segment.DimensionSelector; +import io.druid.segment.data.IndexedInts; + +import java.nio.ByteBuffer; + +public final class Generic2AggPooledTopNScannerPrototype implements Generic2AggPooledTopNScanner +{ + @Override + public long scanAndAggregate( + DimensionSelector dimensionSelector, + BufferAggregator aggregator1, + int aggregator1Size, + BufferAggregator aggregator2, + int aggregator2Size, + Cursor cursor, + int[] positions, + ByteBuffer resultsBuffer + ) + { + int totalAggregatorsSize = aggregator1Size + aggregator2Size; + long scannedRows = 0; + int positionToAllocate = 0; + while (!cursor.isDoneOrInterrupted()) { + final IndexedInts dimValues = dimensionSelector.getRow(); + final int dimSize = dimValues.size(); + for (int i = 0; i < dimSize; i++) { + int dimIndex = dimValues.get(i); + int position = positions[dimIndex]; + if (position >= 0) { + aggregator1.aggregate(resultsBuffer, position); + aggregator2.aggregate(resultsBuffer, position + aggregator1Size); + } else if (position == TopNAlgorithm.INIT_POSITION_VALUE) { + positions[dimIndex] = positionToAllocate; + position = positionToAllocate; + aggregator1.init(resultsBuffer, position); + aggregator1.aggregate(resultsBuffer, position); + position += aggregator1Size; + aggregator2.init(resultsBuffer, position); + aggregator2.aggregate(resultsBuffer, position); + positionToAllocate += totalAggregatorsSize; + } + } + scannedRows++; + cursor.advanceUninterruptibly(); + } + return scannedRows; + } +} diff --git a/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java b/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java index 62737a295fb5..b14cfd8fb7f9 100644 --- a/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java +++ b/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java @@ -19,13 +19,18 @@ package io.druid.query.topn; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import io.druid.collections.ResourceHolder; import io.druid.collections.StupidPool; import io.druid.java.util.common.Pair; import io.druid.java.util.common.guava.CloseQuietly; -import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.BaseQuery; import io.druid.query.ColumnSelectorPlus; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.monomorphicprocessing.SpecializationService; +import io.druid.query.monomorphicprocessing.SpecializationState; +import io.druid.query.monomorphicprocessing.StringRuntimeShape; import io.druid.segment.Capabilities; import io.druid.segment.Cursor; import io.druid.segment.DimensionSelector; @@ -40,6 +45,19 @@ public class PooledTopNAlgorithm extends BaseTopNAlgorithm { + /** Non-final fields for testing, see TopNQueryRunnerTest */ + @VisibleForTesting + static boolean specializeGeneric1AggPooledTopN = + !Boolean.getBoolean("dontSpecializeGeneric1AggPooledTopN"); + @VisibleForTesting + static boolean specializeGeneric2AggPooledTopN = + !Boolean.getBoolean("dontSpecializeGeneric2AggPooledTopN"); + + private static final Generic1AggPooledTopNScanner defaultGeneric1AggScanner = + new Generic1AggPooledTopNScannerPrototype(); + private static final Generic2AggPooledTopNScanner defaultGeneric2AggScanner = + new Generic2AggPooledTopNScannerPrototype(); + private final Capabilities capabilities; private final TopNQuery query; private final StupidPool bufferPool; @@ -166,6 +184,75 @@ protected BufferAggregator[] makeDimValAggregateStore(PooledTopNParams params) { return makeBufferAggregators(params.getCursor(), query.getAggregatorSpecs()); } + + @Override + protected void scanAndAggregate( + final PooledTopNParams params, + final int[] positions, + final BufferAggregator[] theAggregators, + final int numProcessed + ) + { + final Cursor cursor = params.getCursor(); + if (specializeGeneric1AggPooledTopN && theAggregators.length == 1) { + scanAndAggregateGeneric1Agg(params, positions, theAggregators[0], cursor); + } else if (specializeGeneric2AggPooledTopN && theAggregators.length == 2) { + scanAndAggregateGeneric2Agg(params, positions, theAggregators, cursor); + } else { + scanAndAggregateDefault(params, positions, theAggregators); + } + BaseQuery.checkInterrupted(); + } + + private static void scanAndAggregateGeneric1Agg( + PooledTopNParams params, + int[] positions, + BufferAggregator aggregator, + Cursor cursor + ) + { + String runtimeShape = StringRuntimeShape.of(aggregator); + Class prototypeClass = Generic1AggPooledTopNScannerPrototype.class; + SpecializationState specializationState = SpecializationService + .getSpecializationState(prototypeClass, runtimeShape); + Generic1AggPooledTopNScanner scanner = specializationState.getSpecializedOrDefault(defaultGeneric1AggScanner); + long scannedRows = scanner.scanAndAggregate( + params.getDimSelector(), + aggregator, + params.getAggregatorSizes()[0], + cursor, + positions, + params.getResultsBuf() + ); + specializationState.accountLoopIterations(scannedRows); + } + + private static void scanAndAggregateGeneric2Agg( + PooledTopNParams params, + int[] positions, + BufferAggregator[] theAggregators, + Cursor cursor + ) + { + String runtimeShape = StringRuntimeShape.of(theAggregators); + Class prototypeClass = Generic2AggPooledTopNScannerPrototype.class; + SpecializationState specializationState = SpecializationService + .getSpecializationState(prototypeClass, runtimeShape); + Generic2AggPooledTopNScanner scanner = specializationState.getSpecializedOrDefault(defaultGeneric2AggScanner); + int[] aggregatorSizes = params.getAggregatorSizes(); + long scannedRows = scanner.scanAndAggregate( + params.getDimSelector(), + theAggregators[0], + aggregatorSizes[0], + theAggregators[1], + aggregatorSizes[1], + cursor, + positions, + params.getResultsBuf() + ); + specializationState.accountLoopIterations(scannedRows); + } + /** * Use aggressive loop unrolling to aggregate the data * @@ -184,12 +271,10 @@ protected BufferAggregator[] makeDimValAggregateStore(PooledTopNParams params) * still optimizes the high quantity of aggregate queries which benefit greatly from any speed improvements * (they simply take longer to start with). */ - @Override - protected void scanAndAggregate( + private static void scanAndAggregateDefault( final PooledTopNParams params, final int[] positions, - final BufferAggregator[] theAggregators, - final int numProcessed + final BufferAggregator[] theAggregators ) { if (params.getCardinality() < 0) { @@ -211,8 +296,7 @@ protected void scanAndAggregate( final int aggSize = theAggregators.length; final int aggExtra = aggSize % AGG_UNROLL_COUNT; int currentPosition = 0; - - while (!cursor.isDone()) { + while (!cursor.isDoneOrInterrupted()) { final IndexedInts dimValues = dimSelector.getRow(); final int dimSize = dimValues.size(); @@ -393,7 +477,7 @@ protected void scanAndAggregate( currentPosition ); } - cursor.advance(); + cursor.advanceUninterruptibly(); } } diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java index ae7100af27d9..bdf8dc6153a9 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java @@ -310,9 +310,10 @@ public TopNQueryBuilder granularity(Granularity g) return this; } - public TopNQueryBuilder aggregators(List a) + @SuppressWarnings("unchecked") + public TopNQueryBuilder aggregators(List a) { - aggregatorSpecs = a; + aggregatorSpecs = (List) a; return this; } diff --git a/processing/src/main/java/io/druid/segment/BitmapOffset.java b/processing/src/main/java/io/druid/segment/BitmapOffset.java index 5e4ae18309ba..8b1d02979523 100644 --- a/processing/src/main/java/io/druid/segment/BitmapOffset.java +++ b/processing/src/main/java/io/druid/segment/BitmapOffset.java @@ -23,10 +23,17 @@ import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.collections.bitmap.MutableBitmap; import io.druid.collections.bitmap.WrappedImmutableRoaringBitmap; +import io.druid.collections.bitmap.WrappedRoaringBitmap; +import io.druid.extendedset.intset.EmptyIntIterator; +import io.druid.java.util.common.RE; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.data.Offset; import io.druid.segment.data.RoaringBitmapSerdeFactory; import org.roaringbitmap.IntIterator; +import java.util.Arrays; +import java.util.HashSet; + /** */ public class BitmapOffset implements Offset @@ -34,12 +41,93 @@ public class BitmapOffset implements Offset private static final int INVALID_VALUE = -1; private static final BitmapFactory ROARING_BITMAP_FACTORY = new RoaringBitmapSerdeFactory(false).getBitmapFactory(); - private final IntIterator itr; - private final BitmapFactory bitmapFactory; - private final ImmutableBitmap bitmapIndex; - private final boolean descending; + /** + * Currently the default stops are not consciously optimized for the goals described in {@link #factorizeFullness}. + * They are chosen intuitively. There was no experimentation with different bitmapFullnessFactorizationStops. + * Experimentation and performance feedback with a different set of stops is welcome. + */ + private static final String DEFAULT_FULLNESS_FACTORIZATION_STOPS = "0.01,0.1,0.3,0.5,0.7,0.9,0.99"; + private static final double[] BITMAP_FULLNESS_FACTORIZATION_STOPS; + private static final String[] FACTORIZED_FULLNESS; + static + { + String stopString = System.getProperty("bitmapFullnessFactorizationStops", DEFAULT_FULLNESS_FACTORIZATION_STOPS); + String[] stopsArray = stopString.split(","); + if (stopsArray.length == 0) { + throw new RE("Empty bitmapFullnessFactorizationStops: " + stopString); + } + if (new HashSet<>(Arrays.asList(stopsArray)).size() != stopsArray.length) { + throw new RE("Non unique bitmapFullnessFactorizationStops: " + stopString); + } + + BITMAP_FULLNESS_FACTORIZATION_STOPS = new double[stopsArray.length]; + for (int i = 0; i < stopsArray.length; i++) { + String stop = stopsArray[i]; + BITMAP_FULLNESS_FACTORIZATION_STOPS[i] = Double.parseDouble(stop); + + } + Arrays.sort(BITMAP_FULLNESS_FACTORIZATION_STOPS); + + + double firstStop = BITMAP_FULLNESS_FACTORIZATION_STOPS[0]; + if (Double.isNaN(firstStop) || firstStop <= 0.0) { + throw new RE("First bitmapFullnessFactorizationStop[%d] should be > 0", firstStop); + } + double lastStop = BITMAP_FULLNESS_FACTORIZATION_STOPS[stopsArray.length - 1]; + if (Double.isNaN(lastStop) || lastStop >= 1) { + throw new RE("Last bitmapFullnessFactorizationStop[%d] should be < 1", lastStop); + } + + String prevStop = "0"; + FACTORIZED_FULLNESS = new String[stopsArray.length + 1]; + for (int i = 0; i < stopsArray.length; i++) { + String stop = String.valueOf(BITMAP_FULLNESS_FACTORIZATION_STOPS[i]); + FACTORIZED_FULLNESS[i] = "(" + prevStop + ", " + stop + "]"; + prevStop = stop; + } + FACTORIZED_FULLNESS[stopsArray.length] = "(" + prevStop + ", 1)"; + } + + /** + * Processing of queries with BitmapOffsets, whose Bitmaps has different factorized fullness (bucket), reported from + * this method, uses different copies of the same code, so JIT compiler analyzes and compiles the code for different + * factorized fullness separately. The goal is to capture frequency of abstraction usage in compressed bitmap + * algorithms, i. e. + * - "Zero sequence" vs. "Literal" vs. "One sequence" in {@link io.druid.extendedset.intset.ImmutableConciseSet} + * - {@link org.roaringbitmap.ArrayContainer} vs {@link org.roaringbitmap.BitmapContainer} in Roaring + * and then https://shipilev.net/blog/2015/black-magic-method-dispatch/ comes into play. The secondary goal is to + * capture HotSpot's thresholds, which it uses to compile conditional blocks differently inside bitmap impls. See + * https://bugs.openjdk.java.net/browse/JDK-6743900. The default BlockLayoutMinDiamondPercentage=20, i. e. if + * probability of taking some branch is less than 20%, it is moved out of the hot path (to save some icache?). + * + * On the other hand, we don't want to factor fullness into too small pieces, because + * - too little queries may fall into those small buckets, and they are not compiled with Hotspot's C2 compiler + * - if there are a lot of queries for each small factorized fullness and their copies of the code is compiled by + * C2, this pollutes code cache and takes time to perform too many compilations, while some of them likely produce + * identical code. + * + * Ideally there should be as much buckets as possible as long as Hotspot's C2 output for each bucket is different. + */ + private static String factorizeFullness(long bitmapCardinality, long numRows) + { + if (bitmapCardinality == 0) { + return "0"; + } else if (bitmapCardinality == numRows) { + return "1"; + } else { + double fullness = bitmapCardinality / (double) numRows; + int index = Arrays.binarySearch(BITMAP_FULLNESS_FACTORIZATION_STOPS, fullness); + if (index < 0) { + index = ~index; + } + return FACTORIZED_FULLNESS[index]; + } + } + + final IntIterator itr; + final String fullness; - private volatile int val; + int val; public static IntIterator getReverseBitmapOffsetIterator(ImmutableBitmap bitmapIndex) { @@ -55,16 +143,25 @@ public static IntIterator getReverseBitmapOffsetIterator(ImmutableBitmap bitmapI return ((WrappedImmutableRoaringBitmap) roaringBitmap).getBitmap().getReverseIntIterator(); } - public BitmapOffset(BitmapFactory bitmapFactory, ImmutableBitmap bitmapIndex, boolean descending) + public static BitmapOffset of(ImmutableBitmap bitmapIndex, boolean descending, long numRows) { - this.bitmapFactory = bitmapFactory; - this.bitmapIndex = bitmapIndex; - this.descending = descending; - this.itr = newIterator(); + if (bitmapIndex instanceof WrappedImmutableRoaringBitmap || + bitmapIndex instanceof WrappedRoaringBitmap || + descending) { + return new RoaringBitmapOffset(bitmapIndex, descending, numRows); + } else { + return new BitmapOffset(bitmapIndex, descending, numRows); + } + } + + private BitmapOffset(ImmutableBitmap bitmapIndex, boolean descending, long numRows) + { + this.itr = newIterator(bitmapIndex, descending); + this.fullness = factorizeFullness(bitmapIndex.size(), numRows); increment(); } - private IntIterator newIterator() + private IntIterator newIterator(ImmutableBitmap bitmapIndex, boolean descending) { if (!descending) { return bitmapIndex.iterator(); @@ -73,13 +170,11 @@ private IntIterator newIterator() } } - private BitmapOffset(BitmapOffset otherOffset) + private BitmapOffset(String fullness, IntIterator itr, int val) { - this.bitmapFactory = otherOffset.bitmapFactory; - this.bitmapIndex = otherOffset.bitmapIndex; - this.descending = otherOffset.descending; - this.itr = otherOffset.itr.clone(); - this.val = otherOffset.val; + this.fullness = fullness; + this.itr = itr; + this.val = val; } @Override @@ -101,11 +196,7 @@ public boolean withinBounds() @Override public Offset clone() { - if (bitmapIndex == null || bitmapIndex.size() == 0) { - return new BitmapOffset(bitmapFactory, bitmapFactory.makeEmptyImmutableBitmap(), descending); - } - - return new BitmapOffset(this); + return new BitmapOffset(fullness, itr.clone(), val); } @Override @@ -113,4 +204,31 @@ public int getOffset() { return val; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("itr", itr); + inspector.visit("fullness", fullness); + } + + public static class RoaringBitmapOffset extends BitmapOffset + { + + public RoaringBitmapOffset(ImmutableBitmap bitmapIndex, boolean descending, long numRows) + { + super(bitmapIndex, descending, numRows); + } + + RoaringBitmapOffset(String fullness, IntIterator itr, int val) + { + super(fullness, itr, val); + } + + @Override + public Offset clone() + { + return new RoaringBitmapOffset(fullness, itr.hasNext() ? itr.clone() : EmptyIntIterator.instance(), val); + } + } } diff --git a/processing/src/main/java/io/druid/segment/Cursor.java b/processing/src/main/java/io/druid/segment/Cursor.java index aa9b73926058..0af9f4a9710a 100644 --- a/processing/src/main/java/io/druid/segment/Cursor.java +++ b/processing/src/main/java/io/druid/segment/Cursor.java @@ -24,9 +24,11 @@ public interface Cursor extends ColumnSelectorFactory { - public DateTime getTime(); - public void advance(); - public void advanceTo(int offset); - public boolean isDone(); - public void reset(); + DateTime getTime(); + void advance(); + void advanceUninterruptibly(); + void advanceTo(int offset); + boolean isDone(); + boolean isDoneOrInterrupted(); + void reset(); } diff --git a/processing/src/main/java/io/druid/segment/DimensionSelector.java b/processing/src/main/java/io/druid/segment/DimensionSelector.java index 9341b2255adb..cce1f1c9c839 100644 --- a/processing/src/main/java/io/druid/segment/DimensionSelector.java +++ b/processing/src/main/java/io/druid/segment/DimensionSelector.java @@ -21,13 +21,15 @@ import com.google.common.base.Predicate; import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.CalledFromHotLoop; +import io.druid.query.monomorphicprocessing.HotLoopCallee; import io.druid.segment.data.IndexedInts; import javax.annotation.Nullable; /** */ -public interface DimensionSelector extends ColumnValueSelector +public interface DimensionSelector extends ColumnValueSelector, HotLoopCallee { public static int CARDINALITY_UNKNOWN = -1; @@ -38,6 +40,7 @@ public interface DimensionSelector extends ColumnValueSelector * * @return all values for the row as an IntBuffer */ + @CalledFromHotLoop public IndexedInts getRow(); /** @@ -90,6 +93,7 @@ public interface DimensionSelector extends ColumnValueSelector * @param id id to lookup the field name for * @return the field name for the given id */ + @CalledFromHotLoop public String lookupName(int id); /** diff --git a/processing/src/main/java/io/druid/segment/FloatColumnSelector.java b/processing/src/main/java/io/druid/segment/FloatColumnSelector.java index 608c6c1577d3..706bd3d7ca81 100644 --- a/processing/src/main/java/io/druid/segment/FloatColumnSelector.java +++ b/processing/src/main/java/io/druid/segment/FloatColumnSelector.java @@ -19,12 +19,16 @@ package io.druid.segment; +import io.druid.query.monomorphicprocessing.CalledFromHotLoop; +import io.druid.query.monomorphicprocessing.HotLoopCallee; + /** * An object that gets a metric value. Metric values are always floats and there is an assumption that the * FloatColumnSelector has a handle onto some other stateful object (e.g. an Offset) which is changing between calls * to get() (though, that doesn't have to be the case if you always want the same value...). */ -public interface FloatColumnSelector extends ColumnValueSelector +public interface FloatColumnSelector extends ColumnValueSelector, HotLoopCallee { + @CalledFromHotLoop public float get(); } diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java index 65faa810ec13..36e0d2ce3ca5 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java @@ -22,6 +22,7 @@ import io.druid.collections.bitmap.BitmapFactory; import io.druid.collections.bitmap.MutableBitmap; import io.druid.query.dimension.DimensionSpec; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.column.ValueType; import io.druid.segment.data.Indexed; import io.druid.segment.incremental.IncrementalIndex; @@ -115,6 +116,11 @@ public long get() float floatVal = (Float) dims[dimIndex]; return (long) floatVal; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } } return new IndexerLongColumnSelector(); @@ -140,6 +146,11 @@ public float get() return (Float) dims[dimIndex]; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } } return new IndexerFloatColumnSelector(); diff --git a/processing/src/main/java/io/druid/segment/FloatWrappingDimensionSelector.java b/processing/src/main/java/io/druid/segment/FloatWrappingDimensionSelector.java index 259744bb9acc..6a6a060695c2 100644 --- a/processing/src/main/java/io/druid/segment/FloatWrappingDimensionSelector.java +++ b/processing/src/main/java/io/druid/segment/FloatWrappingDimensionSelector.java @@ -20,6 +20,7 @@ package io.druid.segment; import io.druid.query.extraction.ExtractionFn; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.virtual.BaseSingleValueDimensionSelector; public class FloatWrappingDimensionSelector extends BaseSingleValueDimensionSelector @@ -42,4 +43,11 @@ protected String getValue() return extractionFn.apply(selector.get()); } } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + inspector.visit("extractionFn", extractionFn); + } } diff --git a/processing/src/main/java/io/druid/segment/LongColumnSelector.java b/processing/src/main/java/io/druid/segment/LongColumnSelector.java index 869ab49178e9..0da638e5290c 100644 --- a/processing/src/main/java/io/druid/segment/LongColumnSelector.java +++ b/processing/src/main/java/io/druid/segment/LongColumnSelector.java @@ -19,9 +19,13 @@ package io.druid.segment; +import io.druid.query.monomorphicprocessing.CalledFromHotLoop; +import io.druid.query.monomorphicprocessing.HotLoopCallee; + /** */ -public interface LongColumnSelector extends ColumnValueSelector +public interface LongColumnSelector extends ColumnValueSelector, HotLoopCallee { + @CalledFromHotLoop public long get(); } diff --git a/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java b/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java index d01e05f90f87..963c03383a9d 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java @@ -22,6 +22,7 @@ import io.druid.collections.bitmap.BitmapFactory; import io.druid.collections.bitmap.MutableBitmap; import io.druid.query.dimension.DimensionSpec; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.column.ValueType; import io.druid.segment.data.Indexed; import io.druid.segment.incremental.IncrementalIndex; @@ -114,6 +115,11 @@ public long get() return (Long) dims[dimIndex]; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } } return new IndexerLongColumnSelector(); @@ -140,6 +146,11 @@ public float get() long longVal = (Long) dims[dimIndex]; return (float) longVal; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } } return new IndexerFloatColumnSelector(); diff --git a/processing/src/main/java/io/druid/segment/LongWrappingDimensionSelector.java b/processing/src/main/java/io/druid/segment/LongWrappingDimensionSelector.java index 4783a8eb3ca5..5d7a34306266 100644 --- a/processing/src/main/java/io/druid/segment/LongWrappingDimensionSelector.java +++ b/processing/src/main/java/io/druid/segment/LongWrappingDimensionSelector.java @@ -20,6 +20,7 @@ package io.druid.segment; import io.druid.query.extraction.ExtractionFn; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.virtual.BaseSingleValueDimensionSelector; public class LongWrappingDimensionSelector extends BaseSingleValueDimensionSelector @@ -42,4 +43,11 @@ protected String getValue() return extractionFn.apply(selector.get()); } } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + inspector.visit("extractionFn", extractionFn); + } } diff --git a/processing/src/main/java/io/druid/segment/NullDimensionSelector.java b/processing/src/main/java/io/druid/segment/NullDimensionSelector.java index b1c7d03b006e..385bbad0c7d6 100644 --- a/processing/src/main/java/io/druid/segment/NullDimensionSelector.java +++ b/processing/src/main/java/io/druid/segment/NullDimensionSelector.java @@ -22,6 +22,7 @@ import com.google.common.base.Predicate; import com.google.common.base.Strings; import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.data.IndexedInts; import io.druid.segment.data.ZeroIndexedInts; import io.druid.segment.filter.BooleanValueMatcher; @@ -91,4 +92,9 @@ public int lookupId(String name) { return Strings.isNullOrEmpty(name) ? 0 : -1; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } } diff --git a/processing/src/main/java/io/druid/segment/ObjectColumnSelector.java b/processing/src/main/java/io/druid/segment/ObjectColumnSelector.java index 71258f4cd2f7..ec9ab94ab45a 100644 --- a/processing/src/main/java/io/druid/segment/ObjectColumnSelector.java +++ b/processing/src/main/java/io/druid/segment/ObjectColumnSelector.java @@ -19,8 +19,12 @@ package io.druid.segment; +import io.druid.query.monomorphicprocessing.CalledFromHotLoop; + public interface ObjectColumnSelector extends ColumnValueSelector { public Class classOfObject(); + + @CalledFromHotLoop public T get(); } diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java index 1cc59392dcc0..ffa6d527fe48 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java @@ -31,13 +31,14 @@ import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; -import io.druid.query.QueryInterruptedException; +import io.druid.query.BaseQuery; import io.druid.query.dimension.DimensionSpec; import io.druid.query.extraction.ExtractionFn; import io.druid.query.filter.BooleanFilter; import io.druid.query.filter.Filter; import io.druid.query.filter.RowOffsetMatcherFactory; import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.column.BitmapIndex; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; @@ -274,10 +275,10 @@ public Sequence makeCursors( offset = new NoFilterOffset(0, index.getNumRows(), descending); } else { // Use AndFilter.getBitmapIndex to intersect the preFilters to get its short-circuiting behavior. - offset = new BitmapOffset( - selector.getBitmapFactory(), + offset = BitmapOffset.of( AndFilter.getBitmapIndex(selector, preFilters), - descending + descending, + (long) getNumRows() ); } } @@ -472,10 +473,20 @@ private DimensionSelector makeDimensionSelectorUndecorated( final DictionaryEncodedColumn column = cachedColumn; + abstract class QueryableDimensionSelector implements DimensionSelector, IdLookup + { + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("column", column); + inspector.visit("cursorOffset", cursorOffset); + inspector.visit("extractionFn", extractionFn); + } + } if (column == null) { return NullDimensionSelector.instance(); } else if (columnDesc.getCapabilities().hasMultipleValues()) { - class MultiValueDimensionSelector implements DimensionSelector, IdLookup + class MultiValueDimensionSelector extends QueryableDimensionSelector { @Override public IndexedInts getRow() @@ -536,7 +547,7 @@ public int lookupId(String name) } return new MultiValueDimensionSelector(); } else { - class SingleValueDimensionSelector implements DimensionSelector, IdLookup + class SingleValueDimensionSelector extends QueryableDimensionSelector { @Override public IndexedInts getRow() @@ -690,6 +701,13 @@ public float get() { return metricVals.getFloatSingleValueRow(cursorOffset.getOffset()); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("metricVals", metricVals); + inspector.visit("cursorOffset", cursorOffset); + } }; } @@ -724,6 +742,13 @@ public long get() { return metricVals.getLongSingleValueRow(cursorOffset.getOffset()); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("metricVals", metricVals); + inspector.visit("cursorOffset", cursorOffset); + } }; } @@ -912,9 +937,13 @@ public DateTime getTime() @Override public void advance() { - if (Thread.interrupted()) { - throw new QueryInterruptedException(new InterruptedException()); - } + BaseQuery.checkInterrupted(); + cursorOffset.increment(); + } + + @Override + public void advanceUninterruptibly() + { cursorOffset.increment(); } @@ -934,6 +963,12 @@ public boolean isDone() return !cursorOffset.withinBounds(); } + @Override + public boolean isDoneOrInterrupted() + { + return isDone() || Thread.currentThread().isInterrupted(); + } + @Override public void reset() { @@ -981,15 +1016,28 @@ public DateTime getTime() @Override public void advance() { - if (Thread.interrupted()) { - throw new QueryInterruptedException(new InterruptedException()); - } + BaseQuery.checkInterrupted(); cursorOffset.increment(); while (!isDone()) { - if (Thread.interrupted()) { - throw new QueryInterruptedException(new InterruptedException()); + BaseQuery.checkInterrupted(); + if (filterMatcher.matches()) { + return; + } else { + cursorOffset.increment(); } + } + } + + @Override + public void advanceUninterruptibly() + { + if (Thread.currentThread().isInterrupted()) { + return; + } + cursorOffset.increment(); + + while (!isDoneOrInterrupted()) { if (filterMatcher.matches()) { return; } else { @@ -1014,6 +1062,12 @@ public boolean isDone() return !cursorOffset.withinBounds(); } + @Override + public boolean isDoneOrInterrupted() + { + return isDone() || Thread.currentThread().isInterrupted(); + } + @Override public void reset() { @@ -1166,6 +1220,14 @@ public Offset clone() { throw new IllegalStateException("clone"); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("baseOffset", baseOffset); + inspector.visit("timestamps", timestamps); + inspector.visit("allWithinThreshold", allWithinThreshold); + } } private static class AscendingTimestampCheckingOffset extends TimestampCheckingOffset @@ -1275,6 +1337,12 @@ public String toString() { return currentOffset + "/" + rowCount + (descending ? "(DSC)" : ""); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("descending", descending); + } } @Override diff --git a/processing/src/main/java/io/druid/segment/SingleScanTimeDimSelector.java b/processing/src/main/java/io/druid/segment/SingleScanTimeDimSelector.java index 43ffefa3be8f..9e4f296b2102 100644 --- a/processing/src/main/java/io/druid/segment/SingleScanTimeDimSelector.java +++ b/processing/src/main/java/io/druid/segment/SingleScanTimeDimSelector.java @@ -22,6 +22,7 @@ import com.google.common.base.Predicate; import io.druid.query.extraction.ExtractionFn; import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.data.IndexedInts; import io.druid.segment.data.SingleIndexedInt; @@ -156,4 +157,12 @@ public IdLookup idLookup() { return null; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + inspector.visit("extractionFn", extractionFn); + inspector.visit("descending", descending); + } } diff --git a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java index 0b12e24b6762..f7cd8b622fc3 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java @@ -32,6 +32,7 @@ import io.druid.query.dimension.DimensionSpec; import io.druid.query.extraction.ExtractionFn; import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.column.ValueType; import io.druid.segment.data.ArrayBasedIndexedInts; import io.druid.segment.data.Indexed; @@ -541,6 +542,12 @@ public int lookupId(String name) } return getEncodedValue(name, false); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("currEntry", currEntry); + } } return new IndexerDimensionSelector(); } diff --git a/processing/src/main/java/io/druid/segment/ZeroFloatColumnSelector.java b/processing/src/main/java/io/druid/segment/ZeroFloatColumnSelector.java index 888214862f81..295ecd23539d 100644 --- a/processing/src/main/java/io/druid/segment/ZeroFloatColumnSelector.java +++ b/processing/src/main/java/io/druid/segment/ZeroFloatColumnSelector.java @@ -19,6 +19,8 @@ package io.druid.segment; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; + public final class ZeroFloatColumnSelector implements FloatColumnSelector { private static final ZeroFloatColumnSelector INSTANCE = new ZeroFloatColumnSelector(); @@ -38,4 +40,9 @@ public float get() { return 0.0f; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } } diff --git a/processing/src/main/java/io/druid/segment/ZeroLongColumnSelector.java b/processing/src/main/java/io/druid/segment/ZeroLongColumnSelector.java index 911c0e24b265..fbf765fd93e0 100644 --- a/processing/src/main/java/io/druid/segment/ZeroLongColumnSelector.java +++ b/processing/src/main/java/io/druid/segment/ZeroLongColumnSelector.java @@ -19,6 +19,8 @@ package io.druid.segment; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; + public final class ZeroLongColumnSelector implements LongColumnSelector { private static final ZeroLongColumnSelector INSTANCE = new ZeroLongColumnSelector(); @@ -38,4 +40,9 @@ public long get() { return 0; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } } diff --git a/processing/src/main/java/io/druid/segment/column/GenericColumn.java b/processing/src/main/java/io/druid/segment/column/GenericColumn.java index 27ab6d7e3650..5407a5ad5d5a 100644 --- a/processing/src/main/java/io/druid/segment/column/GenericColumn.java +++ b/processing/src/main/java/io/druid/segment/column/GenericColumn.java @@ -19,6 +19,8 @@ package io.druid.segment.column; +import io.druid.query.monomorphicprocessing.CalledFromHotLoop; +import io.druid.query.monomorphicprocessing.HotLoopCallee; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedFloats; import io.druid.segment.data.IndexedLongs; @@ -27,17 +29,23 @@ /** */ -public interface GenericColumn extends Closeable +public interface GenericColumn extends HotLoopCallee, Closeable { public int length(); public ValueType getType(); public boolean hasMultipleValues(); + @CalledFromHotLoop public String getStringSingleValueRow(int rowNum); + @CalledFromHotLoop public Indexed getStringMultiValueRow(int rowNum); + @CalledFromHotLoop public float getFloatSingleValueRow(int rowNum); + @CalledFromHotLoop public IndexedFloats getFloatMultiValueRow(int rowNum); + @CalledFromHotLoop public long getLongSingleValueRow(int rowNum); + @CalledFromHotLoop public IndexedLongs getLongMultiValueRow(int rowNum); @Override diff --git a/processing/src/main/java/io/druid/segment/column/IndexedFloatsGenericColumn.java b/processing/src/main/java/io/druid/segment/column/IndexedFloatsGenericColumn.java index 160e4181e56c..535ff0318714 100644 --- a/processing/src/main/java/io/druid/segment/column/IndexedFloatsGenericColumn.java +++ b/processing/src/main/java/io/druid/segment/column/IndexedFloatsGenericColumn.java @@ -19,6 +19,7 @@ package io.druid.segment.column; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedFloats; import io.druid.segment.data.IndexedLongs; @@ -94,4 +95,10 @@ public void close() { column.close(); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("column", column); + } } diff --git a/processing/src/main/java/io/druid/segment/column/IndexedLongsGenericColumn.java b/processing/src/main/java/io/druid/segment/column/IndexedLongsGenericColumn.java index bc5691764959..13864a7d43a2 100644 --- a/processing/src/main/java/io/druid/segment/column/IndexedLongsGenericColumn.java +++ b/processing/src/main/java/io/druid/segment/column/IndexedLongsGenericColumn.java @@ -19,6 +19,7 @@ package io.druid.segment.column; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedFloats; import io.druid.segment.data.IndexedLongs; @@ -94,4 +95,10 @@ public void close() { column.close(); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("column", column); + } } diff --git a/processing/src/main/java/io/druid/segment/data/ArrayBasedOffset.java b/processing/src/main/java/io/druid/segment/data/ArrayBasedOffset.java index ffcd333cb9c1..41385ec77c7d 100644 --- a/processing/src/main/java/io/druid/segment/data/ArrayBasedOffset.java +++ b/processing/src/main/java/io/druid/segment/data/ArrayBasedOffset.java @@ -19,6 +19,8 @@ package io.druid.segment.data; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; + /** */ public class ArrayBasedOffset implements Offset @@ -67,4 +69,9 @@ public Offset clone() retVal.currIndex = currIndex; return retVal; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } } diff --git a/processing/src/main/java/io/druid/segment/data/IntersectingOffset.java b/processing/src/main/java/io/druid/segment/data/IntersectingOffset.java index 3db91fe22b77..f3da84d0a1e1 100644 --- a/processing/src/main/java/io/druid/segment/data/IntersectingOffset.java +++ b/processing/src/main/java/io/druid/segment/data/IntersectingOffset.java @@ -19,6 +19,8 @@ package io.druid.segment.data; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; + /** */ public class IntersectingOffset implements Offset { @@ -91,4 +93,11 @@ public Offset clone() final Offset rhsClone = rhs.clone(); return new IntersectingOffset(lhsClone, rhsClone); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("lhs", lhs); + inspector.visit("rhs", rhs); + } } diff --git a/processing/src/main/java/io/druid/segment/data/Offset.java b/processing/src/main/java/io/druid/segment/data/Offset.java index e6421e4c0f08..7e44dd7b8e7f 100644 --- a/processing/src/main/java/io/druid/segment/data/Offset.java +++ b/processing/src/main/java/io/druid/segment/data/Offset.java @@ -19,14 +19,18 @@ package io.druid.segment.data; +import io.druid.query.monomorphicprocessing.CalledFromHotLoop; + /** * The "mutable" version of a ReadableOffset. Introduces "increment()" and "withinBounds()" methods, which are * very similar to "next()" and "hasNext()" on the Iterator interface except increment() does not return a value. */ public interface Offset extends ReadableOffset { + @CalledFromHotLoop void increment(); + @CalledFromHotLoop boolean withinBounds(); Offset clone(); diff --git a/processing/src/main/java/io/druid/segment/data/ReadableOffset.java b/processing/src/main/java/io/druid/segment/data/ReadableOffset.java index f93892c302cc..753df459da00 100644 --- a/processing/src/main/java/io/druid/segment/data/ReadableOffset.java +++ b/processing/src/main/java/io/druid/segment/data/ReadableOffset.java @@ -19,6 +19,9 @@ package io.druid.segment.data; +import io.druid.query.monomorphicprocessing.CalledFromHotLoop; +import io.druid.query.monomorphicprocessing.HotLoopCallee; + /** * A ReadableOffset is an object that provides an integer offset, ostensibly as an index into an array. * @@ -26,8 +29,9 @@ * given to classes (e.g. FloatColumnSelector objects) by something which keeps a reference to the base Offset object * and increments it. */ -public interface ReadableOffset +public interface ReadableOffset extends HotLoopCallee { + @CalledFromHotLoop int getOffset(); } diff --git a/processing/src/main/java/io/druid/segment/data/UnioningOffset.java b/processing/src/main/java/io/druid/segment/data/UnioningOffset.java index 51fd4af2ef20..05be50016b5e 100644 --- a/processing/src/main/java/io/druid/segment/data/UnioningOffset.java +++ b/processing/src/main/java/io/druid/segment/data/UnioningOffset.java @@ -19,6 +19,8 @@ package io.druid.segment.data; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; + /** */ public class UnioningOffset implements Offset @@ -135,4 +137,11 @@ public Offset clone() return new UnioningOffset(newOffsets, newOffsetValues, nextOffsetIndex); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("lhs", offsets[0]); + inspector.visit("rhs", offsets[1]); + } } diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java index dbbfa14eddab..ea524bbb7114 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -26,11 +26,12 @@ import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; -import io.druid.query.QueryInterruptedException; +import io.druid.query.BaseQuery; import io.druid.query.dimension.DimensionSpec; import io.druid.query.extraction.ExtractionFn; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.Capabilities; import io.druid.segment.Cursor; import io.druid.segment.DimensionHandler; @@ -273,8 +274,31 @@ public void advance() } while (baseIter.hasNext()) { - if (Thread.interrupted()) { - throw new QueryInterruptedException(new InterruptedException()); + BaseQuery.checkInterrupted(); + + currEntry.set(baseIter.next()); + + if (filterMatcher.matches()) { + return; + } + } + + if (!filterMatcher.matches()) { + done = true; + } + } + + @Override + public void advanceUninterruptibly() + { + if (!baseIter.hasNext()) { + done = true; + return; + } + + while (baseIter.hasNext()) { + if (Thread.currentThread().isInterrupted()) { + return; } currEntry.set(baseIter.next()); @@ -305,6 +329,12 @@ public boolean isDone() return done; } + @Override + public boolean isDoneOrInterrupted() + { + return isDone() || Thread.currentThread().isInterrupted(); + } + @Override public void reset() { @@ -316,9 +346,7 @@ public void reset() Iterators.advance(baseIter, numAdvanced); } - if (Thread.interrupted()) { - throw new QueryInterruptedException(new InterruptedException()); - } + BaseQuery.checkInterrupted(); boolean foundMatched = false; while (baseIter.hasNext()) { @@ -414,6 +442,12 @@ public float get() { return index.getMetricFloatValue(currEntry.getValue(), metricIndex); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("index", index); + } }; } @@ -425,14 +459,20 @@ public LongColumnSelector makeLongColumnSelector(String columnName) } if (columnName.equals(Column.TIME_COLUMN_NAME)) { - return new LongColumnSelector() + class TimeLongColumnSelector implements LongColumnSelector { @Override public long get() { return currEntry.getKey().getTimestamp(); } - }; + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } + } + return new TimeLongColumnSelector(); } final Integer dimIndex = index.getDimensionIndex(columnName); @@ -462,6 +502,12 @@ public long get() metricIndex ); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("index", index); + } }; } diff --git a/processing/src/main/java/io/druid/segment/virtual/BaseSingleValueDimensionSelector.java b/processing/src/main/java/io/druid/segment/virtual/BaseSingleValueDimensionSelector.java index 5fb64c653eaf..1207c275c9f8 100644 --- a/processing/src/main/java/io/druid/segment/virtual/BaseSingleValueDimensionSelector.java +++ b/processing/src/main/java/io/druid/segment/virtual/BaseSingleValueDimensionSelector.java @@ -21,6 +21,7 @@ import com.google.common.base.Predicate; import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.CalledFromHotLoop; import io.druid.segment.DimensionSelector; import io.druid.segment.IdLookup; import io.druid.segment.data.IndexedInts; @@ -31,6 +32,7 @@ public abstract class BaseSingleValueDimensionSelector implements DimensionSelector { + @CalledFromHotLoop protected abstract String getValue(); @Override diff --git a/processing/src/main/java/io/druid/segment/virtual/ExpressionSelectors.java b/processing/src/main/java/io/druid/segment/virtual/ExpressionSelectors.java index f1d02439b4e0..9db9f9ddab45 100644 --- a/processing/src/main/java/io/druid/segment/virtual/ExpressionSelectors.java +++ b/processing/src/main/java/io/druid/segment/virtual/ExpressionSelectors.java @@ -21,6 +21,7 @@ import io.druid.math.expr.Expr; import io.druid.query.extraction.ExtractionFn; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.DimensionSelector; import io.druid.segment.FloatColumnSelector; @@ -56,6 +57,12 @@ public long get() final Number number = baseSelector.get(); return number != null ? number.longValue() : nullValue; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("baseSelector", baseSelector); + } } return new ExpressionLongColumnSelector(); } @@ -75,6 +82,12 @@ public float get() final Number number = baseSelector.get(); return number != null ? number.floatValue() : nullValue; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("baseSelector", baseSelector); + } } return new ExpressionFloatColumnSelector(); } @@ -96,6 +109,12 @@ protected String getValue() final Number number = baseSelector.get(); return number == null ? null : String.valueOf(number); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("baseSelector", baseSelector); + } } return new DefaultExpressionDimensionSelector(); } else { @@ -106,6 +125,12 @@ protected String getValue() { return extractionFn.apply(baseSelector.get()); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("baseSelector", baseSelector); + } } return new ExtractionExpressionDimensionSelector(); } diff --git a/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java index f11553599847..b2db0636c671 100644 --- a/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java @@ -36,6 +36,7 @@ import io.druid.query.filter.SearchQueryDimFilter; import io.druid.query.filter.SelectorDimFilter; import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.query.ordering.StringComparators; import io.druid.query.search.search.ContainsSearchQuerySpec; import io.druid.segment.ColumnSelectorFactory; @@ -167,6 +168,11 @@ public int lookupId(String name) } }; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } } ); } else { diff --git a/processing/src/test/java/io/druid/query/aggregation/MetricManipulatorFnsTest.java b/processing/src/test/java/io/druid/query/aggregation/MetricManipulatorFnsTest.java index 76213c0624fd..9a2bb83e5efe 100644 --- a/processing/src/test/java/io/druid/query/aggregation/MetricManipulatorFnsTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/MetricManipulatorFnsTest.java @@ -21,7 +21,7 @@ import io.druid.hll.HyperLogLogCollector; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; -import io.druid.segment.LongColumnSelector; +import io.druid.segment.TestLongColumnSelector; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -42,7 +42,7 @@ public static Iterable constructorFeeder() final ArrayList constructorArrays = new ArrayList<>(); final long longVal = 13789; LongMinAggregator longMinAggregator = new LongMinAggregator( - new LongColumnSelector() + new TestLongColumnSelector() { @Override public long get() @@ -81,7 +81,7 @@ public long get() LongSumAggregatorFactory longSumAggregatorFactory = new LongSumAggregatorFactory(NAME, FIELD); LongSumAggregator longSumAggregator = new LongSumAggregator( - new LongColumnSelector() + new TestLongColumnSelector() { @Override public long get() diff --git a/processing/src/test/java/io/druid/query/aggregation/TestFloatColumnSelector.java b/processing/src/test/java/io/druid/query/aggregation/TestFloatColumnSelector.java index bcce469e0775..75477a9c00e4 100644 --- a/processing/src/test/java/io/druid/query/aggregation/TestFloatColumnSelector.java +++ b/processing/src/test/java/io/druid/query/aggregation/TestFloatColumnSelector.java @@ -19,11 +19,9 @@ package io.druid.query.aggregation; -import io.druid.segment.FloatColumnSelector; - /** */ -public class TestFloatColumnSelector implements FloatColumnSelector +public class TestFloatColumnSelector extends io.druid.segment.TestFloatColumnSelector { private final float[] floats; diff --git a/processing/src/test/java/io/druid/query/aggregation/TestLongColumnSelector.java b/processing/src/test/java/io/druid/query/aggregation/TestLongColumnSelector.java index 67c206d5d6cb..3a63282e1036 100644 --- a/processing/src/test/java/io/druid/query/aggregation/TestLongColumnSelector.java +++ b/processing/src/test/java/io/druid/query/aggregation/TestLongColumnSelector.java @@ -19,11 +19,9 @@ package io.druid.query.aggregation; -import io.druid.segment.LongColumnSelector; - /** */ -public class TestLongColumnSelector implements LongColumnSelector +public class TestLongColumnSelector extends io.druid.segment.TestLongColumnSelector { private final long[] longs; diff --git a/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorBenchmark.java b/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorBenchmark.java index 2687003d0ea6..718b358ea4be 100644 --- a/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorBenchmark.java +++ b/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorBenchmark.java @@ -44,7 +44,7 @@ public class CardinalityAggregatorBenchmark extends SimpleBenchmark CardinalityBufferAggregator agg; List selectorList; - List> dimInfoList; + ColumnSelectorPlus[] dimInfos; ByteBuffer buf; int pos; @@ -93,12 +93,9 @@ public String[] apply(Integer input) (DimensionSelector) dim1 ); - dimInfoList = Lists.newArrayList(dimInfo1); + dimInfos = new ColumnSelectorPlus[] {dimInfo1}; - agg = new CardinalityBufferAggregator( - dimInfoList, - byRow - ); + agg = new CardinalityBufferAggregator(dimInfos, byRow); CardinalityAggregatorFactory factory = new CardinalityAggregatorFactory( "billy", diff --git a/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java index f51b5db3aa0d..a19a7396fcd9 100644 --- a/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java @@ -43,6 +43,7 @@ import io.druid.query.extraction.JavaScriptExtractionFn; import io.druid.query.extraction.RegexDimExtractionFn; import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.DimensionSelector; import io.druid.segment.DimensionSelectorUtils; import io.druid.segment.IdLookup; @@ -205,6 +206,11 @@ public int lookupId(String s) } }; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } } /* @@ -418,7 +424,7 @@ public void testAggregateValues() throws Exception public void testBufferAggregateRows() throws Exception { CardinalityBufferAggregator agg = new CardinalityBufferAggregator( - dimInfoList, + dimInfoList.toArray(new ColumnSelectorPlus[] {}), true ); @@ -439,7 +445,7 @@ public void testBufferAggregateRows() throws Exception public void testBufferAggregateValues() throws Exception { CardinalityBufferAggregator agg = new CardinalityBufferAggregator( - dimInfoList, + dimInfoList.toArray(new ColumnSelectorPlus[] {}), false ); diff --git a/processing/src/test/java/io/druid/query/dimension/TestDimensionSelector.java b/processing/src/test/java/io/druid/query/dimension/TestDimensionSelector.java index 2498b98183ee..e31b51a7ac13 100644 --- a/processing/src/test/java/io/druid/query/dimension/TestDimensionSelector.java +++ b/processing/src/test/java/io/druid/query/dimension/TestDimensionSelector.java @@ -21,6 +21,7 @@ import com.google.common.base.Predicate; import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.DimensionSelector; import io.druid.segment.DimensionSelectorUtils; import io.druid.segment.IdLookup; @@ -92,4 +93,9 @@ public int lookupId(String name) } }; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } } diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/TestColumnSelectorFactory.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/TestColumnSelectorFactory.java index 77c57c0bf3b7..b3d5fb0dbd96 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/TestColumnSelectorFactory.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/TestColumnSelectorFactory.java @@ -26,6 +26,8 @@ import io.druid.segment.FloatColumnSelector; import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; +import io.druid.segment.TestFloatColumnSelector; +import io.druid.segment.TestLongColumnSelector; import io.druid.segment.column.ColumnCapabilities; public class TestColumnSelectorFactory implements ColumnSelectorFactory @@ -46,7 +48,7 @@ public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) @Override public FloatColumnSelector makeFloatColumnSelector(final String columnName) { - return new FloatColumnSelector() + return new TestFloatColumnSelector() { @Override public float get() @@ -59,7 +61,7 @@ public float get() @Override public LongColumnSelector makeLongColumnSelector(final String columnName) { - return new LongColumnSelector() + return new TestLongColumnSelector() { @Override public long get() diff --git a/processing/src/test/java/io/druid/query/monomorphicprocessing/StringRuntimeShapeTest.java b/processing/src/test/java/io/druid/query/monomorphicprocessing/StringRuntimeShapeTest.java new file mode 100644 index 000000000000..189230710a0b --- /dev/null +++ b/processing/src/test/java/io/druid/query/monomorphicprocessing/StringRuntimeShapeTest.java @@ -0,0 +1,113 @@ +/* + * 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.monomorphicprocessing; + +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.util.HashSet; +import java.util.Set; +import java.util.TreeSet; + +public class StringRuntimeShapeTest +{ + static class Empty implements HotLoopCallee + { + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } + } + + static class Foo implements HotLoopCallee + { + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("flag1", true); + inspector.visit("flag2", false); + inspector.visit("key", "value"); + inspector.visit("empty", new Empty()); + inspector.visit("object", ByteBuffer.allocate(1)); + inspector.visit("array", new Set[] {new HashSet(), new TreeSet()}); + inspector.visit("emptyArray", new Set[] {}); + } + } + + static class Bar implements HotLoopCallee + { + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("foo", new Foo()); + inspector.visit("array", new Foo[] {new Foo(), new Foo()}); + } + } + + @Test + public void testStringRuntimeShape() + { + String barRuntimeShape = StringRuntimeShape.of(new Bar()); + Assert.assertEquals( + "io.druid.query.monomorphicprocessing.StringRuntimeShapeTest$Bar {\n" + + " foo: io.druid.query.monomorphicprocessing.StringRuntimeShapeTest$Foo {\n" + + " flag1: true,\n" + + " flag2: false,\n" + + " key: value,\n" + + " empty: io.druid.query.monomorphicprocessing.StringRuntimeShapeTest$Empty,\n" + + " object: java.nio.HeapByteBuffer {order: BIG_ENDIAN},\n" + + " array: [\n" + + " java.util.HashSet,\n" + + " java.util.TreeSet\n" + + " ],\n" + + " emptyArray: []\n" + + " },\n" + + " array: [\n" + + " io.druid.query.monomorphicprocessing.StringRuntimeShapeTest$Foo {\n" + + " flag1: true,\n" + + " flag2: false,\n" + + " key: value,\n" + + " empty: io.druid.query.monomorphicprocessing.StringRuntimeShapeTest$Empty,\n" + + " object: java.nio.HeapByteBuffer {order: BIG_ENDIAN},\n" + + " array: [\n" + + " java.util.HashSet,\n" + + " java.util.TreeSet\n" + + " ],\n" + + " emptyArray: []\n" + + " },\n" + + " io.druid.query.monomorphicprocessing.StringRuntimeShapeTest$Foo {\n" + + " flag1: true,\n" + + " flag2: false,\n" + + " key: value,\n" + + " empty: io.druid.query.monomorphicprocessing.StringRuntimeShapeTest$Empty,\n" + + " object: java.nio.HeapByteBuffer {order: BIG_ENDIAN},\n" + + " array: [\n" + + " java.util.HashSet,\n" + + " java.util.TreeSet\n" + + " ],\n" + + " emptyArray: []\n" + + " }\n" + + " ]\n" + + "}", + barRuntimeShape + ); + } +} diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java index 06801ea55206..841efc6d4345 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java @@ -45,6 +45,7 @@ import io.druid.query.Result; import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleMaxAggregatorFactory; import io.druid.query.aggregation.DoubleMinAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -107,6 +108,23 @@ public class TopNQueryRunnerTest { @Parameterized.Parameters(name="{0}") public static Iterable constructorFeeder() throws IOException + { + List>> retVal = queryRunners(); + List parameters = new ArrayList<>(); + for (int i = 0; i < 8; i++) { + for (QueryRunner> firstParameter : retVal) { + Object[] params = new Object[4]; + params[0] = firstParameter; + params[1] = (i & 1) != 0; + params[2] = (i & 2) != 0; + params[3] = (i & 4) != 0; + parameters.add(params); + } + } + return parameters; + } + + public static List>> queryRunners() throws IOException { List>> retVal = Lists.newArrayList(); retVal.addAll( @@ -143,20 +161,53 @@ public ByteBuffer get() ) ) ); - - return QueryRunnerTestHelper.transformToConstructionFeeder(retVal); + return retVal; } private final QueryRunner> runner; + private final boolean duplicateSingleAggregatorQueries; @Rule public ExpectedException expectedException = ExpectedException.none(); public TopNQueryRunnerTest( - QueryRunner> runner + QueryRunner> runner, + boolean specializeGeneric1AggPooledTopN, + boolean specializeGeneric2AggPooledTopN, + boolean duplicateSingleAggregatorQueries ) { this.runner = runner; + PooledTopNAlgorithm.specializeGeneric1AggPooledTopN = specializeGeneric1AggPooledTopN; + PooledTopNAlgorithm.specializeGeneric2AggPooledTopN = specializeGeneric2AggPooledTopN; + this.duplicateSingleAggregatorQueries = duplicateSingleAggregatorQueries; + } + + private List duplicateAggregators(AggregatorFactory aggregatorFactory, AggregatorFactory duplicate) + { + if (duplicateSingleAggregatorQueries) { + return ImmutableList.of(aggregatorFactory, duplicate); + } else { + return Collections.singletonList(aggregatorFactory); + } + } + + private List> withDuplicateResults( + List> results, + String key, + String duplicateKey + ) + { + if (!duplicateSingleAggregatorQueries) { + return (List>) results; + } + List> resultsWithDuplicates = new ArrayList<>(); + for (Map result : results) { + resultsWithDuplicates.add( + ImmutableMap.builder().putAll(result).put(duplicateKey, result.get(key)).build() + ); + } + return resultsWithDuplicates; } private Sequence> assertExpectedResults( @@ -2992,6 +3043,10 @@ public void testInvertedTopNQuery() @Test public void testTopNQueryByComplexMetric() { + ImmutableList aggregatorDimensionSpecs = ImmutableList.of(new DefaultDimensionSpec( + QueryRunnerTestHelper.qualityDimension, + QueryRunnerTestHelper.qualityDimension + )); TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) @@ -3000,37 +3055,33 @@ public void testTopNQueryByComplexMetric() .metric(new NumericTopNMetricSpec("numVals")) .threshold(10) .intervals(QueryRunnerTestHelper.firstToThird) - .aggregators( - Lists.newArrayList( - new CardinalityAggregatorFactory( - "numVals", - ImmutableList.of(new DefaultDimensionSpec( - QueryRunnerTestHelper.qualityDimension, - QueryRunnerTestHelper.qualityDimension - )), - false - ) - ) - ) + .aggregators(duplicateAggregators( + new CardinalityAggregatorFactory("numVals", aggregatorDimensionSpecs, false), + new CardinalityAggregatorFactory("numVals1", aggregatorDimensionSpecs, false) + )) .build(); - List> expectedResults = Arrays.asList( + List> expectedResults = Collections.singletonList( new Result<>( new DateTime("2011-04-01T00:00:00.000Z"), new TopNResultValue( - Arrays.>asList( - ImmutableMap.of( - "market", "spot", - "numVals", 9.019833517963864d - ), - ImmutableMap.of( - "market", "total_market", - "numVals", 2.000977198748901d + withDuplicateResults( + Arrays.>asList( + ImmutableMap.of( + "market", "spot", + "numVals", 9.019833517963864d + ), + ImmutableMap.of( + "market", "total_market", + "numVals", 2.000977198748901d + ), + ImmutableMap.of( + "market", "upfront", + "numVals", 2.000977198748901d + ) ), - ImmutableMap.of( - "market", "upfront", - "numVals", 2.000977198748901d - ) + "numVals", + "numVals1" ) ) ) @@ -3048,6 +3099,11 @@ public void testTopNQueryCardinalityAggregatorWithExtractionFn() QueryRunnerTestHelper.marketDimension, helloFn); + ImmutableList aggregatorDimensionSpecs = ImmutableList.of(new ExtractionDimensionSpec( + QueryRunnerTestHelper.qualityDimension, + QueryRunnerTestHelper.qualityDimension, + helloFn + )); TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) @@ -3056,30 +3112,25 @@ public void testTopNQueryCardinalityAggregatorWithExtractionFn() .metric(new NumericTopNMetricSpec("numVals")) .threshold(10) .intervals(QueryRunnerTestHelper.firstToThird) - .aggregators( - Lists.newArrayList( - new CardinalityAggregatorFactory( - "numVals", - ImmutableList.of(new ExtractionDimensionSpec( - QueryRunnerTestHelper.qualityDimension, - QueryRunnerTestHelper.qualityDimension, - helloFn - )), - false - ) - ) - ) + .aggregators(duplicateAggregators( + new CardinalityAggregatorFactory("numVals", aggregatorDimensionSpecs,false), + new CardinalityAggregatorFactory("numVals1",aggregatorDimensionSpecs,false) + )) .build(); - List> expectedResults = Arrays.asList( + List> expectedResults = Collections.singletonList( new Result<>( new DateTime("2011-04-01T00:00:00.000Z"), new TopNResultValue( - Arrays.>asList( - ImmutableMap.of( - "market", "hello", - "numVals", 1.0002442201269182d - ) + withDuplicateResults( + Collections.singletonList( + ImmutableMap.of( + "market", "hello", + "numVals", 1.0002442201269182d + ) + ), + "numVals", + "numVals1" ) ) ) @@ -3567,21 +3618,28 @@ public void testAlphaNumericTopNWithNullPreviousStop() .metric(new DimensionTopNMetricSpec(null, StringComparators.ALPHANUMERIC)) .threshold(2) .intervals(QueryRunnerTestHelper.secondOnly) - .aggregators(Lists.newArrayList(QueryRunnerTestHelper.rowsCount)) + .aggregators(duplicateAggregators( + QueryRunnerTestHelper.rowsCount, + new CountAggregatorFactory("rows1") + )) .build(); List> expectedResults = Arrays.asList( new Result<>( new DateTime("2011-04-02T00:00:00.000Z"), new TopNResultValue( - Arrays.asList( - ImmutableMap.of( - "market", "spot", - "rows", 9L + withDuplicateResults( + Arrays.asList( + ImmutableMap.of( + "market", "spot", + "rows", 9L + ), + ImmutableMap.of( + "market", "total_market", + "rows", 2L + ) ), - ImmutableMap.of( - "market", "total_market", - "rows", 2L - ) + "rows", + "rows1" ) ) ) @@ -3599,21 +3657,28 @@ public void testNumericDimensionTopNWithNullPreviousStop() .metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC)) .threshold(2) .intervals(QueryRunnerTestHelper.secondOnly) - .aggregators(Lists.newArrayList(QueryRunnerTestHelper.rowsCount)) + .aggregators(duplicateAggregators( + QueryRunnerTestHelper.rowsCount, + new CountAggregatorFactory("rows1") + )) .build(); List> expectedResults = Arrays.asList( new Result<>( new DateTime("2011-04-02T00:00:00.000Z"), new TopNResultValue( - Arrays.asList( - ImmutableMap.of( - "market", "spot", - "rows", 9L + withDuplicateResults( + Arrays.asList( + ImmutableMap.of( + "market", "spot", + "rows", 9L + ), + ImmutableMap.of( + "market", "total_market", + "rows", 2L + ) ), - ImmutableMap.of( - "market", "total_market", - "rows", 2L - ) + "rows", + "rows1" ) ) ) diff --git a/processing/src/test/java/io/druid/segment/BitmapOffsetTest.java b/processing/src/test/java/io/druid/segment/BitmapOffsetTest.java index dd8161023229..b3bfdf5bf55b 100644 --- a/processing/src/test/java/io/druid/segment/BitmapOffsetTest.java +++ b/processing/src/test/java/io/druid/segment/BitmapOffsetTest.java @@ -26,6 +26,7 @@ import io.druid.collections.bitmap.BitSetBitmapFactory; import io.druid.collections.bitmap.BitmapFactory; import io.druid.collections.bitmap.ConciseBitmapFactory; +import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.collections.bitmap.MutableBitmap; import io.druid.collections.bitmap.RoaringBitmapFactory; import io.druid.segment.data.Offset; @@ -81,7 +82,8 @@ public void testSanity() throws Exception mutable.add(val); } - final BitmapOffset offset = new BitmapOffset(factory, factory.makeImmutableBitmap(mutable), descending); + ImmutableBitmap bitmap = factory.makeImmutableBitmap(mutable); + final BitmapOffset offset = BitmapOffset.of(bitmap, descending, bitmap.size()); final int[] expected = descending ? TEST_VALS_FLIP : TEST_VALS; int count = 0; diff --git a/processing/src/main/java/io/druid/segment/data/StartLimitedOffset.java b/processing/src/test/java/io/druid/segment/TestFloatColumnSelector.java similarity index 53% rename from processing/src/main/java/io/druid/segment/data/StartLimitedOffset.java rename to processing/src/test/java/io/druid/segment/TestFloatColumnSelector.java index 0d8203765c8c..3d53ac0effc4 100644 --- a/processing/src/main/java/io/druid/segment/data/StartLimitedOffset.java +++ b/processing/src/test/java/io/druid/segment/TestFloatColumnSelector.java @@ -17,49 +17,17 @@ * under the License. */ -package io.druid.segment.data; +package io.druid.segment; -/** - */ -public class StartLimitedOffset implements Offset -{ - private final Offset baseOffset; - private final int limit; - - public StartLimitedOffset( - Offset baseOffset, - int limit - ) - { - this.baseOffset = baseOffset; - this.limit = limit; - - while (baseOffset.withinBounds() && baseOffset.getOffset() < limit) { - baseOffset.increment(); - } - } - - @Override - public void increment() - { - baseOffset.increment(); - } - - @Override - public boolean withinBounds() - { - return baseOffset.withinBounds(); - } - - @Override - public Offset clone() - { - return new StartLimitedOffset(baseOffset.clone(), limit); - } +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +public abstract class TestFloatColumnSelector implements FloatColumnSelector +{ + /** + * Don't care about runtime shape in tests + */ @Override - public int getOffset() + public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - return baseOffset.getOffset(); } } diff --git a/processing/src/test/java/io/druid/segment/TestLongColumnSelector.java b/processing/src/test/java/io/druid/segment/TestLongColumnSelector.java new file mode 100644 index 000000000000..f4b5a32f2585 --- /dev/null +++ b/processing/src/test/java/io/druid/segment/TestLongColumnSelector.java @@ -0,0 +1,33 @@ +/* + * 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.segment; + +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; + +public abstract class TestLongColumnSelector implements LongColumnSelector +{ + /** + * Don't care about runtime shape in tests + */ + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } +} diff --git a/processing/src/test/java/io/druid/segment/virtual/ExpressionObjectSelectorTest.java b/processing/src/test/java/io/druid/segment/virtual/ExpressionObjectSelectorTest.java index b2064c30dc61..9718f195a00b 100644 --- a/processing/src/test/java/io/druid/segment/virtual/ExpressionObjectSelectorTest.java +++ b/processing/src/test/java/io/druid/segment/virtual/ExpressionObjectSelectorTest.java @@ -21,9 +21,9 @@ import com.google.common.base.Supplier; import io.druid.common.guava.SettableSupplier; -import io.druid.segment.FloatColumnSelector; -import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; +import io.druid.segment.TestFloatColumnSelector; +import io.druid.segment.TestLongColumnSelector; import org.junit.Assert; import org.junit.Test; @@ -35,7 +35,7 @@ public class ExpressionObjectSelectorTest public void testSupplierFromLongSelector() { final Supplier supplier = ExpressionObjectSelector.supplierFromLongSelector( - new LongColumnSelector() + new TestLongColumnSelector() { @Override public long get() @@ -52,7 +52,7 @@ public long get() public void testSupplierFromFloatSelector() { final Supplier supplier = ExpressionObjectSelector.supplierFromFloatSelector( - new FloatColumnSelector() + new TestFloatColumnSelector() { @Override public float get() diff --git a/processing/src/test/java/io/druid/segment/virtual/VirtualColumnsTest.java b/processing/src/test/java/io/druid/segment/virtual/VirtualColumnsTest.java index f37d34940bfc..7b8f8c8abf32 100644 --- a/processing/src/test/java/io/druid/segment/virtual/VirtualColumnsTest.java +++ b/processing/src/test/java/io/druid/segment/virtual/VirtualColumnsTest.java @@ -30,6 +30,7 @@ import io.druid.query.extraction.BucketExtractionFn; import io.druid.query.extraction.ExtractionFn; import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.DimensionSelector; import io.druid.segment.DimensionSelectorUtils; @@ -37,6 +38,8 @@ import io.druid.segment.IdLookup; import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; +import io.druid.segment.TestFloatColumnSelector; +import io.druid.segment.TestLongColumnSelector; import io.druid.segment.VirtualColumn; import io.druid.segment.VirtualColumns; import io.druid.segment.column.ColumnCapabilities; @@ -355,6 +358,11 @@ public int lookupId(final String name) } }; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } }; return dimensionSpec.decorate(dimensionSelector); @@ -364,7 +372,7 @@ public int lookupId(final String name) public FloatColumnSelector makeFloatColumnSelector(String columnName, ColumnSelectorFactory factory) { final LongColumnSelector selector = makeLongColumnSelector(columnName, factory); - return new FloatColumnSelector() + return new TestFloatColumnSelector() { @Override public float get() @@ -380,7 +388,7 @@ public LongColumnSelector makeLongColumnSelector(String columnName, ColumnSelect final String subColumn = VirtualColumns.splitColumnName(columnName).rhs; final Long boxed = subColumn == null ? null : Longs.tryParse(subColumn); final long theLong = boxed == null ? -1 : boxed; - return new LongColumnSelector() + return new TestLongColumnSelector() { @Override public long get()