diff --git a/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java index 8e18ce61228..ee5ee674293 100644 --- a/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java @@ -76,6 +76,12 @@ public class DefaultDimensionSpec implements DimensionSpec .array(); } + @Override + public boolean preservesOrdering() + { + return true; + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/query/dimension/DimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/DimensionSpec.java index 3c552e20763..b8cbc58dfc5 100644 --- a/processing/src/main/java/io/druid/query/dimension/DimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/DimensionSpec.java @@ -33,7 +33,12 @@ import io.druid.query.extraction.DimExtractionFn; public interface DimensionSpec { public String getDimension(); + public String getOutputName(); + public DimExtractionFn getDimExtractionFn(); + public byte[] getCacheKey(); + + public boolean preservesOrdering(); } diff --git a/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java index 9fe480e396d..878de5bbad5 100644 --- a/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java @@ -83,6 +83,12 @@ public class ExtractionDimensionSpec implements DimensionSpec .array(); } + @Override + public boolean preservesOrdering() + { + return dimExtractionFn.preservesOrdering(); + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/query/extraction/DimExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/DimExtractionFn.java index 0509c92714a..638204b76f8 100644 --- a/processing/src/main/java/io/druid/query/extraction/DimExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/DimExtractionFn.java @@ -24,7 +24,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; /** */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property="type") +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @JsonSubTypes.Type(name = "time", value = TimeDimExtractionFn.class), @JsonSubTypes.Type(name = "regex", value = RegexDimExtractionFn.class), @@ -35,5 +35,8 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; public interface DimExtractionFn { public byte[] getCacheKey(); + public String apply(String dimValue); + + public boolean preservesOrdering(); } diff --git a/processing/src/main/java/io/druid/query/extraction/JavascriptDimExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/JavascriptDimExtractionFn.java index f552ef4e9cd..d61d3b26b41 100644 --- a/processing/src/main/java/io/druid/query/extraction/JavascriptDimExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/JavascriptDimExtractionFn.java @@ -92,4 +92,10 @@ public class JavascriptDimExtractionFn implements DimExtractionFn { return fn.apply(dimValue); } + + @Override + public boolean preservesOrdering() + { + return false; + } } diff --git a/processing/src/main/java/io/druid/query/extraction/PartialDimExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/PartialDimExtractionFn.java index 5ae4e2bf019..8b6a3b58017 100644 --- a/processing/src/main/java/io/druid/query/extraction/PartialDimExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/PartialDimExtractionFn.java @@ -67,6 +67,12 @@ public class PartialDimExtractionFn implements DimExtractionFn return expr; } + @Override + public boolean preservesOrdering() + { + return true; + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/query/extraction/RegexDimExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/RegexDimExtractionFn.java index c25d0ba6c7a..50a418afed3 100644 --- a/processing/src/main/java/io/druid/query/extraction/RegexDimExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/RegexDimExtractionFn.java @@ -67,6 +67,12 @@ public class RegexDimExtractionFn implements DimExtractionFn return expr; } + @Override + public boolean preservesOrdering() + { + return false; + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/query/extraction/SearchQuerySpecDimExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/SearchQuerySpecDimExtractionFn.java index 02c13eca630..b7579ea1399 100644 --- a/processing/src/main/java/io/druid/query/extraction/SearchQuerySpecDimExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/SearchQuerySpecDimExtractionFn.java @@ -63,6 +63,12 @@ public class SearchQuerySpecDimExtractionFn implements DimExtractionFn return searchQuerySpec.accept(dimValue) ? dimValue : null; } + @Override + public boolean preservesOrdering() + { + return true; + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/query/extraction/TimeDimExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/TimeDimExtractionFn.java index 57b7f6617d8..215f8c32aee 100644 --- a/processing/src/main/java/io/druid/query/extraction/TimeDimExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/TimeDimExtractionFn.java @@ -87,6 +87,12 @@ public class TimeDimExtractionFn implements DimExtractionFn return resultFormat; } + @Override + public boolean preservesOrdering() + { + return false; + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java b/processing/src/main/java/io/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java index 254d13d581b..cd9593192a6 100644 --- a/processing/src/main/java/io/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java +++ b/processing/src/main/java/io/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java @@ -31,7 +31,6 @@ import io.druid.segment.DimensionSelector; import java.nio.ByteBuffer; import java.util.Arrays; -import java.util.Comparator; import java.util.Iterator; import java.util.List; @@ -41,7 +40,6 @@ public class AggregateTopNMetricFirstAlgorithm implements TopNAlgorithm comparator; private final StupidPool bufferPool; public AggregateTopNMetricFirstAlgorithm( @@ -52,8 +50,6 @@ public class AggregateTopNMetricFirstAlgorithm implements TopNAlgorithm, TopNParams> { private final TopNQuery query; - private final Comparator comparator; public DimExtractionTopNAlgorithm( Capabilities capabilities, @@ -44,8 +42,6 @@ public class DimExtractionTopNAlgorithm extends BaseTopNAlgorithm aggregatorSpecs, - List postAggregatorSpecs + final List aggregatorSpecs, + final List postAggregatorSpecs ) { - return Comparators.inverse(delegate.getComparator(aggregatorSpecs, postAggregatorSpecs)); + return Comparators.inverse( + new Comparator() + { + @Override + public int compare(Object o1, Object o2) + { + // nulls last + if (o1 == null) { + return 1; + } + if (o2 == null) { + return -1; + } + return delegate.getComparator(aggregatorSpecs, postAggregatorSpecs).compare(o1, o2); + } + } + ); } @Override @@ -80,7 +97,14 @@ public class InvertedTopNMetricSpec implements TopNMetricSpec List postAggs ) { - return delegate.getResultBuilder(timestamp, dimSpec, threshold, comparator, aggFactories, postAggs); + return delegate.getResultBuilder( + timestamp, + dimSpec, + threshold, + comparator, + aggFactories, + postAggs + ); } @Override @@ -94,6 +118,9 @@ public class InvertedTopNMetricSpec implements TopNMetricSpec @Override public TopNMetricSpecBuilder configureOptimizer(TopNMetricSpecBuilder builder) { + if (!canBeOptimizedUnordered()) { + return builder; + } return delegate.configureOptimizer(builder); } @@ -109,6 +136,12 @@ public class InvertedTopNMetricSpec implements TopNMetricSpec return delegate.getMetricName(dimSpec); } + @Override + public boolean canBeOptimizedUnordered() + { + return delegate.canBeOptimizedUnordered(); + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/io/druid/query/topn/LexicographicTopNMetricSpec.java b/processing/src/main/java/io/druid/query/topn/LexicographicTopNMetricSpec.java index b7c7c6a2565..a23dd2f186f 100644 --- a/processing/src/main/java/io/druid/query/topn/LexicographicTopNMetricSpec.java +++ b/processing/src/main/java/io/druid/query/topn/LexicographicTopNMetricSpec.java @@ -43,6 +43,13 @@ public class LexicographicTopNMetricSpec implements TopNMetricSpec @Override public int compare(String s, String s2) { + // null first + if (s == null) { + return -1; + } + if (s2 == null) { + return 1; + } return UnsignedBytes.lexicographicalComparator().compare(s.getBytes(Charsets.UTF_8), s2.getBytes(Charsets.UTF_8)); } }; @@ -54,7 +61,7 @@ public class LexicographicTopNMetricSpec implements TopNMetricSpec @JsonProperty("previousStop") String previousStop ) { - this.previousStop = (previousStop == null) ? "" : previousStop; + this.previousStop = previousStop; } @Override @@ -85,13 +92,20 @@ public class LexicographicTopNMetricSpec implements TopNMetricSpec List postAggs ) { - return new TopNLexicographicResultBuilder(timestamp, dimSpec, threshold, previousStop, comparator, aggFactories); + return new TopNLexicographicResultBuilder( + timestamp, + dimSpec, + threshold, + previousStop, + comparator, + aggFactories + ); } @Override public byte[] getCacheKey() { - byte[] previousStopBytes = previousStop.getBytes(Charsets.UTF_8); + byte[] previousStopBytes = previousStop == null ? new byte[]{} : previousStop.getBytes(Charsets.UTF_8); return ByteBuffer.allocate(1 + previousStopBytes.length) .put(CACHE_TYPE_ID) @@ -119,6 +133,12 @@ public class LexicographicTopNMetricSpec implements TopNMetricSpec return dimSpec.getOutputName(); } + @Override + public boolean canBeOptimizedUnordered() + { + return false; + } + @Override public String toString() { @@ -130,12 +150,18 @@ public class LexicographicTopNMetricSpec implements TopNMetricSpec @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } LexicographicTopNMetricSpec that = (LexicographicTopNMetricSpec) o; - if (previousStop != null ? !previousStop.equals(that.previousStop) : that.previousStop != null) return false; + if (previousStop != null ? !previousStop.equals(that.previousStop) : that.previousStop != null) { + return false; + } return true; } diff --git a/processing/src/main/java/io/druid/query/topn/NumericTopNMetricSpec.java b/processing/src/main/java/io/druid/query/topn/NumericTopNMetricSpec.java index 9ad97e239cd..ad0a80fe719 100644 --- a/processing/src/main/java/io/druid/query/topn/NumericTopNMetricSpec.java +++ b/processing/src/main/java/io/druid/query/topn/NumericTopNMetricSpec.java @@ -158,6 +158,12 @@ public class NumericTopNMetricSpec implements TopNMetricSpec return metric; } + @Override + public boolean canBeOptimizedUnordered() + { + return true; + } + @Override public String toString() { 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 a8e3f324467..ea4fd37547d 100644 --- a/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java +++ b/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java @@ -31,7 +31,6 @@ import io.druid.segment.data.IndexedInts; import java.nio.ByteBuffer; import java.util.Arrays; -import java.util.Comparator; /** */ @@ -40,7 +39,6 @@ public class PooledTopNAlgorithm { private final Capabilities capabilities; private final TopNQuery query; - private final Comparator comparator; private final StupidPool bufferPool; public PooledTopNAlgorithm( @@ -53,8 +51,6 @@ public class PooledTopNAlgorithm this.capabilities = capabilities; this.query = query; - this.comparator = query.getTopNMetricSpec() - .getComparator(query.getAggregatorSpecs(), query.getPostAggregatorSpecs()); this.bufferPool = bufferPool; } @@ -115,12 +111,15 @@ public class PooledTopNAlgorithm } - @Override protected int[] makeDimValSelector(PooledTopNParams params, int numProcessed, int numToProcess) { final TopNMetricSpecBuilder arrayProvider = params.getArrayProvider(); + if (!query.getDimensionSpec().preservesOrdering()) { + return arrayProvider.build(); + } + arrayProvider.ignoreFirstN(numProcessed); arrayProvider.keepOnlyN(numToProcess); return query.getTopNMetricSpec().configureOptimizer(arrayProvider).build(); diff --git a/processing/src/main/java/io/druid/query/topn/TopNLexicographicResultBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNLexicographicResultBuilder.java index bf49f627252..ba669899fc8 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNLexicographicResultBuilder.java +++ b/processing/src/main/java/io/druid/query/topn/TopNLexicographicResultBuilder.java @@ -23,7 +23,6 @@ import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DimensionSpec; import org.joda.time.DateTime; @@ -40,6 +39,7 @@ public class TopNLexicographicResultBuilder implements TopNResultBuilder private final DateTime timestamp; private final DimensionSpec dimSpec; private final String previousStop; + private final Comparator comparator; private final List aggFactories; private MinMaxPriorityQueue pQueue = null; @@ -55,6 +55,7 @@ public class TopNLexicographicResultBuilder implements TopNResultBuilder this.timestamp = timestamp; this.dimSpec = dimSpec; this.previousStop = previousStop; + this.comparator = comparator; this.aggFactories = aggFactories; instantiatePQueue(threshold, comparator); @@ -69,7 +70,7 @@ public class TopNLexicographicResultBuilder implements TopNResultBuilder { Map metricValues = Maps.newLinkedHashMap(); - if (dimName.compareTo(previousStop) > 0) { + if (comparator.compare(dimName, previousStop) > 0) { metricValues.put(dimSpec.getOutputName(), dimName); Iterator aggsIter = aggFactories.iterator(); for (Object metricVal : metricVals) { diff --git a/processing/src/main/java/io/druid/query/topn/TopNMetricSpec.java b/processing/src/main/java/io/druid/query/topn/TopNMetricSpec.java index 6e934e32dd6..25e710d1cff 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNMetricSpec.java +++ b/processing/src/main/java/io/druid/query/topn/TopNMetricSpec.java @@ -60,4 +60,6 @@ public interface TopNMetricSpec public void initTopNAlgorithmSelector(TopNAlgorithmSelector selector); public String getMetricName(DimensionSpec dimSpec); + + public boolean canBeOptimizedUnordered(); } diff --git a/processing/src/main/java/io/druid/query/topn/TopNParams.java b/processing/src/main/java/io/druid/query/topn/TopNParams.java index 8ccc85da284..fbeab73792e 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNParams.java +++ b/processing/src/main/java/io/druid/query/topn/TopNParams.java @@ -31,7 +31,12 @@ public class TopNParams private final int cardinality; private final int numValuesPerPass; - protected TopNParams(DimensionSelector dimSelector, Cursor cursor, int cardinality, int numValuesPerPass) + protected TopNParams( + DimensionSelector dimSelector, + Cursor cursor, + int cardinality, + int numValuesPerPass + ) { this.dimSelector = dimSelector; this.cursor = cursor; 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 c8628bfdca5..22b750faf00 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java @@ -134,32 +134,32 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put(providerDimension, "total_market") - .put("rows", 186L) - .put("index", 215679.82879638672D) - .put("addRowsIndexConstant", 215866.82879638672D) - .put("uniques", QueryRunnerTestHelper.UNIQUES_2) - .put("maxIndex", 1743.9217529296875D) - .put("minIndex", 792.3260498046875D) - .build(), + .put(providerDimension, "total_market") + .put("rows", 186L) + .put("index", 215679.82879638672D) + .put("addRowsIndexConstant", 215866.82879638672D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_2) + .put("maxIndex", 1743.9217529296875D) + .put("minIndex", 792.3260498046875D) + .build(), ImmutableMap.builder() - .put(providerDimension, "upfront") - .put("rows", 186L) - .put("index", 192046.1060180664D) - .put("addRowsIndexConstant", 192233.1060180664D) - .put("uniques", QueryRunnerTestHelper.UNIQUES_2) - .put("maxIndex", 1870.06103515625D) - .put("minIndex", 545.9906005859375D) - .build(), + .put(providerDimension, "upfront") + .put("rows", 186L) + .put("index", 192046.1060180664D) + .put("addRowsIndexConstant", 192233.1060180664D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_2) + .put("maxIndex", 1870.06103515625D) + .put("minIndex", 545.9906005859375D) + .build(), ImmutableMap.builder() - .put(providerDimension, "spot") - .put("rows", 837L) - .put("index", 95606.57232284546D) - .put("addRowsIndexConstant", 96444.57232284546D) - .put("uniques", QueryRunnerTestHelper.UNIQUES_9) - .put("maxIndex", 277.2735290527344D) - .put("minIndex", 59.02102279663086D) - .build() + .put(providerDimension, "spot") + .put("rows", 837L) + .put("index", 95606.57232284546D) + .put("addRowsIndexConstant", 96444.57232284546D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_9) + .put("maxIndex", 277.2735290527344D) + .put("minIndex", 59.02102279663086D) + .build() ) ) ) @@ -198,32 +198,32 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put(providerDimension, "total_market") - .put("rows", 186L) - .put("index", 215679.82879638672D) - .put("addRowsIndexConstant", 215866.82879638672D) - .put("uniques", QueryRunnerTestHelper.UNIQUES_2) - .put("maxIndex", 1743.9217529296875D) - .put("minIndex", 792.3260498046875D) - .build(), + .put(providerDimension, "total_market") + .put("rows", 186L) + .put("index", 215679.82879638672D) + .put("addRowsIndexConstant", 215866.82879638672D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_2) + .put("maxIndex", 1743.9217529296875D) + .put("minIndex", 792.3260498046875D) + .build(), ImmutableMap.builder() - .put(providerDimension, "upfront") - .put("rows", 186L) - .put("index", 192046.1060180664D) - .put("addRowsIndexConstant", 192233.1060180664D) - .put("uniques", QueryRunnerTestHelper.UNIQUES_2) - .put("maxIndex", 1870.06103515625D) - .put("minIndex", 545.9906005859375D) - .build(), + .put(providerDimension, "upfront") + .put("rows", 186L) + .put("index", 192046.1060180664D) + .put("addRowsIndexConstant", 192233.1060180664D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_2) + .put("maxIndex", 1870.06103515625D) + .put("minIndex", 545.9906005859375D) + .build(), ImmutableMap.builder() - .put(providerDimension, "spot") - .put("rows", 837L) - .put("index", 95606.57232284546D) - .put("addRowsIndexConstant", 96444.57232284546D) - .put("uniques", QueryRunnerTestHelper.UNIQUES_9) - .put("maxIndex", 277.2735290527344D) - .put("minIndex", 59.02102279663086D) - .build() + .put(providerDimension, "spot") + .put("rows", 837L) + .put("index", 95606.57232284546D) + .put("addRowsIndexConstant", 96444.57232284546D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_9) + .put("maxIndex", 277.2735290527344D) + .put("minIndex", 59.02102279663086D) + .build() ) ) ) @@ -263,32 +263,32 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "spot") - .put("rows", 837L) - .put("index", 95606.57232284546D) - .put("addRowsIndexConstant", 96444.57232284546D) - .put("uniques", QueryRunnerTestHelper.UNIQUES_9) - .put("maxIndex", 277.2735290527344D) - .put("minIndex", 59.02102279663086D) - .build(), + .put("provider", "spot") + .put("rows", 837L) + .put("index", 95606.57232284546D) + .put("addRowsIndexConstant", 96444.57232284546D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_9) + .put("maxIndex", 277.2735290527344D) + .put("minIndex", 59.02102279663086D) + .build(), ImmutableMap.builder() - .put("provider", "total_market") - .put("rows", 186L) - .put("index", 215679.82879638672D) - .put("addRowsIndexConstant", 215866.82879638672D) - .put("uniques", QueryRunnerTestHelper.UNIQUES_2) - .put("maxIndex", 1743.9217529296875D) - .put("minIndex", 792.3260498046875D) - .build(), + .put("provider", "total_market") + .put("rows", 186L) + .put("index", 215679.82879638672D) + .put("addRowsIndexConstant", 215866.82879638672D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_2) + .put("maxIndex", 1743.9217529296875D) + .put("minIndex", 792.3260498046875D) + .build(), ImmutableMap.builder() - .put("provider", "upfront") - .put("rows", 186L) - .put("index", 192046.1060180664D) - .put("addRowsIndexConstant", 192233.1060180664D) - .put("uniques", QueryRunnerTestHelper.UNIQUES_2) - .put("maxIndex", 1870.06103515625D) - .put("minIndex", 545.9906005859375D) - .build() + .put("provider", "upfront") + .put("rows", 186L) + .put("index", 192046.1060180664D) + .put("addRowsIndexConstant", 192233.1060180664D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_2) + .put("maxIndex", 1870.06103515625D) + .put("minIndex", 545.9906005859375D) + .build() ) ) ) @@ -696,18 +696,18 @@ public class TopNQueryRunnerTest public void testTopNWithNonExistentFilterMultiDim() { AndDimFilter andDimFilter = Druids.newAndDimFilterBuilder() - .fields( - Lists.newArrayList( - Druids.newSelectorDimFilterBuilder() - .dimension(providerDimension) - .value("billyblank") - .build(), - Druids.newSelectorDimFilterBuilder() - .dimension(QueryRunnerTestHelper.qualityDimension) - .value("mezzanine") - .build() - ) - ).build(); + .fields( + Lists.newArrayList( + Druids.newSelectorDimFilterBuilder() + .dimension(providerDimension) + .value("billyblank") + .build(), + Druids.newSelectorDimFilterBuilder() + .dimension(QueryRunnerTestHelper.qualityDimension) + .value("mezzanine") + .build() + ) + ).build(); TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) @@ -1077,6 +1077,88 @@ public class TopNQueryRunnerTest TestHelper.assertExpectedResults(expectedResults, runner.run(query)); } + @Test + public void testTopNInvertedLexicographicWithPreviousStop() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(providerDimension) + .metric(new InvertedTopNMetricSpec(new LexicographicTopNMetricSpec("upfront"))) + .threshold(4) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + providerDimension, "total_market", + "rows", 4L, + "index", 5351.814697265625D, + "addRowsIndexConstant", 5356.814697265625D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 + ), + ImmutableMap.of( + providerDimension, "spot", + "rows", 18L, + "index", 2231.8768157958984D, + "addRowsIndexConstant", 2250.8768157958984D, + "uniques", QueryRunnerTestHelper.UNIQUES_9 + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTopNInvertedLexicographicWithNonExistingPreviousStop() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(providerDimension) + .metric(new InvertedTopNMetricSpec(new LexicographicTopNMetricSpec("u"))) + .threshold(4) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + providerDimension, "total_market", + "rows", 4L, + "index", 5351.814697265625D, + "addRowsIndexConstant", 5356.814697265625D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 + ), + ImmutableMap.of( + providerDimension, "spot", + "rows", 18L, + "index", 2231.8768157958984D, + "addRowsIndexConstant", 2250.8768157958984D, + "uniques", QueryRunnerTestHelper.UNIQUES_9 + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + @Test public void testTopNDimExtraction() { @@ -1129,6 +1211,246 @@ public class TopNQueryRunnerTest TestHelper.assertExpectedResults(expectedResults, runner.run(query)); } + @Test + public void testTopNLexicographicDimExtraction() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension( + new ExtractionDimensionSpec( + providerDimension, providerDimension, new RegexDimExtractionFn("(.)") + ) + ) + .metric(new LexicographicTopNMetricSpec(null)) + .threshold(4) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + providerDimension, "s", + "rows", 18L, + "index", 2231.8768157958984D, + "addRowsIndexConstant", 2250.8768157958984D, + "uniques", QueryRunnerTestHelper.UNIQUES_9 + ), + ImmutableMap.of( + providerDimension, "t", + "rows", 4L, + "index", 5351.814697265625D, + "addRowsIndexConstant", 5356.814697265625D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 + ), + ImmutableMap.of( + providerDimension, "u", + "rows", 4L, + "index", 4875.669677734375D, + "addRowsIndexConstant", 4880.669677734375D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testInvertedTopNLexicographicDimExtraction2() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension( + new ExtractionDimensionSpec( + providerDimension, providerDimension, new RegexDimExtractionFn("..(.)") + ) + ) + .metric(new InvertedTopNMetricSpec(new LexicographicTopNMetricSpec(null))) + .threshold(4) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + providerDimension, "t", + "rows", 4L, + "index", 5351.814697265625D, + "addRowsIndexConstant", 5356.814697265625D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 + ), + ImmutableMap.of( + providerDimension, "o", + "rows", 18L, + "index", 2231.8768157958984D, + "addRowsIndexConstant", 2250.8768157958984D, + "uniques", QueryRunnerTestHelper.UNIQUES_9 + ), + ImmutableMap.of( + providerDimension, "f", + "rows", 4L, + "index", 4875.669677734375D, + "addRowsIndexConstant", 4880.669677734375D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTopNLexicographicDimExtractionWithPreviousStop() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension( + new ExtractionDimensionSpec( + providerDimension, providerDimension, new RegexDimExtractionFn("(.)") + ) + ) + .metric(new LexicographicTopNMetricSpec("spot")) + .threshold(4) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + providerDimension, "t", + "rows", 4L, + "index", 5351.814697265625D, + "addRowsIndexConstant", 5356.814697265625D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 + ), + ImmutableMap.of( + providerDimension, "u", + "rows", 4L, + "index", 4875.669677734375D, + "addRowsIndexConstant", 4880.669677734375D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + + @Test + public void testInvertedTopNLexicographicDimExtractionWithPreviousStop() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension( + new ExtractionDimensionSpec( + providerDimension, providerDimension, new RegexDimExtractionFn("(.)") + ) + ) + .metric(new InvertedTopNMetricSpec(new LexicographicTopNMetricSpec("u"))) + .threshold(4) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + providerDimension, "t", + "rows", 4L, + "index", 5351.814697265625D, + "addRowsIndexConstant", 5356.814697265625D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 + ), + ImmutableMap.of( + providerDimension, "s", + "rows", 18L, + "index", 2231.8768157958984D, + "addRowsIndexConstant", 2250.8768157958984D, + "uniques", QueryRunnerTestHelper.UNIQUES_9 + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testInvertedTopNLexicographicDimExtractionWithPreviousStop2() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension( + new ExtractionDimensionSpec( + providerDimension, providerDimension, new RegexDimExtractionFn("..(.)") + ) + ) + .metric(new InvertedTopNMetricSpec(new LexicographicTopNMetricSpec("p"))) + .threshold(4) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + providerDimension, "o", + "rows", 18L, + "index", 2231.8768157958984D, + "addRowsIndexConstant", 2250.8768157958984D, + "uniques", QueryRunnerTestHelper.UNIQUES_9 + ), + ImmutableMap.of( + providerDimension, "f", + "rows", 4L, + "index", 4875.669677734375D, + "addRowsIndexConstant", 4880.669677734375D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + @Test public void testInvertedTopNQuery() { @@ -1179,7 +1501,8 @@ public class TopNQueryRunnerTest } @Test - public void testTopNDependentPostAgg() { + public void testTopNDependentPostAgg() + { TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran)