From 50a6839693a9acc48405d29f9a6c92e29cb29c12 Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 8 Jan 2014 15:51:03 -0800 Subject: [PATCH 1/2] inital commit of topNs --- docs/content/TopNMetricSpec.md | 45 + docs/content/TopNQuery.md | 119 ++ docs/content/toc.textile | 2 + .../src/main/java/io/druid/query/Query.java | 5 +- .../AggregateTopNMetricFirstAlgorithm.java | 174 +++ .../druid/query/topn/BaseTopNAlgorithm.java | 234 ++++ .../query/topn/BySegmentTopNResultValue.java | 90 ++ .../topn/DimExtractionTopNAlgorithm.java | 169 +++ .../io/druid/query/topn/DimValHolder.java | 110 ++ .../DimensionAndMetricValueExtractor.java | 83 ++ .../query/topn/InvertedTopNMetricSpec.java | 104 ++ .../query/topn/LegacyTopNMetricSpec.java | 51 + .../topn/LexicographicTopNMetricSpec.java | 121 ++ .../query/topn/NumericTopNMetricSpec.java | 160 +++ .../druid/query/topn/PooledTopNAlgorithm.java | 401 ++++++ .../io/druid/query/topn/TopNAlgorithm.java | 45 + .../query/topn/TopNAlgorithmSelector.java | 73 ++ .../io/druid/query/topn/TopNBinaryFn.java | 126 ++ .../topn/TopNLexicographicResultBuilder.java | 134 ++ .../java/io/druid/query/topn/TopNMapFn.java | 65 + .../io/druid/query/topn/TopNMetricSpec.java | 58 + .../query/topn/TopNMetricSpecBuilder.java | 35 + .../query/topn/TopNNumericResultBuilder.java | 153 +++ .../java/io/druid/query/topn/TopNParams.java | 61 + .../java/io/druid/query/topn/TopNQuery.java | 211 ++++ .../io/druid/query/topn/TopNQueryBuilder.java | 290 +++++ .../io/druid/query/topn/TopNQueryConfig.java | 39 + .../io/druid/query/topn/TopNQueryEngine.java | 117 ++ .../query/topn/TopNQueryQueryToolChest.java | 405 +++++++ .../query/topn/TopNQueryRunnerFactory.java | 108 ++ .../druid/query/topn/TopNResultBuilder.java | 48 + .../io/druid/query/topn/TopNResultMerger.java | 40 + .../io/druid/query/topn/TopNResultValue.java | 107 ++ .../java/io/druid/query/TestQueryRunners.java | 83 ++ .../io/druid/query/topn/TopNBinaryFnTest.java | 458 +++++++ .../druid/query/topn/TopNQueryRunnerTest.java | 1077 +++++++++++++++++ .../query/topn/TopNQueryRunnerTestHelper.java | 73 ++ .../druid/guice/QueryRunnerFactoryModule.java | 3 + .../io/druid/guice/QueryToolChestModule.java | 5 + 39 files changed, 5681 insertions(+), 1 deletion(-) create mode 100644 docs/content/TopNMetricSpec.md create mode 100644 docs/content/TopNQuery.md create mode 100644 processing/src/main/java/io/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java create mode 100644 processing/src/main/java/io/druid/query/topn/BaseTopNAlgorithm.java create mode 100644 processing/src/main/java/io/druid/query/topn/BySegmentTopNResultValue.java create mode 100644 processing/src/main/java/io/druid/query/topn/DimExtractionTopNAlgorithm.java create mode 100644 processing/src/main/java/io/druid/query/topn/DimValHolder.java create mode 100644 processing/src/main/java/io/druid/query/topn/DimensionAndMetricValueExtractor.java create mode 100644 processing/src/main/java/io/druid/query/topn/InvertedTopNMetricSpec.java create mode 100644 processing/src/main/java/io/druid/query/topn/LegacyTopNMetricSpec.java create mode 100644 processing/src/main/java/io/druid/query/topn/LexicographicTopNMetricSpec.java create mode 100644 processing/src/main/java/io/druid/query/topn/NumericTopNMetricSpec.java create mode 100644 processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java create mode 100644 processing/src/main/java/io/druid/query/topn/TopNAlgorithm.java create mode 100644 processing/src/main/java/io/druid/query/topn/TopNAlgorithmSelector.java create mode 100644 processing/src/main/java/io/druid/query/topn/TopNBinaryFn.java create mode 100644 processing/src/main/java/io/druid/query/topn/TopNLexicographicResultBuilder.java create mode 100644 processing/src/main/java/io/druid/query/topn/TopNMapFn.java create mode 100644 processing/src/main/java/io/druid/query/topn/TopNMetricSpec.java create mode 100644 processing/src/main/java/io/druid/query/topn/TopNMetricSpecBuilder.java create mode 100644 processing/src/main/java/io/druid/query/topn/TopNNumericResultBuilder.java create mode 100644 processing/src/main/java/io/druid/query/topn/TopNParams.java create mode 100644 processing/src/main/java/io/druid/query/topn/TopNQuery.java create mode 100644 processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java create mode 100644 processing/src/main/java/io/druid/query/topn/TopNQueryConfig.java create mode 100644 processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java create mode 100644 processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java create mode 100644 processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java create mode 100644 processing/src/main/java/io/druid/query/topn/TopNResultBuilder.java create mode 100644 processing/src/main/java/io/druid/query/topn/TopNResultMerger.java create mode 100644 processing/src/main/java/io/druid/query/topn/TopNResultValue.java create mode 100644 processing/src/test/java/io/druid/query/TestQueryRunners.java create mode 100644 processing/src/test/java/io/druid/query/topn/TopNBinaryFnTest.java create mode 100644 processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java create mode 100644 processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTestHelper.java diff --git a/docs/content/TopNMetricSpec.md b/docs/content/TopNMetricSpec.md new file mode 100644 index 00000000000..672f5d352f6 --- /dev/null +++ b/docs/content/TopNMetricSpec.md @@ -0,0 +1,45 @@ +--- +layout: doc_page +--- +TopNMetricSpec +================== + +The topN metric spec specifies how topN values should be sorted. + +## Numeric TopNMetricSpec + +The simplest metric specification is a String value indicating the metric to sort topN results by. They are included in a topN query with: + +```json +"metric": +``` + +The metric field can also be given as a JSON object. The grammar for dimension values sorted by numeric value is shown below: + +```json +"metric": { + "type": "numeric", + "metric": "" +} +``` + +|property|description|required?| +|--------|-----------|---------| +|type|this indicates a numeric sort|yes| +|metric|the actual metric field in which results will be sorted by|yes| + +## Lexicographic TopNMetricSpec + +The grammar for dimension values sorted lexicographically is as follows: + +```json +"metric": { + "type": "lexicographic", + "previousStop": "" +} +``` + +|property|description|required?| +|--------|-----------|---------| +|type|this indicates a lexicographic sort|yes| +|previousStop|the starting point of the lexicographic sort. For example, if a previousStop value is 'b', all values before 'b' are discarded. This field can be used to paginate through all the dimension values.|no| diff --git a/docs/content/TopNQuery.md b/docs/content/TopNQuery.md new file mode 100644 index 00000000000..e0a0fc37077 --- /dev/null +++ b/docs/content/TopNQuery.md @@ -0,0 +1,119 @@ +--- +layout: doc_page +--- +TopN queries +================== + +TopN queries return a sorted set of results for the values in a given dimension according to some criteria. Conceptually, they can be thought of as an approximate [GroupByQuery](GroupByQuery.html) over a single dimension with an [Ordering](Ordering.html) spec. TopNs are much faster and resource efficient than GroupBys for this use case. These types of queries take a topN query object and return an array of JSON objects where each object represents a value asked for by the topN query. + +A topN query object looks like: + +```json + "queryType": "topN", + "dataSource": "sample_data", + "dimension": "sample_dim", + "threshold": 5, + "metric": "count", + "granularity": "all", + "filter": { + "type": "and", + "fields": [ + { + "type": "selector", + "dimension": "dim1", + "value": "some_value" + }, + { + "type": "selector", + "dimension": "dim2", + "value": "some_other_val" + } + ] + }, + "aggregations": [ + { + "type": "longSum", + "name": "count", + "fieldName": "count" + }, + { + "type": "doubleSum", + "name": "some_metric", + "fieldName": "some_metric" + } + ], + "postAggregations": [ + { + "type": "arithmetic", + "name": "sample_divide", + "fn": "/", + "fields": [ + { + "type": "fieldAccess", + "name": "some_metric", + "fieldName": "some_metric" + }, + { + "type": "fieldAccess", + "name": "count", + "fieldName": "count" + } + ] + } + ], + "intervals": [ + "2013-08-31T00:00:00.000/2013-09-03T00:00:00.000" + ] +} +``` + +There are 10 parts to a topN query, but 7 of them are shared with [TimeseriesQuery](TimeseriesQuery.html). Please review [TimeseriesQuery](TimeseriesQuery.html) for meanings of fields not defined below. + +|property|description|required?| +|--------|-----------|---------| +|dimension|A JSON object defining the dimension that you want the top taken for. For more info, see [DimensionSpecs](DimensionSpecs.html)|yes| +|threshold|An integer defining the N in the topN (i.e. how many you want in the top list)|yes| +|metric|A JSON object specifying the metric to sort by for the top list. For more info, see [TopNMetricSpec](TopNMetricSpec.html).|yes| + +Please note the context JSON object is also available for topN queries and should be used with the same caution as the timeseries case. +The format of the results would look like so: + +```json +[ + { + "timestamp": "2013-08-31T00:00:00.000Z", + "result": [ + { + "user": "67.173.175.77", + "count": 111, + "some_metrics": 10669, + "average": 96.11711711711712 + }, + { + "user": "24.10.49.170", + "count": 88, + "some_metrics": 28344, + "average": 322.09090909090907 + }, + { + "user": "72.193.24.148", + "count": 70, + "some_metrics": 871, + "average": 12.442857142857143 + }, + { + "user": "108.46.28.47", + "count": 62, + "some_metrics": 815, + "average": 13.14516129032258 + }, + { + "user": "99.181.143.133", + "count": 60, + "some_metrics": 2787, + "average": 46.45 + } + ] + } +] +``` diff --git a/docs/content/toc.textile b/docs/content/toc.textile index 802d94ef434..2a5f7ed185d 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -42,6 +42,8 @@ h2. Querying ** "SegmentMetadataQuery":./SegmentMetadataQuery.html ** "TimeBoundaryQuery":./TimeBoundaryQuery.html ** "TimeseriesQuery":./TimeseriesQuery.html +** "TopNQuery":./TopNQuery.html +*** "TopNMetricSpec":./TopNMetricSpec.html h2. Architecture * "Design":./Design.html diff --git a/processing/src/main/java/io/druid/query/Query.java b/processing/src/main/java/io/druid/query/Query.java index daf1410308e..f4ad958e75d 100644 --- a/processing/src/main/java/io/druid/query/Query.java +++ b/processing/src/main/java/io/druid/query/Query.java @@ -28,6 +28,7 @@ import io.druid.query.search.search.SearchQuery; import io.druid.query.spec.QuerySegmentSpec; import io.druid.query.timeboundary.TimeBoundaryQuery; import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.topn.TopNQuery; import org.joda.time.Duration; import org.joda.time.Interval; @@ -40,7 +41,8 @@ import java.util.Map; @JsonSubTypes.Type(name = Query.SEARCH, value = SearchQuery.class), @JsonSubTypes.Type(name = Query.TIME_BOUNDARY, value = TimeBoundaryQuery.class), @JsonSubTypes.Type(name = Query.GROUP_BY, value = GroupByQuery.class), - @JsonSubTypes.Type(name = Query.SEGMENT_METADATA, value = SegmentMetadataQuery.class) + @JsonSubTypes.Type(name = Query.SEGMENT_METADATA, value = SegmentMetadataQuery.class), + @JsonSubTypes.Type(name = Query.TOPN, value = TopNQuery.class) }) public interface Query { @@ -49,6 +51,7 @@ public interface Query public static final String TIME_BOUNDARY = "timeBoundary"; public static final String GROUP_BY = "groupBy"; public static final String SEGMENT_METADATA = "segmentMetadata"; + public static final String TOPN = "topN"; public String getDataSource(); diff --git a/processing/src/main/java/io/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java b/processing/src/main/java/io/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java new file mode 100644 index 00000000000..ea878736bbd --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java @@ -0,0 +1,174 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.google.common.collect.Lists; +import com.metamx.common.ISE; +import io.druid.collections.StupidPool; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.segment.Capabilities; +import io.druid.segment.Cursor; +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; + +/** + */ +public class AggregateTopNMetricFirstAlgorithm implements TopNAlgorithm +{ + private final Capabilities capabilities; + private final TopNQuery query; + private final Comparator comparator; + private final StupidPool bufferPool; + + public AggregateTopNMetricFirstAlgorithm( + Capabilities capabilities, + TopNQuery query, + StupidPool bufferPool + ) + { + this.capabilities = capabilities; + this.query = query; + this.comparator = query.getTopNMetricSpec() + .getComparator(query.getAggregatorSpecs(), query.getPostAggregatorSpecs()); + this.bufferPool = bufferPool; + } + + + @Override + public TopNParams makeInitParams( + DimensionSelector dimSelector, Cursor cursor + ) + { + return new TopNParams(dimSelector, cursor, dimSelector.getValueCardinality(), Integer.MAX_VALUE); + } + + @Override + public TopNResultBuilder makeResultBuilder(TopNParams params) + { + return query.getTopNMetricSpec().getResultBuilder( + params.getCursor().getTime(), query.getDimensionSpec(), query.getThreshold(), comparator + ); + } + + @Override + public void run( + TopNParams params, TopNResultBuilder resultBuilder, int[] ints + ) + { + final TopNResultBuilder singleMetricResultBuilder = makeResultBuilder(params); + final String metric = ((NumericTopNMetricSpec) query.getTopNMetricSpec()).getMetric(); + + // Find either the aggregator or post aggregator to do the topN over + List condensedAggs = Lists.newArrayList(); + for (AggregatorFactory aggregatorSpec : query.getAggregatorSpecs()) { + if (aggregatorSpec.getName().equalsIgnoreCase(metric)) { + condensedAggs.add(aggregatorSpec); + break; + } + } + List condensedPostAggs = Lists.newArrayList(); + if (condensedAggs.isEmpty()) { + for (PostAggregator postAggregator : query.getPostAggregatorSpecs()) { + if (postAggregator.getName().equalsIgnoreCase(metric)) { + condensedPostAggs.add(postAggregator); + + // Add all dependent metrics + for (AggregatorFactory aggregatorSpec : query.getAggregatorSpecs()) { + if (postAggregator.getDependentFields().contains(aggregatorSpec.getName())) { + condensedAggs.add(aggregatorSpec); + } + } + break; + } + } + } + if (condensedAggs.isEmpty() && condensedPostAggs.isEmpty()) { + throw new ISE("WTF! Can't find the metric to do topN over?"); + } + + // Run topN for only a single metric + TopNQuery singleMetricQuery = new TopNQueryBuilder().copy(query) + .aggregators(condensedAggs) + .postAggregators(condensedPostAggs) + .build(); + + PooledTopNAlgorithm singleMetricAlgo = new PooledTopNAlgorithm(capabilities, singleMetricQuery, bufferPool); + PooledTopNAlgorithm.PooledTopNParams singleMetricParam = null; + int[] dimValSelector = null; + try { + singleMetricParam = singleMetricAlgo.makeInitParams(params.getDimSelector(), params.getCursor()); + singleMetricAlgo.run( + singleMetricParam, + singleMetricResultBuilder, + null + ); + + // Get only the topN dimension values + dimValSelector = getDimValSelectorForTopNMetric(singleMetricParam, singleMetricResultBuilder); + } + finally { + if (singleMetricParam != null) { + singleMetricAlgo.cleanup(singleMetricParam); + } + } + + PooledTopNAlgorithm allMetricAlgo = new PooledTopNAlgorithm(capabilities, query, bufferPool); + PooledTopNAlgorithm.PooledTopNParams allMetricsParam = null; + try { + // Run topN for all metrics for top N dimension values + allMetricsParam = allMetricAlgo.makeInitParams(params.getDimSelector(), params.getCursor()); + allMetricAlgo.run( + allMetricsParam, + resultBuilder, + dimValSelector + ); + } + finally { + if (allMetricsParam != null) { + allMetricAlgo.cleanup(allMetricsParam); + } + } + } + + @Override + public void cleanup(TopNParams params) + { + } + + private int[] getDimValSelectorForTopNMetric(TopNParams params, TopNResultBuilder resultBuilder) + { + int[] dimValSelector = new int[params.getDimSelector().getValueCardinality()]; + Arrays.fill(dimValSelector, SKIP_POSITION_VALUE); + + Iterator dimValIter = resultBuilder.getTopNIterator(); + while (dimValIter.hasNext()) { + int dimValIndex = (Integer) dimValIter.next().getDimValIndex(); + dimValSelector[dimValIndex] = INIT_POSITION_VALUE; + } + + return dimValSelector; + } +} diff --git a/processing/src/main/java/io/druid/query/topn/BaseTopNAlgorithm.java b/processing/src/main/java/io/druid/query/topn/BaseTopNAlgorithm.java new file mode 100644 index 00000000000..47093cea8a5 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/BaseTopNAlgorithm.java @@ -0,0 +1,234 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.metamx.common.Pair; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.Capabilities; +import io.druid.segment.Cursor; +import io.druid.segment.DimensionSelector; + +import java.util.Arrays; +import java.util.List; + +/** + */ +public abstract class BaseTopNAlgorithm + implements TopNAlgorithm +{ + protected static Aggregator[] makeAggregators(Cursor cursor, List aggregatorSpecs) + { + Aggregator[] aggregators = new Aggregator[aggregatorSpecs.size()]; + int aggregatorIndex = 0; + for (AggregatorFactory spec : aggregatorSpecs) { + aggregators[aggregatorIndex] = spec.factorize(cursor); + ++aggregatorIndex; + } + return aggregators; + } + + protected static BufferAggregator[] makeBufferAggregators(Cursor cursor, List aggregatorSpecs) + { + BufferAggregator[] aggregators = new BufferAggregator[aggregatorSpecs.size()]; + int aggregatorIndex = 0; + for (AggregatorFactory spec : aggregatorSpecs) { + aggregators[aggregatorIndex] = spec.factorizeBuffered(cursor); + ++aggregatorIndex; + } + return aggregators; + } + + private final Capabilities capabilities; + + protected BaseTopNAlgorithm(Capabilities capabilities) + { + this.capabilities = capabilities; + } + + @Override + public void run( + Parameters params, + TopNResultBuilder resultBuilder, + DimValSelector dimValSelector + ) + { + boolean hasDimValSelector = (dimValSelector != null); + + final int cardinality = params.getCardinality(); + int numProcessed = 0; + while (numProcessed < cardinality) { + final int numToProcess = Math.min(params.getNumValuesPerPass(), cardinality - numProcessed); + + params.getCursor().reset(); + + DimValSelector theDimValSelector; + if (!hasDimValSelector) { + theDimValSelector = makeDimValSelector(params, numProcessed, numToProcess); + } else { + theDimValSelector = updateDimValSelector(dimValSelector, numProcessed, numToProcess); + } + + DimValAggregateStore aggregatesStore = makeDimValAggregateStore(params); + + scanAndAggregate(params, theDimValSelector, aggregatesStore, numProcessed); + + updateResults(params, theDimValSelector, aggregatesStore, resultBuilder); + + closeAggregators(aggregatesStore); + + numProcessed += numToProcess; + } + } + + protected abstract DimValSelector makeDimValSelector(Parameters params, int numProcessed, int numToProcess); + + protected abstract DimValSelector updateDimValSelector( + DimValSelector dimValSelector, + int numProcessed, + int numToProcess + ); + + protected abstract DimValAggregateStore makeDimValAggregateStore(Parameters params); + + protected abstract void scanAndAggregate( + Parameters params, + DimValSelector dimValSelector, + DimValAggregateStore dimValAggregateStore, + int numProcessed + ); + + protected abstract void updateResults( + Parameters params, + DimValSelector dimValSelector, + DimValAggregateStore dimValAggregateStore, + TopNResultBuilder resultBuilder + ); + + protected abstract void closeAggregators( + DimValAggregateStore dimValAggregateStore + ); + + protected class AggregatorArrayProvider extends BaseArrayProvider + { + Aggregator[][] expansionAggs; + int cardinality; + + public AggregatorArrayProvider(DimensionSelector dimSelector, TopNQuery query, int cardinality) + { + super(dimSelector, query, capabilities); + + this.expansionAggs = new Aggregator[cardinality][]; + this.cardinality = cardinality; + } + + @Override + public Aggregator[][] build() + { + Pair startEnd = computeStartEnd(cardinality); + + Arrays.fill(expansionAggs, 0, startEnd.lhs, EMPTY_ARRAY); + Arrays.fill(expansionAggs, startEnd.lhs, startEnd.rhs, null); + Arrays.fill(expansionAggs, startEnd.rhs, expansionAggs.length, EMPTY_ARRAY); + + return expansionAggs; + } + } + + protected static abstract class BaseArrayProvider implements TopNMetricSpecBuilder + { + private volatile String previousStop; + private volatile boolean ignoreAfterThreshold; + private volatile int ignoreFirstN; + private volatile int keepOnlyN; + + private final DimensionSelector dimSelector; + private final TopNQuery query; + private final Capabilities capabilities; + + public BaseArrayProvider( + DimensionSelector dimSelector, + TopNQuery query, + Capabilities capabilities + ) + { + this.dimSelector = dimSelector; + this.query = query; + this.capabilities = capabilities; + + previousStop = null; + ignoreAfterThreshold = false; + ignoreFirstN = 0; + keepOnlyN = dimSelector.getValueCardinality(); + } + + @Override + public void skipTo(String previousStop) + { + if (capabilities.dimensionValuesSorted()) { + this.previousStop = previousStop; + } + } + + @Override + public void ignoreAfterThreshold() + { + ignoreAfterThreshold = true; + } + + @Override + public void ignoreFirstN(int n) + { + ignoreFirstN = n; + } + + @Override + public void keepOnlyN(int n) + { + keepOnlyN = n; + } + + protected Pair computeStartEnd(int cardinality) + { + int startIndex = ignoreFirstN; + + if (previousStop != null) { + int lookupId = dimSelector.lookupId(previousStop) + 1; + if (lookupId < 0) { + lookupId *= -1; + } + if (lookupId > ignoreFirstN + keepOnlyN) { + startIndex = ignoreFirstN + keepOnlyN; + } else { + startIndex = Math.max(lookupId, startIndex); + } + } + + int endIndex = Math.min(ignoreFirstN + keepOnlyN, cardinality); + + if (ignoreAfterThreshold && query.getDimensionsFilter() == null) { + endIndex = Math.min(endIndex, startIndex + query.getThreshold()); + } + + return Pair.of(startIndex, endIndex); + } + } +} diff --git a/processing/src/main/java/io/druid/query/topn/BySegmentTopNResultValue.java b/processing/src/main/java/io/druid/query/topn/BySegmentTopNResultValue.java new file mode 100644 index 00000000000..56849d6fd47 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/BySegmentTopNResultValue.java @@ -0,0 +1,90 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonValue; +import io.druid.query.BySegmentResultValue; +import io.druid.query.Result; + +import java.util.List; + +/** + */ +public class BySegmentTopNResultValue extends TopNResultValue implements BySegmentResultValue +{ + private final List> results; + private final String segmentId; + private final String intervalString; + + @JsonCreator + public BySegmentTopNResultValue( + @JsonProperty("results") List> results, + @JsonProperty("segment") String segmentId, + @JsonProperty("interval") String intervalString + ) + { + super(null); + + this.results = results; + this.segmentId = segmentId; + this.intervalString = intervalString; + } + + @Override + @JsonValue(false) + public List getValue() + { + throw new UnsupportedOperationException(); + } + + + @Override + @JsonProperty("results") + public List> getResults() + { + return results; + } + + @Override + @JsonProperty("segment") + public String getSegmentId() + { + return segmentId; + } + + @Override + @JsonProperty("interval") + public String getIntervalString() + { + return intervalString; + } + + @Override + public String toString() + { + return "BySegmentTopNResultValue{" + + "results=" + results + + ", segmentId='" + segmentId + '\'' + + ", intervalString='" + intervalString + '\'' + + '}'; + } +} diff --git a/processing/src/main/java/io/druid/query/topn/DimExtractionTopNAlgorithm.java b/processing/src/main/java/io/druid/query/topn/DimExtractionTopNAlgorithm.java new file mode 100644 index 00000000000..95ce5312e9d --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/DimExtractionTopNAlgorithm.java @@ -0,0 +1,169 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.google.common.collect.Maps; +import io.druid.query.aggregation.Aggregator; +import io.druid.segment.Capabilities; +import io.druid.segment.Cursor; +import io.druid.segment.DimensionSelector; +import io.druid.segment.data.IndexedInts; + +import java.util.Comparator; +import java.util.Map; + +/** + */ +public class DimExtractionTopNAlgorithm extends BaseTopNAlgorithm, TopNParams> +{ + private final TopNQuery query; + private final Comparator comparator; + + public DimExtractionTopNAlgorithm( + Capabilities capabilities, + TopNQuery query + ) + { + super(capabilities); + + this.query = query; + this.comparator = query.getTopNMetricSpec() + .getComparator(query.getAggregatorSpecs(), query.getPostAggregatorSpecs()); + } + + @Override + public TopNParams makeInitParams( + final DimensionSelector dimSelector, final Cursor cursor + ) + { + return new TopNParams(dimSelector, cursor, dimSelector.getValueCardinality(), Integer.MAX_VALUE); + } + + @Override + public TopNResultBuilder makeResultBuilder(TopNParams params) + { + return query.getTopNMetricSpec().getResultBuilder( + params.getCursor().getTime(), query.getDimensionSpec(), query.getThreshold(), comparator + ); + } + + @Override + protected Aggregator[][] makeDimValSelector(TopNParams params, int numProcessed, int numToProcess) + { + return query.getTopNMetricSpec().configureOptimizer( + new AggregatorArrayProvider(params.getDimSelector(), query, params.getCardinality()) + ).build(); + } + + @Override + protected Aggregator[][] updateDimValSelector(Aggregator[][] aggregators, int numProcessed, int numToProcess) + { + return aggregators; + } + + @Override + protected Map makeDimValAggregateStore(TopNParams params) + { + return Maps.newHashMap(); + } + + @Override + public void scanAndAggregate( + TopNParams params, + Aggregator[][] rowSelector, + Map aggregatesStore, + int numProcessed + ) + { + final Cursor cursor = params.getCursor(); + final DimensionSelector dimSelector = params.getDimSelector(); + + while (!cursor.isDone()) { + final IndexedInts dimValues = dimSelector.getRow(); + + for (int i = 0; i < dimValues.size(); ++i) { + final int dimIndex = dimValues.get(i); + + Aggregator[] theAggregators = rowSelector[dimIndex]; + if (theAggregators == null) { + String key = query.getDimensionSpec().getDimExtractionFn().apply(dimSelector.lookupName(dimIndex)); + if (key == null) { + rowSelector[dimIndex] = EMPTY_ARRAY; + continue; + } + theAggregators = aggregatesStore.get(key); + if (theAggregators == null) { + theAggregators = makeAggregators(cursor, query.getAggregatorSpecs()); + aggregatesStore.put(key, theAggregators); + } + rowSelector[dimIndex] = theAggregators; + } + + for (Aggregator aggregator : theAggregators) { + aggregator.aggregate(); + } + } + + cursor.advance(); + } + } + + @Override + protected void updateResults( + TopNParams params, + Aggregator[][] rowSelector, + Map aggregatesStore, + TopNResultBuilder resultBuilder + ) + { + for (Map.Entry entry : aggregatesStore.entrySet()) { + Aggregator[] aggs = entry.getValue(); + if (aggs != null && aggs.length > 0) { + Object[] vals = new Object[aggs.length]; + for (int i = 0; i < aggs.length; i++) { + vals[i] = aggs[i].get(); + } + + resultBuilder.addEntry( + entry.getKey(), + entry.getKey(), + vals, + query.getAggregatorSpecs(), + query.getPostAggregatorSpecs() + ); + } + } + } + + @Override + protected void closeAggregators(Map stringMap) + { + for (Aggregator[] aggregators : stringMap.values()) { + for (Aggregator agg : aggregators) { + agg.close(); + } + } + } + + @Override + public void cleanup(TopNParams params) + { + } +} diff --git a/processing/src/main/java/io/druid/query/topn/DimValHolder.java b/processing/src/main/java/io/druid/query/topn/DimValHolder.java new file mode 100644 index 00000000000..f17c77685c0 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/DimValHolder.java @@ -0,0 +1,110 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import java.util.Map; + +/** + */ +public class DimValHolder +{ + private final Object topNMetricVal; + private final String dimName; + private final Object dimValIndex; + private final Map metricValues; + + public DimValHolder( + Object topNMetricVal, + String dimName, + Object dimValIndex, + Map metricValues + ) + { + this.topNMetricVal = topNMetricVal; + this.dimName = dimName; + this.dimValIndex = dimValIndex; + this.metricValues = metricValues; + } + + public Object getTopNMetricVal() + { + return topNMetricVal; + } + + public String getDimName() + { + return dimName; + } + + public Object getDimValIndex() + { + return dimValIndex; + } + + public Map getMetricValues() + { + return metricValues; + } + + public static class Builder + { + private Object topNMetricVal; + private String dirName; + private Object dimValIndex; + private Map metricValues; + + public Builder() + { + topNMetricVal = null; + dirName = null; + dimValIndex = null; + metricValues = null; + } + + public Builder withTopNMetricVal(Object topNMetricVal) + { + this.topNMetricVal = topNMetricVal; + return this; + } + + public Builder withDirName(String dirName) + { + this.dirName = dirName; + return this; + } + + public Builder withDimValIndex(Object dimValIndex) + { + this.dimValIndex = dimValIndex; + return this; + } + + public Builder withMetricValues(Map metricValues) + { + this.metricValues = metricValues; + return this; + } + + public DimValHolder build() + { + return new DimValHolder(topNMetricVal, dirName, dimValIndex, metricValues); + } + } +} diff --git a/processing/src/main/java/io/druid/query/topn/DimensionAndMetricValueExtractor.java b/processing/src/main/java/io/druid/query/topn/DimensionAndMetricValueExtractor.java new file mode 100644 index 00000000000..500074cebbd --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/DimensionAndMetricValueExtractor.java @@ -0,0 +1,83 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.fasterxml.jackson.annotation.JsonCreator; +import io.druid.query.MetricValueExtractor; + +import java.util.Map; + +/** + */ +public class DimensionAndMetricValueExtractor extends MetricValueExtractor +{ + private final Map value; + + @JsonCreator + public DimensionAndMetricValueExtractor(Map value) + { + super(value); + + this.value = value; + } + + public String getStringDimensionValue(String dimension) + { + return (String) value.get(dimension); + } + + public Object getDimensionValue(String dimension) + { + return value.get(dimension); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + DimensionAndMetricValueExtractor that = (DimensionAndMetricValueExtractor) o; + + if (value != null ? !value.equals(that.value) : that.value != null) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + return value != null ? value.hashCode() : 0; + } + + @Override + public String toString() + { + return "DimensionAndMetricValueExtractor{" + + "value=" + value + + '}'; + } +} diff --git a/processing/src/main/java/io/druid/query/topn/InvertedTopNMetricSpec.java b/processing/src/main/java/io/druid/query/topn/InvertedTopNMetricSpec.java new file mode 100644 index 00000000000..2e458295387 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/InvertedTopNMetricSpec.java @@ -0,0 +1,104 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.metamx.common.guava.Comparators; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.dimension.DimensionSpec; +import org.joda.time.DateTime; + +import java.nio.ByteBuffer; +import java.util.Comparator; +import java.util.List; + +/** + */ +public class InvertedTopNMetricSpec implements TopNMetricSpec +{ + private static final byte CACHE_TYPE_ID = 0x3; + + private final TopNMetricSpec delegate; + + @JsonCreator + public InvertedTopNMetricSpec( + @JsonProperty("metric") TopNMetricSpec delegate + ) + { + this.delegate = delegate; + } + + @Override + public void verifyPreconditions( + List aggregatorSpecs, + List postAggregatorSpecs + ) + { + delegate.verifyPreconditions(aggregatorSpecs, postAggregatorSpecs); + } + + @JsonProperty("metric") + public TopNMetricSpec getDelegate() + { + return delegate; + } + + @Override + public Comparator getComparator( + List aggregatorSpecs, + List postAggregatorSpecs + ) + { + return Comparators.inverse(delegate.getComparator(aggregatorSpecs, postAggregatorSpecs)); + } + + @Override + public TopNResultBuilder getResultBuilder( + DateTime timestamp, + DimensionSpec dimSpec, + int threshold, + Comparator comparator + ) + { + return delegate.getResultBuilder(timestamp, dimSpec, threshold, comparator); + } + + @Override + public byte[] getCacheKey() + { + final byte[] cacheKey = delegate.getCacheKey(); + + return ByteBuffer.allocate(1 + cacheKey.length).put(CACHE_TYPE_ID).put(cacheKey).array(); + } + + @Override + public TopNMetricSpecBuilder configureOptimizer(TopNMetricSpecBuilder builder) + { + return delegate.configureOptimizer(builder); + } + + @Override + public void initTopNAlgorithmSelector(TopNAlgorithmSelector selector) + { + delegate.initTopNAlgorithmSelector(selector); + } +} diff --git a/processing/src/main/java/io/druid/query/topn/LegacyTopNMetricSpec.java b/processing/src/main/java/io/druid/query/topn/LegacyTopNMetricSpec.java new file mode 100644 index 00000000000..f185d799ea1 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/LegacyTopNMetricSpec.java @@ -0,0 +1,51 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.metamx.common.IAE; + +import java.util.Map; + +/** + */ +public class LegacyTopNMetricSpec extends NumericTopNMetricSpec +{ + private static final String convertValue(Object metric) + { + final String retVal; + + if (metric instanceof String) { + retVal = (String) metric; + } else if (metric instanceof Map) { + retVal = (String) ((Map) metric).get("metric"); + } else { + throw new IAE("Unknown type[%s] for metric[%s]", metric.getClass(), metric); + } + + return retVal; + } + + @JsonCreator + public LegacyTopNMetricSpec(Object metric) + { + super(convertValue(metric)); + } +} diff --git a/processing/src/main/java/io/druid/query/topn/LexicographicTopNMetricSpec.java b/processing/src/main/java/io/druid/query/topn/LexicographicTopNMetricSpec.java new file mode 100644 index 00000000000..dca6d3ed651 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/LexicographicTopNMetricSpec.java @@ -0,0 +1,121 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; +import com.google.common.primitives.UnsignedBytes; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.dimension.DimensionSpec; +import org.joda.time.DateTime; + +import java.nio.ByteBuffer; +import java.util.Comparator; +import java.util.List; + +/** + */ +public class LexicographicTopNMetricSpec implements TopNMetricSpec +{ + private static final byte CACHE_TYPE_ID = 0x1; + + private static Comparator comparator = new Comparator() + { + @Override + public int compare(String s, String s2) + { + return UnsignedBytes.lexicographicalComparator().compare(s.getBytes(Charsets.UTF_8), s2.getBytes(Charsets.UTF_8)); + } + }; + + private final String previousStop; + + @JsonCreator + public LexicographicTopNMetricSpec( + @JsonProperty("previousStop") String previousStop + ) + { + this.previousStop = (previousStop == null) ? "" : previousStop; + } + + @Override + public void verifyPreconditions(List aggregatorSpecs, List postAggregatorSpecs) + { + } + + @JsonProperty + public String getPreviousStop() + { + return previousStop; + } + + + @Override + public Comparator getComparator(List aggregatorSpecs, List postAggregatorSpecs) + { + return comparator; + } + + @Override + public TopNResultBuilder getResultBuilder( + DateTime timestamp, + DimensionSpec dimSpec, + int threshold, + Comparator comparator + ) + { + return new TopNLexicographicResultBuilder(timestamp, dimSpec, threshold, previousStop, comparator); + } + + @Override + public byte[] getCacheKey() + { + byte[] previousStopBytes = previousStop.getBytes(Charsets.UTF_8); + + return ByteBuffer.allocate(1 + previousStopBytes.length) + .put(CACHE_TYPE_ID) + .put(previousStopBytes) + .array(); + } + + @Override + public TopNMetricSpecBuilder configureOptimizer(TopNMetricSpecBuilder builder) + { + builder.skipTo(previousStop); + builder.ignoreAfterThreshold(); + return builder; + } + + @Override + public void initTopNAlgorithmSelector(TopNAlgorithmSelector selector) + { + selector.setAggregateAllMetrics(true); + } + + @Override + public String toString() + { + return "LexicographicTopNMetricSpec{" + + "previousStop='" + previousStop + '\'' + + '}'; + } +} diff --git a/processing/src/main/java/io/druid/query/topn/NumericTopNMetricSpec.java b/processing/src/main/java/io/druid/query/topn/NumericTopNMetricSpec.java new file mode 100644 index 00000000000..4f0b6ebdc22 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/NumericTopNMetricSpec.java @@ -0,0 +1,160 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; +import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.dimension.DimensionSpec; +import org.joda.time.DateTime; + +import java.nio.ByteBuffer; +import java.util.Comparator; +import java.util.List; + +/** + */ +public class NumericTopNMetricSpec implements TopNMetricSpec +{ + private static final byte CACHE_TYPE_ID = 0x0; + + private final String metric; + + @JsonCreator + public NumericTopNMetricSpec( + @JsonProperty("metric") String metric + ) + { + this.metric = metric; + } + + @Override + public void verifyPreconditions(List aggregatorSpecs, List postAggregatorSpecs) + { + Preconditions.checkNotNull(metric, "metric can't be null"); + Preconditions.checkNotNull(aggregatorSpecs, "aggregations cannot be null"); + Preconditions.checkArgument(aggregatorSpecs.size() > 0, "Must have at least one AggregatorFactory"); + + final AggregatorFactory aggregator = Iterables.tryFind( + aggregatorSpecs, + new Predicate() + { + @Override + public boolean apply(AggregatorFactory input) + { + return input.getName().equals(metric); + } + } + ).orNull(); + + final PostAggregator postAggregator = Iterables.tryFind( + postAggregatorSpecs, + new Predicate() + { + @Override + public boolean apply(PostAggregator input) + { + return input.getName().equals(metric); + } + } + ).orNull(); + + Preconditions.checkArgument( + aggregator != null || postAggregator != null, + "Must have an AggregatorFactory or PostAggregator for metric[%s], gave[%s] and [%s]", + metric, + aggregatorSpecs, + postAggregatorSpecs + ); + } + + @JsonProperty + public String getMetric() + { + return metric; + } + + @Override + public Comparator getComparator(List aggregatorSpecs, List postAggregatorSpecs) + { + Comparator comp = null; + for (AggregatorFactory factory : aggregatorSpecs) { + if (metric.equals(factory.getName())) { + comp = factory.getComparator(); + break; + } + } + for (PostAggregator pf : postAggregatorSpecs) { + if (metric.equals(pf.getName())) { + comp = pf.getComparator(); + break; + } + } + + return comp; + } + + @Override + public TopNResultBuilder getResultBuilder( + DateTime timestamp, + DimensionSpec dimSpec, + int threshold, + Comparator comparator + ) + { + return new TopNNumericResultBuilder(timestamp, dimSpec, metric, threshold, comparator); + } + + @Override + public byte[] getCacheKey() + { + byte[] metricBytes = metric.getBytes(Charsets.UTF_8); + + return ByteBuffer.allocate(1 + metricBytes.length) + .put(CACHE_TYPE_ID) + .put(metricBytes) + .array(); + } + + @Override + public TopNMetricSpecBuilder configureOptimizer(TopNMetricSpecBuilder builder) + { + return builder; + } + + @Override + public void initTopNAlgorithmSelector(TopNAlgorithmSelector selector) + { + selector.setAggregateTopNMetricFirst(true); + } + + @Override + public String toString() + { + return "NumericTopNMetricSpec{" + + "metric='" + metric + '\'' + + '}'; + } +} diff --git a/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java b/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java new file mode 100644 index 00000000000..d87631c7b57 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java @@ -0,0 +1,401 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.google.common.io.Closeables; +import com.metamx.common.Pair; +import io.druid.collections.ResourceHolder; +import io.druid.collections.StupidPool; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.Capabilities; +import io.druid.segment.Cursor; +import io.druid.segment.DimensionSelector; +import io.druid.segment.data.IndexedInts; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Comparator; + +/** + */ +public class PooledTopNAlgorithm extends BaseTopNAlgorithm +{ + private final Capabilities capabilities; + private final TopNQuery query; + private final Comparator comparator; + private final StupidPool bufferPool; + + public PooledTopNAlgorithm( + Capabilities capabilities, + TopNQuery query, + StupidPool bufferPool + ) + { + super(capabilities); + + this.capabilities = capabilities; + this.query = query; + this.comparator = query.getTopNMetricSpec() + .getComparator(query.getAggregatorSpecs(), query.getPostAggregatorSpecs()); + this.bufferPool = bufferPool; + } + + @Override + public PooledTopNParams makeInitParams( + DimensionSelector dimSelector, Cursor cursor + ) + { + ResourceHolder resultsBufHolder = bufferPool.take(); + ByteBuffer resultsBuf = resultsBufHolder.get(); + resultsBuf.clear(); + + final int cardinality = dimSelector.getValueCardinality(); + + final TopNMetricSpecBuilder arrayProvider = new BaseArrayProvider( + dimSelector, + query, + capabilities + ) + { + private final int[] positions = new int[cardinality]; + + @Override + public int[] build() + { + Pair startEnd = computeStartEnd(cardinality); + + Arrays.fill(positions, 0, startEnd.lhs, SKIP_POSITION_VALUE); + Arrays.fill(positions, startEnd.lhs, startEnd.rhs, INIT_POSITION_VALUE); + Arrays.fill(positions, startEnd.rhs, positions.length, SKIP_POSITION_VALUE); + + return positions; + } + }; + + final int numBytesToWorkWith = resultsBuf.remaining(); + final int[] aggregatorSizes = new int[query.getAggregatorSpecs().size()]; + int numBytesPerRecord = 0; + + for (int i = 0; i < query.getAggregatorSpecs().size(); ++i) { + aggregatorSizes[i] = query.getAggregatorSpecs().get(i).getMaxIntermediateSize(); + numBytesPerRecord += aggregatorSizes[i]; + } + + final int numValuesPerPass = numBytesToWorkWith / numBytesPerRecord; + + return PooledTopNParams.builder() + .withDimSelector(dimSelector) + .withCursor(cursor) + .withCardinality(cardinality) + .withResultsBufHolder(resultsBufHolder) + .withResultsBuf(resultsBuf) + .withArrayProvider(arrayProvider) + .withNumBytesPerRecord(numBytesPerRecord) + .withNumValuesPerPass(numValuesPerPass) + .withAggregatorSizes(aggregatorSizes) + .build(); + } + + @Override + public TopNResultBuilder makeResultBuilder(PooledTopNParams params) + { + return query.getTopNMetricSpec().getResultBuilder( + params.getCursor().getTime(), query.getDimensionSpec(), query.getThreshold(), comparator + ); + } + + @Override + protected int[] makeDimValSelector(PooledTopNParams params, int numProcessed, int numToProcess) + { + final TopNMetricSpecBuilder arrayProvider = params.getArrayProvider(); + + arrayProvider.ignoreFirstN(numProcessed); + arrayProvider.keepOnlyN(numToProcess); + return query.getTopNMetricSpec().configureOptimizer(arrayProvider).build(); + } + + protected int[] updateDimValSelector(int[] dimValSelector, int numProcessed, int numToProcess) + { + final int[] retVal = Arrays.copyOf(dimValSelector, dimValSelector.length); + + final int validEnd = Math.min(retVal.length, numProcessed + numToProcess); + final int end = Math.max(retVal.length, validEnd); + + Arrays.fill(retVal, 0, numProcessed, SKIP_POSITION_VALUE); + Arrays.fill(retVal, validEnd, end, SKIP_POSITION_VALUE); + + return retVal; + } + + @Override + protected BufferAggregator[] makeDimValAggregateStore(PooledTopNParams params) + { + return makeBufferAggregators(params.getCursor(), query.getAggregatorSpecs()); + } + + @Override + protected void scanAndAggregate( + PooledTopNParams params, + int[] positions, + BufferAggregator[] theAggregators, + int numProcessed + ) + { + final ByteBuffer resultsBuf = params.getResultsBuf(); + final int numBytesPerRecord = params.getNumBytesPerRecord(); + final int[] aggregatorSizes = params.getAggregatorSizes(); + final Cursor cursor = params.getCursor(); + final DimensionSelector dimSelector = params.getDimSelector(); + + while (!cursor.isDone()) { + final IndexedInts dimValues = dimSelector.getRow(); + + for (int i = 0; i < dimValues.size(); ++i) { + final int dimIndex = dimValues.get(i); + int position = positions[dimIndex]; + switch (position) { + case SKIP_POSITION_VALUE: + break; + case INIT_POSITION_VALUE: + positions[dimIndex] = (dimIndex - numProcessed) * numBytesPerRecord; + position = positions[dimIndex]; + for (int j = 0; j < theAggregators.length; ++j) { + theAggregators[j].init(resultsBuf, position); + position += aggregatorSizes[j]; + } + position = positions[dimIndex]; + default: + for (int j = 0; j < theAggregators.length; ++j) { + theAggregators[j].aggregate(resultsBuf, position); + position += aggregatorSizes[j]; + } + } + } + + cursor.advance(); + } + } + + @Override + protected void updateResults( + PooledTopNParams params, + int[] positions, + BufferAggregator[] theAggregators, + TopNResultBuilder resultBuilder + ) + { + final ByteBuffer resultsBuf = params.getResultsBuf(); + final int[] aggregatorSizes = params.getAggregatorSizes(); + final DimensionSelector dimSelector = params.getDimSelector(); + + for (int i = 0; i < positions.length; i++) { + int position = positions[i]; + if (position >= 0) { + Object[] vals = new Object[theAggregators.length]; + for (int j = 0; j < theAggregators.length; j++) { + vals[j] = theAggregators[j].get(resultsBuf, position); + position += aggregatorSizes[j]; + } + + resultBuilder.addEntry( + dimSelector.lookupName(i), + i, + vals, + query.getAggregatorSpecs(), + query.getPostAggregatorSpecs() + ); + } + } + } + + @Override + protected void closeAggregators(BufferAggregator[] bufferAggregators) + { + for(BufferAggregator agg : bufferAggregators) { + agg.close(); + } + } + + @Override + public void cleanup(PooledTopNParams params) + { + ResourceHolder resultsBufHolder = params.getResultsBufHolder(); + + if (resultsBufHolder != null) { + resultsBufHolder.get().clear(); + } + Closeables.closeQuietly(resultsBufHolder); + } + + public static class PooledTopNParams extends TopNParams + { + public static Builder builder() + { + return new Builder(); + } + + private final ResourceHolder resultsBufHolder; + private final ByteBuffer resultsBuf; + private final int[] aggregatorSizes; + private final int numBytesPerRecord; + private final TopNMetricSpecBuilder arrayProvider; + + public PooledTopNParams( + DimensionSelector dimSelector, + Cursor cursor, + int cardinality, + ResourceHolder resultsBufHolder, + ByteBuffer resultsBuf, + int[] aggregatorSizes, + int numBytesPerRecord, + int numValuesPerPass, + TopNMetricSpecBuilder arrayProvider + ) + { + super(dimSelector, cursor, cardinality, numValuesPerPass); + + this.resultsBufHolder = resultsBufHolder; + this.resultsBuf = resultsBuf; + this.aggregatorSizes = aggregatorSizes; + this.numBytesPerRecord = numBytesPerRecord; + this.arrayProvider = arrayProvider; + } + + public ResourceHolder getResultsBufHolder() + { + return resultsBufHolder; + } + + public ByteBuffer getResultsBuf() + { + return resultsBuf; + } + + public int[] getAggregatorSizes() + { + return aggregatorSizes; + } + + public int getNumBytesPerRecord() + { + return numBytesPerRecord; + } + + public TopNMetricSpecBuilder getArrayProvider() + { + return arrayProvider; + } + + public static class Builder + { + private DimensionSelector dimSelector; + private Cursor cursor; + private int cardinality; + private ResourceHolder resultsBufHolder; + private ByteBuffer resultsBuf; + private int[] aggregatorSizes; + private int numBytesPerRecord; + private int numValuesPerPass; + private TopNMetricSpecBuilder arrayProvider; + + public Builder() + { + dimSelector = null; + cursor = null; + cardinality = 0; + resultsBufHolder = null; + resultsBuf = null; + aggregatorSizes = null; + numBytesPerRecord = 0; + numValuesPerPass = 0; + arrayProvider = null; + } + + public Builder withDimSelector(DimensionSelector dimSelector) + { + this.dimSelector = dimSelector; + return this; + } + + public Builder withCursor(Cursor cursor) + { + this.cursor = cursor; + return this; + } + + public Builder withCardinality(int cardinality) + { + this.cardinality = cardinality; + return this; + } + + public Builder withResultsBufHolder(ResourceHolder resultsBufHolder) + { + this.resultsBufHolder = resultsBufHolder; + return this; + } + + public Builder withResultsBuf(ByteBuffer resultsBuf) + { + this.resultsBuf = resultsBuf; + return this; + } + + public Builder withAggregatorSizes(int[] aggregatorSizes) + { + this.aggregatorSizes = aggregatorSizes; + return this; + } + + public Builder withNumBytesPerRecord(int numBytesPerRecord) + { + this.numBytesPerRecord = numBytesPerRecord; + return this; + } + + public Builder withNumValuesPerPass(int numValuesPerPass) + { + this.numValuesPerPass = numValuesPerPass; + return this; + } + + public Builder withArrayProvider(TopNMetricSpecBuilder arrayProvider) + { + this.arrayProvider = arrayProvider; + return this; + } + + public PooledTopNParams build() + { + return new PooledTopNParams( + dimSelector, + cursor, + cardinality, + resultsBufHolder, + resultsBuf, + aggregatorSizes, + numBytesPerRecord, + numValuesPerPass, + arrayProvider + ); + } + } + } +} diff --git a/processing/src/main/java/io/druid/query/topn/TopNAlgorithm.java b/processing/src/main/java/io/druid/query/topn/TopNAlgorithm.java new file mode 100644 index 00000000000..89bac871441 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/TopNAlgorithm.java @@ -0,0 +1,45 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import io.druid.query.aggregation.Aggregator; +import io.druid.segment.Cursor; +import io.druid.segment.DimensionSelector; + +/** + */ +public interface TopNAlgorithm +{ + public static final Aggregator[] EMPTY_ARRAY = {}; + public static final int INIT_POSITION_VALUE = -1; + public static final int SKIP_POSITION_VALUE = -2; + + public TopNParams makeInitParams(DimensionSelector dimSelector, Cursor cursor); + + public TopNResultBuilder makeResultBuilder(Parameters params); + + public void run( + Parameters params, + TopNResultBuilder resultBuilder, + DimValSelector dimValSelector + ); + + public void cleanup(Parameters params); +} diff --git a/processing/src/main/java/io/druid/query/topn/TopNAlgorithmSelector.java b/processing/src/main/java/io/druid/query/topn/TopNAlgorithmSelector.java new file mode 100644 index 00000000000..424a3aa5950 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/TopNAlgorithmSelector.java @@ -0,0 +1,73 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +/** + */ +public class TopNAlgorithmSelector +{ + private final int cardinality; + private final int numBytesPerRecord; + + private volatile boolean hasDimExtractionFn; + private volatile boolean aggregateAllMetrics; + private volatile boolean aggregateTopNMetricFirst; + + public TopNAlgorithmSelector(int cardinality, int numBytesPerRecord) + { + this.cardinality = cardinality; + this.numBytesPerRecord = numBytesPerRecord; + } + + public void setHasDimExtractionFn(boolean hasDimExtractionFn) + { + this.hasDimExtractionFn = hasDimExtractionFn; + } + + public void setAggregateAllMetrics(boolean aggregateAllMetrics) + { + this.aggregateAllMetrics = aggregateAllMetrics; + } + + public void setAggregateTopNMetricFirst(boolean aggregateTopNMetricFirst) + { + // These are just heuristics based on an analysis of where an inflection point may lie to switch + // between different algorithms + // More info: https://metamarkets.atlassian.net/wiki/display/APP/Top+n+speeds+with+uniques + if (cardinality > 400000 && numBytesPerRecord > 100) { + this.aggregateTopNMetricFirst = aggregateTopNMetricFirst; + } + } + + public boolean isHasDimExtractionFn() + { + return hasDimExtractionFn; + } + + public boolean isAggregateAllMetrics() + { + return aggregateAllMetrics; + } + + public boolean isAggregateTopNMetricFirst() + { + return aggregateTopNMetricFirst; + } +} diff --git a/processing/src/main/java/io/druid/query/topn/TopNBinaryFn.java b/processing/src/main/java/io/druid/query/topn/TopNBinaryFn.java new file mode 100644 index 00000000000..437c28f640f --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/TopNBinaryFn.java @@ -0,0 +1,126 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.metamx.common.guava.nary.BinaryFn; +import io.druid.granularity.AllGranularity; +import io.druid.granularity.QueryGranularity; +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; + +import java.util.Comparator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +/** + */ +public class TopNBinaryFn implements BinaryFn, Result, Result> +{ + private final TopNResultMerger merger; + private final DimensionSpec dimSpec; + private final QueryGranularity gran; + private final String dimension; + private final TopNMetricSpec topNMetricSpec; + private final int threshold; + private final List aggregations; + private final List postAggregations; + private final Comparator comparator; + + public TopNBinaryFn( + final TopNResultMerger merger, + final QueryGranularity granularity, + final DimensionSpec dimSpec, + final TopNMetricSpec topNMetricSpec, + final int threshold, + final List aggregatorSpecs, + final List postAggregatorSpecs + ) + { + this.merger = merger; + this.dimSpec = dimSpec; + this.gran = granularity; + this.topNMetricSpec = topNMetricSpec; + this.threshold = threshold; + this.aggregations = aggregatorSpecs; + this.postAggregations = postAggregatorSpecs; + + this.dimension = dimSpec.getOutputName(); + this.comparator = topNMetricSpec.getComparator(aggregatorSpecs, postAggregatorSpecs); + } + + @Override + public Result apply(Result arg1, Result arg2) + { + if (arg1 == null) { + return merger.getResult(arg2, comparator); + } + if (arg2 == null) { + return merger.getResult(arg1, comparator); + } + + Map retVals = new LinkedHashMap(); + + TopNResultValue arg1Vals = arg1.getValue(); + TopNResultValue arg2Vals = arg2.getValue(); + + for (DimensionAndMetricValueExtractor arg1Val : arg1Vals) { + retVals.put(arg1Val.getStringDimensionValue(dimension), arg1Val); + } + for (DimensionAndMetricValueExtractor arg2Val : arg2Vals) { + final String dimensionValue = arg2Val.getStringDimensionValue(dimension); + DimensionAndMetricValueExtractor arg1Val = retVals.get(dimensionValue); + + if (arg1Val != null) { + Map retVal = new LinkedHashMap(); + + retVal.put(dimension, dimensionValue); + for (AggregatorFactory factory : aggregations) { + final String metricName = factory.getName(); + retVal.put(metricName, factory.combine(arg1Val.getMetric(metricName), arg2Val.getMetric(metricName))); + } + + for (PostAggregator pf : postAggregations) { + retVal.put(pf.getName(), pf.compute(retVal)); + } + + retVals.put(dimensionValue, new DimensionAndMetricValueExtractor(retVal)); + } else { + retVals.put(dimensionValue, arg2Val); + } + } + + final DateTime timestamp; + if (gran instanceof AllGranularity) { + timestamp = arg1.getTimestamp(); + } else { + timestamp = gran.toDateTime(gran.truncate(arg1.getTimestamp().getMillis())); + } + + TopNResultBuilder bob = topNMetricSpec.getResultBuilder(timestamp, dimSpec, threshold, comparator); + for (DimensionAndMetricValueExtractor extractor : retVals.values()) { + bob.addEntry(extractor); + } + return bob.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 new file mode 100644 index 00000000000..37360dfb1cd --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/TopNLexicographicResultBuilder.java @@ -0,0 +1,134 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +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; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + */ +public class TopNLexicographicResultBuilder implements TopNResultBuilder +{ + private final DateTime timestamp; + private final DimensionSpec dimSpec; + private final String previousStop; + + private MinMaxPriorityQueue pQueue = null; + + public TopNLexicographicResultBuilder( + DateTime timestamp, + DimensionSpec dimSpec, + int threshold, + String previousStop, + final Comparator comparator + ) + { + this.timestamp = timestamp; + this.dimSpec = dimSpec; + this.previousStop = previousStop; + + instantiatePQueue(threshold, comparator); + } + + @Override + public TopNResultBuilder addEntry( + String dimName, + Object dimValIndex, + Object[] metricVals, + List aggFactories, + List postAggs + ) + { + Map metricValues = Maps.newLinkedHashMap(); + + if (dimName.compareTo(previousStop) > 0) { + metricValues.put(dimSpec.getOutputName(), dimName); + Iterator aggsIter = aggFactories.iterator(); + for (Object metricVal : metricVals) { + metricValues.put(aggsIter.next().getName(), metricVal); + } + for (PostAggregator postAgg : postAggs) { + metricValues.put(postAgg.getName(), postAgg.compute(metricValues)); + } + + pQueue.add(new DimValHolder.Builder().withDirName(dimName).withMetricValues(metricValues).build()); + } + + return this; + } + + @Override + public TopNResultBuilder addEntry(DimensionAndMetricValueExtractor dimensionAndMetricValueExtractor) + { + pQueue.add( + new DimValHolder.Builder().withDirName(dimensionAndMetricValueExtractor.getStringDimensionValue(dimSpec.getOutputName())) + .withMetricValues(dimensionAndMetricValueExtractor.getBaseObject()) + .build() + ); + + return this; + } + + @Override + public Iterator getTopNIterator() + { + return pQueue.iterator(); + } + + @Override + public Result build() + { + // Pull out top aggregated values + List> values = new ArrayList>(pQueue.size()); + while (!pQueue.isEmpty()) { + values.add(pQueue.remove().getMetricValues()); + } + + return new Result(timestamp, new TopNResultValue(values)); + } + + private void instantiatePQueue(int threshold, final Comparator comparator) + { + this.pQueue = MinMaxPriorityQueue.orderedBy( + new Comparator() + { + @Override + public int compare( + DimValHolder o1, + DimValHolder o2 + ) + { + return comparator.compare(o1.getDimName(), o2.getDimName()); + } + } + ).maximumSize(threshold).create(); + } +} diff --git a/processing/src/main/java/io/druid/query/topn/TopNMapFn.java b/processing/src/main/java/io/druid/query/topn/TopNMapFn.java new file mode 100644 index 00000000000..c013d546f7f --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/TopNMapFn.java @@ -0,0 +1,65 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.google.common.base.Function; +import io.druid.query.Result; +import io.druid.segment.Cursor; +import io.druid.segment.DimensionSelector; + +public class TopNMapFn implements Function> +{ + private final TopNQuery query; + private final TopNAlgorithm topNAlgorithm; + + + public TopNMapFn( + TopNQuery query, + TopNAlgorithm topNAlgorithm + ) + { + this.query = query; + this.topNAlgorithm = topNAlgorithm; + } + + @Override + @SuppressWarnings("unchecked") + public Result apply(Cursor cursor) + { + final DimensionSelector dimSelector = cursor.makeDimensionSelector(query.getDimensionSpec().getDimension()); + if (dimSelector == null) { + return null; + } + + TopNParams params = null; + try { + params = topNAlgorithm.makeInitParams(dimSelector, cursor); + + TopNResultBuilder resultBuilder = topNAlgorithm.makeResultBuilder(params); + + topNAlgorithm.run(params, resultBuilder, null); + + return resultBuilder.build(); + } + finally { + topNAlgorithm.cleanup(params); + } + } +} diff --git a/processing/src/main/java/io/druid/query/topn/TopNMetricSpec.java b/processing/src/main/java/io/druid/query/topn/TopNMetricSpec.java new file mode 100644 index 00000000000..c2baf13e3eb --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/TopNMetricSpec.java @@ -0,0 +1,58 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.dimension.DimensionSpec; +import org.joda.time.DateTime; + +import java.util.Comparator; +import java.util.List; + +/** + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = LegacyTopNMetricSpec.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "numeric", value = NumericTopNMetricSpec.class), + @JsonSubTypes.Type(name = "lexicographic", value = LexicographicTopNMetricSpec.class), + @JsonSubTypes.Type(name = "inverted", value = InvertedTopNMetricSpec.class) +}) +public interface TopNMetricSpec +{ + public void verifyPreconditions(List aggregatorSpecs, List postAggregatorSpecs); + + public Comparator getComparator(List aggregatorSpecs, List postAggregatorSpecs); + + public TopNResultBuilder getResultBuilder( + DateTime timestamp, + DimensionSpec dimSpec, + int threshold, + Comparator comparator + ); + + public byte[] getCacheKey(); + + public TopNMetricSpecBuilder configureOptimizer(TopNMetricSpecBuilder builder); + + public void initTopNAlgorithmSelector(TopNAlgorithmSelector selector); +} diff --git a/processing/src/main/java/io/druid/query/topn/TopNMetricSpecBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNMetricSpecBuilder.java new file mode 100644 index 00000000000..f1f5cdc9d0c --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/TopNMetricSpecBuilder.java @@ -0,0 +1,35 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +/** + */ +public interface TopNMetricSpecBuilder +{ + public void skipTo(String previousStop); + + public void ignoreAfterThreshold(); + + public void ignoreFirstN(int n); + + public void keepOnlyN(int n); + + public T build(); +} diff --git a/processing/src/main/java/io/druid/query/topn/TopNNumericResultBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNNumericResultBuilder.java new file mode 100644 index 00000000000..9f6479baee4 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/TopNNumericResultBuilder.java @@ -0,0 +1,153 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +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; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + */ +public class TopNNumericResultBuilder implements TopNResultBuilder +{ + private final DateTime timestamp; + private final DimensionSpec dimSpec; + private final String metricName; + + private MinMaxPriorityQueue pQueue = null; + + public TopNNumericResultBuilder( + DateTime timestamp, + DimensionSpec dimSpec, + String metricName, + int threshold, + final Comparator comparator + ) + { + this.timestamp = timestamp; + this.dimSpec = dimSpec; + this.metricName = metricName; + + instantiatePQueue(threshold, comparator); + } + + @Override + public TopNResultBuilder addEntry( + String dimName, + Object dimValIndex, + Object[] metricVals, + List aggFactories, + List postAggs + ) + { + Map metricValues = Maps.newLinkedHashMap(); + + metricValues.put(dimSpec.getOutputName(), dimName); + + Iterator aggFactoryIter = aggFactories.iterator(); + for (Object metricVal : metricVals) { + metricValues.put(aggFactoryIter.next().getName(), metricVal); + } + for (PostAggregator postAgg : postAggs) { + metricValues.put(postAgg.getName(), postAgg.compute(metricValues)); + } + + Object topNMetricVal = metricValues.get(metricName); + pQueue.add( + new DimValHolder.Builder().withTopNMetricVal(topNMetricVal) + .withDirName(dimName) + .withDimValIndex(dimValIndex) + .withMetricValues(metricValues) + .build() + ); + + return this; + } + + @Override + public TopNResultBuilder addEntry(DimensionAndMetricValueExtractor dimensionAndMetricValueExtractor) + { + pQueue.add( + new DimValHolder.Builder().withTopNMetricVal(dimensionAndMetricValueExtractor.getDimensionValue(metricName)) + .withDirName(dimSpec.getOutputName()) + .withMetricValues(dimensionAndMetricValueExtractor.getBaseObject()) + .build() + ); + + return this; + } + + @Override + public Iterator getTopNIterator() + { + return pQueue.iterator(); + } + + @Override + public Result build() + { + // Pull out top aggregated values + List> values = new ArrayList>(pQueue.size()); + while (!pQueue.isEmpty()) { + values.add(pQueue.remove().getMetricValues()); + } + + return new Result( + timestamp, + new TopNResultValue(values) + ); + } + + private void instantiatePQueue(int threshold, final Comparator comparator) + { + this.pQueue = MinMaxPriorityQueue.orderedBy( + new Comparator() + { + @Override + public int compare(DimValHolder d1, DimValHolder d2) + { + int retVal = comparator.compare(d2.getTopNMetricVal(), d1.getTopNMetricVal()); + + if (retVal == 0) { + if (d1.getDimName() == null) { + retVal = -1; + } else if (d2.getDimName() == null) { + retVal = 1; + } else { + retVal = d1.getDimName().compareTo(d2.getDimName()); + } + } + + return retVal; + } + } + ).maximumSize(threshold).create(); + } +} diff --git a/processing/src/main/java/io/druid/query/topn/TopNParams.java b/processing/src/main/java/io/druid/query/topn/TopNParams.java new file mode 100644 index 00000000000..8ccc85da284 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/TopNParams.java @@ -0,0 +1,61 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import io.druid.segment.Cursor; +import io.druid.segment.DimensionSelector; + +/** + */ +public class TopNParams +{ + private final DimensionSelector dimSelector; + private final Cursor cursor; + private final int cardinality; + private final int numValuesPerPass; + + protected TopNParams(DimensionSelector dimSelector, Cursor cursor, int cardinality, int numValuesPerPass) + { + this.dimSelector = dimSelector; + this.cursor = cursor; + this.cardinality = cardinality; + this.numValuesPerPass = numValuesPerPass; + } + + public DimensionSelector getDimSelector() + { + return dimSelector; + } + + public Cursor getCursor() + { + return cursor; + } + + public int getCardinality() + { + return cardinality; + } + + public int getNumValuesPerPass() + { + return numValuesPerPass; + } +} diff --git a/processing/src/main/java/io/druid/query/topn/TopNQuery.java b/processing/src/main/java/io/druid/query/topn/TopNQuery.java new file mode 100644 index 00000000000..10b27deb6e3 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/TopNQuery.java @@ -0,0 +1,211 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import io.druid.granularity.QueryGranularity; +import io.druid.query.BaseQuery; +import io.druid.query.Queries; +import io.druid.query.Result; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.filter.DimFilter; +import io.druid.query.spec.QuerySegmentSpec; + +import java.util.List; +import java.util.Map; + +/** + */ +public class TopNQuery extends BaseQuery> +{ + public static final String TOPN = "topN"; + + private final DimensionSpec dimensionSpec; + private final TopNMetricSpec topNMetricSpec; + private final int threshold; + private final DimFilter dimFilter; + private final QueryGranularity granularity; + private final List aggregatorSpecs; + private final List postAggregatorSpecs; + + @JsonCreator + public TopNQuery( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("dimension") DimensionSpec dimensionSpec, + @JsonProperty("metric") TopNMetricSpec topNMetricSpec, + @JsonProperty("threshold") int threshold, + @JsonProperty("intervals") QuerySegmentSpec querySegmentSpec, + @JsonProperty("filter") DimFilter dimFilter, + @JsonProperty("granularity") QueryGranularity granularity, + @JsonProperty("aggregations") List aggregatorSpecs, + @JsonProperty("postAggregations") List postAggregatorSpecs, + @JsonProperty("context") Map context + ) + { + super(dataSource, querySegmentSpec, context); + this.dimensionSpec = dimensionSpec; + this.topNMetricSpec = topNMetricSpec; + this.threshold = threshold; + + this.dimFilter = dimFilter; + this.granularity = granularity; + this.aggregatorSpecs = aggregatorSpecs; + this.postAggregatorSpecs = postAggregatorSpecs == null ? ImmutableList.of() : postAggregatorSpecs; + + Preconditions.checkNotNull(dimensionSpec, "dimensionSpec can't be null"); + Preconditions.checkNotNull(topNMetricSpec, "must specify a metric"); + + Preconditions.checkArgument(threshold != 0, "Threshold cannot be equal to 0."); + topNMetricSpec.verifyPreconditions(this.aggregatorSpecs, this.postAggregatorSpecs); + + Queries.verifyAggregations(this.aggregatorSpecs, this.postAggregatorSpecs); + } + + @Override + public boolean hasFilters() + { + return dimFilter != null; + } + + @Override + public String getType() + { + return TOPN; + } + + @JsonProperty("dimension") + public DimensionSpec getDimensionSpec() + { + return dimensionSpec; + } + + @JsonProperty("metric") + public TopNMetricSpec getTopNMetricSpec() + { + return topNMetricSpec; + } + + @JsonProperty("threshold") + public int getThreshold() + { + return threshold; + } + + @JsonProperty("filter") + public DimFilter getDimensionsFilter() + { + return dimFilter; + } + + @JsonProperty + public QueryGranularity getGranularity() + { + return granularity; + } + + @JsonProperty("aggregations") + public List getAggregatorSpecs() + { + return aggregatorSpecs; + } + + @JsonProperty("postAggregations") + public List getPostAggregatorSpecs() + { + return postAggregatorSpecs; + } + + public void initTopNAlgorithmSelector(TopNAlgorithmSelector selector) + { + if (dimensionSpec.getDimExtractionFn() != null) { + selector.setHasDimExtractionFn(true); + } + topNMetricSpec.initTopNAlgorithmSelector(selector); + } + + public TopNQuery withQuerySegmentSpec(QuerySegmentSpec querySegmentSpec) + { + return new TopNQuery( + getDataSource(), + dimensionSpec, + topNMetricSpec, + threshold, + querySegmentSpec, + dimFilter, + granularity, + aggregatorSpecs, + postAggregatorSpecs, + getContext() + ); + } + + public TopNQuery withThreshold(int threshold) + { + return new TopNQuery( + getDataSource(), + dimensionSpec, + topNMetricSpec, + threshold, + getQuerySegmentSpec(), + dimFilter, + granularity, + aggregatorSpecs, + postAggregatorSpecs, + getContext() + ); + } + + public TopNQuery withOverriddenContext(Map contextOverrides) + { + return new TopNQuery( + getDataSource(), + dimensionSpec, + topNMetricSpec, + threshold, + getQuerySegmentSpec(), + dimFilter, + granularity, + aggregatorSpecs, + postAggregatorSpecs, + computeOverridenContext(contextOverrides) + ); + } + + @Override + public String toString() + { + return "TopNQuery{" + + "dataSource='" + getDataSource() + '\'' + + ", dimensionSpec=" + dimensionSpec + + ", topNMetricSpec=" + topNMetricSpec + + ", threshold=" + threshold + + ", querySegmentSpec=" + getQuerySegmentSpec() + + ", dimFilter=" + dimFilter + + ", granularity='" + granularity + '\'' + + ", aggregatorSpecs=" + aggregatorSpecs + + ", postAggregatorSpecs=" + postAggregatorSpecs + + '}'; + } +} diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java new file mode 100644 index 00000000000..1bfb690f490 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java @@ -0,0 +1,290 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.google.common.collect.Lists; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.dimension.DefaultDimensionSpec; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.filter.DimFilter; +import io.druid.query.filter.OrDimFilter; +import io.druid.query.filter.SelectorDimFilter; +import io.druid.query.spec.LegacySegmentSpec; +import io.druid.query.spec.QuerySegmentSpec; +import org.joda.time.Interval; + +import java.util.List; +import java.util.Map; + +/** + * A Builder for TopNQuery. + *

+ * Required: dataSource(), intervals(), metric() and threshold() must be called before build() + * Additional requirement for numeric metric sorts: aggregators() must be called before build() + *

+ * Optional: filters(), granularity(), postAggregators() and context() can be called before build() + *

+ * Usage example: + *


+ *   TopNQuery query = new TopNQueryBuilder()
+ *                                  .dataSource("Example")
+ *                                  .dimension("example_dim")
+ *                                  .metric("example_metric")
+ *                                  .threshold(100)
+ *                                  .intervals("2012-01-01/2012-01-02")
+ *                                  .build();
+ * 
+ * + * @see io.druid.query.topn.TopNQuery + */ +public class TopNQueryBuilder +{ + private String dataSource; + private DimensionSpec dimensionSpec; + private TopNMetricSpec topNMetricSpec; + private int threshold; + private QuerySegmentSpec querySegmentSpec; + private DimFilter dimFilter; + private QueryGranularity granularity; + private List aggregatorSpecs; + private List postAggregatorSpecs; + private Map context; + + public TopNQueryBuilder() + { + dataSource = ""; + dimensionSpec = null; + topNMetricSpec = null; + threshold = 0; + querySegmentSpec = null; + dimFilter = null; + granularity = QueryGranularity.ALL; + aggregatorSpecs = Lists.newArrayList(); + postAggregatorSpecs = Lists.newArrayList(); + context = null; + } + + public String getDataSource() + { + return dataSource; + } + + public DimensionSpec getDimensionSpec() + { + return dimensionSpec; + } + + public TopNMetricSpec getTopNMetricSpec() + { + return topNMetricSpec; + } + + public int getThreshold() + { + return threshold; + } + + public QuerySegmentSpec getQuerySegmentSpec() + { + return querySegmentSpec; + } + + public DimFilter getDimFilter() + { + return dimFilter; + } + + public QueryGranularity getGranularity() + { + return granularity; + } + + public List getAggregatorSpecs() + { + return aggregatorSpecs; + } + + public List getPostAggregatorSpecs() + { + return postAggregatorSpecs; + } + + public Map getContext() + { + return context; + } + + public TopNQuery build() + { + return new TopNQuery( + dataSource, + dimensionSpec, + topNMetricSpec, + threshold, + querySegmentSpec, + dimFilter, + granularity, + aggregatorSpecs, + postAggregatorSpecs, + context + ); + } + + public TopNQueryBuilder copy(TopNQuery query) + { + return new TopNQueryBuilder() + .dataSource(query.getDataSource()) + .dimension(query.getDimensionSpec()) + .metric(query.getTopNMetricSpec()) + .threshold(query.getThreshold()) + .intervals(query.getIntervals()) + .filters(query.getDimensionsFilter()) + .granularity(query.getGranularity()) + .aggregators(query.getAggregatorSpecs()) + .postAggregators(query.getPostAggregatorSpecs()) + .context(query.getContext()); + } + + public TopNQueryBuilder copy(TopNQueryBuilder builder) + { + return new TopNQueryBuilder() + .dataSource(builder.dataSource) + .dimension(builder.dimensionSpec) + .metric(builder.topNMetricSpec) + .threshold(builder.threshold) + .intervals(builder.querySegmentSpec) + .filters(builder.dimFilter) + .granularity(builder.granularity) + .aggregators(builder.aggregatorSpecs) + .postAggregators(builder.postAggregatorSpecs) + .context(builder.context); + } + + public TopNQueryBuilder dataSource(String d) + { + dataSource = d; + return this; + } + + public TopNQueryBuilder dimension(String d) + { + return dimension(d, null); + } + + public TopNQueryBuilder dimension(String d, String outputName) + { + return dimension(new DefaultDimensionSpec(d, outputName)); + } + + public TopNQueryBuilder dimension(DimensionSpec d) + { + dimensionSpec = d; + return this; + } + + public TopNQueryBuilder metric(String s) + { + return metric(new NumericTopNMetricSpec(s)); + } + + public TopNQueryBuilder metric(TopNMetricSpec t) + { + topNMetricSpec = t; + return this; + } + + public TopNQueryBuilder threshold(int i) + { + threshold = i; + return this; + } + + public TopNQueryBuilder intervals(QuerySegmentSpec q) + { + querySegmentSpec = q; + return this; + } + + public TopNQueryBuilder intervals(String s) + { + querySegmentSpec = new LegacySegmentSpec(s); + return this; + } + + public TopNQueryBuilder intervals(List l) + { + querySegmentSpec = new LegacySegmentSpec(l); + return this; + } + + public TopNQueryBuilder filters(String dimensionName, String value) + { + dimFilter = new SelectorDimFilter(dimensionName, value); + return this; + } + + public TopNQueryBuilder filters(String dimensionName, String value, String... values) + { + List fields = Lists.newArrayList(new SelectorDimFilter(dimensionName, value)); + for (String val : values) { + fields.add(new SelectorDimFilter(dimensionName, val)); + } + dimFilter = new OrDimFilter(fields); + return this; + } + + public TopNQueryBuilder filters(DimFilter f) + { + dimFilter = f; + return this; + } + + public TopNQueryBuilder granularity(String g) + { + granularity = QueryGranularity.fromString(g); + return this; + } + + public TopNQueryBuilder granularity(QueryGranularity g) + { + granularity = g; + return this; + } + + public TopNQueryBuilder aggregators(List a) + { + aggregatorSpecs = a; + return this; + } + + public TopNQueryBuilder postAggregators(List p) + { + postAggregatorSpecs = p; + return this; + } + + public TopNQueryBuilder context(Map c) + { + context = c; + return this; + } +} diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryConfig.java b/processing/src/main/java/io/druid/query/topn/TopNQueryConfig.java new file mode 100644 index 00000000000..32de88f0cb3 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryConfig.java @@ -0,0 +1,39 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.query.QueryConfig; + +import javax.validation.constraints.Min; + +/** + */ +public class TopNQueryConfig extends QueryConfig +{ + @JsonProperty + @Min(1) + private int minTopNThreshold = 1000; + + public int getMinTopNThreshold() + { + return minTopNThreshold; + } +} diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java new file mode 100644 index 00000000000..09a158b31de --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java @@ -0,0 +1,117 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.google.common.base.Function; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.metamx.common.guava.FunctionalIterable; +import com.metamx.common.logger.Logger; +import io.druid.collections.StupidPool; +import io.druid.granularity.QueryGranularity; +import io.druid.query.Result; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.filter.Filter; +import io.druid.segment.Capabilities; +import io.druid.segment.Cursor; +import io.druid.segment.StorageAdapter; +import io.druid.segment.filter.Filters; +import org.joda.time.Interval; + +import java.nio.ByteBuffer; +import java.util.List; + +/** + */ +public class TopNQueryEngine +{ + private static final Logger log = new Logger(TopNQueryEngine.class); + + private final StupidPool bufferPool; + + public TopNQueryEngine(StupidPool bufferPool) + { + this.bufferPool = bufferPool; + } + + public Iterable> query(final TopNQuery query, final StorageAdapter adapter) + { + final List queryIntervals = query.getQuerySegmentSpec().getIntervals(); + final Filter filter = Filters.convertDimensionFilters(query.getDimensionsFilter()); + final QueryGranularity granularity = query.getGranularity(); + final Function> mapFn = getMapFn(query, adapter); + + Preconditions.checkArgument( + queryIntervals.size() == 1, "Can only handle a single interval, got[%s]", queryIntervals + ); + + if (mapFn == null) { + return Lists.newArrayList(); + } + + return FunctionalIterable + .create(adapter.makeCursors(filter, queryIntervals.get(0), granularity)) + .transform( + new Function() + { + @Override + public Cursor apply(Cursor input) + { + log.debug("Running over cursor[%s]", adapter.getInterval(), input.getTime()); + return input; + } + } + ) + .keep(mapFn); + } + + private Function> getMapFn(TopNQuery query, final StorageAdapter adapter) + { + if (adapter == null) { + log.warn( + "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped. Returning empty results." + ); + return null; + } + + final Capabilities capabilities = adapter.getCapabilities(); + final int cardinality = adapter.getDimensionCardinality(query.getDimensionSpec().getDimension()); + int numBytesPerRecord = 0; + for (AggregatorFactory aggregatorFactory : query.getAggregatorSpecs()) { + numBytesPerRecord += aggregatorFactory.getMaxIntermediateSize(); + } + + final TopNAlgorithmSelector selector = new TopNAlgorithmSelector(cardinality, numBytesPerRecord); + query.initTopNAlgorithmSelector(selector); + + TopNAlgorithm topNAlgorithm = null; + if (selector.isHasDimExtractionFn()) { + topNAlgorithm = new DimExtractionTopNAlgorithm(capabilities, query); + } else if (selector.isAggregateAllMetrics()) { + topNAlgorithm = new PooledTopNAlgorithm(capabilities, query, bufferPool); + } else if (selector.isAggregateTopNMetricFirst()) { + topNAlgorithm = new AggregateTopNMetricFirstAlgorithm(capabilities, query, bufferPool); + } else { + topNAlgorithm = new PooledTopNAlgorithm(capabilities, query, bufferPool); + } + + return new TopNMapFn(query, topNAlgorithm); + } +} diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java new file mode 100644 index 00000000000..6e1d816cbd9 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java @@ -0,0 +1,405 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Ordering; +import com.google.common.primitives.Ints; +import com.google.inject.Inject; +import com.metamx.common.ISE; +import com.metamx.common.guava.MergeSequence; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import com.metamx.common.guava.nary.BinaryFn; +import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.collections.OrderedMergeSequence; +import io.druid.granularity.QueryGranularity; +import io.druid.query.CacheStrategy; +import io.druid.query.IntervalChunkingQueryRunner; +import io.druid.query.Query; +import io.druid.query.QueryCacheHelper; +import io.druid.query.QueryRunner; +import io.druid.query.QueryToolChest; +import io.druid.query.Result; +import io.druid.query.ResultGranularTimestampComparator; +import io.druid.query.ResultMergeQueryRunner; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.MetricManipulationFn; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.filter.DimFilter; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.Minutes; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + */ +public class TopNQueryQueryToolChest extends QueryToolChest, TopNQuery> +{ + private static final byte TOPN_QUERY = 0x1; + + private static final Joiner COMMA_JOIN = Joiner.on(","); + private static final TypeReference> TYPE_REFERENCE = new TypeReference>(){}; + + private static final TypeReference OBJECT_TYPE_REFERENCE = new TypeReference(){}; + private final TopNQueryConfig config; + + @Inject + public TopNQueryQueryToolChest( + TopNQueryConfig config + ) + { + this.config = config; + } + + @Override + public QueryRunner> mergeResults(QueryRunner> runner) + { + return new ResultMergeQueryRunner>(runner) + { + @Override + protected Ordering> makeOrdering(Query> query) + { + return Ordering.from( + new ResultGranularTimestampComparator( + ((TopNQuery) query).getGranularity() + ) + ); + } + + @Override + protected BinaryFn, Result, Result> createMergeFn( + Query> input + ) + { + TopNQuery query = (TopNQuery) input; + return new TopNBinaryFn( + TopNResultMerger.identity, + query.getGranularity(), + query.getDimensionSpec(), + query.getTopNMetricSpec(), + query.getThreshold(), + query.getAggregatorSpecs(), + query.getPostAggregatorSpecs() + ); + } + }; + } + + @Override + public Sequence> mergeSequences(Sequence>> seqOfSequences) + { + return new OrderedMergeSequence>(getOrdering(), seqOfSequences); + } + + @Override + public ServiceMetricEvent.Builder makeMetricBuilder(TopNQuery query) + { + int numMinutes = 0; + for (Interval interval : query.getIntervals()) { + numMinutes += Minutes.minutesIn(interval).getMinutes(); + } + + return new ServiceMetricEvent.Builder() + .setUser2(query.getDataSource()) + .setUser4(String.format("topN/%s/%s", query.getThreshold(), query.getDimensionSpec().getDimension())) + .setUser5(COMMA_JOIN.join(query.getIntervals())) + .setUser6(String.valueOf(query.hasFilters())) + .setUser7(String.format("%,d aggs", query.getAggregatorSpecs().size())) + .setUser9(Minutes.minutes(numMinutes).toString()); + } + + @Override + public Function, Result> makeMetricManipulatorFn( + final TopNQuery query, final MetricManipulationFn fn + ) + { + return new Function, Result>() + { + private String dimension = query.getDimensionSpec().getOutputName(); + + @Override + public Result apply(@Nullable Result result) + { + List> serializedValues = Lists.newArrayList( + Iterables.transform( + result.getValue(), + new Function>() + { + @Override + public Map apply(@Nullable DimensionAndMetricValueExtractor input) + { + final Map values = Maps.newHashMap(); + for (AggregatorFactory agg : query.getAggregatorSpecs()) { + values.put(agg.getName(), fn.manipulate(agg, input.getMetric(agg.getName()))); + } + for (PostAggregator postAgg : query.getPostAggregatorSpecs()) { + values.put(postAgg.getName(), input.getMetric(postAgg.getName())); + } + values.put(dimension, input.getDimensionValue(dimension)); + + return values; + } + } + ) + ); + + return new Result( + result.getTimestamp(), + new TopNResultValue(serializedValues) + ); + } + }; + } + + @Override + public TypeReference> getResultTypeReference() + { + return TYPE_REFERENCE; + } + + @Override + public CacheStrategy, Object, TopNQuery> getCacheStrategy(final TopNQuery query) + { + return new CacheStrategy, Object, TopNQuery>() + { + private final List aggs = query.getAggregatorSpecs(); + private final List postAggs = query.getPostAggregatorSpecs(); + + @Override + public byte[] computeCacheKey(TopNQuery query) + { + final byte[] dimensionSpecBytes = query.getDimensionSpec().getCacheKey(); + final byte[] metricSpecBytes = query.getTopNMetricSpec().getCacheKey(); + + final DimFilter dimFilter = query.getDimensionsFilter(); + final byte[] filterBytes = dimFilter == null ? new byte[]{} : dimFilter.getCacheKey(); + final byte[] aggregatorBytes = QueryCacheHelper.computeAggregatorBytes(query.getAggregatorSpecs()); + final byte[] granularityBytes = query.getGranularity().cacheKey(); + + return ByteBuffer + .allocate( + 1 + dimensionSpecBytes.length + metricSpecBytes.length + 4 + + granularityBytes.length + filterBytes.length + aggregatorBytes.length + ) + .put(TOPN_QUERY) + .put(dimensionSpecBytes) + .put(metricSpecBytes) + .put(Ints.toByteArray(query.getThreshold())) + .put(granularityBytes) + .put(filterBytes) + .put(aggregatorBytes) + .array(); + } + + @Override + public TypeReference getCacheObjectClazz() + { + return OBJECT_TYPE_REFERENCE; + } + + @Override + public Function, Object> prepareForCache() + { + return new Function, Object>() + { + @Override + public Object apply(@Nullable final Result input) + { + List results = Lists.newArrayList(input.getValue()); + final List retVal = Lists.newArrayListWithCapacity(results.size() + 1); + + // make sure to preserve timezone information when caching results + retVal.add(input.getTimestamp().getMillis()); + for (DimensionAndMetricValueExtractor result : results) { + List vals = Lists.newArrayListWithCapacity(aggs.size() + 2); + vals.add(result.getStringDimensionValue(query.getDimensionSpec().getOutputName())); + for (AggregatorFactory agg : aggs) { + vals.add(result.getMetric(agg.getName())); + } + retVal.add(vals); + } + return retVal; + } + }; + } + + @Override + public Function> pullFromCache() + { + return new Function>() + { + private final QueryGranularity granularity = query.getGranularity(); + + @Override + public Result apply(@Nullable Object input) + { + List results = (List) input; + List> retVal = Lists.newArrayListWithCapacity(results.size()); + + Iterator inputIter = results.iterator(); + DateTime timestamp = granularity.toDateTime(new DateTime(inputIter.next()).getMillis()); + + while (inputIter.hasNext()) { + List result = (List) inputIter.next(); + Map vals = Maps.newLinkedHashMap(); + + Iterator aggIter = aggs.iterator(); + Iterator resultIter = result.iterator(); + + vals.put(query.getDimensionSpec().getOutputName(), resultIter.next()); + + while (aggIter.hasNext() && resultIter.hasNext()) { + final AggregatorFactory factory = aggIter.next(); + vals.put(factory.getName(), factory.deserialize(resultIter.next())); + } + + for (PostAggregator postAgg : postAggs) { + vals.put(postAgg.getName(), postAgg.compute(vals)); + } + + retVal.add(vals); + } + + return new Result(timestamp, new TopNResultValue(retVal)); + } + }; + } + + @Override + public Sequence> mergeSequences(Sequence>> seqOfSequences) + { + return new MergeSequence>(getOrdering(), seqOfSequences); + } + }; + } + + @Override + public QueryRunner> preMergeQueryDecoration(QueryRunner> runner) + { + return new IntervalChunkingQueryRunner>(runner, config.getChunkPeriod()); + } + + @Override + public QueryRunner> postMergeQueryDecoration(final QueryRunner> runner) + { + return new ThresholdAdjustingQueryRunner(runner, config.getMinTopNThreshold()); + } + + private static class ThresholdAdjustingQueryRunner implements QueryRunner> + { + private final QueryRunner> runner; + private final int minTopNThreshold; + + public ThresholdAdjustingQueryRunner( + QueryRunner> runner, + int minTopNThreshold + ) + { + this.runner = runner; + this.minTopNThreshold = minTopNThreshold; + } + + @Override + public Sequence> run(Query> input) + { + if (!(input instanceof TopNQuery)) { + throw new ISE("Can only handle [%s], got [%s]", TopNQuery.class, input.getClass()); + } + + final TopNQuery query = (TopNQuery) input; + if (query.getThreshold() > minTopNThreshold) { + return runner.run(query); + } + + final boolean isBySegment = Boolean.parseBoolean(query.getContextValue("bySegment", "false")); + + return Sequences.map( + runner.run(query.withThreshold(minTopNThreshold)), + new Function, Result>() + { + @Override + public Result apply(Result input) + { + if (isBySegment) { + BySegmentTopNResultValue value = (BySegmentTopNResultValue) input.getValue(); + + return new Result( + input.getTimestamp(), + new BySegmentTopNResultValue( + Lists.transform( + value.getResults(), + new Function, Result>() + { + @Override + public Result apply(@Nullable Result input) + { + return new Result( + input.getTimestamp(), + new TopNResultValue( + Lists.newArrayList( + Iterables.limit( + input.getValue(), + query.getThreshold() + ) + ) + ) + ); + } + } + ), + value.getSegmentId(), + value.getIntervalString() + ) + ); + } + + return new Result( + input.getTimestamp(), + new TopNResultValue( + Lists.newArrayList( + Iterables.limit( + input.getValue(), + query.getThreshold() + ) + ) + ) + ); + } + } + ); + } + } + + public Ordering> getOrdering() + { + return Ordering.natural(); + } +} diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java new file mode 100644 index 00000000000..d342d764d50 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java @@ -0,0 +1,108 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.google.inject.Inject; +import com.metamx.common.ISE; +import com.metamx.common.guava.BaseSequence; +import com.metamx.common.guava.Sequence; +import io.druid.collections.StupidPool; +import io.druid.guice.annotations.Global; +import io.druid.query.ChainedExecutionQueryRunner; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryToolChest; +import io.druid.query.Result; +import io.druid.segment.Segment; + +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.concurrent.ExecutorService; + +/** + */ +public class TopNQueryRunnerFactory implements QueryRunnerFactory, TopNQuery> +{ + private final StupidPool computationBufferPool; + private final TopNQueryQueryToolChest toolchest; + + @Inject + public TopNQueryRunnerFactory( + @Global StupidPool computationBufferPool, + TopNQueryQueryToolChest toolchest + ) + { + this.computationBufferPool = computationBufferPool; + this.toolchest = toolchest; + } + + @Override + public QueryRunner> createRunner(final Segment segment) + { + final TopNQueryEngine queryEngine = new TopNQueryEngine(computationBufferPool); + return new QueryRunner>() + { + @Override + public Sequence> run(Query> input) + { + if (!(input instanceof TopNQuery)) { + throw new ISE("Got a [%s] which isn't a %s", input.getClass(), TopNQuery.class); + } + + final TopNQuery legacyQuery = (TopNQuery) input; + + return new BaseSequence, Iterator>>( + new BaseSequence.IteratorMaker, Iterator>>() + { + @Override + public Iterator> make() + { + return queryEngine.query(legacyQuery, segment.asStorageAdapter()).iterator(); + } + + @Override + public void cleanup(Iterator> toClean) + { + + } + } + ); + } + }; + + } + + @Override + public QueryRunner> mergeRunners( + ExecutorService queryExecutor, Iterable>> queryRunners + ) + { + return new ChainedExecutionQueryRunner>( + queryExecutor, toolchest.getOrdering(), queryRunners + ); + } + + @Override + public QueryToolChest, TopNQuery> getToolchest() + { + return toolchest; + } +} diff --git a/processing/src/main/java/io/druid/query/topn/TopNResultBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNResultBuilder.java new file mode 100644 index 00000000000..5823ee3eece --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/TopNResultBuilder.java @@ -0,0 +1,48 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import io.druid.query.Result; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; + +import java.util.Iterator; +import java.util.List; + +/** + */ +public interface TopNResultBuilder +{ + public TopNResultBuilder addEntry( + String dimName, + Object dimValIndex, + Object[] metricVals, + List aggFactories, + List postAggs + ); + + public TopNResultBuilder addEntry( + DimensionAndMetricValueExtractor dimensionAndMetricValueExtractor + ); + + public Iterator getTopNIterator(); + + public Result build(); +} diff --git a/processing/src/main/java/io/druid/query/topn/TopNResultMerger.java b/processing/src/main/java/io/druid/query/topn/TopNResultMerger.java new file mode 100644 index 00000000000..f4ff8ca9b06 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/TopNResultMerger.java @@ -0,0 +1,40 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import io.druid.query.Result; + +import java.util.Comparator; + +/** + */ +public interface TopNResultMerger +{ + public static TopNResultMerger identity = new TopNResultMerger() + { + @Override + public Result getResult(Result result, Comparator comparator) + { + return result; + } + }; + + public Result getResult(Result result, Comparator comparator); +} diff --git a/processing/src/main/java/io/druid/query/topn/TopNResultValue.java b/processing/src/main/java/io/druid/query/topn/TopNResultValue.java new file mode 100644 index 00000000000..b65bb1f815a --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/TopNResultValue.java @@ -0,0 +1,107 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import com.metamx.common.IAE; + +import javax.annotation.Nullable; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + */ +public class TopNResultValue implements Iterable +{ + private final List value; + + @JsonCreator + public TopNResultValue( + List value + ) + { + this.value = (value == null) ? Lists.newArrayList() : Lists.transform( + value, + new Function() + { + @Override + public DimensionAndMetricValueExtractor apply(@Nullable Object input) + { + if (input instanceof Map) { + return new DimensionAndMetricValueExtractor((Map) input); + } else if (input instanceof DimensionAndMetricValueExtractor) { + return (DimensionAndMetricValueExtractor) input; + } else { + throw new IAE("Unknown type for input[%s]", input.getClass()); + } + } + } + ); + } + + @JsonValue + public List getValue() + { + return value; + } + + @Override + public Iterator iterator() + { + return value.iterator(); + } + + @Override + public String toString() + { + return "TopNResultValue{" + + "value=" + value + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + TopNResultValue that = (TopNResultValue) o; + + if (value != null ? !value.equals(that.value) : that.value != null) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + return value != null ? value.hashCode() : 0; + } +} diff --git a/processing/src/test/java/io/druid/query/TestQueryRunners.java b/processing/src/test/java/io/druid/query/TestQueryRunners.java new file mode 100644 index 00000000000..c4767c1c6f9 --- /dev/null +++ b/processing/src/test/java/io/druid/query/TestQueryRunners.java @@ -0,0 +1,83 @@ +package io.druid.query; + +import com.google.common.base.Supplier; +import io.druid.collections.StupidPool; +import io.druid.query.search.SearchQueryQueryToolChest; +import io.druid.query.search.SearchQueryRunnerFactory; +import io.druid.query.search.search.SearchQueryConfig; +import io.druid.query.timeboundary.TimeBoundaryQueryRunnerFactory; +import io.druid.query.timeseries.TimeseriesQueryRunnerFactory; +import io.druid.query.topn.TopNQueryConfig; +import io.druid.query.topn.TopNQueryQueryToolChest; +import io.druid.query.topn.TopNQueryRunnerFactory; +import io.druid.segment.Segment; + +import java.nio.ByteBuffer; + +/** + */ +public class TestQueryRunners +{ + public static final StupidPool pool = new StupidPool( + new Supplier() + { + @Override + public ByteBuffer get() + { + return ByteBuffer.allocate(1024 * 10); + } + } + ); + + public static final TopNQueryConfig topNConfig = new TopNQueryConfig(); + + public static StupidPool getPool() + { + return pool; + } + + public static QueryRunner makeTopNQueryRunner( + Segment adapter + ) + { + QueryRunnerFactory factory = new TopNQueryRunnerFactory(pool, new TopNQueryQueryToolChest(topNConfig)); + return new FinalizeResultsQueryRunner( + factory.createRunner(adapter), + factory.getToolchest() + ); + } + + public static QueryRunner makeTimeSeriesQueryRunner( + Segment adapter + ) + { + QueryRunnerFactory factory = TimeseriesQueryRunnerFactory.create(); + return new FinalizeResultsQueryRunner( + factory.createRunner(adapter), + factory.getToolchest() + ); + } + + public static QueryRunner makeSearchQueryRunner( + Segment adapter + ) + { + QueryRunnerFactory factory = new SearchQueryRunnerFactory(new SearchQueryQueryToolChest(new SearchQueryConfig())); + return new FinalizeResultsQueryRunner( + factory.createRunner(adapter), + factory.getToolchest() + ); + } + + public static QueryRunner makeTimeBoundaryQueryRunner( + Segment adapter + ) + { + QueryRunnerFactory factory = new TimeBoundaryQueryRunnerFactory(); + return new FinalizeResultsQueryRunner( + factory.createRunner(adapter), + factory.getToolchest() + ); + } + +} \ No newline at end of file diff --git a/processing/src/test/java/io/druid/query/topn/TopNBinaryFnTest.java b/processing/src/test/java/io/druid/query/topn/TopNBinaryFnTest.java new file mode 100644 index 00000000000..2ce63a693e2 --- /dev/null +++ b/processing/src/test/java/io/druid/query/topn/TopNBinaryFnTest.java @@ -0,0 +1,458 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import io.druid.granularity.QueryGranularity; +import io.druid.query.Result; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.post.ArithmeticPostAggregator; +import io.druid.query.aggregation.post.ConstantPostAggregator; +import io.druid.query.aggregation.post.FieldAccessPostAggregator; +import io.druid.query.dimension.DefaultDimensionSpec; +import junit.framework.Assert; +import org.joda.time.DateTime; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + */ +public class TopNBinaryFnTest +{ + final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows"); + final LongSumAggregatorFactory indexLongSum = new LongSumAggregatorFactory("index", "index"); + final ConstantPostAggregator constant = new ConstantPostAggregator("const", 1L); + final FieldAccessPostAggregator rowsPostAgg = new FieldAccessPostAggregator("rows", "rows"); + final FieldAccessPostAggregator indexPostAgg = new FieldAccessPostAggregator("index", "index"); + final ArithmeticPostAggregator addrowsindexconstant = new ArithmeticPostAggregator( + "addrowsindexconstant", + "+", + Lists.newArrayList(constant, rowsPostAgg, indexPostAgg) + ); + final List aggregatorFactories = Arrays.asList( + rowsCount, + indexLongSum + ); + final List postAggregators = Arrays.asList( + addrowsindexconstant + ); + private final DateTime currTime = new DateTime(); + + private void assertTopNMergeResult(Object o1, Object o2) + { + Iterator i1 = ((Iterable) o1).iterator(); + Iterator i2 = ((Iterable) o2).iterator(); + while (i1.hasNext() && i2.hasNext()) { + Assert.assertEquals(i1.next(), i2.next()); + } + Assert.assertTrue(!i1.hasNext() && !i2.hasNext()); + } + + @Test + public void testMerge() + { + Result result1 = new Result( + currTime, + new TopNResultValue( + ImmutableList.>of( + ImmutableMap.of( + "rows", 1L, + "index", 2L, + "testdim", "1" + ), + ImmutableMap.of( + "rows", 2L, + "index", 4L, + "testdim", "2" + ), + ImmutableMap.of( + "rows", 0L, + "index", 2L, + "testdim", "3" + ) + ) + ) + ); + Result result2 = new Result( + currTime, + new TopNResultValue( + ImmutableList.>of( + ImmutableMap.of( + "rows", 2L, + "index", 3L, + "testdim", "1" + ), + ImmutableMap.of( + "rows", 2L, + "index", 0L, + "testdim", "2" + ), + ImmutableMap.of( + "rows", 0L, + "index", 1L, + "testdim", "3" + ) + ) + ) + ); + + Result expected = new Result( + currTime, + new TopNResultValue( + ImmutableList.>of( + ImmutableMap.of( + "testdim", "1", + "rows", 3L, + "index", 5L, + "addrowsindexconstant", 9.0 + ), + + ImmutableMap.of( + "testdim", "2", + "rows", 4L, + "index", 4L, + "addrowsindexconstant", 9.0 + ) + ) + ) + ); + + Result actual = new TopNBinaryFn( + TopNResultMerger.identity, + QueryGranularity.ALL, + new DefaultDimensionSpec("testdim", null), + new NumericTopNMetricSpec("index"), + 2, + aggregatorFactories, + postAggregators + ).apply( + result1, + result2 + ); + Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp()); + assertTopNMergeResult(expected.getValue(), actual.getValue()); + } + + @Test + public void testMergeDay() + { + Result result1 = new Result( + currTime, + new TopNResultValue( + ImmutableList.>of( + ImmutableMap.of( + "rows", 1L, + "index", 2L, + "testdim", "1" + ), + ImmutableMap.of( + "rows", 2L, + "index", 4L, + "testdim", "2" + ), + ImmutableMap.of( + "rows", 0L, + "index", 2L, + "testdim", "3" + ) + ) + ) + ); + Result result2 = new Result( + currTime, + new TopNResultValue( + ImmutableList.>of( + ImmutableMap.of( + "rows", 2L, + "index", 3L, + "testdim", "1" + ), + ImmutableMap.of( + "rows", 2L, + "index", 0L, + "testdim", "2" + ), + ImmutableMap.of( + "rows", 0L, + "index", 1L, + "testdim", "3" + ) + ) + ) + ); + + Result expected = new Result( + new DateTime(QueryGranularity.DAY.truncate(currTime.getMillis())), + new TopNResultValue( + ImmutableList.>of( + ImmutableMap.of( + "testdim", "1", + "rows", 3L, + "index", 5L, + "addrowsindexconstant", 9.0 + ), + ImmutableMap.of( + "testdim", "2", + "rows", 4L, + "index", 4L, + "addrowsindexconstant", 9.0 + ) + ) + ) + ); + + Result actual = new TopNBinaryFn( + TopNResultMerger.identity, + QueryGranularity.DAY, + new DefaultDimensionSpec("testdim", null), + new NumericTopNMetricSpec("index"), + 2, + aggregatorFactories, + postAggregators + ).apply( + result1, + result2 + ); + Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp()); + assertTopNMergeResult(expected.getValue(), actual.getValue()); + } + + @Test + public void testMergeOneResultNull() + { + Result result1 = new Result( + currTime, + new TopNResultValue( + ImmutableList.>of( + ImmutableMap.of( + "rows", 1L, + "index", 2L, + "testdim", "1" + ), + ImmutableMap.of( + "rows", 2L, + "index", 4L, + "testdim", "2" + ), + ImmutableMap.of( + "rows", 0L, + "index", 2L, + "testdim", "3" + ) + ) + ) + ); + Result result2 = null; + + Result expected = result1; + + Result actual = new TopNBinaryFn( + TopNResultMerger.identity, + QueryGranularity.ALL, + new DefaultDimensionSpec("testdim", null), + new NumericTopNMetricSpec("index"), + 2, + aggregatorFactories, + postAggregators + ).apply( + result1, + result2 + ); + Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp()); + assertTopNMergeResult(expected.getValue(), actual.getValue()); + } + + @Test + public void testMergeByPostAgg() + { + Result result1 = new Result( + currTime, + new TopNResultValue( + ImmutableList.>of( + ImmutableMap.of( + "rows", 1L, + "index", 2L, + "testdim", "1" + ), + ImmutableMap.of( + "rows", 2L, + "index", 4L, + "testdim", "2" + ), + ImmutableMap.of( + "rows", 0L, + "index", 2L, + "testdim", "3" + ) + ) + ) + ); + Result result2 = new Result( + currTime, + new TopNResultValue( + ImmutableList.>of( + ImmutableMap.of( + "rows", 2L, + "index", 3L, + "testdim", "1" + ), + ImmutableMap.of( + "rows", 2L, + "index", 0L, + "testdim", "2" + ), + ImmutableMap.of( + "rows", 0L, + "index", 1L, + "testdim", "3" + ) + ) + ) + ); + + Result expected = new Result( + currTime, + new TopNResultValue( + ImmutableList.>of( + ImmutableMap.of( + "testdim", "1", + "rows", 3L, + "index", 5L, + "addrowsindexconstant", 9.0 + ), + ImmutableMap.of( + "testdim", "2", + "rows", 4L, + "index", 4L, + "addrowsindexconstant", 9.0 + ) + ) + ) + ); + + Result actual = new TopNBinaryFn( + TopNResultMerger.identity, + QueryGranularity.ALL, + new DefaultDimensionSpec("testdim", null), + new NumericTopNMetricSpec("addrowsindexconstant"), + 2, + aggregatorFactories, + postAggregators + ).apply( + result1, + result2 + ); + Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp()); + assertTopNMergeResult(expected.getValue(), actual.getValue()); + } + + @Test + public void testMergeShiftedTimestamp() + { + Result result1 = new Result( + currTime, + new TopNResultValue( + ImmutableList.>of( + ImmutableMap.of( + "rows", 1L, + "index", 2L, + "testdim", "1" + ), + ImmutableMap.of( + "rows", 2L, + "index", 4L, + "testdim", "2" + ), + ImmutableMap.of( + "rows", 0L, + "index", 2L, + "testdim", "3" + ) + ) + ) + ); + Result result2 = new Result( + currTime.plusHours(2), + new TopNResultValue( + ImmutableList.>of( + ImmutableMap.of( + "rows", 2L, + "index", 3L, + "testdim", "1" + ), + ImmutableMap.of( + "rows", 2L, + "index", 0L, + "testdim", "2" + ), + ImmutableMap.of( + "rows", 0L, + "index", 1L, + "testdim", "3" + ) + ) + ) + ); + + Result expected = new Result( + currTime, + new TopNResultValue( + ImmutableList.>of( + ImmutableMap.of( + "testdim", "1", + "rows", 3L, + "index", 5L, + "addrowsindexconstant", 9.0 + ), + ImmutableMap.of( + "testdim", "2", + "rows", 4L, + "index", 4L, + "addrowsindexconstant", 9.0 + ) + ) + ) + ); + + Result actual = new TopNBinaryFn( + TopNResultMerger.identity, + QueryGranularity.ALL, + new DefaultDimensionSpec("testdim", null), + new NumericTopNMetricSpec("index"), + 2, + aggregatorFactories, + postAggregators + ).apply( + result1, + result2 + ); + Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp()); + assertTopNMergeResult(expected.getValue(), actual.getValue()); + } +} diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java new file mode 100644 index 00000000000..839f82c3cbc --- /dev/null +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java @@ -0,0 +1,1077 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.metamx.common.guava.Sequences; +import io.druid.collections.StupidPool; +import io.druid.granularity.QueryGranularity; +import io.druid.query.Druids; +import io.druid.query.QueryRunner; +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.DoubleSumAggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.aggregation.MaxAggregatorFactory; +import io.druid.query.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.post.ArithmeticPostAggregator; +import io.druid.query.aggregation.post.ConstantPostAggregator; +import io.druid.query.aggregation.post.FieldAccessPostAggregator; +import io.druid.query.dimension.ExtractionDimensionSpec; +import io.druid.query.extraction.RegexDimExtractionFn; +import io.druid.query.filter.AndDimFilter; +import io.druid.query.filter.DimFilter; +import io.druid.query.spec.MultipleIntervalSegmentSpec; +import io.druid.query.spec.QuerySegmentSpec; +import io.druid.segment.TestHelper; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; + +/** + */ +@RunWith(Parameterized.class) +public class TopNQueryRunnerTest +{ + @Parameterized.Parameters + public static Collection constructorFeeder() throws IOException + { + List retVal = Lists.newArrayList(); + retVal.addAll( + TopNQueryRunnerTestHelper.makeQueryRunners( + new TopNQueryRunnerFactory( + TestQueryRunners.getPool(), + new TopNQueryQueryToolChest(new TopNQueryConfig()) + ) + ) + ); + retVal.addAll( + TopNQueryRunnerTestHelper.makeQueryRunners( + new TopNQueryRunnerFactory( + new StupidPool( + new Supplier() + { + @Override + public ByteBuffer get() + { + return ByteBuffer.allocate(2000); + } + } + ), + new TopNQueryQueryToolChest(new TopNQueryConfig()) + ) + ) + ); + + return retVal; + } + + private final QueryRunner runner; + + public TopNQueryRunnerTest( + QueryRunner runner + ) + { + this.runner = runner; + } + + final String dataSource = "testing"; + final QueryGranularity gran = QueryGranularity.DAY; + final QueryGranularity allGran = QueryGranularity.ALL; + final String providerDimension = "provider"; + final String qualityDimension = "quality"; + final String placementishDimension = "placementish"; + final String indexMetric = "index"; + final String addRowsIndexConstantMetric = "addRowsIndexConstant"; + final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows"); + final LongSumAggregatorFactory indexLongSum = new LongSumAggregatorFactory("index", "index"); + final DoubleSumAggregatorFactory indexDoubleSum = new DoubleSumAggregatorFactory("index", "index"); + final ConstantPostAggregator constant = new ConstantPostAggregator("const", 1L); + final FieldAccessPostAggregator rowsPostAgg = new FieldAccessPostAggregator("rows", "rows"); + final FieldAccessPostAggregator indexPostAgg = new FieldAccessPostAggregator("index", "index"); + final ArithmeticPostAggregator addRowsIndexConstant = + new ArithmeticPostAggregator( + "addRowsIndexConstant", "+", Lists.newArrayList(constant, rowsPostAgg, indexPostAgg) + ); + final List commonAggregators = Arrays.asList(rowsCount, indexDoubleSum); + + + final String[] expectedFullOnIndexValues = new String[]{ + "4500.0", "6077.949111938477", "4922.488838195801", "5726.140853881836", "4698.468170166016", + "4651.030891418457", "4398.145851135254", "4596.068244934082", "4434.630561828613", "0.0", + "6162.801361083984", "5590.292701721191", "4994.298484802246", "5179.679672241211", "6288.556800842285", + "6025.663551330566", "5772.855537414551", "5346.517524719238", "5497.331253051758", "5909.684387207031", + "5862.711364746094", "5958.373008728027", "5224.882194519043", "5456.789611816406", "5456.095397949219", + "4642.481948852539", "5023.572692871094", "5155.821723937988", "5350.3723220825195", "5236.997489929199", + "4910.097717285156", "4507.608840942383", "4659.80500793457", "5354.878845214844", "4945.796455383301", + "6459.080368041992", "4390.493583679199", "6545.758262634277", "6922.801231384277", "6023.452911376953", + "6812.107475280762", "6368.713348388672", "6381.748748779297", "5631.245086669922", "4976.192253112793", + "6541.463027954102", "5983.8513107299805", "5967.189498901367", "5567.139289855957", "4863.5944747924805", + "4681.164360046387", "6122.321441650391", "5410.308860778809", "4846.676376342773", "5333.872688293457", + "5013.053741455078", "4836.85563659668", "5264.486434936523", "4581.821243286133", "4680.233596801758", + "4771.363662719727", "5038.354717254639", "4816.808464050293", "4684.095504760742", "5023.663467407227", + "5889.72257232666", "4984.973915100098", "5664.220512390137", "5572.653915405273", "5537.123138427734", + "5980.422874450684", "6243.834693908691", "5372.147285461426", "5690.728981018066", "5827.796455383301", + "6141.0769119262695", "6082.3237228393555", "5678.771339416504", "6814.467971801758", "6626.151596069336", + "5833.2095947265625", "4679.222328186035", "5367.9403076171875", "5410.445640563965", "5689.197135925293", + "5240.5018310546875", "4790.912239074707", "4992.670921325684", "4796.888023376465", "5479.439590454102", + "5506.567192077637", "4743.144546508789", "4913.282669067383", "4723.869743347168" + }; + + final DateTime skippedDay = new DateTime("2011-01-21T00:00:00.000Z"); + + final QuerySegmentSpec firstToThird = new MultipleIntervalSegmentSpec( + Arrays.asList(new Interval("2011-04-01T00:00:00.000Z/2011-04-03T00:00:00.000Z")) + ); + final QuerySegmentSpec fullOnInterval = new MultipleIntervalSegmentSpec( + Arrays.asList(new Interval("1970-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z")) + ); + + + @Test + public void testFullOnTopN() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .dimension(providerDimension) + .metric(indexMetric) + .threshold(4) + .intervals(fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + commonAggregators, + Lists.newArrayList( + new MaxAggregatorFactory("maxIndex", "index"), + new MinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("provider", "total_market") + .put("rows", 186L) + .put("index", 215679.82879638672D) + .put("addRowsIndexConstant", 215866.82879638672D) + .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("maxIndex", 1870.06103515625D) + .put("minIndex", 545.9906005859375D) + .build(), + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 837L) + .put("index", 95606.57232284546D) + .put("addRowsIndexConstant", 96444.57232284546D) + .put("maxIndex", 277.2735290527344D) + .put("minIndex", 59.02102279663086D) + .build() + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testFullOnTopNOverPostAggs() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .dimension(providerDimension) + .metric(addRowsIndexConstantMetric) + .threshold(4) + .intervals(fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + commonAggregators, + Lists.newArrayList( + new MaxAggregatorFactory("maxIndex", "index"), + new MinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("provider", "total_market") + .put("rows", 186L) + .put("index", 215679.82879638672D) + .put("addRowsIndexConstant", 215866.82879638672D) + .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("maxIndex", 1870.06103515625D) + .put("minIndex", 545.9906005859375D) + .build(), + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 837L) + .put("index", 95606.57232284546D) + .put("addRowsIndexConstant", 96444.57232284546D) + .put("maxIndex", 277.2735290527344D) + .put("minIndex", 59.02102279663086D) + .build() + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTopN() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .dimension(providerDimension) + .metric(indexMetric) + .threshold(4) + .intervals(firstToThird) + .aggregators(commonAggregators) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + "provider", "total_market", + "rows", 4L, + "index", 5351.814697265625D, + "addRowsIndexConstant", 5356.814697265625D + ), + ImmutableMap.of( + "provider", "upfront", + "rows", 4L, + "index", 4875.669677734375D, + "addRowsIndexConstant", 4880.669677734375D + ), + ImmutableMap.of( + "provider", "spot", + "rows", 18L, + "index", 2231.8768157958984D, + "addRowsIndexConstant", 2250.8768157958984D + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTopNWithOrFilter1() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .filters(providerDimension, "total_market", "upfront", "spot") + .dimension(providerDimension) + .metric(indexMetric) + .threshold(4) + .intervals(firstToThird) + .aggregators(commonAggregators) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + "provider", "total_market", + "rows", 4L, + "index", 5351.814697265625D, + "addRowsIndexConstant", 5356.814697265625D + ), + ImmutableMap.of( + "provider", "upfront", + "rows", 4L, + "index", 4875.669677734375D, + "addRowsIndexConstant", 4880.669677734375D + ), + ImmutableMap.of( + "provider", "spot", + "rows", 18L, + "index", 2231.8768157958984D, + "addRowsIndexConstant", 2250.8768157958984D + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTopNWithOrFilter2() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .filters(providerDimension, "total_market", "upfront") + .dimension(providerDimension) + .metric(indexMetric) + .threshold(4) + .intervals(firstToThird) + .aggregators(commonAggregators) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + "provider", "total_market", + "rows", 4L, + "index", 5351.814697265625D, + "addRowsIndexConstant", 5356.814697265625D + ), + ImmutableMap.of( + "provider", "upfront", + "rows", 4L, + "index", 4875.669677734375D, + "addRowsIndexConstant", 4880.669677734375D + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTopNWithFilter1() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .filters(providerDimension, "upfront") + .dimension(providerDimension) + .metric(indexMetric) + .threshold(4) + .intervals(firstToThird) + .aggregators(commonAggregators) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + "provider", "upfront", + "rows", 4L, + "index", 4875.669677734375D, + "addRowsIndexConstant", 4880.669677734375D + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTopNWithFilter2() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .filters(qualityDimension, "mezzanine") + .dimension(providerDimension) + .metric(indexMetric) + .threshold(4) + .intervals(firstToThird) + .aggregators(commonAggregators) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + "provider", "upfront", + "rows", 2L, + "index", 2591.68359375D, + "addRowsIndexConstant", 2594.68359375D + ), + ImmutableMap.of( + "provider", "total_market", + "rows", 2L, + "index", 2508.39599609375D, + "addRowsIndexConstant", 2511.39599609375D + ), + ImmutableMap.of( + "provider", "spot", + "rows", 2L, + "index", 220.63774871826172D, + "addRowsIndexConstant", 223.63774871826172D + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTopNWithFilter2OneDay() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .filters(qualityDimension, "mezzanine") + .dimension(providerDimension) + .metric(indexMetric) + .threshold(4) + .intervals( + new MultipleIntervalSegmentSpec( + Arrays.asList(new Interval("2011-04-01T00:00:00.000Z/2011-04-02T00:00:00.000Z")) + ) + ) + .aggregators(commonAggregators) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + "provider", "upfront", + "rows", 1L, + "index", new Float(1447.341160).doubleValue(), + "addRowsIndexConstant", new Float(1449.341160).doubleValue() + ), + ImmutableMap.of( + "provider", "total_market", + "rows", 1L, + "index", new Float(1314.839715).doubleValue(), + "addRowsIndexConstant", new Float(1316.839715).doubleValue() + ), + ImmutableMap.of( + "provider", "spot", + "rows", 1L, + "index", new Float(109.705815).doubleValue(), + "addRowsIndexConstant", new Float(111.705815).doubleValue() + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTopNWithNonExistentFilterInOr() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .filters(providerDimension, "total_market", "upfront", "billyblank") + .dimension(providerDimension) + .metric(indexMetric) + .threshold(4) + .intervals(firstToThird) + .aggregators(commonAggregators) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + "provider", "total_market", + "rows", 4L, + "index", 5351.814697265625D, + "addRowsIndexConstant", 5356.814697265625D + ), + ImmutableMap.of( + "provider", "upfront", + "rows", 4L, + "index", 4875.669677734375D, + "addRowsIndexConstant", 4880.669677734375D + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTopNWithNonExistentFilter() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .filters(providerDimension, "billyblank") + .dimension(providerDimension) + .metric(indexMetric) + .threshold(4) + .intervals(firstToThird) + .aggregators(commonAggregators) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + TestHelper.assertExpectedResults( + Lists.>newArrayList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue(Lists.>newArrayList()) + ) + ), + runner.run(query) + ); + } + + @Test + public void testTopNWithNonExistentFilterMultiDim() + { + AndDimFilter andDimFilter = Druids.newAndDimFilterBuilder() + .fields( + Lists.newArrayList( + Druids.newSelectorDimFilterBuilder() + .dimension(providerDimension) + .value("billyblank") + .build(), + Druids.newSelectorDimFilterBuilder() + .dimension(qualityDimension) + .value("mezzanine") + .build() + ) + ).build(); + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .filters(andDimFilter) + .dimension(providerDimension) + .metric(indexMetric) + .threshold(4) + .intervals(firstToThird) + .aggregators(commonAggregators) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + TestHelper.assertExpectedResults( + Lists.>newArrayList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue(Lists.>newArrayList()) + ) + ), + runner.run(query) + ); + } + + @Test + public void testTopNWithMultiValueDimFilter1() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .filters(placementishDimension, "m") + .dimension(providerDimension) + .metric(indexMetric) + .threshold(4) + .intervals(firstToThird) + .aggregators(commonAggregators) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + TestHelper.assertExpectedResults( + Sequences.toList( + runner.run( + new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .filters(qualityDimension, "mezzanine") + .dimension(providerDimension) + .metric(indexMetric) + .threshold(4) + .intervals(firstToThird) + .aggregators(commonAggregators) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build() + ), Lists.>newArrayList() + ), runner.run(query) + ); + } + + @Test + public void testTopNWithMultiValueDimFilter2() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .filters(placementishDimension, "m", "a", "b") + .dimension(qualityDimension) + .metric(indexMetric) + .threshold(4) + .intervals(firstToThird) + .aggregators(commonAggregators) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + TestHelper.assertExpectedResults( + Sequences.toList( + runner.run( + new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .filters(qualityDimension, "mezzanine", "automotive", "business") + .dimension(qualityDimension) + .metric(indexMetric) + .threshold(4) + .intervals(firstToThird) + .aggregators(commonAggregators) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build() + ), Lists.>newArrayList() + ) + , runner.run(query) + ); + } + + @Test + public void testTopNWithMultiValueDimFilter3() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .filters(placementishDimension, "a") + .dimension(placementishDimension) + .metric(indexMetric) + .threshold(4) + .intervals(firstToThird) + .aggregators(commonAggregators) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + final ArrayList> expectedResults = Lists.newArrayList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + "placementish", "a", + "rows", 2L, + "index", 283.31103515625D, + "addRowsIndexConstant", 286.31103515625D + ), + ImmutableMap.of( + "placementish", "preferred", + "rows", 2L, + "index", 283.31103515625D, + "addRowsIndexConstant", 286.31103515625D + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTopNWithMultiValueDimFilter4() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .filters(placementishDimension, "a", "b") + .dimension(placementishDimension) + .metric(indexMetric) + .threshold(4) + .intervals(firstToThird) + .aggregators(commonAggregators) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + final ArrayList> expectedResults = Lists.newArrayList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + "placementish", "preferred", + "rows", 4L, + "index", 514.868408203125D, + "addRowsIndexConstant", 519.868408203125D + ), + ImmutableMap.of( + "placementish", + "a", "rows", 2L, + "index", 283.31103515625D, + "addRowsIndexConstant", 286.31103515625D + ), + ImmutableMap.of( + "placementish", "b", + "rows", 2L, + "index", 231.557373046875D, + "addRowsIndexConstant", 234.557373046875D + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTopNWithMultiValueDimFilter5() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .filters(placementishDimension, "preferred") + .dimension(placementishDimension) + .metric(indexMetric) + .threshold(4) + .intervals(firstToThird) + .aggregators(commonAggregators) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + final ArrayList> expectedResults = Lists.newArrayList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + "placementish", "preferred", + "rows", 26L, + "index", 12459.361190795898D, + "addRowsIndexConstant", 12486.361190795898D + ), + ImmutableMap.of( + "placementish", "p", + "rows", 6L, + "index", 5407.213653564453D, + "addRowsIndexConstant", 5414.213653564453D + ), + ImmutableMap.of( + "placementish", "m", + "rows", 6L, + "index", 5320.717338562012D, + "addRowsIndexConstant", 5327.717338562012D + ), + ImmutableMap.of( + "placementish", "t", + "rows", 4L, + "index", 422.3440856933594D, + "addRowsIndexConstant", 427.3440856933594D + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTopNLexicographic() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .dimension(providerDimension) + .metric(new LexicographicTopNMetricSpec("")) + .threshold(4) + .intervals(firstToThird) + .aggregators(commonAggregators) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + "provider", "spot", + "rows", 18L, + "index", 2231.8768157958984D, + "addRowsIndexConstant", 2250.8768157958984D + ), + ImmutableMap.of( + "provider", "total_market", + "rows", 4L, + "index", 5351.814697265625D, + "addRowsIndexConstant", 5356.814697265625D + ), + ImmutableMap.of( + "provider", "upfront", + "rows", 4L, + "index", 4875.669677734375D, + "addRowsIndexConstant", 4880.669677734375D + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTopNLexicographicWithPreviousStop() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .dimension(providerDimension) + .metric(new LexicographicTopNMetricSpec("spot")) + .threshold(4) + .intervals(firstToThird) + .aggregators(commonAggregators) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + "provider", "total_market", + "rows", 4L, + "index", 5351.814697265625D, + "addRowsIndexConstant", 5356.814697265625D + ), + ImmutableMap.of( + "provider", "upfront", + "rows", 4L, + "index", 4875.669677734375D, + "addRowsIndexConstant", 4880.669677734375D + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTopNLexicographicWithNonExistingPreviousStop() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .dimension(providerDimension) + .metric(new LexicographicTopNMetricSpec("t")) + .threshold(4) + .intervals(firstToThird) + .aggregators(commonAggregators) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + "provider", "total_market", + "rows", 4L, + "index", 5351.814697265625D, + "addRowsIndexConstant", 5356.814697265625D + ), + ImmutableMap.of( + "provider", "upfront", + "rows", 4L, + "index", 4875.669677734375D, + "addRowsIndexConstant", 4880.669677734375D + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTopNDimExtraction() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .dimension( + new ExtractionDimensionSpec( + providerDimension, providerDimension, new RegexDimExtractionFn("(.)") + ) + ) + .metric("rows") + .threshold(4) + .intervals(firstToThird) + .aggregators(commonAggregators) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + "provider", "s", + "rows", 18L, + "index", 2231.8768157958984D, + "addRowsIndexConstant", 2250.8768157958984D + ), + ImmutableMap.of( + "provider", "t", + "rows", 4L, + "index", 5351.814697265625D, + "addRowsIndexConstant", 5356.814697265625D + ), + ImmutableMap.of( + "provider", "u", + "rows", 4L, + "index", 4875.669677734375D, + "addRowsIndexConstant", 4880.669677734375D + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testInvertedTopNQuery() + { + TopNQuery query = + new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .dimension(providerDimension) + .metric(new InvertedTopNMetricSpec(new NumericTopNMetricSpec(indexMetric))) + .threshold(3) + .intervals(firstToThird) + .aggregators(commonAggregators) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + "provider", "spot", + "rows", 18L, + "index", 2231.8768157958984D, + "addRowsIndexConstant", 2250.8768157958984D + ), + ImmutableMap.of( + "provider", "upfront", + "rows", 4L, + "index", 4875.669677734375D, + "addRowsIndexConstant", 4880.669677734375D + ), + ImmutableMap.of( + "provider", "total_market", + "rows", 4L, + "index", 5351.814697265625D, + "addRowsIndexConstant", 5356.814697265625D + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } +} \ No newline at end of file diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTestHelper.java new file mode 100644 index 00000000000..977cd9e7cdf --- /dev/null +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTestHelper.java @@ -0,0 +1,73 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.query.topn; + +import io.druid.query.FinalizeResultsQueryRunner; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.segment.IncrementalIndexSegment; +import io.druid.segment.QueryableIndex; +import io.druid.segment.QueryableIndexSegment; +import io.druid.segment.Segment; +import io.druid.segment.TestIndex; +import io.druid.segment.incremental.IncrementalIndex; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; + +public class TopNQueryRunnerTestHelper +{ + @SuppressWarnings("unchecked") + public static Collection makeQueryRunners( + QueryRunnerFactory factory + ) + throws IOException + { + final IncrementalIndex rtIndex = TestIndex.getIncrementalTestIndex(); + final QueryableIndex mMappedTestIndex = TestIndex.getMMappedTestIndex(); + final QueryableIndex mergedRealtimeIndex = TestIndex.mergedRealtimeIndex(); + return Arrays.asList( + new Object[][]{ + { + makeQueryRunner(factory, new IncrementalIndexSegment(rtIndex)) + }, + { + makeQueryRunner(factory, new QueryableIndexSegment(null, mMappedTestIndex)) + }, + { + makeQueryRunner(factory, new QueryableIndexSegment(null, mergedRealtimeIndex)) + } + } + ); + } + + public static QueryRunner makeQueryRunner( + QueryRunnerFactory> factory, + Segment adapter + ) + { + return new FinalizeResultsQueryRunner( + factory.createRunner(adapter), + factory.getToolchest() + ); + } +} \ No newline at end of file diff --git a/server/src/main/java/io/druid/guice/QueryRunnerFactoryModule.java b/server/src/main/java/io/druid/guice/QueryRunnerFactoryModule.java index 0f9970a6f30..fc1ab48fcfc 100644 --- a/server/src/main/java/io/druid/guice/QueryRunnerFactoryModule.java +++ b/server/src/main/java/io/druid/guice/QueryRunnerFactoryModule.java @@ -35,6 +35,8 @@ import io.druid.query.timeboundary.TimeBoundaryQuery; import io.druid.query.timeboundary.TimeBoundaryQueryRunnerFactory; import io.druid.query.timeseries.TimeseriesQuery; import io.druid.query.timeseries.TimeseriesQueryRunnerFactory; +import io.druid.query.topn.TopNQuery; +import io.druid.query.topn.TopNQueryRunnerFactory; import java.util.Map; @@ -49,6 +51,7 @@ public class QueryRunnerFactoryModule extends QueryToolChestModule .put(TimeBoundaryQuery.class, TimeBoundaryQueryRunnerFactory.class) .put(SegmentMetadataQuery.class, SegmentMetadataQueryRunnerFactory.class) .put(GroupByQuery.class, GroupByQueryRunnerFactory.class) + .put(TopNQuery.class, TopNQueryRunnerFactory.class) .build(); @Override diff --git a/server/src/main/java/io/druid/guice/QueryToolChestModule.java b/server/src/main/java/io/druid/guice/QueryToolChestModule.java index 7c4981560d8..4e103db29de 100644 --- a/server/src/main/java/io/druid/guice/QueryToolChestModule.java +++ b/server/src/main/java/io/druid/guice/QueryToolChestModule.java @@ -38,6 +38,9 @@ import io.druid.query.timeboundary.TimeBoundaryQuery; import io.druid.query.timeboundary.TimeBoundaryQueryQueryToolChest; import io.druid.query.timeseries.TimeseriesQuery; import io.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import io.druid.query.topn.TopNQuery; +import io.druid.query.topn.TopNQueryConfig; +import io.druid.query.topn.TopNQueryQueryToolChest; import java.util.Map; @@ -52,6 +55,7 @@ public class QueryToolChestModule implements Module .put(TimeBoundaryQuery.class, TimeBoundaryQueryQueryToolChest.class) .put(SegmentMetadataQuery.class, SegmentMetadataQueryQueryToolChest.class) .put(GroupByQuery.class, GroupByQueryQueryToolChest.class) + .put(TopNQuery.class, TopNQueryQueryToolChest.class) .build(); @Override @@ -67,5 +71,6 @@ public class QueryToolChestModule implements Module JsonConfigProvider.bind(binder, "druid.query", QueryConfig.class); JsonConfigProvider.bind(binder, "druid.query.groupBy", GroupByQueryConfig.class); JsonConfigProvider.bind(binder, "druid.query.search", SearchQueryConfig.class); + JsonConfigProvider.bind(binder, "druid.query.topN", TopNQueryConfig.class); } } From 0301dde671a7db828acb4fb9c90652597cacd772 Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 8 Jan 2014 15:56:29 -0800 Subject: [PATCH 2/2] remove unnecessary doc line --- .../src/main/java/io/druid/query/topn/TopNAlgorithmSelector.java | 1 - 1 file changed, 1 deletion(-) diff --git a/processing/src/main/java/io/druid/query/topn/TopNAlgorithmSelector.java b/processing/src/main/java/io/druid/query/topn/TopNAlgorithmSelector.java index 424a3aa5950..a65b78dbc96 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNAlgorithmSelector.java +++ b/processing/src/main/java/io/druid/query/topn/TopNAlgorithmSelector.java @@ -50,7 +50,6 @@ public class TopNAlgorithmSelector { // These are just heuristics based on an analysis of where an inflection point may lie to switch // between different algorithms - // More info: https://metamarkets.atlassian.net/wiki/display/APP/Top+n+speeds+with+uniques if (cardinality > 400000 && numBytesPerRecord > 100) { this.aggregateTopNMetricFirst = aggregateTopNMetricFirst; }