From 4c8495662f672f146bd1f7eb64cd3bd3096a7aed Mon Sep 17 00:00:00 2001 From: Yuval Oren Date: Wed, 15 Jan 2014 15:38:57 -0800 Subject: [PATCH 01/66] Subquery support for GroupBy queries --- .../main/java/io/druid/query/BaseQuery.java | 38 ++- .../main/java/io/druid/query/DataSource.java | 38 +++ .../src/main/java/io/druid/query/Druids.java | 38 ++- .../src/main/java/io/druid/query/Query.java | 3 +- .../java/io/druid/query/QueryDataSource.java | 79 +++++ .../io/druid/query/SubqueryQueryRunner.java | 50 +++ .../java/io/druid/query/TableDataSource.java | 73 +++++ .../aggregation/CountAggregatorFactory.java | 19 ++ .../DoubleSumAggregatorFactory.java | 22 ++ .../HistogramAggregatorFactory.java | 26 ++ .../JavaScriptAggregatorFactory.java | 31 ++ .../aggregation/LongSumAggregatorFactory.java | 22 ++ .../aggregation/MaxAggregatorFactory.java | 22 ++ .../aggregation/MinAggregatorFactory.java | 22 ++ .../ToLowerCaseAggregatorFactory.java | 20 ++ .../post/ArithmeticPostAggregator.java | 26 ++ .../post/ConstantPostAggregator.java | 29 ++ .../post/FieldAccessPostAggregator.java | 22 ++ .../post/JavaScriptPostAggregator.java | 26 ++ .../query/dimension/DefaultDimensionSpec.java | 22 ++ .../dimension/ExtractionDimensionSpec.java | 25 ++ .../io/druid/query/groupby/GroupByQuery.java | 81 ++++- .../groupby/GroupByQueryQueryToolChest.java | 80 +++-- .../groupby/orderby/DefaultLimitSpec.java | 22 ++ .../query/groupby/orderby/NoopLimitSpec.java | 11 + .../SegmentMetadataQueryQueryToolChest.java | 8 +- .../metadata/SegmentMetadataQuery.java | 35 ++- .../search/SearchQueryQueryToolChest.java | 11 +- .../search/FragmentSearchQuerySpec.java | 19 ++ .../InsensitiveContainsSearchQuerySpec.java | 19 ++ .../search/LexicographicSearchSortSpec.java | 5 + .../query/search/search/SearchQuery.java | 40 ++- .../spec/MultipleIntervalSegmentSpec.java | 19 ++ .../spec/MultipleSpecificSegmentSpec.java | 22 ++ .../druid/query/spec/SpecificSegmentSpec.java | 19 ++ .../query/timeboundary/TimeBoundaryQuery.java | 19 +- .../TimeBoundaryQueryQueryToolChest.java | 9 +- .../query/timeseries/TimeseriesQuery.java | 37 ++- .../timeseries/TimeseriesQueryEngine.java | 2 + .../TimeseriesQueryQueryToolChest.java | 13 +- .../TimeseriesQueryRunnerFactory.java | 8 +- .../query/topn/InvertedTopNMetricSpec.java | 19 ++ .../topn/LexicographicTopNMetricSpec.java | 19 ++ .../query/topn/NumericTopNMetricSpec.java | 19 ++ .../java/io/druid/query/topn/TopNQuery.java | 45 ++- .../io/druid/query/topn/TopNQueryBuilder.java | 16 +- .../query/topn/TopNQueryQueryToolChest.java | 2 +- .../java/io/druid/query/DataSourceTest.java | 88 ++++++ .../io/druid/query/QueryRunnerTestHelper.java | 4 + .../query/groupby/GroupByQueryRunnerTest.java | 287 ++++++++++++++---- .../druid/query/groupby/GroupByQueryTest.java | 68 +++++ .../GroupByTimeseriesQueryRunnerTest.java | 46 +-- .../druid/query/search/SearchQueryTest.java | 62 ++++ .../timeboundary/TimeBoundaryQueryTest.java | 51 ++++ .../query/timeseries/TimeseriesQueryTest.java | 62 ++++ .../druid/query/topn/TopNQueryRunnerTest.java | 126 +++----- .../io/druid/query/topn/TopNQueryTest.java | 75 +++++ .../io/druid/client/BrokerServerView.java | 19 +- .../druid/client/CachingClusteredClient.java | 1 + .../io/druid/client/TimelineServerView.java | 3 +- .../java/io/druid/server/QueryResource.java | 2 +- .../server/coordination/ServerManager.java | 59 ++-- 62 files changed, 1894 insertions(+), 311 deletions(-) create mode 100644 processing/src/main/java/io/druid/query/DataSource.java create mode 100644 processing/src/main/java/io/druid/query/QueryDataSource.java create mode 100644 processing/src/main/java/io/druid/query/SubqueryQueryRunner.java create mode 100644 processing/src/main/java/io/druid/query/TableDataSource.java create mode 100644 processing/src/test/java/io/druid/query/DataSourceTest.java create mode 100644 processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java create mode 100644 processing/src/test/java/io/druid/query/search/SearchQueryTest.java create mode 100644 processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryTest.java create mode 100644 processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryTest.java create mode 100644 processing/src/test/java/io/druid/query/topn/TopNQueryTest.java diff --git a/processing/src/main/java/io/druid/query/BaseQuery.java b/processing/src/main/java/io/druid/query/BaseQuery.java index b1e1a587123..ef837cc67b0 100644 --- a/processing/src/main/java/io/druid/query/BaseQuery.java +++ b/processing/src/main/java/io/druid/query/BaseQuery.java @@ -34,14 +34,14 @@ import java.util.Map; */ public abstract class BaseQuery implements Query { - private final String dataSource; + private final DataSource dataSource; private final Map context; private final QuerySegmentSpec querySegmentSpec; private volatile Duration duration; public BaseQuery( - String dataSource, + DataSource dataSource, QuerySegmentSpec querySegmentSpec, Map context ) @@ -49,14 +49,14 @@ public abstract class BaseQuery implements Query Preconditions.checkNotNull(dataSource, "dataSource can't be null"); Preconditions.checkNotNull(querySegmentSpec, "querySegmentSpec can't be null"); - this.dataSource = dataSource.toLowerCase(); + this.dataSource = dataSource; this.context = context; this.querySegmentSpec = querySegmentSpec; } @JsonProperty @Override - public String getDataSource() + public DataSource getDataSource() { return dataSource; } @@ -130,4 +130,34 @@ public abstract class BaseQuery implements Query return overridden; } + + /** + * Compare the BaseQuery fields with another BaseQuery. For use in subclasses implementing equals() + * @param that + * @return + */ + protected boolean partialEquals(BaseQuery that) + { + if (context != null ? !context.equals(that.context) : that.context != null) return false; + if (dataSource != null ? !dataSource.equals(that.dataSource) : that.dataSource != null) return false; + if (duration != null ? !duration.equals(that.duration) : that.duration != null) return false; + if (querySegmentSpec != null ? !querySegmentSpec.equals(that.querySegmentSpec) : that.querySegmentSpec != null) + return false; + + return true; + } + + /** + * Hash the fields within BaseQuery. For use in subclasses implementing hashCode() + * @return + */ + protected int partialHashCode() + { + int result = dataSource != null ? dataSource.hashCode() : 0; + result = 31 * result + (context != null ? context.hashCode() : 0); + result = 31 * result + (querySegmentSpec != null ? querySegmentSpec.hashCode() : 0); + result = 31 * result + (duration != null ? duration.hashCode() : 0); + return result; + } + } diff --git a/processing/src/main/java/io/druid/query/DataSource.java b/processing/src/main/java/io/druid/query/DataSource.java new file mode 100644 index 00000000000..a9a4ed94b17 --- /dev/null +++ b/processing/src/main/java/io/druid/query/DataSource.java @@ -0,0 +1,38 @@ +/* + * Druid - a distributed column store. + * + * 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. + * + * This file Copyright (C) 2014 N3TWORK, Inc. and contributed to the Druid project + * under the Druid Corporate Contributor License Agreement. + */ + +package io.druid.query; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +import java.io.Serializable; +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, + include = JsonTypeInfo.As.PROPERTY, + property = "type", + defaultImpl = TableDataSource.class) +@JsonSubTypes({ + @JsonSubTypes.Type(value = TableDataSource.class, name = "table"), + @JsonSubTypes.Type(value = QueryDataSource.class, name = "query") +}) +public interface DataSource extends Serializable +{ +} diff --git a/processing/src/main/java/io/druid/query/Druids.java b/processing/src/main/java/io/druid/query/Druids.java index 99d5acdc6f6..fd2abd13ab8 100644 --- a/processing/src/main/java/io/druid/query/Druids.java +++ b/processing/src/main/java/io/druid/query/Druids.java @@ -298,7 +298,7 @@ public class Druids */ public static class TimeseriesQueryBuilder { - private String dataSource; + private DataSource dataSource; private QuerySegmentSpec querySegmentSpec; private DimFilter dimFilter; private QueryGranularity granularity; @@ -308,7 +308,7 @@ public class Druids private TimeseriesQueryBuilder() { - dataSource = ""; + dataSource = null; querySegmentSpec = null; dimFilter = null; granularity = QueryGranularity.ALL; @@ -354,7 +354,7 @@ public class Druids .context(builder.context); } - public String getDataSource() + public DataSource getDataSource() { return dataSource; } @@ -390,6 +390,12 @@ public class Druids } public TimeseriesQueryBuilder dataSource(String ds) + { + dataSource = new TableDataSource(ds); + return this; + } + + public TimeseriesQueryBuilder dataSource(DataSource ds) { dataSource = ds; return this; @@ -492,7 +498,7 @@ public class Druids */ public static class SearchQueryBuilder { - private String dataSource; + private DataSource dataSource; private DimFilter dimFilter; private QueryGranularity granularity; private int limit; @@ -503,7 +509,7 @@ public class Druids public SearchQueryBuilder() { - dataSource = ""; + dataSource = null; dimFilter = null; granularity = QueryGranularity.ALL; limit = 0; @@ -531,7 +537,7 @@ public class Druids public SearchQueryBuilder copy(SearchQuery query) { return new SearchQueryBuilder() - .dataSource(query.getDataSource()) + .dataSource(((TableDataSource)query.getDataSource()).getName()) .intervals(query.getQuerySegmentSpec()) .filters(query.getDimensionsFilter()) .granularity(query.getGranularity()) @@ -555,6 +561,12 @@ public class Druids } public SearchQueryBuilder dataSource(String d) + { + dataSource = new TableDataSource(d); + return this; + } + + public SearchQueryBuilder dataSource(DataSource d) { dataSource = d; return this; @@ -676,13 +688,13 @@ public class Druids */ public static class TimeBoundaryQueryBuilder { - private String dataSource; + private DataSource dataSource; private QuerySegmentSpec querySegmentSpec; private Map context; public TimeBoundaryQueryBuilder() { - dataSource = ""; + dataSource = null; querySegmentSpec = null; context = null; } @@ -704,9 +716,15 @@ public class Druids .context(builder.context); } - public TimeBoundaryQueryBuilder dataSource(String d) + public TimeBoundaryQueryBuilder dataSource(String ds) { - dataSource = d; + dataSource = new TableDataSource(ds); + return this; + } + + public TimeBoundaryQueryBuilder dataSource(DataSource ds) + { + dataSource = ds; return this; } diff --git a/processing/src/main/java/io/druid/query/Query.java b/processing/src/main/java/io/druid/query/Query.java index f4ad958e75d..34b145a2cb1 100644 --- a/processing/src/main/java/io/druid/query/Query.java +++ b/processing/src/main/java/io/druid/query/Query.java @@ -19,6 +19,7 @@ package io.druid.query; +import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.common.guava.Sequence; @@ -53,7 +54,7 @@ public interface Query public static final String SEGMENT_METADATA = "segmentMetadata"; public static final String TOPN = "topN"; - public String getDataSource(); + public DataSource getDataSource(); public boolean hasFilters(); diff --git a/processing/src/main/java/io/druid/query/QueryDataSource.java b/processing/src/main/java/io/druid/query/QueryDataSource.java new file mode 100644 index 00000000000..b1721e54704 --- /dev/null +++ b/processing/src/main/java/io/druid/query/QueryDataSource.java @@ -0,0 +1,79 @@ +/* + * Druid - a distributed column store. + * + * 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. + * + * This file Copyright (C) 2014 N3TWORK, Inc. and contributed to the Druid project + * under the Druid Corporate Contributor License Agreement. + */ + +package io.druid.query; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import io.druid.query.groupby.GroupByQuery; +import io.druid.query.metadata.metadata.SegmentMetadataQuery; +import io.druid.query.search.search.SearchQuery; +import io.druid.query.timeboundary.TimeBoundaryQuery; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.topn.TopNQuery; + +public class QueryDataSource implements DataSource +{ + @JsonProperty + private Query query; + + public QueryDataSource() + { + } + + public QueryDataSource(Query query) + { + this.query = query; + } + + public Query getQuery() + { + return query; + } + + public void setQuery(Query query) + { + this.query = query; + } + + public String toString() { return query.toString(); } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + QueryDataSource that = (QueryDataSource) o; + + if (!query.equals(that.query)) return false; + + return true; + } + + @Override + public int hashCode() + { + return query.hashCode(); + } +} diff --git a/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java b/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java new file mode 100644 index 00000000000..86ce37b5cc8 --- /dev/null +++ b/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java @@ -0,0 +1,50 @@ +/* + * Druid - a distributed column store. + * + * 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. + * + * This file Copyright (C) 2014 N3TWORK, Inc. and contributed to the Druid project + * under the Druid Corporate Contributor License Agreement. + */ + +package io.druid.query; + +import com.metamx.common.guava.Sequence; +import org.joda.time.Period; + +/** + * If there's a subquery, run it instead of the outer query + */ +public class SubqueryQueryRunner implements QueryRunner +{ + private final QueryRunner baseRunner; + + public SubqueryQueryRunner(QueryRunner baseRunner) + { + this.baseRunner = baseRunner; + } + + @Override + public Sequence run(final Query query) + { + DataSource dataSource = query.getDataSource(); + if (dataSource instanceof QueryDataSource) { + return run((Query) ((QueryDataSource)dataSource).getQuery()); + } + else { + return baseRunner.run(query); + } + } +} diff --git a/processing/src/main/java/io/druid/query/TableDataSource.java b/processing/src/main/java/io/druid/query/TableDataSource.java new file mode 100644 index 00000000000..a6b091bdc89 --- /dev/null +++ b/processing/src/main/java/io/druid/query/TableDataSource.java @@ -0,0 +1,73 @@ +/* + * Druid - a distributed column store. + * + * 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. + * + * This file Copyright (C) 2014 N3TWORK, Inc. and contributed to the Druid project + * under the Druid Corporate Contributor License Agreement. + */ +package io.druid.query; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class TableDataSource implements DataSource +{ + @JsonProperty + private String name; + + @JsonCreator + public TableDataSource() + { + + } + + @JsonCreator + public TableDataSource(String name) + { + this.name = name==null? name : name.toLowerCase(); + } + + public String getName() + { + return name; + } + + public void setName(String name) + { + this.name = name; + } + + public String toString() { return name; } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + TableDataSource that = (TableDataSource) o; + + if (!name.equals(that.name)) return false; + + return true; + } + + @Override + public int hashCode() + { + return name.hashCode(); + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java index 1b02f923996..e47999e8719 100644 --- a/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java @@ -132,4 +132,23 @@ public class CountAggregatorFactory implements AggregatorFactory "name='" + name + '\'' + '}'; } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + CountAggregatorFactory that = (CountAggregatorFactory) o; + + if (name != null ? !name.equals(that.name) : that.name != null) return false; + + return true; + } + + @Override + public int hashCode() + { + return name != null ? name.hashCode() : 0; + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java index f85d0c677f5..ebd4e185ea3 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java @@ -150,4 +150,26 @@ public class DoubleSumAggregatorFactory implements AggregatorFactory ", name='" + name + '\'' + '}'; } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + DoubleSumAggregatorFactory that = (DoubleSumAggregatorFactory) o; + + if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) return false; + if (name != null ? !name.equals(that.name) : that.name != null) return false; + + return true; + } + + @Override + public int hashCode() + { + int result = fieldName != null ? fieldName.hashCode() : 0; + result = 31 * result + (name != null ? name.hashCode() : 0); + return result; + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java index 2abd19d2330..060d40d2798 100644 --- a/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java @@ -179,4 +179,30 @@ public class HistogramAggregatorFactory implements AggregatorFactory ", breaks=" + Arrays.toString(breaks) + '}'; } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + HistogramAggregatorFactory that = (HistogramAggregatorFactory) o; + + if (!Arrays.equals(breaks, that.breaks)) return false; + if (breaksList != null ? !breaksList.equals(that.breaksList) : that.breaksList != null) return false; + if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) return false; + if (name != null ? !name.equals(that.name) : that.name != null) return false; + + return true; + } + + @Override + public int hashCode() + { + int result = name != null ? name.hashCode() : 0; + result = 31 * result + (fieldName != null ? fieldName.hashCode() : 0); + result = 31 * result + (breaksList != null ? breaksList.hashCode() : 0); + result = 31 * result + (breaks != null ? Arrays.hashCode(breaks) : 0); + return result; + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java index 927ab89676f..6de6be09ad8 100644 --- a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java @@ -317,4 +317,35 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory } }; } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + JavaScriptAggregatorFactory that = (JavaScriptAggregatorFactory) o; + + if (compiledScript != null ? !compiledScript.equals(that.compiledScript) : that.compiledScript != null) + return false; + if (fieldNames != null ? !fieldNames.equals(that.fieldNames) : that.fieldNames != null) return false; + if (fnAggregate != null ? !fnAggregate.equals(that.fnAggregate) : that.fnAggregate != null) return false; + if (fnCombine != null ? !fnCombine.equals(that.fnCombine) : that.fnCombine != null) return false; + if (fnReset != null ? !fnReset.equals(that.fnReset) : that.fnReset != null) return false; + if (name != null ? !name.equals(that.name) : that.name != null) return false; + + return true; + } + + @Override + public int hashCode() + { + int result = name != null ? name.hashCode() : 0; + result = 31 * result + (fieldNames != null ? fieldNames.hashCode() : 0); + result = 31 * result + (fnAggregate != null ? fnAggregate.hashCode() : 0); + result = 31 * result + (fnReset != null ? fnReset.hashCode() : 0); + result = 31 * result + (fnCombine != null ? fnCombine.hashCode() : 0); + result = 31 * result + (compiledScript != null ? compiledScript.hashCode() : 0); + return result; + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java index f1372ff024c..50ef5130756 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java @@ -150,4 +150,26 @@ public class LongSumAggregatorFactory implements AggregatorFactory ", name='" + name + '\'' + '}'; } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + LongSumAggregatorFactory that = (LongSumAggregatorFactory) o; + + if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) return false; + if (name != null ? !name.equals(that.name) : that.name != null) return false; + + return true; + } + + @Override + public int hashCode() + { + int result = fieldName != null ? fieldName.hashCode() : 0; + result = 31 * result + (name != null ? name.hashCode() : 0); + return result; + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java index d66d9c5b2d9..ee8217f820b 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java @@ -150,4 +150,26 @@ public class MaxAggregatorFactory implements AggregatorFactory ", name='" + name + '\'' + '}'; } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + MaxAggregatorFactory that = (MaxAggregatorFactory) o; + + if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) return false; + if (name != null ? !name.equals(that.name) : that.name != null) return false; + + return true; + } + + @Override + public int hashCode() + { + int result = fieldName != null ? fieldName.hashCode() : 0; + result = 31 * result + (name != null ? name.hashCode() : 0); + return result; + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java index 0a168114358..9c3d560bacf 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java @@ -150,4 +150,26 @@ public class MinAggregatorFactory implements AggregatorFactory ", name='" + name + '\'' + '}'; } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + MinAggregatorFactory that = (MinAggregatorFactory) o; + + if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) return false; + if (name != null ? !name.equals(that.name) : that.name != null) return false; + + return true; + } + + @Override + public int hashCode() + { + int result = fieldName != null ? fieldName.hashCode() : 0; + result = 31 * result + (name != null ? name.hashCode() : 0); + return result; + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/ToLowerCaseAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/ToLowerCaseAggregatorFactory.java index 457d22f2cdf..6c559ba8ec6 100644 --- a/processing/src/main/java/io/druid/query/aggregation/ToLowerCaseAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/ToLowerCaseAggregatorFactory.java @@ -112,4 +112,24 @@ public class ToLowerCaseAggregatorFactory implements AggregatorFactory { return baseAggregatorFactory.getAggregatorStartValue(); } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + ToLowerCaseAggregatorFactory that = (ToLowerCaseAggregatorFactory) o; + + if (baseAggregatorFactory != null ? !baseAggregatorFactory.equals(that.baseAggregatorFactory) : that.baseAggregatorFactory != null) + return false; + + return true; + } + + @Override + public int hashCode() + { + return baseAggregatorFactory != null ? baseAggregatorFactory.hashCode() : 0; + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/post/ArithmeticPostAggregator.java b/processing/src/main/java/io/druid/query/aggregation/post/ArithmeticPostAggregator.java index cf6881d18ad..87138dadef8 100644 --- a/processing/src/main/java/io/druid/query/aggregation/post/ArithmeticPostAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/post/ArithmeticPostAggregator.java @@ -193,4 +193,30 @@ public class ArithmeticPostAggregator implements PostAggregator return lookupMap.keySet(); } } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + ArithmeticPostAggregator that = (ArithmeticPostAggregator) o; + + if (fields != null ? !fields.equals(that.fields) : that.fields != null) return false; + if (fnName != null ? !fnName.equals(that.fnName) : that.fnName != null) return false; + if (name != null ? !name.equals(that.name) : that.name != null) return false; + if (op != that.op) return false; + + return true; + } + + @Override + public int hashCode() + { + int result = name != null ? name.hashCode() : 0; + result = 31 * result + (fnName != null ? fnName.hashCode() : 0); + result = 31 * result + (fields != null ? fields.hashCode() : 0); + result = 31 * result + (op != null ? op.hashCode() : 0); + return result; + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/post/ConstantPostAggregator.java b/processing/src/main/java/io/druid/query/aggregation/post/ConstantPostAggregator.java index d5dce4d3140..7bf4ed23d4c 100644 --- a/processing/src/main/java/io/druid/query/aggregation/post/ConstantPostAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/post/ConstantPostAggregator.java @@ -91,4 +91,33 @@ public class ConstantPostAggregator implements PostAggregator ", constantValue=" + constantValue + '}'; } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + ConstantPostAggregator that = (ConstantPostAggregator) o; + + if (constantValue != null && that.constantValue != null) { + if (constantValue.doubleValue() != that.constantValue.doubleValue()) + return false; + } + else if (constantValue != that.constantValue) { + return false; + } + + if (name != null ? !name.equals(that.name) : that.name != null) return false; + + return true; + } + + @Override + public int hashCode() + { + int result = name != null ? name.hashCode() : 0; + result = 31 * result + (constantValue != null ? constantValue.hashCode() : 0); + return result; + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/post/FieldAccessPostAggregator.java b/processing/src/main/java/io/druid/query/aggregation/post/FieldAccessPostAggregator.java index 59962836d2e..6c2321b10fd 100644 --- a/processing/src/main/java/io/druid/query/aggregation/post/FieldAccessPostAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/post/FieldAccessPostAggregator.java @@ -84,4 +84,26 @@ public class FieldAccessPostAggregator implements PostAggregator ", fieldName='" + fieldName + '\'' + '}'; } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + FieldAccessPostAggregator that = (FieldAccessPostAggregator) o; + + if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) return false; + if (name != null ? !name.equals(that.name) : that.name != null) return false; + + return true; + } + + @Override + public int hashCode() + { + int result = name != null ? name.hashCode() : 0; + result = 31 * result + (fieldName != null ? fieldName.hashCode() : 0); + return result; + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/post/JavaScriptPostAggregator.java b/processing/src/main/java/io/druid/query/aggregation/post/JavaScriptPostAggregator.java index fef4b26f0f2..666dbc64a82 100644 --- a/processing/src/main/java/io/druid/query/aggregation/post/JavaScriptPostAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/post/JavaScriptPostAggregator.java @@ -142,4 +142,30 @@ public class JavaScriptPostAggregator implements PostAggregator { return function; } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + JavaScriptPostAggregator that = (JavaScriptPostAggregator) o; + + if (fieldNames != null ? !fieldNames.equals(that.fieldNames) : that.fieldNames != null) return false; + if (fn != null ? !fn.equals(that.fn) : that.fn != null) return false; + if (function != null ? !function.equals(that.function) : that.function != null) return false; + if (name != null ? !name.equals(that.name) : that.name != null) return false; + + return true; + } + + @Override + public int hashCode() + { + int result = name != null ? name.hashCode() : 0; + result = 31 * result + (fieldNames != null ? fieldNames.hashCode() : 0); + result = 31 * result + (function != null ? function.hashCode() : 0); + result = 31 * result + (fn != null ? fn.hashCode() : 0); + return result; + } } diff --git a/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java index 3a9137b008e..8e18ce61228 100644 --- a/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java @@ -84,4 +84,26 @@ public class DefaultDimensionSpec implements DimensionSpec ", outputName='" + outputName + '\'' + '}'; } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + DefaultDimensionSpec that = (DefaultDimensionSpec) o; + + if (dimension != null ? !dimension.equals(that.dimension) : that.dimension != null) return false; + if (outputName != null ? !outputName.equals(that.outputName) : that.outputName != null) return false; + + return true; + } + + @Override + public int hashCode() + { + int result = dimension != null ? dimension.hashCode() : 0; + result = 31 * result + (outputName != null ? outputName.hashCode() : 0); + return result; + } } diff --git a/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java index 82fba73d0a1..9fe480e396d 100644 --- a/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java @@ -92,4 +92,29 @@ public class ExtractionDimensionSpec implements DimensionSpec ", outputName='" + outputName + '\'' + '}'; } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + ExtractionDimensionSpec that = (ExtractionDimensionSpec) o; + + if (dimExtractionFn != null ? !dimExtractionFn.equals(that.dimExtractionFn) : that.dimExtractionFn != null) + return false; + if (dimension != null ? !dimension.equals(that.dimension) : that.dimension != null) return false; + if (outputName != null ? !outputName.equals(that.outputName) : that.outputName != null) return false; + + return true; + } + + @Override + public int hashCode() + { + int result = dimension != null ? dimension.hashCode() : 0; + result = 31 * result + (dimExtractionFn != null ? dimExtractionFn.hashCode() : 0); + result = 31 * result + (outputName != null ? outputName.hashCode() : 0); + return result; + } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java index 93db02101ba..ebdd7a06489 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java @@ -32,8 +32,7 @@ import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import io.druid.data.input.Row; import io.druid.granularity.QueryGranularity; -import io.druid.query.BaseQuery; -import io.druid.query.Queries; +import io.druid.query.*; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DefaultDimensionSpec; @@ -72,7 +71,7 @@ public class GroupByQuery extends BaseQuery @JsonCreator public GroupByQuery( - @JsonProperty("dataSource") String dataSource, + @JsonProperty("dataSource") DataSource dataSource, @JsonProperty("intervals") QuerySegmentSpec querySegmentSpec, @JsonProperty("filter") DimFilter dimFilter, @JsonProperty("granularity") QueryGranularity granularity, @@ -133,7 +132,7 @@ public class GroupByQuery extends BaseQuery * have already passed in order for the object to exist. */ private GroupByQuery( - String dataSource, + DataSource dataSource, QuerySegmentSpec querySegmentSpec, DimFilter dimFilter, QueryGranularity granularity, @@ -255,7 +254,7 @@ public class GroupByQuery extends BaseQuery public static class Builder { - private String dataSource; + private DataSource dataSource; private QuerySegmentSpec querySegmentSpec; private DimFilter dimFilter; private QueryGranularity granularity; @@ -270,7 +269,9 @@ public class GroupByQuery extends BaseQuery private List orderByColumnSpecs = Lists.newArrayList(); private int limit = Integer.MAX_VALUE; - private Builder() {} + private Builder() + { + } private Builder(Builder builder) { @@ -288,12 +289,24 @@ public class GroupByQuery extends BaseQuery context = builder.context; } - public Builder setDataSource(String dataSource) + public Builder setDataSource(DataSource dataSource) { this.dataSource = dataSource; return this; } + public Builder setDataSource(String dataSource) + { + this.dataSource = new TableDataSource(dataSource); + return this; + } + + public Builder setDataSource(Query query) + { + this.dataSource = new QueryDataSource(query); + return this; + } + public Builder setInterval(Object interval) { return setQuerySegmentSpec(new LegacySegmentSpec(interval)); @@ -479,13 +492,51 @@ public class GroupByQuery extends BaseQuery public String toString() { return "GroupByQuery{" + - "limitSpec=" + limitSpec + - ", dimFilter=" + dimFilter + - ", granularity=" + granularity + - ", dimensions=" + dimensions + - ", aggregatorSpecs=" + aggregatorSpecs + - ", postAggregatorSpecs=" + postAggregatorSpecs + - ", orderByLimitFn=" + orderByLimitFn + - '}'; + "limitSpec=" + limitSpec + + ", dimFilter=" + dimFilter + + ", granularity=" + granularity + + ", dimensions=" + dimensions + + ", aggregatorSpecs=" + aggregatorSpecs + + ", postAggregatorSpecs=" + postAggregatorSpecs + + ", orderByLimitFn=" + orderByLimitFn + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + GroupByQuery that = (GroupByQuery) o; + if (!partialEquals(that)) + return false; + + if (aggregatorSpecs != null ? !aggregatorSpecs.equals(that.aggregatorSpecs) : that.aggregatorSpecs != null) + return false; + if (dimFilter != null ? !dimFilter.equals(that.dimFilter) : that.dimFilter != null) return false; + if (dimensions != null ? !dimensions.equals(that.dimensions) : that.dimensions != null) return false; + if (granularity != null ? !granularity.equals(that.granularity) : that.granularity != null) return false; + if (havingSpec != null ? !havingSpec.equals(that.havingSpec) : that.havingSpec != null) return false; + if (limitSpec != null ? !limitSpec.equals(that.limitSpec) : that.limitSpec != null) return false; + if (postAggregatorSpecs != null ? !postAggregatorSpecs.equals(that.postAggregatorSpecs) : that.postAggregatorSpecs != null) + return false; + + return true; + } + + @Override + public int hashCode() + { + int result = limitSpec != null ? limitSpec.hashCode() : 0; + result = 31 * result + partialHashCode(); + result = 31 * result + (havingSpec != null ? havingSpec.hashCode() : 0); + result = 31 * result + (dimFilter != null ? dimFilter.hashCode() : 0); + result = 31 * result + (granularity != null ? granularity.hashCode() : 0); + result = 31 * result + (dimensions != null ? dimensions.hashCode() : 0); + result = 31 * result + (aggregatorSpecs != null ? aggregatorSpecs.hashCode() : 0); + result = 31 * result + (postAggregatorSpecs != null ? postAggregatorSpecs.hashCode() : 0); + result = 31 * result + (orderByLimitFn != null ? orderByLimitFn.hashCode() : 0); + return result; } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index 9bae22699c6..f85cf4ceab1 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -37,14 +37,12 @@ import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; import io.druid.data.input.Rows; import io.druid.granularity.QueryGranularity; -import io.druid.query.IntervalChunkingQueryRunner; -import io.druid.query.Query; -import io.druid.query.QueryRunner; -import io.druid.query.QueryToolChest; +import io.druid.query.*; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.MetricManipulationFn; import io.druid.query.dimension.DimensionSpec; import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.joda.time.Interval; import org.joda.time.Minutes; @@ -63,13 +61,16 @@ public class GroupByQueryQueryToolChest extends QueryToolChest NO_MERGE_CONTEXT = ImmutableMap.of(GROUP_BY_MERGE_KEY, "false"); private final Supplier configSupplier; + private GroupByQueryEngine engine; // For running the outer query around a subquery @Inject public GroupByQueryQueryToolChest( - Supplier configSupplier + Supplier configSupplier, + GroupByQueryEngine engine ) { this.configSupplier = configSupplier; + this.engine = engine; } @Override @@ -90,6 +91,52 @@ public class GroupByQueryQueryToolChest extends QueryToolChest mergeGroupByResults(final GroupByQuery query, QueryRunner runner) + { + + Sequence result; + + // If there's a subquery, merge subquery results and then apply the aggregator + DataSource dataSource = query.getDataSource(); + if (dataSource instanceof QueryDataSource) { + GroupByQuery subquery; + try { + subquery = (GroupByQuery) ((QueryDataSource)dataSource).getQuery(); + } catch (ClassCastException e) { + throw new UnsupportedOperationException("Subqueries must be of type 'group by'"); + } + Sequence subqueryResult = mergeGroupByResults(subquery, runner); + IncrementalIndexStorageAdapter adapter + = new IncrementalIndexStorageAdapter(makeIncrementalIndex(subquery, subqueryResult)); + result = engine.process(query, adapter); + } + else { + result = runner.run(query); + } + + return postAggregate(query, result); + } + + private Sequence postAggregate(final GroupByQuery query, Sequence result) + { + IncrementalIndex index = makeIncrementalIndex(query, result); + // convert millis back to timestamp according to granularity to preserve time zone information + Sequence retVal = Sequences.map( + Sequences.simple(index.iterableWithPostAggregations(query.getPostAggregatorSpecs())), + new Function() + { + @Override + public Row apply(Row input) + { + final MapBasedRow row = (MapBasedRow) input; + return new MapBasedRow(query.getGranularity().toDateTime(row.getTimestampFromEpoch()), row.getEvent()); + } + } + ); + + return query.applyLimit(retVal); + } + + private IncrementalIndex makeIncrementalIndex(GroupByQuery query, Sequence result) { final GroupByQueryConfig config = configSupplier.get(); final QueryGranularity gran = query.getGranularity(); @@ -122,7 +169,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest retVal = Sequences.map( - Sequences.simple(index.iterableWithPostAggregations(query.getPostAggregatorSpecs())), - new Function() - { - @Override - public Row apply(Row input) - { - final MapBasedRow row = (MapBasedRow) input; - return new MapBasedRow(gran.toDateTime(row.getTimestampFromEpoch()), row.getEvent()); - } - } - ); - - return query.applyLimit(retVal); + return index; } @Override @@ -176,7 +209,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest preMergeQueryDecoration(QueryRunner runner) { - return new IntervalChunkingQueryRunner(runner, configSupplier.get().getChunkPeriod()); + return new SubqueryQueryRunner( + new IntervalChunkingQueryRunner(runner, configSupplier.get().getChunkPeriod())); } } diff --git a/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java index 4cd052058bf..376a982ebe3 100644 --- a/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java @@ -232,4 +232,26 @@ public class DefaultLimitSpec implements LimitSpec return Sequences.simple(sorter.toTopN(materializedList, limit)); } } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + DefaultLimitSpec that = (DefaultLimitSpec) o; + + if (limit != that.limit) return false; + if (columns != null ? !columns.equals(that.columns) : that.columns != null) return false; + + return true; + } + + @Override + public int hashCode() + { + int result = columns != null ? columns.hashCode() : 0; + result = 31 * result + limit; + return result; + } } diff --git a/processing/src/main/java/io/druid/query/groupby/orderby/NoopLimitSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/NoopLimitSpec.java index 6fbc063c72d..d975e24a65f 100644 --- a/processing/src/main/java/io/druid/query/groupby/orderby/NoopLimitSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/NoopLimitSpec.java @@ -46,4 +46,15 @@ public class NoopLimitSpec implements LimitSpec { return "NoopLimitSpec"; } + + @Override + public boolean equals(Object other) + { + return (other instanceof NoopLimitSpec); + } + + @Override + public int hashCode() { + return 0; + } } diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index a5e1a1a8319..aff0d90d147 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -34,11 +34,7 @@ import com.metamx.common.guava.nary.BinaryFn; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.collections.OrderedMergeSequence; import io.druid.common.utils.JodaUtils; -import io.druid.query.CacheStrategy; -import io.druid.query.Query; -import io.druid.query.QueryRunner; -import io.druid.query.QueryToolChest; -import io.druid.query.ResultMergeQueryRunner; +import io.druid.query.*; import io.druid.query.aggregation.MetricManipulationFn; import io.druid.query.metadata.metadata.ColumnAnalysis; import io.druid.query.metadata.metadata.SegmentAnalysis; @@ -147,7 +143,7 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest @JsonProperty("context") Map context ) { - super(dataSource, querySegmentSpec, context); + super(new TableDataSource(dataSource), querySegmentSpec, context); this.toInclude = toInclude == null ? new AllColumnIncluderator() : toInclude; this.merge = merge == null ? false : merge; @@ -76,13 +77,41 @@ public class SegmentMetadataQuery extends BaseQuery public Query withOverriddenContext(Map contextOverride) { return new SegmentMetadataQuery( - getDataSource(), getQuerySegmentSpec(), toInclude, merge, computeOverridenContext(contextOverride) + ((TableDataSource)getDataSource()).getName(), + getQuerySegmentSpec(), toInclude, merge, computeOverridenContext(contextOverride) ); } @Override public Query withQuerySegmentSpec(QuerySegmentSpec spec) { - return new SegmentMetadataQuery(getDataSource(), spec, toInclude, merge, getContext()); + return new SegmentMetadataQuery( + ((TableDataSource)getDataSource()).getName(), + spec, toInclude, merge, getContext()); + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + SegmentMetadataQuery that = (SegmentMetadataQuery) o; + if (!partialEquals(that)) + return false; + + if (merge != that.merge) return false; + if (toInclude != null ? !toInclude.equals(that.toInclude) : that.toInclude != null) return false; + + return true; + } + + @Override + public int hashCode() + { + int result = toInclude != null ? toInclude.hashCode() : 0; + result = 31 * result + partialHashCode(); + result = 31 * result + (merge ? 1 : 0); + return result; } } diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java index 07f51a0f743..99e592a19f3 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -37,14 +37,7 @@ 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.query.CacheStrategy; -import io.druid.query.IntervalChunkingQueryRunner; -import io.druid.query.Query; -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.*; import io.druid.query.aggregation.MetricManipulationFn; import io.druid.query.filter.DimFilter; import io.druid.query.search.search.SearchHit; @@ -119,7 +112,7 @@ public class SearchQueryQueryToolChest extends QueryToolChest> @JsonCreator public SearchQuery( - @JsonProperty("dataSource") String dataSource, + @JsonProperty("dataSource") DataSource dataSource, @JsonProperty("filter") DimFilter dimFilter, @JsonProperty("granularity") QueryGranularity granularity, @JsonProperty("limit") int limit, @@ -190,4 +188,38 @@ public class SearchQuery extends BaseQuery> ", limit=" + limit + '}'; } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + SearchQuery that = (SearchQuery) o; + + if (!partialEquals(that)) + return false; + + if (limit != that.limit) return false; + if (dimFilter != null ? !dimFilter.equals(that.dimFilter) : that.dimFilter != null) return false; + if (dimensions != null ? !dimensions.equals(that.dimensions) : that.dimensions != null) return false; + if (granularity != null ? !granularity.equals(that.granularity) : that.granularity != null) return false; + if (querySpec != null ? !querySpec.equals(that.querySpec) : that.querySpec != null) return false; + if (sortSpec != null ? !sortSpec.equals(that.sortSpec) : that.sortSpec != null) return false; + + return true; + } + + @Override + public int hashCode() + { + int result = dimFilter != null ? dimFilter.hashCode() : 0; + result = 31 * result + partialHashCode(); + result = 31 * result + (sortSpec != null ? sortSpec.hashCode() : 0); + result = 31 * result + (granularity != null ? granularity.hashCode() : 0); + result = 31 * result + (dimensions != null ? dimensions.hashCode() : 0); + result = 31 * result + (querySpec != null ? querySpec.hashCode() : 0); + result = 31 * result + limit; + return result; + } } diff --git a/processing/src/main/java/io/druid/query/spec/MultipleIntervalSegmentSpec.java b/processing/src/main/java/io/druid/query/spec/MultipleIntervalSegmentSpec.java index 07d4436416c..f50aef4eaea 100644 --- a/processing/src/main/java/io/druid/query/spec/MultipleIntervalSegmentSpec.java +++ b/processing/src/main/java/io/druid/query/spec/MultipleIntervalSegmentSpec.java @@ -64,4 +64,23 @@ public class MultipleIntervalSegmentSpec implements QuerySegmentSpec "intervals=" + intervals + '}'; } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + MultipleIntervalSegmentSpec that = (MultipleIntervalSegmentSpec) o; + + if (intervals != null ? !intervals.equals(that.intervals) : that.intervals != null) return false; + + return true; + } + + @Override + public int hashCode() + { + return intervals != null ? intervals.hashCode() : 0; + } } diff --git a/processing/src/main/java/io/druid/query/spec/MultipleSpecificSegmentSpec.java b/processing/src/main/java/io/druid/query/spec/MultipleSpecificSegmentSpec.java index 59dda1e0d67..125e959e8da 100644 --- a/processing/src/main/java/io/druid/query/spec/MultipleSpecificSegmentSpec.java +++ b/processing/src/main/java/io/druid/query/spec/MultipleSpecificSegmentSpec.java @@ -91,4 +91,26 @@ public class MultipleSpecificSegmentSpec implements QuerySegmentSpec "descriptors=" + descriptors + '}'; } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + MultipleSpecificSegmentSpec that = (MultipleSpecificSegmentSpec) o; + + if (descriptors != null ? !descriptors.equals(that.descriptors) : that.descriptors != null) return false; + if (intervals != null ? !intervals.equals(that.intervals) : that.intervals != null) return false; + + return true; + } + + @Override + public int hashCode() + { + int result = descriptors != null ? descriptors.hashCode() : 0; + result = 31 * result + (intervals != null ? intervals.hashCode() : 0); + return result; + } } diff --git a/processing/src/main/java/io/druid/query/spec/SpecificSegmentSpec.java b/processing/src/main/java/io/druid/query/spec/SpecificSegmentSpec.java index be48d55e651..d83ef60b894 100644 --- a/processing/src/main/java/io/druid/query/spec/SpecificSegmentSpec.java +++ b/processing/src/main/java/io/druid/query/spec/SpecificSegmentSpec.java @@ -51,4 +51,23 @@ public class SpecificSegmentSpec implements QuerySegmentSpec { return walker.getQueryRunnerForSegments(query, Arrays.asList(descriptor)); } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + SpecificSegmentSpec that = (SpecificSegmentSpec) o; + + if (descriptor != null ? !descriptor.equals(that.descriptor) : that.descriptor != null) return false; + + return true; + } + + @Override + public int hashCode() + { + return descriptor != null ? descriptor.hashCode() : 0; + } } diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java index 957390acf15..ba1e9e9a237 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java @@ -24,9 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import io.druid.query.BaseQuery; -import io.druid.query.Query; -import io.druid.query.Result; +import io.druid.query.*; import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.DateTime; @@ -51,7 +49,7 @@ public class TimeBoundaryQuery extends BaseQuery @JsonCreator public TimeBoundaryQuery( - @JsonProperty("dataSource") String dataSource, + @JsonProperty("dataSource") DataSource dataSource, @JsonProperty("intervals") QuerySegmentSpec querySegmentSpec, @JsonProperty("context") Map context ) @@ -164,4 +162,17 @@ public class TimeBoundaryQuery extends BaseQuery ) ); } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + return partialEquals((TimeBoundaryQuery) o); + } + + @Override + public int hashCode() { + return partialHashCode(); + } } diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index ba2ab0feb9e..fd6c49c1aed 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -31,12 +31,7 @@ import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.collections.OrderedMergeSequence; -import io.druid.query.BySegmentSkippingQueryRunner; -import io.druid.query.CacheStrategy; -import io.druid.query.Query; -import io.druid.query.QueryRunner; -import io.druid.query.QueryToolChest; -import io.druid.query.Result; +import io.druid.query.*; import io.druid.query.aggregation.MetricManipulationFn; import io.druid.timeline.LogicalSegment; import org.joda.time.DateTime; @@ -117,7 +112,7 @@ public class TimeBoundaryQueryQueryToolChest public ServiceMetricEvent.Builder makeMetricBuilder(TimeBoundaryQuery query) { return new ServiceMetricEvent.Builder() - .setUser2(query.getDataSource()) + .setUser2(query.getDataSource().toString()) .setUser4(query.getType()) .setUser6("false"); } diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java index ab5b649a896..b86ec6f6469 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java @@ -24,10 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; 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.Query; -import io.druid.query.Result; +import io.druid.query.*; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.filter.DimFilter; @@ -48,7 +45,7 @@ public class TimeseriesQuery extends BaseQuery> @JsonCreator public TimeseriesQuery( - @JsonProperty("dataSource") String dataSource, + @JsonProperty("dataSource") DataSource dataSource, @JsonProperty("intervals") QuerySegmentSpec querySegmentSpec, @JsonProperty("filter") DimFilter dimFilter, @JsonProperty("granularity") QueryGranularity granularity, @@ -142,4 +139,34 @@ public class TimeseriesQuery extends BaseQuery> '}'; } + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + TimeseriesQuery that = (TimeseriesQuery) o; + if (!partialEquals(that)) + return false; + + if (aggregatorSpecs != null ? !aggregatorSpecs.equals(that.aggregatorSpecs) : that.aggregatorSpecs != null) + return false; + if (dimFilter != null ? !dimFilter.equals(that.dimFilter) : that.dimFilter != null) return false; + if (granularity != null ? !granularity.equals(that.granularity) : that.granularity != null) return false; + if (postAggregatorSpecs != null ? !postAggregatorSpecs.equals(that.postAggregatorSpecs) : that.postAggregatorSpecs != null) + return false; + + return true; + } + + @Override + public int hashCode() + { + int result = dimFilter != null ? dimFilter.hashCode() : 0; + result = 31 * result + partialHashCode(); + result = 31 * result + (granularity != null ? granularity.hashCode() : 0); + result = 31 * result + (aggregatorSpecs != null ? aggregatorSpecs.hashCode() : 0); + result = 31 * result + (postAggregatorSpecs != null ? postAggregatorSpecs.hashCode() : 0); + return result; + } } diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryEngine.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryEngine.java index b57105aa654..c6d9d31d7e4 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryEngine.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryEngine.java @@ -22,6 +22,8 @@ package io.druid.query.timeseries; import com.google.common.base.Function; import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.Sequence; +import io.druid.query.DataSource; +import io.druid.query.Query; import io.druid.query.QueryRunnerHelper; import io.druid.query.Result; import io.druid.query.aggregation.Aggregator; diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index c328822e754..7a11d42a325 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -32,16 +32,7 @@ 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.QueryConfig; -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.*; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.MetricManipulationFn; import io.druid.query.aggregation.PostAggregator; @@ -122,7 +113,7 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest> @JsonCreator public TopNQuery( - @JsonProperty("dataSource") String dataSource, + @JsonProperty("dataSource") DataSource dataSource, @JsonProperty("dimension") DimensionSpec dimensionSpec, @JsonProperty("metric") TopNMetricSpec topNMetricSpec, @JsonProperty("threshold") int threshold, @@ -208,4 +206,43 @@ public class TopNQuery extends BaseQuery> ", postAggregatorSpecs=" + postAggregatorSpecs + '}'; } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + TopNQuery topNQuery = (TopNQuery) o; + if (!partialEquals(topNQuery)) + return false; + + if (threshold != topNQuery.threshold) return false; + if (aggregatorSpecs != null ? !aggregatorSpecs.equals(topNQuery.aggregatorSpecs) : topNQuery.aggregatorSpecs != null) + return false; + if (dimFilter != null ? !dimFilter.equals(topNQuery.dimFilter) : topNQuery.dimFilter != null) return false; + if (dimensionSpec != null ? !dimensionSpec.equals(topNQuery.dimensionSpec) : topNQuery.dimensionSpec != null) + return false; + if (granularity != null ? !granularity.equals(topNQuery.granularity) : topNQuery.granularity != null) return false; + if (postAggregatorSpecs != null ? !postAggregatorSpecs.equals(topNQuery.postAggregatorSpecs) : topNQuery.postAggregatorSpecs != null) + return false; + if (topNMetricSpec != null ? !topNMetricSpec.equals(topNQuery.topNMetricSpec) : topNQuery.topNMetricSpec != null) + return false; + + return true; + } + + @Override + public int hashCode() + { + int result = dimensionSpec != null ? dimensionSpec.hashCode() : 0; + result = 31 * result + partialHashCode(); + result = 31 * result + (topNMetricSpec != null ? topNMetricSpec.hashCode() : 0); + result = 31 * result + threshold; + result = 31 * result + (dimFilter != null ? dimFilter.hashCode() : 0); + result = 31 * result + (granularity != null ? granularity.hashCode() : 0); + result = 31 * result + (aggregatorSpecs != null ? aggregatorSpecs.hashCode() : 0); + result = 31 * result + (postAggregatorSpecs != null ? postAggregatorSpecs.hashCode() : 0); + return result; + } } diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java index 1bfb690f490..21efd3b8351 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java @@ -21,6 +21,8 @@ package io.druid.query.topn; import com.google.common.collect.Lists; import io.druid.granularity.QueryGranularity; +import io.druid.query.DataSource; +import io.druid.query.TableDataSource; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DefaultDimensionSpec; @@ -58,7 +60,7 @@ import java.util.Map; */ public class TopNQueryBuilder { - private String dataSource; + private DataSource dataSource; private DimensionSpec dimensionSpec; private TopNMetricSpec topNMetricSpec; private int threshold; @@ -71,7 +73,7 @@ public class TopNQueryBuilder public TopNQueryBuilder() { - dataSource = ""; + dataSource = null; dimensionSpec = null; topNMetricSpec = null; threshold = 0; @@ -83,7 +85,7 @@ public class TopNQueryBuilder context = null; } - public String getDataSource() + public DataSource getDataSource() { return dataSource; } @@ -152,7 +154,7 @@ public class TopNQueryBuilder public TopNQueryBuilder copy(TopNQuery query) { return new TopNQueryBuilder() - .dataSource(query.getDataSource()) + .dataSource(query.getDataSource().toString()) .dimension(query.getDimensionSpec()) .metric(query.getTopNMetricSpec()) .threshold(query.getThreshold()) @@ -180,6 +182,12 @@ public class TopNQueryBuilder } public TopNQueryBuilder dataSource(String d) + { + dataSource = new TableDataSource(d); + return this; + } + + public TopNQueryBuilder dataSource(DataSource d) { dataSource = d; return this; diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java index 6e1d816cbd9..6f0400a1f1b 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java @@ -128,7 +128,7 @@ public class TopNQueryQueryToolChest extends QueryToolChestnewArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + String dataSourceJSON = "{\"type\":\"query\", \"query\":" + jsonMapper.writeValueAsString(query) + "}"; + + DataSource dataSource = jsonMapper.readValue(dataSourceJSON, DataSource.class); + Assert.assertEquals(new QueryDataSource(query), dataSource); + } + +} diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index 58c1e97034c..91243d22679 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -55,6 +55,7 @@ public class QueryRunnerTestHelper public static final String qualityDimension = "quality"; public static final String placementishDimension = "placementish"; public static final String indexMetric = "index"; + public static final String addRowsIndexConstantMetric = "addRowsIndexConstant"; public static final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows"); public static final LongSumAggregatorFactory indexLongSum = new LongSumAggregatorFactory("index", "index"); public static final DoubleSumAggregatorFactory indexDoubleSum = new DoubleSumAggregatorFactory("index", "index"); @@ -94,6 +95,9 @@ public class QueryRunnerTestHelper public static final QuerySegmentSpec firstToThird = new MultipleIntervalSegmentSpec( Arrays.asList(new Interval("2011-04-01T00:00:00.000Z/2011-04-03T00:00:00.000Z")) ); + public static final QuerySegmentSpec secondOnly = new MultipleIntervalSegmentSpec( + Arrays.asList(new Interval("2011-04-02T00:00:00.000Z/P1D")) + ); public static final QuerySegmentSpec fullOnInterval = new MultipleIntervalSegmentSpec( Arrays.asList(new Interval("1970-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z")) ); diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index e1556e94da7..102bf24f3d3 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -38,9 +38,11 @@ import io.druid.granularity.QueryGranularity; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.QueryToolChest; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.aggregation.MaxAggregatorFactory; import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DimensionSpec; import io.druid.query.filter.RegexDimFilter; @@ -90,22 +92,24 @@ public class GroupByQueryRunnerTest config.setMaxIntermediateRows(10000); final Supplier configSupplier = Suppliers.ofInstance(config); - final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( - new GroupByQueryEngine( - configSupplier, - new StupidPool( - new Supplier() - { - @Override - public ByteBuffer get() - { - return ByteBuffer.allocate(1024 * 1024); - } - } - ) - ), + final GroupByQueryEngine engine = new GroupByQueryEngine( configSupplier, - new GroupByQueryQueryToolChest(configSupplier) + new StupidPool( + new Supplier() + { + @Override + public ByteBuffer get() + { + return ByteBuffer.allocate(1024 * 1024); + } + } + ) + ); + + final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( + engine, + configSupplier, + new GroupByQueryQueryToolChest(configSupplier, engine) ); return Lists.newArrayList( @@ -167,8 +171,7 @@ public class GroupByQueryRunnerTest createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) ); - Iterable results = Sequences.toList(runner.run(query), Lists.newArrayList()); - + Iterable results = runQuery(query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -178,33 +181,33 @@ public class GroupByQueryRunnerTest DateTimeZone tz = DateTimeZone.forID("America/Los_Angeles"); GroupByQuery query = GroupByQuery.builder() - .setDataSource(QueryRunnerTestHelper.dataSource) - .setInterval("2011-03-31T00:00:00-07:00/2011-04-02T00:00:00-07:00") - .setDimensions( - Lists.newArrayList( - (DimensionSpec) new DefaultDimensionSpec( - "quality", - "alias" - ) - ) - ) - .setAggregatorSpecs( - Arrays.asList( - QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory( - "idx", - "index" - ) - ) - ) - .setGranularity( - new PeriodGranularity( - new Period("P1D"), - null, - tz - ) - ) - .build(); + .setDataSource(QueryRunnerTestHelper.dataSource) + .setInterval("2011-03-31T00:00:00-07:00/2011-04-02T00:00:00-07:00") + .setDimensions( + Lists.newArrayList( + (DimensionSpec) new DefaultDimensionSpec( + "quality", + "alias" + ) + ) + ) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory( + "idx", + "index" + ) + ) + ) + .setGranularity( + new PeriodGranularity( + new Period("P1D"), + null, + tz + ) + ) + .build(); List expectedResults = Arrays.asList( createExpectedRow(new DateTime("2011-03-31", tz), "alias", "automotive", "rows", 1L, "idx", 135L), @@ -228,11 +231,7 @@ public class GroupByQueryRunnerTest createExpectedRow(new DateTime("2011-04-01", tz), "alias", "travel", "rows", 1L, "idx", 126L) ); - Iterable results = Sequences.toList( - runner.run(query), - Lists.newArrayList() - ); - + Iterable results = runQuery(query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } @@ -661,7 +660,21 @@ public class GroupByQueryRunnerTest createExpectedRow("2011-04-01", "quality", "automotive", "rows", 2L) ); - QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier).mergeResults(runner); + final GroupByQueryEngine engine = new GroupByQueryEngine( + configSupplier, + new StupidPool( + new Supplier() + { + @Override + public ByteBuffer get() + { + return ByteBuffer.allocate(1024 * 1024); + } + } + ) + ); + + QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine).mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); } @@ -696,7 +709,21 @@ public class GroupByQueryRunnerTest ); TestHelper.assertExpectedObjects(expectedResults, runner.run(query), "normal"); - QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier).mergeResults(runner); + final GroupByQueryEngine engine = new GroupByQueryEngine( + configSupplier, + new StupidPool( + new Supplier() + { + @Override + public ByteBuffer get() + { + return ByteBuffer.allocate(1024 * 1024); + } + } + ) + ); + + QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine).mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); } @@ -731,10 +758,166 @@ public class GroupByQueryRunnerTest ); TestHelper.assertExpectedObjects(expectedResults, runner.run(query), "normal"); - QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier).mergeResults(runner); + final GroupByQueryEngine engine = new GroupByQueryEngine( + configSupplier, + new StupidPool( + new Supplier() + { + @Override + public ByteBuffer get() + { + return ByteBuffer.allocate(1024 * 1024); + } + } + ) + ); + + QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine).mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); } + // A subquery identical to the query should yield identical results + @Test + public void testIdenticalSubquery() + { + GroupByQuery subquery = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(subquery) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias"))) + .setAggregatorSpecs( + Arrays.asList( + new LongSumAggregatorFactory("rows", "rows"), + new LongSumAggregatorFactory("idx", "idx") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + List expectedResults = Arrays.asList( + createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), + createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), + + createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), + createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), + createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), + createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), + createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), + createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), + createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), + createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), + createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) + ); + + // Subqueries are handled by the ToolChest + Iterable results = runQuery(query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testDifferentGroupingSubquery() + { + GroupByQuery subquery = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(subquery) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setAggregatorSpecs( + Arrays.asList( + new MaxAggregatorFactory("idx", "idx") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + List expectedResults = Arrays.asList( + createExpectedRow("2011-04-01", "idx", 2900.0), + createExpectedRow("2011-04-02", "idx", 2505.0) + ); + + Iterable results = runQuery(query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + @Test + public void testDifferentIntervalSubquery() + { + GroupByQuery subquery = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(subquery) + .setQuerySegmentSpec(QueryRunnerTestHelper.secondOnly) + .setAggregatorSpecs( + Arrays.asList( + new MaxAggregatorFactory("idx", "idx") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + List expectedResults = Arrays.asList( + createExpectedRow("2011-04-02", "idx", 2505.0) + ); + + Iterable results = runQuery(query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + private Iterable runQuery(GroupByQuery query) + { + QueryToolChest toolChest = factory.getToolchest(); + Sequence queryResult = toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner)).run(query); + return Sequences.toList(queryResult, Lists.newArrayList()); + } + + private Row createExpectedRow(final String timestamp, Object... vals) { return createExpectedRow(new DateTime(timestamp), vals); diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java new file mode 100644 index 00000000000..8557ba04c24 --- /dev/null +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java @@ -0,0 +1,68 @@ +/* + * Druid - a distributed column store. + * + * 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. + * + * This file Copyright (C) 2014 N3TWORK, Inc. and contributed to the Druid project + * under the Druid Corporate Contributor License Agreement. + */ + +package io.druid.query.groupby; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.Druids; +import io.druid.query.Query; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.dimension.DefaultDimensionSpec; +import io.druid.query.dimension.DimensionSpec; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Arrays; + +public class GroupByQueryTest +{ + private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); + + @Test + public void testQuerySerialization() throws IOException + { + Query query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + String json = jsonMapper.writeValueAsString(query); + Query serdeQuery = jsonMapper.readValue(json, Query.class); + + Assert.assertEquals(query, serdeQuery); + } + +} diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java index 562527556a9..2538e91bc76 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java @@ -56,22 +56,24 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest config.setMaxIntermediateRows(10000); final Supplier configSupplier = Suppliers.ofInstance(config); - final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( - new GroupByQueryEngine( - configSupplier, - new StupidPool( - new Supplier() - { - @Override - public ByteBuffer get() - { - return ByteBuffer.allocate(1024 * 1024); - } - } - ) - ), + final GroupByQueryEngine engine = new GroupByQueryEngine( configSupplier, - new GroupByQueryQueryToolChest(configSupplier) + new StupidPool( + new Supplier() + { + @Override + public ByteBuffer get() + { + return ByteBuffer.allocate(1024 * 1024); + } + } + ) + ); + + final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( + engine, + configSupplier, + new GroupByQueryQueryToolChest(configSupplier, engine) ); final Collection objects = QueryRunnerTestHelper.makeQueryRunners(factory); @@ -95,13 +97,13 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest return Sequences.map( groupByRunner.run( GroupByQuery.builder() - .setDataSource(tsQuery.getDataSource()) - .setQuerySegmentSpec(tsQuery.getQuerySegmentSpec()) - .setGranularity(tsQuery.getGranularity()) - .setDimFilter(tsQuery.getDimensionsFilter()) - .setAggregatorSpecs(tsQuery.getAggregatorSpecs()) - .setPostAggregatorSpecs(tsQuery.getPostAggregatorSpecs()) - .build() + .setDataSource(tsQuery.getDataSource()) + .setQuerySegmentSpec(tsQuery.getQuerySegmentSpec()) + .setGranularity(tsQuery.getGranularity()) + .setDimFilter(tsQuery.getDimensionsFilter()) + .setAggregatorSpecs(tsQuery.getAggregatorSpecs()) + .setPostAggregatorSpecs(tsQuery.getPostAggregatorSpecs()) + .build() ), new Function>() { diff --git a/processing/src/test/java/io/druid/query/search/SearchQueryTest.java b/processing/src/test/java/io/druid/query/search/SearchQueryTest.java new file mode 100644 index 00000000000..7a5695a6ab6 --- /dev/null +++ b/processing/src/test/java/io/druid/query/search/SearchQueryTest.java @@ -0,0 +1,62 @@ +/* + * Druid - a distributed column store. + * + * 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. + * + * This file Copyright (C) 2014 N3TWORK, Inc. and contributed to the Druid project + * under the Druid Corporate Contributor License Agreement. + */ + +package io.druid.query.search; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.Druids; +import io.druid.query.Query; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.dimension.DefaultDimensionSpec; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.groupby.GroupByQuery; +import io.druid.query.search.search.SearchQuery; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Arrays; + +public class SearchQueryTest +{ + private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); + + @Test + public void testQuerySerialization() throws IOException + { + Query query = Druids.newSearchQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .query("a") + .build(); + + String json = jsonMapper.writeValueAsString(query); + Query serdeQuery = jsonMapper.readValue(json, Query.class); + + Assert.assertEquals(query, serdeQuery); + } + +} diff --git a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryTest.java b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryTest.java new file mode 100644 index 00000000000..1f7fb96a2b8 --- /dev/null +++ b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryTest.java @@ -0,0 +1,51 @@ +/* + * Druid - a distributed column store. + * + * 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. + * + * This file Copyright (C) 2014 N3TWORK, Inc. and contributed to the Druid project + * under the Druid Corporate Contributor License Agreement. + */ + +package io.druid.query.timeboundary; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.Druids; +import io.druid.query.Query; +import io.druid.query.QueryRunnerTestHelper; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; + +public class TimeBoundaryQueryTest +{ + private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); + + @Test + public void testQuerySerialization() throws IOException + { + Query query = Druids.newTimeBoundaryQueryBuilder() + .dataSource("testing") + .build(); + + String json = jsonMapper.writeValueAsString(query); + Query serdeQuery = jsonMapper.readValue(json, Query.class); + + Assert.assertEquals(query, serdeQuery); + } + +} diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryTest.java new file mode 100644 index 00000000000..43e22ddfb94 --- /dev/null +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryTest.java @@ -0,0 +1,62 @@ +/* + * Druid - a distributed column store. + * + * 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. + * + * This file Copyright (C) 2014 N3TWORK, Inc. and contributed to the Druid project + * under the Druid Corporate Contributor License Agreement. + */ + +package io.druid.query.timeseries; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.Druids; +import io.druid.query.Query; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.aggregation.PostAggregator; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Arrays; + +public class TimeseriesQueryTest +{ + private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); + + @Test + public void testQuerySerialization() throws IOException + { + Query query = Druids.newTimeseriesQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.dayGran) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + QueryRunnerTestHelper.indexDoubleSum + ) + ) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + String json = jsonMapper.writeValueAsString(query); + Query serdeQuery = jsonMapper.readValue(json, Query.class); + + Assert.assertEquals(query, serdeQuery); + } + +} diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java index 839f82c3cbc..af0988fc100 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java @@ -61,6 +61,8 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import static io.druid.query.QueryRunnerTestHelper.*; + /** */ @RunWith(Parameterized.class) @@ -108,58 +110,6 @@ public class TopNQueryRunnerTest 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() @@ -191,7 +141,7 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "total_market") + .put(providerDimension, "total_market") .put("rows", 186L) .put("index", 215679.82879638672D) .put("addRowsIndexConstant", 215866.82879638672D) @@ -199,7 +149,7 @@ public class TopNQueryRunnerTest .put("minIndex", 792.3260498046875D) .build(), ImmutableMap.builder() - .put("provider", "upfront") + .put(providerDimension, "upfront") .put("rows", 186L) .put("index", 192046.1060180664D) .put("addRowsIndexConstant", 192233.1060180664D) @@ -207,7 +157,7 @@ public class TopNQueryRunnerTest .put("minIndex", 545.9906005859375D) .build(), ImmutableMap.builder() - .put("provider", "spot") + .put(providerDimension, "spot") .put("rows", 837L) .put("index", 95606.57232284546D) .put("addRowsIndexConstant", 96444.57232284546D) @@ -252,7 +202,7 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put("provider", "total_market") + .put(providerDimension, "total_market") .put("rows", 186L) .put("index", 215679.82879638672D) .put("addRowsIndexConstant", 215866.82879638672D) @@ -260,7 +210,7 @@ public class TopNQueryRunnerTest .put("minIndex", 792.3260498046875D) .build(), ImmutableMap.builder() - .put("provider", "upfront") + .put(providerDimension, "upfront") .put("rows", 186L) .put("index", 192046.1060180664D) .put("addRowsIndexConstant", 192233.1060180664D) @@ -268,7 +218,7 @@ public class TopNQueryRunnerTest .put("minIndex", 545.9906005859375D) .build(), ImmutableMap.builder() - .put("provider", "spot") + .put(providerDimension, "spot") .put("rows", 837L) .put("index", 95606.57232284546D) .put("addRowsIndexConstant", 96444.57232284546D) @@ -304,19 +254,19 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - "provider", "total_market", + providerDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D ), ImmutableMap.of( - "provider", "upfront", + providerDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D ), ImmutableMap.of( - "provider", "spot", + providerDimension, "spot", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D @@ -350,19 +300,19 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - "provider", "total_market", + providerDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D ), ImmutableMap.of( - "provider", "upfront", + providerDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D ), ImmutableMap.of( - "provider", "spot", + providerDimension, "spot", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D @@ -396,13 +346,13 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - "provider", "total_market", + providerDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D ), ImmutableMap.of( - "provider", "upfront", + providerDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D @@ -436,7 +386,7 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - "provider", "upfront", + providerDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D @@ -470,19 +420,19 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - "provider", "upfront", + providerDimension, "upfront", "rows", 2L, "index", 2591.68359375D, "addRowsIndexConstant", 2594.68359375D ), ImmutableMap.of( - "provider", "total_market", + providerDimension, "total_market", "rows", 2L, "index", 2508.39599609375D, "addRowsIndexConstant", 2511.39599609375D ), ImmutableMap.of( - "provider", "spot", + providerDimension, "spot", "rows", 2L, "index", 220.63774871826172D, "addRowsIndexConstant", 223.63774871826172D @@ -520,19 +470,19 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - "provider", "upfront", + providerDimension, "upfront", "rows", 1L, "index", new Float(1447.341160).doubleValue(), "addRowsIndexConstant", new Float(1449.341160).doubleValue() ), ImmutableMap.of( - "provider", "total_market", + providerDimension, "total_market", "rows", 1L, "index", new Float(1314.839715).doubleValue(), "addRowsIndexConstant", new Float(1316.839715).doubleValue() ), ImmutableMap.of( - "provider", "spot", + providerDimension, "spot", "rows", 1L, "index", new Float(109.705815).doubleValue(), "addRowsIndexConstant", new Float(111.705815).doubleValue() @@ -566,13 +516,13 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - "provider", "total_market", + providerDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D ), ImmutableMap.of( - "provider", "upfront", + providerDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D @@ -877,19 +827,19 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - "provider", "spot", + providerDimension, "spot", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D ), ImmutableMap.of( - "provider", "total_market", + providerDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D ), ImmutableMap.of( - "provider", "upfront", + providerDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D @@ -922,13 +872,13 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - "provider", "total_market", + providerDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D ), ImmutableMap.of( - "provider", "upfront", + providerDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D @@ -961,13 +911,13 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - "provider", "total_market", + providerDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D ), ImmutableMap.of( - "provider", "upfront", + providerDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D @@ -1004,19 +954,19 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - "provider", "s", + providerDimension, "s", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D ), ImmutableMap.of( - "provider", "t", + providerDimension, "t", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D ), ImmutableMap.of( - "provider", "u", + providerDimension, "u", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D @@ -1050,19 +1000,19 @@ public class TopNQueryRunnerTest new TopNResultValue( Arrays.>asList( ImmutableMap.of( - "provider", "spot", + providerDimension, "spot", "rows", 18L, "index", 2231.8768157958984D, "addRowsIndexConstant", 2250.8768157958984D ), ImmutableMap.of( - "provider", "upfront", + providerDimension, "upfront", "rows", 4L, "index", 4875.669677734375D, "addRowsIndexConstant", 4880.669677734375D ), ImmutableMap.of( - "provider", "total_market", + providerDimension, "total_market", "rows", 4L, "index", 5351.814697265625D, "addRowsIndexConstant", 5356.814697265625D diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java new file mode 100644 index 00000000000..61c082039f4 --- /dev/null +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java @@ -0,0 +1,75 @@ +/* + * Druid - a distributed column store. + * + * 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. + * + * This file Copyright (C) 2014 N3TWORK, Inc. and contributed to the Druid project + * under the Druid Corporate Contributor License Agreement. + */ + +package io.druid.query.topn; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.Query; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.MaxAggregatorFactory; +import io.druid.query.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Arrays; + +import static io.druid.query.QueryRunnerTestHelper.*; + +public class TopNQueryTest +{ + private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); + + @Test + public void testQuerySerialization() throws IOException + { + Query 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(); + + String json = jsonMapper.writeValueAsString(query); + Query serdeQuery = jsonMapper.readValue(json, Query.class); + + Assert.assertEquals(query, serdeQuery); + } + +} diff --git a/server/src/main/java/io/druid/client/BrokerServerView.java b/server/src/main/java/io/druid/client/BrokerServerView.java index 3e860a4f23c..4ef7a9f76fc 100644 --- a/server/src/main/java/io/druid/client/BrokerServerView.java +++ b/server/src/main/java/io/druid/client/BrokerServerView.java @@ -30,8 +30,7 @@ import io.druid.client.selector.QueryableDruidServer; import io.druid.client.selector.ServerSelectorStrategy; import io.druid.concurrent.Execs; import io.druid.guice.annotations.Client; -import io.druid.query.QueryRunner; -import io.druid.query.QueryToolChestWarehouse; +import io.druid.query.*; import io.druid.timeline.DataSegment; import io.druid.timeline.VersionedIntervalTimeline; import io.druid.timeline.partition.PartitionChunk; @@ -232,10 +231,22 @@ public class BrokerServerView implements TimelineServerView @Override - public VersionedIntervalTimeline getTimeline(String dataSource) + public VersionedIntervalTimeline getTimeline(DataSource dataSource) { + String table; + while (dataSource instanceof QueryDataSource) { + dataSource = ((QueryDataSource)dataSource).getQuery().getDataSource(); + } + + if (dataSource instanceof TableDataSource) { + table = ((TableDataSource)dataSource).getName(); + } + else { + throw new UnsupportedOperationException("Unsupported data source type: " + dataSource.getClass().getSimpleName()); + } + synchronized (lock) { - return timelines.get(dataSource); + return timelines.get(table); } } diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 31a644cf887..ed30af51a69 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -140,6 +140,7 @@ public class CachingClusteredClient implements QueryRunner final Query rewrittenQuery = query.withOverriddenContext(contextBuilder.build()); + VersionedIntervalTimeline timeline = serverView.getTimeline(query.getDataSource()); if (timeline == null) { return Sequences.empty(); diff --git a/server/src/main/java/io/druid/client/TimelineServerView.java b/server/src/main/java/io/druid/client/TimelineServerView.java index 7082c599c75..0a6a43c8fdb 100644 --- a/server/src/main/java/io/druid/client/TimelineServerView.java +++ b/server/src/main/java/io/druid/client/TimelineServerView.java @@ -20,6 +20,7 @@ package io.druid.client; import io.druid.client.selector.ServerSelector; +import io.druid.query.DataSource; import io.druid.query.QueryRunner; import io.druid.timeline.VersionedIntervalTimeline; @@ -27,6 +28,6 @@ import io.druid.timeline.VersionedIntervalTimeline; */ public interface TimelineServerView extends ServerView { - VersionedIntervalTimeline getTimeline(String dataSource); + VersionedIntervalTimeline getTimeline(DataSource dataSource); QueryRunner getQueryRunner(DruidServer server); } diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 4c490271e95..0fb92b7d0ce 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -124,7 +124,7 @@ public class QueryResource emitter.emit( new ServiceMetricEvent.Builder() - .setUser2(query.getDataSource()) + .setUser2(query.getDataSource().toString()) .setUser4(query.getType()) .setUser5(query.getIntervals().get(0).toString()) .setUser6(String.valueOf(query.hasFilters())) diff --git a/server/src/main/java/io/druid/server/coordination/ServerManager.java b/server/src/main/java/io/druid/server/coordination/ServerManager.java index 950be651e86..f9633e30e0b 100644 --- a/server/src/main/java/io/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/io/druid/server/coordination/ServerManager.java @@ -30,18 +30,7 @@ import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.collections.CountingMap; import io.druid.guice.annotations.Processing; -import io.druid.query.BySegmentQueryRunner; -import io.druid.query.FinalizeResultsQueryRunner; -import io.druid.query.MetricsEmittingQueryRunner; -import io.druid.query.NoopQueryRunner; -import io.druid.query.Query; -import io.druid.query.QueryRunner; -import io.druid.query.QueryRunnerFactory; -import io.druid.query.QueryRunnerFactoryConglomerate; -import io.druid.query.QuerySegmentWalker; -import io.druid.query.QueryToolChest; -import io.druid.query.ReferenceCountingSegmentQueryRunner; -import io.druid.query.SegmentDescriptor; +import io.druid.query.*; import io.druid.query.spec.QuerySegmentSpec; import io.druid.query.spec.SpecificSegmentQueryRunner; import io.druid.query.spec.SpecificSegmentSpec; @@ -118,6 +107,7 @@ public class ServerManager implements QuerySegmentWalker /** * Load a single segment. + * * @param segment segment to load * @return true if the segment was newly loaded, false if it was already loaded * @throws SegmentLoadingException if the segment cannot be loaded @@ -127,12 +117,10 @@ public class ServerManager implements QuerySegmentWalker final Segment adapter; try { adapter = segmentLoader.getSegment(segment); - } - catch (SegmentLoadingException e) { + } catch (SegmentLoadingException e) { try { segmentLoader.cleanup(segment); - } - catch (SegmentLoadingException e1) { + } catch (SegmentLoadingException e1) { // ignore } throw e; @@ -204,12 +192,11 @@ public class ServerManager implements QuerySegmentWalker try { log.info("Attempting to close segment %s", segment.getIdentifier()); oldQueryable.close(); - } - catch (IOException e) { + } catch (IOException e) { log.makeAlert(e, "Exception closing segment") - .addData("dataSource", dataSource) - .addData("segmentId", segment.getIdentifier()) - .emit(); + .addData("dataSource", dataSource) + .addData("segmentId", segment.getIdentifier()) + .emit(); } } else { log.info( @@ -233,7 +220,20 @@ public class ServerManager implements QuerySegmentWalker final QueryToolChest> toolChest = factory.getToolchest(); - final VersionedIntervalTimeline timeline = dataSources.get(query.getDataSource()); + DataSource dataSource = query.getDataSource(); + if (!(dataSource instanceof TableDataSource)) { + throw new UnsupportedOperationException("data source type '" + dataSource.getClass().getName() + "' unsupported"); + } + + String dataSourceName; + try { + dataSourceName = ((TableDataSource)query.getDataSource()).getName(); + } + catch (ClassCastException e) { + throw new UnsupportedOperationException("Subqueries are only supported in the broker"); + } + + final VersionedIntervalTimeline timeline = dataSources.get(dataSourceName); if (timeline == null) { return new NoopQueryRunner(); @@ -294,6 +294,7 @@ public class ServerManager implements QuerySegmentWalker Predicates.>notNull() ); + return new FinalizeResultsQueryRunner(toolChest.mergeResults(factory.mergeRunners(exec, adapters)), toolChest); } @@ -303,14 +304,22 @@ public class ServerManager implements QuerySegmentWalker final QueryRunnerFactory> factory = conglomerate.findFactory(query); if (factory == null) { log.makeAlert("Unknown query type, [%s]", query.getClass()) - .addData("dataSource", query.getDataSource()) - .emit(); + .addData("dataSource", query.getDataSource()) + .emit(); return new NoopQueryRunner(); } final QueryToolChest> toolChest = factory.getToolchest(); - final VersionedIntervalTimeline timeline = dataSources.get(query.getDataSource()); + String dataSourceName; + try { + dataSourceName = ((TableDataSource)query.getDataSource()).getName(); + } + catch (ClassCastException e) { + throw new UnsupportedOperationException("Subqueries are only supported in the broker"); + } + + final VersionedIntervalTimeline timeline = dataSources.get(dataSourceName); if (timeline == null) { return new NoopQueryRunner(); From f2c1f798b9858cb96b3094529a8ec479e19dd01b Mon Sep 17 00:00:00 2001 From: Yuval Oren Date: Fri, 24 Jan 2014 15:03:48 -0800 Subject: [PATCH 02/66] Fix subquery merge issues --- .../src/main/java/io/druid/query/select/SelectQuery.java | 3 ++- .../io/druid/query/select/SelectQueryQueryToolChest.java | 2 +- .../io/druid/query/select/SelectQueryRunnerTest.java | 9 +++++---- .../segment/realtime/plumber/RealtimePlumberSchool.java | 2 +- 4 files changed, 9 insertions(+), 7 deletions(-) diff --git a/processing/src/main/java/io/druid/query/select/SelectQuery.java b/processing/src/main/java/io/druid/query/select/SelectQuery.java index 8c5eb2ba59f..c56d279cf8f 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQuery.java +++ b/processing/src/main/java/io/druid/query/select/SelectQuery.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import io.druid.granularity.QueryGranularity; import io.druid.query.BaseQuery; +import io.druid.query.DataSource; import io.druid.query.Query; import io.druid.query.Result; import io.druid.query.filter.DimFilter; @@ -45,7 +46,7 @@ public class SelectQuery extends BaseQuery> @JsonCreator public SelectQuery( - @JsonProperty("dataSource") String dataSource, + @JsonProperty("dataSource") DataSource dataSource, @JsonProperty("intervals") QuerySegmentSpec querySegmentSpec, @JsonProperty("filter") DimFilter dimFilter, @JsonProperty("granularity") QueryGranularity granularity, diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java index 9e5a365479c..2d64c2f1451 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java @@ -126,7 +126,7 @@ public class SelectQueryQueryToolChest extends QueryToolChest 0) ? maxPendingPersists : defaultPending; - Preconditions.checkArgument(maxPendingPersists <= 0, "RealtimePlumberSchool requires maxPendingPersists > 0"); + Preconditions.checkArgument(maxPendingPersists > 0, "RealtimePlumberSchool requires maxPendingPersists > 0"); Preconditions.checkNotNull(windowPeriod, "RealtimePlumberSchool requires a windowPeriod."); Preconditions.checkNotNull(basePersistDirectory, "RealtimePlumberSchool requires a basePersistDirectory."); Preconditions.checkNotNull(segmentGranularity, "RealtimePlumberSchool requires a segmentGranularity."); From c88cb2af16db60ae9cbc0d89929bfc4610da5fce Mon Sep 17 00:00:00 2001 From: Yuval Oren Date: Fri, 24 Jan 2014 15:58:47 -0800 Subject: [PATCH 03/66] Fix NPE when subquery returns empty result --- .../IncrementalIndexStorageAdapter.java | 5 +++ .../io/druid/query/QueryRunnerTestHelper.java | 3 ++ .../query/groupby/GroupByQueryRunnerTest.java | 35 +++++++++++++++++++ 3 files changed, 43 insertions(+) diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java index d0243f39123..8843879f91b 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -126,8 +126,13 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter @Override public Iterable makeCursors(final Filter filter, final Interval interval, final QueryGranularity gran) { + if (index.isEmpty()) { + return ImmutableList.of(); + } + Interval actualIntervalTmp = interval; + final Interval dataInterval = new Interval(getMinTime().getMillis(), gran.next(getMaxTime().getMillis())); if (!actualIntervalTmp.overlaps(dataInterval)) { return ImmutableList.of(); diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index 6243134ecf1..7434b7301f3 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -103,6 +103,9 @@ public class QueryRunnerTestHelper public static final QuerySegmentSpec fullOnInterval = new MultipleIntervalSegmentSpec( Arrays.asList(new Interval("1970-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z")) ); + public static final QuerySegmentSpec emptyInterval = new MultipleIntervalSegmentSpec( + Arrays.asList(new Interval("2020-04-02T00:00:00.000Z/P1D")) + ); @SuppressWarnings("unchecked") public static Collection makeQueryRunners( diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 102bf24f3d3..4fffcf67c01 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -58,7 +58,9 @@ import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Interval; import org.joda.time.Period; +import org.junit.Assert; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -910,6 +912,39 @@ public class GroupByQueryRunnerTest TestHelper.assertExpectedObjects(expectedResults, results, ""); } + @Test + public void testEmptySubquery() + { + GroupByQuery subquery = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.emptyInterval) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(subquery) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setAggregatorSpecs( + Arrays.asList( + new MaxAggregatorFactory("idx", "idx") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + Iterable results = runQuery(query); + Assert.assertFalse(results.iterator().hasNext()); + } + private Iterable runQuery(GroupByQuery query) { QueryToolChest toolChest = factory.getToolchest(); From 8f39981fab10bc796a5b53d9d6dca3a1fafcf1a9 Mon Sep 17 00:00:00 2001 From: Yuval Oren Date: Fri, 24 Jan 2014 16:12:10 -0800 Subject: [PATCH 04/66] Documentation for query data sources --- docs/content/DataSource.md | 25 +++++++++++++++++++++++++ docs/content/GroupByQuery.md | 2 +- 2 files changed, 26 insertions(+), 1 deletion(-) create mode 100644 docs/content/DataSource.md diff --git a/docs/content/DataSource.md b/docs/content/DataSource.md new file mode 100644 index 00000000000..49c583561ba --- /dev/null +++ b/docs/content/DataSource.md @@ -0,0 +1,25 @@ +--- +layout: doc_page +--- +A data source is the Druid equivalent of a database table. However, a query can also masquerade as a data source, providing subquery-like functionality. Query data sources are currently only supported by [GroupBy](GroupByQuery.html) queries. + +### Table Data Source +The table data source the most common type. It's represented by a string, or by the full structure: + +```json +{ + "type": "table", + "name": +} +``` + +### Query Data Source +```json +{ + "type": "query", + "query": { + "type": "groupBy", + ... + } +} +``` diff --git a/docs/content/GroupByQuery.md b/docs/content/GroupByQuery.md index dd7f49f7179..34a10c28d64 100644 --- a/docs/content/GroupByQuery.md +++ b/docs/content/GroupByQuery.md @@ -47,7 +47,7 @@ There are 9 main parts to a groupBy query: |property|description|required?| |--------|-----------|---------| |queryType|This String should always be "groupBy"; this is the first thing Druid looks at to figure out how to interpret the query|yes| -|dataSource|A String defining the data source to query, very similar to a table in a relational database|yes| +|dataSource|A String defining the data source to query, very similar to a table in a relational database, or a [DataSource](DataSource.html) structure.|yes| |dimensions|A JSON list of dimensions to do the groupBy over|yes| |orderBy|See [OrderBy](OrderBy.html).|no| |having|See [Having](Having.html).|no| From e9cf9355186381a543d8fc8f1b8aea156509fef1 Mon Sep 17 00:00:00 2001 From: Yuval Oren Date: Mon, 27 Jan 2014 18:58:25 -0800 Subject: [PATCH 05/66] Fixed realtime querying after subquery changes --- .../segment/realtime/RealtimeManager.java | 34 +++++++++++++------ 1 file changed, 23 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java index a49760ab5e9..5621ef534e0 100644 --- a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java +++ b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java @@ -30,17 +30,11 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.emitter.EmittingLogger; import io.druid.data.input.Firehose; import io.druid.data.input.InputRow; -import io.druid.query.FinalizeResultsQueryRunner; -import io.druid.query.NoopQueryRunner; -import io.druid.query.Query; -import io.druid.query.QueryRunner; -import io.druid.query.QueryRunnerFactory; -import io.druid.query.QueryRunnerFactoryConglomerate; -import io.druid.query.QuerySegmentWalker; -import io.druid.query.QueryToolChest; -import io.druid.query.SegmentDescriptor; +import io.druid.query.*; +import io.druid.segment.ReferenceCountingSegment; import io.druid.segment.realtime.plumber.Plumber; import io.druid.segment.realtime.plumber.Sink; +import io.druid.timeline.VersionedIntervalTimeline; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; @@ -108,7 +102,7 @@ public class RealtimeManager implements QuerySegmentWalker @Override public QueryRunner getQueryRunnerForIntervals(Query query, Iterable intervals) { - final FireChief chief = chiefs.get(query.getDataSource()); + final FireChief chief = chiefs.get(getDataSourceName(query)); return chief == null ? new NoopQueryRunner() : chief.getQueryRunner(query); } @@ -116,11 +110,29 @@ public class RealtimeManager implements QuerySegmentWalker @Override public QueryRunner getQueryRunnerForSegments(Query query, Iterable specs) { - final FireChief chief = chiefs.get(query.getDataSource()); + final FireChief chief = chiefs.get(getDataSourceName(query)); return chief == null ? new NoopQueryRunner() : chief.getQueryRunner(query); } + private String getDataSourceName(Query query) + { + DataSource dataSource = query.getDataSource(); + if (!(dataSource instanceof TableDataSource)) { + throw new UnsupportedOperationException("data source type '" + dataSource.getClass().getName() + "' unsupported"); + } + + String dataSourceName; + try { + dataSourceName = ((TableDataSource)query.getDataSource()).getName(); + } + catch (ClassCastException e) { + throw new UnsupportedOperationException("Subqueries are only supported in the broker"); + } + return dataSourceName; + } + + private class FireChief extends Thread implements Closeable { private final FireDepartment fireDepartment; From 98956cdadb1c3f2083df4a2f7305993e4d60de88 Mon Sep 17 00:00:00 2001 From: Yuval Oren Date: Mon, 3 Feb 2014 15:53:10 -0800 Subject: [PATCH 06/66] Extra null value check. Not sure why this is needed in the subquery implementation and not regular queries. Bug elsewhere? --- .../segment/filter/JavaScriptFilter.java | 55 +++++++++++-------- .../query/groupby/GroupByQueryRunnerTest.java | 2 + 2 files changed, 34 insertions(+), 23 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/filter/JavaScriptFilter.java b/processing/src/main/java/io/druid/segment/filter/JavaScriptFilter.java index 743a07421cb..75817403571 100644 --- a/processing/src/main/java/io/druid/segment/filter/JavaScriptFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/JavaScriptFilter.java @@ -26,6 +26,7 @@ import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcherFactory; +import io.druid.segment.data.Indexed; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import org.mozilla.javascript.Context; import org.mozilla.javascript.Function; @@ -49,27 +50,33 @@ public class JavaScriptFilter implements Filter { final Context cx = Context.enter(); try { - ImmutableConciseSet conciseSet = ImmutableConciseSet.union( - FunctionalIterable.create(selector.getDimensionValues(dimension)) - .filter(new Predicate() - { - @Override - public boolean apply(@Nullable String input) - { - return predicate.applyInContext(cx, input); - } - }) - .transform( - new com.google.common.base.Function() - { - @Override - public ImmutableConciseSet apply(@Nullable String input) - { - return selector.getConciseInvertedIndex(dimension, input); - } - } - ) - ); + final Indexed dimValues = selector.getDimensionValues(dimension); + ImmutableConciseSet conciseSet; + if (dimValues == null) { + conciseSet = new ImmutableConciseSet(); + } else { + conciseSet = ImmutableConciseSet.union( + FunctionalIterable.create(dimValues) + .filter(new Predicate() + { + @Override + public boolean apply(@Nullable String input) + { + return predicate.applyInContext(cx, input); + } + }) + .transform( + new com.google.common.base.Function() + { + @Override + public ImmutableConciseSet apply(@Nullable String input) + { + return selector.getConciseInvertedIndex(dimension, input); + } + } + ) + ); + } return conciseSet; } finally { Context.exit(); @@ -83,12 +90,14 @@ public class JavaScriptFilter implements Filter return factory.makeValueMatcher(dimension, predicate); } - static class JavaScriptPredicate implements Predicate { + static class JavaScriptPredicate implements Predicate + { final ScriptableObject scope; final Function fnApply; final String script; - public JavaScriptPredicate(final String script) { + public JavaScriptPredicate(final String script) + { Preconditions.checkNotNull(script, "script must not be null"); this.script = script; diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 4fffcf67c01..f336a26884e 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -45,6 +45,7 @@ import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.MaxAggregatorFactory; import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DimensionSpec; +import io.druid.query.filter.JavaScriptDimFilter; import io.druid.query.filter.RegexDimFilter; import io.druid.query.groupby.having.EqualToHavingSpec; import io.druid.query.groupby.having.GreaterThanHavingSpec; @@ -787,6 +788,7 @@ public class GroupByQueryRunnerTest .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setDimFilter(new JavaScriptDimFilter("quality", "function(dim){ return true; }")) .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, From 314e38a2c6a88c47ee3b3a8e5cf416c0d65058f8 Mon Sep 17 00:00:00 2001 From: Yuval Oren Date: Wed, 26 Feb 2014 22:36:27 -0800 Subject: [PATCH 07/66] Fixed realtime index query serving failure due to incorrect treatment of DataSource --- .../indexing/overlord/ThreadPoolTaskRunner.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java index 2cc94ac7400..0d69ed5036c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java @@ -38,6 +38,7 @@ import io.druid.indexing.common.TaskToolboxFactory; import io.druid.indexing.common.task.Task; import io.druid.query.NoopQueryRunner; import io.druid.query.Query; +import io.druid.query.TableDataSource; import io.druid.query.QueryRunner; import io.druid.query.QuerySegmentWalker; import io.druid.query.SegmentDescriptor; @@ -152,10 +153,17 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker private QueryRunner getQueryRunnerImpl(Query query) { QueryRunner queryRunner = null; + String queryDataSource; + try { + queryDataSource = ((TableDataSource)query.getDataSource()).getName(); + } + catch (ClassCastException e) { + throw new IllegalArgumentException("Subqueries are not welcome here"); + } for (final ThreadPoolTaskRunnerWorkItem taskRunnerWorkItem : ImmutableList.copyOf(runningItems)) { final Task task = taskRunnerWorkItem.getTask(); - if (task.getDataSource().equals(query.getDataSource())) { + if (task.getDataSource().equals(queryDataSource)) { final QueryRunner taskQueryRunner = task.getQueryRunner(query); if (taskQueryRunner != null) { @@ -163,7 +171,7 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker queryRunner = taskQueryRunner; } else { log.makeAlert("Found too many query runners for datasource") - .addData("dataSource", query.getDataSource()) + .addData("dataSource", queryDataSource) .emit(); } } From 70db460f972f9b0c3d9dba04c233e08d73343b27 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 5 Mar 2014 10:55:12 -0800 Subject: [PATCH 08/66] Blocking Executors and maxPendingPersists, oh my! - Execs.newBlockingSingleThreaded can now accept capacity = 0. - Changed default maxPendingPersists from 2 to 0. - Fixed serde of maxPendingPersists in RealtimeIndexTasks. --- .../main/java/io/druid/concurrent/Execs.java | 40 ++++++++------ .../java/io/druid/concurrent/ExecsTest.java | 55 ++++++++++++++----- .../common/task/RealtimeIndexTask.java | 10 +++- .../indexing/common/task/TaskSerdeTest.java | 4 +- .../plumber/RealtimePlumberSchool.java | 2 +- 5 files changed, 76 insertions(+), 35 deletions(-) diff --git a/common/src/main/java/io/druid/concurrent/Execs.java b/common/src/main/java/io/druid/concurrent/Execs.java index 308208ef98d..66af2a196ba 100644 --- a/common/src/main/java/io/druid/concurrent/Execs.java +++ b/common/src/main/java/io/druid/concurrent/Execs.java @@ -22,11 +22,13 @@ package io.druid.concurrent; import com.google.common.util.concurrent.ThreadFactoryBuilder; import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.RejectedExecutionHandler; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -60,25 +62,29 @@ public class Execs * @param capacity maximum capacity after which the executorService will block on accepting new tasks * @return ExecutorService which blocks accepting new tasks when the capacity reached */ - public static ExecutorService newBlockingSingleThreaded(String nameFormat, int capacity) + public static ExecutorService newBlockingSingleThreaded(final String nameFormat, final int capacity) { - return new ThreadPoolExecutor( - 1, 1, - 0L, TimeUnit.MILLISECONDS, - new ArrayBlockingQueue(capacity), makeThreadFactory(nameFormat) - , new RejectedExecutionHandler() - { - @Override - public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) - { - try { - ((ArrayBlockingQueue) executor.getQueue()).put(r); - } - catch (InterruptedException e) { - throw new RejectedExecutionException("Got Interrupted while adding to the Queue"); - } - } + final BlockingQueue queue; + if (capacity > 0) { + queue = new ArrayBlockingQueue<>(capacity); + } else { + queue = new SynchronousQueue<>(); } + return new ThreadPoolExecutor( + 1, 1, 0L, TimeUnit.MILLISECONDS, queue, makeThreadFactory(nameFormat), + new RejectedExecutionHandler() + { + @Override + public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) + { + try { + executor.getQueue().put(r); + } + catch (InterruptedException e) { + throw new RejectedExecutionException("Got Interrupted while adding to the Queue"); + } + } + } ); } } diff --git a/common/src/test/java/io/druid/concurrent/ExecsTest.java b/common/src/test/java/io/druid/concurrent/ExecsTest.java index 809ed5eac02..ae2b0e15473 100644 --- a/common/src/test/java/io/druid/concurrent/ExecsTest.java +++ b/common/src/test/java/io/druid/concurrent/ExecsTest.java @@ -20,6 +20,8 @@ package io.druid.concurrent; import com.google.common.base.Throwables; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import com.metamx.common.logger.Logger; import org.junit.Assert; import org.junit.Test; @@ -30,23 +32,46 @@ import java.util.concurrent.atomic.AtomicInteger; public class ExecsTest { + private static final Logger log = new Logger(ExecsTest.class); + @Test - public void testBlockingExecutorService() throws Exception + public void testBlockingExecutorServiceZeroCapacity() throws Exception { - final int capacity = 3; - final ExecutorService blockingExecutor = Execs.newBlockingSingleThreaded("test%d", capacity); - final CountDownLatch queueFullSignal = new CountDownLatch(capacity + 1); - final CountDownLatch taskCompletedSignal = new CountDownLatch(2 * capacity); + runTest(0); + } + + @Test + public void testBlockingExecutorServiceOneCapacity() throws Exception + { + runTest(1); + } + + @Test + public void testBlockingExecutorServiceThreeCapacity() throws Exception + { + runTest(3); + } + + private static void runTest(final int capacity) throws Exception + { + final int nTasks = (capacity + 1) * 3; + final ExecutorService blockingExecutor = Execs.newBlockingSingleThreaded("ExecsTest-Blocking-%d", capacity); + final CountDownLatch queueShouldBeFullSignal = new CountDownLatch(capacity + 1); + final CountDownLatch taskCompletedSignal = new CountDownLatch(nTasks); final CountDownLatch taskStartSignal = new CountDownLatch(1); final AtomicInteger producedCount = new AtomicInteger(); final AtomicInteger consumedCount = new AtomicInteger(); - ExecutorService producer = Executors.newSingleThreadExecutor(); + final ExecutorService producer = Executors.newSingleThreadExecutor( + new ThreadFactoryBuilder().setNameFormat( + "ExecsTest-Producer-%d" + ).build() + ); producer.submit( new Runnable() { public void run() { - for (int i = 0; i < 2 * capacity; i++) { + for (int i = 0; i < nTasks; i++) { final int taskID = i; System.out.println("Produced task" + taskID); blockingExecutor.submit( @@ -55,7 +80,7 @@ public class ExecsTest @Override public void run() { - System.out.println("Starting task" + taskID); + log.info("Starting task: %s", taskID); try { taskStartSignal.await(); consumedCount.incrementAndGet(); @@ -64,29 +89,31 @@ public class ExecsTest catch (Exception e) { throw Throwables.propagate(e); } - System.out.println("Completed task" + taskID); + log.info("Completed task: %s", taskID); } } ); producedCount.incrementAndGet(); - queueFullSignal.countDown(); + queueShouldBeFullSignal.countDown(); } } } ); - queueFullSignal.await(); - // verify that the producer blocks + queueShouldBeFullSignal.await(); + // Verify that the producer blocks. I don't think it's possible to be sure that the producer is blocking (since + // it could be doing nothing for any reason). But waiting a short period of time and checking that it hasn't done + // anything should hopefully be sufficient. + Thread.sleep(500); Assert.assertEquals(capacity + 1, producedCount.get()); // let the tasks run taskStartSignal.countDown(); // wait until all tasks complete taskCompletedSignal.await(); // verify all tasks consumed - Assert.assertEquals(2 * capacity, consumedCount.get()); + Assert.assertEquals(nTasks, consumedCount.get()); // cleanup blockingExecutor.shutdown(); producer.shutdown(); - } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index 7a40035c3e6..09172da3a4d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -109,7 +109,7 @@ public class RealtimeIndexTask extends AbstractTask @JsonProperty("firehose") FirehoseFactory firehoseFactory, @JsonProperty("fireDepartmentConfig") FireDepartmentConfig fireDepartmentConfig, @JsonProperty("windowPeriod") Period windowPeriod, - @JsonProperty("maxPendingPersists") int maxPendingPersists, + @JsonProperty("maxPendingPersists") Integer maxPendingPersists, @JsonProperty("segmentGranularity") IndexGranularity segmentGranularity, @JsonProperty("rejectionPolicy") RejectionPolicyFactory rejectionPolicyFactory ) @@ -139,7 +139,7 @@ public class RealtimeIndexTask extends AbstractTask this.firehoseFactory = firehoseFactory; this.fireDepartmentConfig = fireDepartmentConfig; this.windowPeriod = windowPeriod; - this.maxPendingPersists = (maxPendingPersists == 0) + this.maxPendingPersists = (maxPendingPersists == null) ? RealtimePlumberSchool.DEFAULT_MAX_PENDING_PERSISTS : maxPendingPersists; this.segmentGranularity = segmentGranularity; @@ -398,6 +398,12 @@ public class RealtimeIndexTask extends AbstractTask return windowPeriod; } + @JsonProperty + public int getMaxPendingPersists() + { + return maxPendingPersists; + } + @JsonProperty public IndexGranularity getSegmentGranularity() { diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index b75169c8b9a..ba5afb04d3b 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -198,7 +198,7 @@ public class TaskSerdeTest null, null, new Period("PT10M"), - 1, + 5, IndexGranularity.HOUR, null ); @@ -214,6 +214,7 @@ public class TaskSerdeTest Assert.assertEquals("rofl", task.getTaskResource().getAvailabilityGroup()); Assert.assertEquals(new Period("PT10M"), task.getWindowPeriod()); Assert.assertEquals(IndexGranularity.HOUR, task.getSegmentGranularity()); + Assert.assertEquals(5, task.getMaxPendingPersists()); Assert.assertEquals(task.getId(), task2.getId()); Assert.assertEquals(task.getGroupId(), task2.getGroupId()); @@ -222,6 +223,7 @@ public class TaskSerdeTest Assert.assertEquals(task.getTaskResource().getAvailabilityGroup(), task2.getTaskResource().getAvailabilityGroup()); Assert.assertEquals(task.getWindowPeriod(), task2.getWindowPeriod()); Assert.assertEquals(task.getSegmentGranularity(), task2.getSegmentGranularity()); + Assert.assertEquals(task.getMaxPendingPersists(), task2.getMaxPendingPersists()); } @Test diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java index f7d6398a194..4a8332137d4 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java @@ -44,7 +44,7 @@ import java.util.concurrent.ExecutorService; */ public class RealtimePlumberSchool implements PlumberSchool { - public static final int DEFAULT_MAX_PENDING_PERSISTS = 2; + public static final int DEFAULT_MAX_PENDING_PERSISTS = 0; private static final EmittingLogger log = new EmittingLogger(RealtimePlumberSchool.class); From 5b4c04510f9a268698161f9577c5546f6f5374ae Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 4 Mar 2014 13:24:56 -0800 Subject: [PATCH 09/66] fix populateCache flag not working --- .../druid/client/CachingClusteredClient.java | 40 +++++++++++-------- 1 file changed, 24 insertions(+), 16 deletions(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index dacfc7938ef..c2901c9675f 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -176,32 +176,37 @@ public class CachingClusteredClient implements QueryRunner queryCacheKey = null; } - // Pull cached segments from cache and remove from set of segments to query - if (useCache && queryCacheKey != null) { + if (queryCacheKey != null) { Map, Cache.NamedKey> cacheKeys = Maps.newHashMap(); - for (Pair e : segments) { - cacheKeys.put(e, computeSegmentCacheKey(e.lhs.getSegment().getIdentifier(), e.rhs, queryCacheKey)); + for (Pair segment : segments) { + final Cache.NamedKey segmentCacheKey = computeSegmentCacheKey( + segment.lhs.getSegment().getIdentifier(), + segment.rhs, + queryCacheKey + ); + cacheKeys.put(segment, segmentCacheKey); } - Map cachedValues = cache.getBulk(cacheKeys.values()); + // Pull cached segments from cache and remove from set of segments to query + final Map cachedValues; + if (useCache) { + cachedValues = cache.getBulk(cacheKeys.values()); + } else { + cachedValues = ImmutableMap.of(); + } for (Map.Entry, Cache.NamedKey> entry : cacheKeys.entrySet()) { Pair segment = entry.getKey(); Cache.NamedKey segmentCacheKey = entry.getValue(); - - final ServerSelector selector = segment.lhs; - final SegmentDescriptor descriptor = segment.rhs; - final Interval segmentQueryInterval = descriptor.getInterval(); + final Interval segmentQueryInterval = segment.rhs.getInterval(); final byte[] cachedValue = cachedValues.get(segmentCacheKey); - if (cachedValue != null) { - cachedResults.add(Pair.of(segmentQueryInterval.getStart(), cachedValue)); - // remove cached segment from set of segments to query segments.remove(segment); - } else { - final String segmentIdentifier = selector.getSegment().getIdentifier(); + cachedResults.add(Pair.of(segmentQueryInterval.getStart(), cachedValue)); + } else if (populateCache) { + final String segmentIdentifier = segment.lhs.getSegment().getIdentifier(); cachePopulatorMap.put( String.format("%s_%s", segmentIdentifier, segmentQueryInterval), new CachePopulator(cache, objectMapper, segmentCacheKey) @@ -331,9 +336,12 @@ public class CachingClusteredClient implements QueryRunner String segmentIdentifier = value.getSegmentId(); final Iterable segmentResults = value.getResults(); - cachePopulatorMap.get( + CachePopulator cachePopulator = cachePopulatorMap.get( String.format("%s_%s", segmentIdentifier, value.getInterval()) - ).populate(Iterables.transform(segmentResults, prepareForCache)); + ); + if(cachePopulator != null) { + cachePopulator.populate(Iterables.transform(segmentResults, prepareForCache)); + } return Sequences.simple( Iterables.transform( From 427014d0a07f762ccf93a24b099b0f4a5e61578a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 4 Mar 2014 13:25:02 -0800 Subject: [PATCH 10/66] cosmetics --- .../main/java/io/druid/client/CachingClusteredClient.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index c2901c9675f..c17314c7cf8 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -128,7 +128,7 @@ public class CachingClusteredClient implements QueryRunner final boolean isBySegment = Boolean.parseBoolean(query.getContextValue("bySegment", "false")); - ImmutableMap.Builder contextBuilder = new ImmutableMap.Builder(); + ImmutableMap.Builder contextBuilder = new ImmutableMap.Builder<>(); final String priority = query.getContextValue("priority", "0"); contextBuilder.put("priority", priority); @@ -234,7 +234,7 @@ public class CachingClusteredClient implements QueryRunner } } - return new LazySequence( + return new LazySequence<>( new Supplier>() { @Override @@ -270,7 +270,7 @@ public class CachingClusteredClient implements QueryRunner final TypeReference cacheObjectClazz = strategy.getCacheObjectClazz(); for (Pair cachedResultPair : cachedResults) { final byte[] cachedResult = cachedResultPair.rhs; - Sequence cachedSequence = new BaseSequence>( + Sequence cachedSequence = new BaseSequence<>( new BaseSequence.IteratorMaker>() { @Override From fffb58f6e8234fac41375fcba2adbc2374b620d1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 4 Mar 2014 13:48:43 -0800 Subject: [PATCH 11/66] add CachingClusteredClient tests --- .../client/CachingClusteredClientTest.java | 1275 +++++++++++++++++ .../java/io/druid/client/RangeIterable.java | 89 ++ 2 files changed, 1364 insertions(+) create mode 100644 server/src/test/java/io/druid/client/CachingClusteredClientTest.java create mode 100644 server/src/test/java/io/druid/client/RangeIterable.java diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java new file mode 100644 index 00000000000..9a5154e5824 --- /dev/null +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -0,0 +1,1275 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 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.client; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.annotation.JsonSerialize; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Ordering; +import com.metamx.common.ISE; +import com.metamx.common.guava.FunctionalIterable; +import com.metamx.common.guava.MergeIterable; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import com.metamx.common.guava.nary.TrinaryFn; +import io.druid.query.topn.TopNQuery; +import io.druid.query.topn.TopNQueryBuilder; +import io.druid.query.topn.TopNQueryConfig; +import io.druid.query.topn.TopNQueryQueryToolChest; +import io.druid.query.topn.TopNResultValue; +import io.druid.client.cache.Cache; +import io.druid.client.cache.MapCache; +import io.druid.client.selector.QueryableDruidServer; +import io.druid.client.selector.RandomServerSelectorStrategy; +import io.druid.client.selector.ServerSelector; +import io.druid.granularity.PeriodGranularity; +import io.druid.granularity.QueryGranularity; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.BySegmentResultValueClass; +import io.druid.query.Druids; +import io.druid.query.MapQueryToolChestWarehouse; +import io.druid.query.Query; +import io.druid.query.QueryConfig; +import io.druid.query.QueryRunner; +import io.druid.query.QueryToolChest; +import io.druid.query.Result; +import io.druid.query.SegmentDescriptor; +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.FieldAccessPostAggregator; +import io.druid.query.filter.DimFilter; +import io.druid.query.search.SearchQueryQueryToolChest; +import io.druid.query.search.SearchResultValue; +import io.druid.query.search.search.InsensitiveContainsSearchQuerySpec; +import io.druid.query.search.search.SearchHit; +import io.druid.query.search.search.SearchQuery; +import io.druid.query.search.search.SearchQueryConfig; +import io.druid.query.spec.MultipleIntervalSegmentSpec; +import io.druid.query.timeboundary.TimeBoundaryQuery; +import io.druid.query.timeboundary.TimeBoundaryResultValue; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import io.druid.query.timeseries.TimeseriesResultValue; +import io.druid.segment.TestHelper; +import io.druid.timeline.DataSegment; +import io.druid.timeline.VersionedIntervalTimeline; +import io.druid.timeline.partition.NoneShardSpec; +import io.druid.timeline.partition.PartitionChunk; +import io.druid.timeline.partition.ShardSpec; +import io.druid.timeline.partition.SingleElementPartitionChunk; +import io.druid.timeline.partition.StringPartitionChunk; +import org.easymock.Capture; +import org.easymock.EasyMock; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.TreeMap; +import java.util.concurrent.Executor; + +/** + */ +@RunWith(Parameterized.class) +public class CachingClusteredClientTest +{ + /** + * We want a deterministic test, but we'd also like a bit of randomness for the distribution of segments + * across servers. Thus, we loop multiple times and each time use a deterministically created Random instance. + * Increase this value to increase exposure to random situations at the expense of test run time. + */ + private static final int RANDOMNESS = 10; + + public static final ImmutableMap CONTEXT = ImmutableMap.of(); + public static final MultipleIntervalSegmentSpec SEG_SPEC = new MultipleIntervalSegmentSpec(ImmutableList.of()); + public static final String DATA_SOURCE = "test"; + + private static final List AGGS = Arrays.asList( + new CountAggregatorFactory("rows"), + new LongSumAggregatorFactory("imps", "imps"), + new LongSumAggregatorFactory("impers", "imps") + ); + private static final List POST_AGGS = Arrays.asList( + new ArithmeticPostAggregator( + "avg_imps_per_row", + "/", + Arrays.asList( + new FieldAccessPostAggregator("imps", "imps"), + new FieldAccessPostAggregator("rows", "rows") + ) + ) + ); + private static final List RENAMED_AGGS = Arrays.asList( + new CountAggregatorFactory("rows2"), + new LongSumAggregatorFactory("imps", "imps"), + new LongSumAggregatorFactory("impers2", "imps") + ); + private static final DimFilter DIM_FILTER = null; + private static final List RENAMED_POST_AGGS = Arrays.asList(); + private static final QueryGranularity GRANULARITY = QueryGranularity.DAY; + private static final DateTimeZone TIMEZONE = DateTimeZone.forID("America/Los_Angeles"); + private static final QueryGranularity PT1H_TZ_GRANULARITY = new PeriodGranularity(new Period("PT1H"), null, TIMEZONE); + private static final String TOP_DIM = "a_dim"; + + @Parameterized.Parameters + public static Collection constructorFeeder() throws IOException + { + return Lists.transform( + Lists.newArrayList(new RangeIterable(RANDOMNESS)), + new Function() + { + @Override + public Object apply(@Nullable Integer input) + { + return new Object[]{input}; + } + } + ); + } + + + protected static final DefaultObjectMapper jsonMapper = new DefaultObjectMapper(new SmileFactory()); + + static { + jsonMapper.getFactory().setCodec(jsonMapper); + } + + private final Random random; + + protected VersionedIntervalTimeline timeline; + protected TimelineServerView serverView; + protected Cache cache; + + CachingClusteredClient client; + + DruidServer[] servers; + + public CachingClusteredClientTest(int randomSeed) + { + this.random = new Random(randomSeed); + } + + @Before + public void setUp() throws Exception + { + timeline = new VersionedIntervalTimeline<>(Ordering.natural()); + serverView = EasyMock.createStrictMock(TimelineServerView.class); + cache = MapCache.create(100000); + + client = makeClient(); + + servers = new DruidServer[]{ + new DruidServer("test1", "test1", 10, "historical", "bye", 0), + new DruidServer("test2", "test2", 10, "historical", "bye", 0), + new DruidServer("test3", "test3", 10, "historical", "bye", 0), + new DruidServer("test4", "test4", 10, "historical", "bye", 0), + new DruidServer("test5", "test5", 10, "historical", "bye", 0) + }; + } + + @Test + @SuppressWarnings("unchecked") + public void testTimeseriesCaching() throws Exception + { + final Druids.TimeseriesQueryBuilder builder = Druids.newTimeseriesQueryBuilder() + .dataSource(DATA_SOURCE) + .intervals(SEG_SPEC) + .filters(DIM_FILTER) + .granularity(GRANULARITY) + .aggregators(AGGS) + .postAggregators(POST_AGGS) + .context(CONTEXT); + + testQueryCaching( + builder.build(), + new Interval("2011-01-01/2011-01-02"), makeTimeResults(new DateTime("2011-01-01"), 50, 5000), + new Interval("2011-01-02/2011-01-03"), makeTimeResults(new DateTime("2011-01-02"), 30, 6000), + new Interval("2011-01-04/2011-01-05"), makeTimeResults(new DateTime("2011-01-04"), 23, 85312), + + new Interval("2011-01-05/2011-01-10"), + makeTimeResults( + new DateTime("2011-01-05"), 85, 102, + new DateTime("2011-01-06"), 412, 521, + new DateTime("2011-01-07"), 122, 21894, + new DateTime("2011-01-08"), 5, 20, + new DateTime("2011-01-09"), 18, 521 + ), + + new Interval("2011-01-05/2011-01-10"), + makeTimeResults( + new DateTime("2011-01-05T01"), 80, 100, + new DateTime("2011-01-06T01"), 420, 520, + new DateTime("2011-01-07T01"), 12, 2194, + new DateTime("2011-01-08T01"), 59, 201, + new DateTime("2011-01-09T01"), 181, 52 + ) + ); + + TestHelper.assertExpectedResults( + makeRenamedTimeResults( + new DateTime("2011-01-01"), 50, 5000, + new DateTime("2011-01-02"), 30, 6000, + new DateTime("2011-01-04"), 23, 85312, + new DateTime("2011-01-05"), 85, 102, + new DateTime("2011-01-05T01"), 80, 100, + new DateTime("2011-01-06"), 412, 521, + new DateTime("2011-01-06T01"), 420, 520, + new DateTime("2011-01-07"), 122, 21894, + new DateTime("2011-01-07T01"), 12, 2194, + new DateTime("2011-01-08"), 5, 20, + new DateTime("2011-01-08T01"), 59, 201, + new DateTime("2011-01-09"), 18, 521, + new DateTime("2011-01-09T01"), 181, 52 + ), + client.run( + builder.intervals("2011-01-01/2011-01-10") + .aggregators(RENAMED_AGGS) + .postAggregators(RENAMED_POST_AGGS) + .build() + ) + ); + } + + @Test + @SuppressWarnings("unchecked") + public void testTimeseriesCachingTimeZone() throws Exception + { + final Druids.TimeseriesQueryBuilder builder = Druids.newTimeseriesQueryBuilder() + .dataSource(DATA_SOURCE) + .intervals(SEG_SPEC) + .filters(DIM_FILTER) + .granularity(PT1H_TZ_GRANULARITY) + .aggregators(AGGS) + .postAggregators(POST_AGGS) + .context(CONTEXT); + + testQueryCaching( + builder.build(), + new Interval("2011-11-04/2011-11-08"), + makeTimeResults( + new DateTime("2011-11-04", TIMEZONE), 50, 5000, + new DateTime("2011-11-05", TIMEZONE), 30, 6000, + new DateTime("2011-11-06", TIMEZONE), 23, 85312, + new DateTime("2011-11-07", TIMEZONE), 85, 102 + ) + ); + + TestHelper.assertExpectedResults( + makeRenamedTimeResults( + new DateTime("2011-11-04", TIMEZONE), 50, 5000, + new DateTime("2011-11-05", TIMEZONE), 30, 6000, + new DateTime("2011-11-06", TIMEZONE), 23, 85312, + new DateTime("2011-11-07", TIMEZONE), 85, 102 + ), + client.run( + builder.intervals("2011-11-04/2011-11-08") + .aggregators(RENAMED_AGGS) + .postAggregators(RENAMED_POST_AGGS) + .build() + ) + ); + } + + @Test + @SuppressWarnings("unchecked") + public void testTopNCaching() throws Exception + { + final TopNQueryBuilder builder = new TopNQueryBuilder() + .dataSource(DATA_SOURCE) + .dimension(TOP_DIM) + .metric("imps") + .threshold(3) + .intervals(SEG_SPEC) + .filters(DIM_FILTER) + .granularity(GRANULARITY) + .aggregators(AGGS) + .postAggregators(POST_AGGS) + .context(CONTEXT); + + testQueryCaching( + builder.build(), + new Interval("2011-01-01/2011-01-02"), + makeTopNResults(new DateTime("2011-01-01"), "a", 50, 5000, "b", 50, 4999, "c", 50, 4998), + + new Interval("2011-01-02/2011-01-03"), + makeTopNResults(new DateTime("2011-01-02"), "a", 50, 4997, "b", 50, 4996, "c", 50, 4995), + + new Interval("2011-01-05/2011-01-10"), + makeTopNResults( + new DateTime("2011-01-05"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, + new DateTime("2011-01-06"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-07"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-08"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, + new DateTime("2011-01-09"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983 + ), + + new Interval("2011-01-05/2011-01-10"), + makeTopNResults( + new DateTime("2011-01-05T01"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, + new DateTime("2011-01-06T01"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-07T01"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-08T01"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, + new DateTime("2011-01-09T01"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983 + ) + ); + + TestHelper.assertExpectedResults( + makeRenamedTopNResults( + new DateTime("2011-01-01"), "a", 50, 5000, "b", 50, 4999, "c", 50, 4998, + new DateTime("2011-01-02"), "a", 50, 4997, "b", 50, 4996, "c", 50, 4995, + new DateTime("2011-01-05"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, + new DateTime("2011-01-05T01"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, + new DateTime("2011-01-06"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-06T01"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-07"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-07T01"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-08"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, + new DateTime("2011-01-08T01"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, + new DateTime("2011-01-09"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983, + new DateTime("2011-01-09T01"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983 + ), + client.run( + builder.intervals("2011-01-01/2011-01-10") + .metric("imps") + .aggregators(RENAMED_AGGS) + .postAggregators(RENAMED_POST_AGGS) + .build() + ) + ); + } + + @Test + @SuppressWarnings("unchecked") + public void testTopNCachingTimeZone() throws Exception + { + final TopNQueryBuilder builder = new TopNQueryBuilder() + .dataSource(DATA_SOURCE) + .dimension(TOP_DIM) + .metric("imps") + .threshold(3) + .intervals(SEG_SPEC) + .filters(DIM_FILTER) + .granularity(PT1H_TZ_GRANULARITY) + .aggregators(AGGS) + .postAggregators(POST_AGGS) + .context(CONTEXT); + + testQueryCaching( + builder.build(), + new Interval("2011-11-04/2011-11-08"), + makeTopNResults( + new DateTime("2011-11-04", TIMEZONE), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, + new DateTime("2011-11-05", TIMEZONE), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-11-06", TIMEZONE), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-11-07", TIMEZONE), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986 + ) + ); + + TestHelper.assertExpectedResults( + makeRenamedTopNResults( + + new DateTime("2011-11-04", TIMEZONE), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, + new DateTime("2011-11-05", TIMEZONE), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-11-06", TIMEZONE), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-11-07", TIMEZONE), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986 + ), + client.run( + builder.intervals("2011-11-04/2011-11-08") + .metric("imps") + .aggregators(RENAMED_AGGS) + .postAggregators(RENAMED_POST_AGGS) + .build() + ) + ); + } + + @Test + @SuppressWarnings("unchecked") + public void testTopNCachingEmptyResults() throws Exception + { + final TopNQueryBuilder builder = new TopNQueryBuilder() + .dataSource(DATA_SOURCE) + .dimension(TOP_DIM) + .metric("imps") + .threshold(3) + .intervals(SEG_SPEC) + .filters(DIM_FILTER) + .granularity(GRANULARITY) + .aggregators(AGGS) + .postAggregators(POST_AGGS) + .context(CONTEXT); + + testQueryCaching( + builder.build(), + new Interval("2011-01-01/2011-01-02"), + makeTopNResults(), + + new Interval("2011-01-02/2011-01-03"), + makeTopNResults(), + + new Interval("2011-01-05/2011-01-10"), + makeTopNResults( + new DateTime("2011-01-05"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, + new DateTime("2011-01-06"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-07"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-08"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, + new DateTime("2011-01-09"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983 + ), + + new Interval("2011-01-05/2011-01-10"), + makeTopNResults( + new DateTime("2011-01-05T01"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, + new DateTime("2011-01-06T01"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-07T01"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-08T01"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, + new DateTime("2011-01-09T01"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983 + ) + ); + + TestHelper.assertExpectedResults( + makeRenamedTopNResults( + new DateTime("2011-01-05"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, + new DateTime("2011-01-05T01"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992, + new DateTime("2011-01-06"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-06T01"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-07"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-07T01"), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989, + new DateTime("2011-01-08"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, + new DateTime("2011-01-08T01"), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986, + new DateTime("2011-01-09"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983, + new DateTime("2011-01-09T01"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983 + ), + client.run( + builder.intervals("2011-01-01/2011-01-10") + .metric("imps") + .aggregators(RENAMED_AGGS) + .postAggregators(RENAMED_POST_AGGS) + .build() + ) + ); + } + + @Test + public void testSearchCaching() throws Exception + { + testQueryCaching( + new SearchQuery( + DATA_SOURCE, + DIM_FILTER, + GRANULARITY, + 1000, + SEG_SPEC, + Arrays.asList("a_dim"), + new InsensitiveContainsSearchQuerySpec("how"), + null, + CONTEXT + ), + new Interval("2011-01-01/2011-01-02"), + makeSearchResults(new DateTime("2011-01-01"), "how", "howdy", "howwwwww", "howwy"), + + new Interval("2011-01-02/2011-01-03"), + makeSearchResults(new DateTime("2011-01-02"), "how1", "howdy1", "howwwwww1", "howwy1"), + + new Interval("2011-01-05/2011-01-10"), + makeSearchResults( + new DateTime("2011-01-05"), "how2", "howdy2", "howwwwww2", "howww2", + new DateTime("2011-01-06"), "how3", "howdy3", "howwwwww3", "howww3", + new DateTime("2011-01-07"), "how4", "howdy4", "howwwwww4", "howww4", + new DateTime("2011-01-08"), "how5", "howdy5", "howwwwww5", "howww5", + new DateTime("2011-01-09"), "how6", "howdy6", "howwwwww6", "howww6" + ), + + new Interval("2011-01-05/2011-01-10"), + makeSearchResults( + new DateTime("2011-01-05T01"), "how2", "howdy2", "howwwwww2", "howww2", + new DateTime("2011-01-06T01"), "how3", "howdy3", "howwwwww3", "howww3", + new DateTime("2011-01-07T01"), "how4", "howdy4", "howwwwww4", "howww4", + new DateTime("2011-01-08T01"), "how5", "howdy5", "howwwwww5", "howww5", + new DateTime("2011-01-09T01"), "how6", "howdy6", "howwwwww6", "howww6" + ) + ); + } + + @SuppressWarnings("unchecked") + public void testQueryCaching( + final Query query, Object... args + ) + { + if (args.length % 2 != 0) { + throw new ISE("args.length must be divisible by two, was %d", args.length); + } + + final List queryIntervals = Lists.newArrayListWithCapacity(args.length / 2); + final List>>> expectedResults = Lists.newArrayListWithCapacity(queryIntervals.size()); + + for (int i = 0; i < args.length; i += 2) { + final Interval interval = (Interval) args[i]; + final Iterable> results = (Iterable>) args[i + 1]; + + if (queryIntervals.size() > 0 && interval.equals(queryIntervals.get(queryIntervals.size() - 1))) { + expectedResults.get(expectedResults.size() - 1).add(results); + } else { + queryIntervals.add(interval); + expectedResults.add(Lists.>>newArrayList(results)); + } + } + + for (int i = 0; i < queryIntervals.size(); ++i) { + timeline = new VersionedIntervalTimeline<>(Ordering.natural()); + final int numTimesToQuery = 3; + + List mocks = Lists.newArrayList(); + mocks.add(serverView); + + final Interval actualQueryInterval = new Interval( + queryIntervals.get(0).getStart(), queryIntervals.get(i).getEnd() + ); + + final List> serverExpectationList = Lists.newArrayList(); + + for (int k = 0; k < i + 1; ++k) { + final int numChunks = expectedResults.get(k).size(); + final TreeMap serverExpectations = Maps.newTreeMap(); + serverExpectationList.add(serverExpectations); + for (int j = 0; j < numChunks; ++j) { + DruidServer lastServer = servers[random.nextInt(servers.length)]; + if (!serverExpectations.containsKey(lastServer)) { + serverExpectations.put(lastServer, new ServerExpectations(lastServer, makeMock(mocks, QueryRunner.class))); + } + + ServerExpectation expectation = new ServerExpectation( + String.format("%s_%s", k, j), + queryIntervals.get(i), + makeMock(mocks, DataSegment.class), + expectedResults.get(k).get(j) + ); + serverExpectations.get(lastServer).addExpectation(expectation); + + ServerSelector selector = new ServerSelector(expectation.getSegment(), new RandomServerSelectorStrategy()); + selector.addServer(new QueryableDruidServer(lastServer, null)); + + final PartitionChunk chunk; + if (numChunks == 1) { + chunk = new SingleElementPartitionChunk<>(selector); + } else { + String start = null; + String end = null; + if (j > 0) { + start = String.valueOf(j - 1); + } + if (j + 1 < numChunks) { + end = String.valueOf(j); + } + chunk = new StringPartitionChunk<>(start, end, j, selector); + } + timeline.add(queryIntervals.get(k), String.valueOf(k), chunk); + } + } + + List queryCaptures = Lists.newArrayList(); + final Map finalExpectation = serverExpectationList.get( + serverExpectationList.size() - 1 + ); + for (Map.Entry entry : finalExpectation.entrySet()) { + DruidServer server = entry.getKey(); + ServerExpectations expectations = entry.getValue(); + + EasyMock.expect(serverView.getQueryRunner(server)).andReturn(expectations.getQueryRunner()).once(); + + final Capture capture = new Capture(); + queryCaptures.add(capture); + QueryRunner queryable = expectations.getQueryRunner(); + + if (query instanceof TimeseriesQuery) { + List segmentIds = Lists.newArrayList(); + List intervals = Lists.newArrayList(); + List>> results = Lists.newArrayList(); + for (ServerExpectation expectation : expectations) { + segmentIds.add(expectation.getSegmentId()); + intervals.add(expectation.getInterval()); + results.add(expectation.getResults()); + } + EasyMock.expect(queryable.run(EasyMock.capture(capture))) + .andReturn(toQueryableTimeseriesResults(segmentIds, intervals, results)) + .once(); + } else if (query instanceof TopNQuery) { + List segmentIds = Lists.newArrayList(); + List intervals = Lists.newArrayList(); + List>> results = Lists.newArrayList(); + for (ServerExpectation expectation : expectations) { + segmentIds.add(expectation.getSegmentId()); + intervals.add(expectation.getInterval()); + results.add(expectation.getResults()); + } + EasyMock.expect(queryable.run(EasyMock.capture(capture))) + .andReturn(toQueryableTopNResults(segmentIds, intervals, results)) + .once(); + } else if (query instanceof SearchQuery) { + List segmentIds = Lists.newArrayList(); + List intervals = Lists.newArrayList(); + List>> results = Lists.newArrayList(); + for (ServerExpectation expectation : expectations) { + segmentIds.add(expectation.getSegmentId()); + intervals.add(expectation.getInterval()); + results.add(expectation.getResults()); + } + EasyMock.expect(queryable.run(EasyMock.capture(capture))) + .andReturn(toQueryableSearchResults(segmentIds, intervals, results)) + .once(); + } else if (query instanceof TimeBoundaryQuery) { + List segmentIds = Lists.newArrayList(); + List intervals = Lists.newArrayList(); + List>> results = Lists.newArrayList(); + for (ServerExpectation expectation : expectations) { + segmentIds.add(expectation.getSegmentId()); + intervals.add(expectation.getInterval()); + results.add(expectation.getResults()); + } + EasyMock.expect(queryable.run(EasyMock.capture(capture))) + .andReturn(toQueryableTimeBoundaryResults(segmentIds, intervals, results)) + .once(); + } else { + throw new ISE("Unknown query type[%s]", query.getClass()); + } + } + + final int expectedResultsRangeStart; + final int expectedResultsRangeEnd; + if (query instanceof TimeBoundaryQuery) { + expectedResultsRangeStart = i; + expectedResultsRangeEnd = i + 1; + } else { + expectedResultsRangeStart = 0; + expectedResultsRangeEnd = i + 1; + } + + runWithMocks( + new Runnable() + { + @Override + public void run() + { + for (int i = 0; i < numTimesToQuery; ++i) { + TestHelper.assertExpectedResults( + new MergeIterable<>( + Ordering.>natural().nullsFirst(), + FunctionalIterable + .create(new RangeIterable(expectedResultsRangeStart, expectedResultsRangeEnd)) + .transformCat( + new Function>>>() + { + @Override + public Iterable>> apply(@Nullable Integer input) + { + List>> retVal = Lists.newArrayList(); + + final Map exps = serverExpectationList.get(input); + for (ServerExpectations expectations : exps.values()) { + for (ServerExpectation expectation : expectations) { + retVal.add(expectation.getResults()); + } + } + + return retVal; + } + } + ) + ), + client.run( + query.withQuerySegmentSpec( + new MultipleIntervalSegmentSpec( + Arrays.asList( + actualQueryInterval + ) + ) + ) + ) + ); + } + } + }, + mocks.toArray() + ); + + for (Capture queryCapture : queryCaptures) { + Query capturedQuery = (Query) queryCapture.getValue(); + Assert.assertEquals("true", capturedQuery.getContextValue("bySegment")); + } + } + } + + private Sequence> toQueryableTimeseriesResults( + Iterable segmentIds, + Iterable intervals, + Iterable>> results + ) + { + return Sequences.simple( + FunctionalIterable + .create(segmentIds) + .trinaryTransform( + intervals, + results, + new TrinaryFn>, Result>() + { + @Override + @SuppressWarnings("unchecked") + public Result apply( + final String segmentId, + final Interval interval, + final Iterable> results + ) + { + return new Result( + results.iterator().next().getTimestamp(), + new BySegmentResultValueClass( + Lists.newArrayList(results), + segmentId, + interval + ) + ); + } + } + ) + ); + } + + private Sequence> toQueryableTopNResults( + Iterable segmentIds, Iterable intervals, Iterable>> results + ) + { + return Sequences.simple( + FunctionalIterable + .create(segmentIds) + .trinaryTransform( + intervals, + results, + new TrinaryFn>, Result>() + { + @Override + @SuppressWarnings("unchecked") + public Result apply( + final String segmentId, + final Interval interval, + final Iterable> results + ) + { + return new Result( + interval.getStart(), + new BySegmentResultValueClass( + Lists.newArrayList(results), + segmentId, + interval + ) + ); + } + } + ) + ); + } + + private Sequence> toQueryableSearchResults( + Iterable segmentIds, Iterable intervals, Iterable>> results + ) + { + return Sequences.simple( + FunctionalIterable + .create(segmentIds) + .trinaryTransform( + intervals, + results, + new TrinaryFn>, Result>() + { + @Override + @SuppressWarnings("unchecked") + public Result apply( + final String segmentId, + final Interval interval, + final Iterable> results + ) + { + return new Result( + results.iterator().next().getTimestamp(), + new BySegmentResultValueClass( + Lists.newArrayList(results), + segmentId, + interval + ) + ); + } + } + ) + ); + } + + private Sequence> toQueryableTimeBoundaryResults( + Iterable segmentIds, + Iterable intervals, + Iterable>> results + ) + { + return Sequences.simple( + FunctionalIterable + .create(segmentIds) + .trinaryTransform( + intervals, + results, + new TrinaryFn>, Result>() + { + @Override + @SuppressWarnings("unchecked") + public Result apply( + final String segmentId, + final Interval interval, + final Iterable> results + ) + { + return new Result( + results.iterator().next().getTimestamp(), + new BySegmentResultValueClass( + Lists.newArrayList(results), + segmentId, + interval + ) + ); + } + } + ) + ); + } + + private Iterable> makeTimeResults + (Object... objects) + { + if (objects.length % 3 != 0) { + throw new ISE("makeTimeResults must be passed arguments in groups of 3, got[%d]", objects.length); + } + + List> retVal = Lists.newArrayListWithCapacity(objects.length / 3); + for (int i = 0; i < objects.length; i += 3) { + retVal.add( + new Result<>( + (DateTime) objects[i], + new TimeseriesResultValue( + ImmutableMap.of( + "rows", objects[i + 1], + "imps", objects[i + 2], + "impers", objects[i + 2], + "avg_imps_per_row", + ((Number) objects[i + 2]).doubleValue() / ((Number) objects[i + 1]).doubleValue() + ) + ) + ) + ); + } + return retVal; + } + + private Iterable> makeRenamedTimeResults + (Object... objects) + { + if (objects.length % 3 != 0) { + throw new ISE("makeTimeResults must be passed arguments in groups of 3, got[%d]", objects.length); + } + + List> retVal = Lists.newArrayListWithCapacity(objects.length / 3); + for (int i = 0; i < objects.length; i += 3) { + retVal.add( + new Result<>( + (DateTime) objects[i], + new TimeseriesResultValue( + ImmutableMap.of( + "rows2", objects[i + 1], + "imps", objects[i + 2], + "impers2", objects[i + 2] + ) + ) + ) + ); + } + return retVal; + } + + private Iterable> makeTopNResults + (Object... objects) + { + List> retVal = Lists.newArrayList(); + int index = 0; + while (index < objects.length) { + DateTime timestamp = (DateTime) objects[index++]; + + List> values = Lists.newArrayList(); + while (index < objects.length && !(objects[index] instanceof DateTime)) { + if (objects.length - index < 3) { + throw new ISE( + "expect 3 values for each entry in the top list, had %d values left.", objects.length - index + ); + } + final double imps = ((Number) objects[index + 2]).doubleValue(); + final double rows = ((Number) objects[index + 1]).doubleValue(); + values.add( + ImmutableMap.of( + TOP_DIM, objects[index], + "rows", rows, + "imps", imps, + "impers", imps, + "avg_imps_per_row", imps / rows + ) + ); + index += 3; + } + + retVal.add(new Result<>(timestamp, new TopNResultValue(values))); + } + return retVal; + } + + private Iterable> makeRenamedTopNResults + (Object... objects) + { + List> retVal = Lists.newArrayList(); + int index = 0; + while (index < objects.length) { + DateTime timestamp = (DateTime) objects[index++]; + + List> values = Lists.newArrayList(); + while (index < objects.length && !(objects[index] instanceof DateTime)) { + if (objects.length - index < 3) { + throw new ISE( + "expect 3 values for each entry in the top list, had %d values left.", objects.length - index + ); + } + final double imps = ((Number) objects[index + 2]).doubleValue(); + final double rows = ((Number) objects[index + 1]).doubleValue(); + values.add( + ImmutableMap.of( + TOP_DIM, objects[index], + "rows2", rows, + "imps", imps, + "impers2", imps + ) + ); + index += 3; + } + + retVal.add(new Result<>(timestamp, new TopNResultValue(values))); + } + return retVal; + } + + private Iterable> makeSearchResults + (Object... objects) + { + List> retVal = Lists.newArrayList(); + int index = 0; + while (index < objects.length) { + DateTime timestamp = (DateTime) objects[index++]; + + List values = Lists.newArrayList(); + while (index < objects.length && !(objects[index] instanceof DateTime)) { + values.add(new SearchHit(TOP_DIM, objects[index++].toString())); + } + + retVal.add(new Result<>(timestamp, new SearchResultValue(values))); + } + return retVal; + } + + private T makeMock(List mocks, Class clazz) + { + T obj = EasyMock.createMock(clazz); + mocks.add(obj); + return obj; + } + + private void runWithMocks(Runnable toRun, Object... mocks) + { + EasyMock.replay(mocks); + + toRun.run(); + + EasyMock.verify(mocks); + EasyMock.reset(mocks); + } + + protected CachingClusteredClient makeClient() + { + return new CachingClusteredClient( + new MapQueryToolChestWarehouse( + ImmutableMap., QueryToolChest>builder() + .put( + TimeseriesQuery.class, + new TimeseriesQueryQueryToolChest(new QueryConfig()) + ) + .put(TopNQuery.class, new TopNQueryQueryToolChest(new TopNQueryConfig())) + .put(SearchQuery.class, new SearchQueryQueryToolChest(new SearchQueryConfig())) + .build() + ), + new TimelineServerView() + { + @Override + public void registerSegmentCallback(Executor exec, SegmentCallback callback) + { + } + + @Override + public VersionedIntervalTimeline getTimeline(String dataSource) + { + return timeline; + } + + @Override + public QueryRunner getQueryRunner(DruidServer server) + { + return serverView.getQueryRunner(server); + } + + @Override + public void registerServerCallback(Executor exec, ServerCallback callback) + { + + } + }, + cache, + jsonMapper + ); + } + + private static class ServerExpectation + { + private final String segmentId; + private final Interval interval; + private final DataSegment segment; + private final Iterable> results; + + public ServerExpectation( + String segmentId, + Interval interval, + DataSegment segment, + Iterable> results + ) + { + this.segmentId = segmentId; + this.interval = interval; + this.segment = segment; + this.results = results; + } + + public String getSegmentId() + { + return segmentId; + } + + public Interval getInterval() + { + return interval; + } + + public DataSegment getSegment() + { + return new MyDataSegment(); + } + + public Iterable> getResults() + { + return results; + } + + private class MyDataSegment extends DataSegment + { + private MyDataSegment() + { + super( + "", + new Interval(0, 1), + "", + null, + null, + null, + new NoneShardSpec(), + null, + -1 + ); + } + + private final DataSegment baseSegment = segment; + + @Override + @JsonProperty + public String getDataSource() + { + return baseSegment.getDataSource(); + } + + @Override + @JsonProperty + public Interval getInterval() + { + return baseSegment.getInterval(); + } + + @Override + @JsonProperty + public Map getLoadSpec() + { + return baseSegment.getLoadSpec(); + } + + @Override + @JsonProperty + public String getVersion() + { + return baseSegment.getVersion(); + } + + @Override + @JsonSerialize + @JsonProperty + public List getDimensions() + { + return baseSegment.getDimensions(); + } + + @Override + @JsonSerialize + @JsonProperty + public List getMetrics() + { + return baseSegment.getMetrics(); + } + + @Override + @JsonProperty + public ShardSpec getShardSpec() + { + return baseSegment.getShardSpec(); + } + + @Override + @JsonProperty + public long getSize() + { + return baseSegment.getSize(); + } + + @Override + public String getIdentifier() + { + return segmentId; + } + + @Override + public SegmentDescriptor toDescriptor() + { + return baseSegment.toDescriptor(); + } + + @Override + public int compareTo(DataSegment dataSegment) + { + return baseSegment.compareTo(dataSegment); + } + + @Override + public boolean equals(Object o) + { + return baseSegment.equals(o); + } + + @Override + public int hashCode() + { + return baseSegment.hashCode(); + } + + @Override + public String toString() + { + return baseSegment.toString(); + } + } + } + + private static class ServerExpectations implements Iterable + { + private final DruidServer server; + private final QueryRunner queryRunner; + + private final List expectations = Lists.newArrayList(); + + public ServerExpectations( + DruidServer server, + QueryRunner queryRunner + ) + { + this.server = server; + this.queryRunner = queryRunner; + } + + public DruidServer getServer() + { + return server; + } + + public QueryRunner getQueryRunner() + { + return queryRunner; + } + + public List getExpectations() + { + return expectations; + } + + public void addExpectation( + ServerExpectation expectation + ) + { + expectations.add(expectation); + } + + @Override + public Iterator iterator() + { + return expectations.iterator(); + } + } +} diff --git a/server/src/test/java/io/druid/client/RangeIterable.java b/server/src/test/java/io/druid/client/RangeIterable.java new file mode 100644 index 00000000000..132e0cb013d --- /dev/null +++ b/server/src/test/java/io/druid/client/RangeIterable.java @@ -0,0 +1,89 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 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.client; + +import java.util.Iterator; + +/** + */ +public class RangeIterable implements Iterable +{ + private final int end; + private final int start; + private final int increment; + + public RangeIterable( + int end + ) + { + this(0, end); + } + + public RangeIterable( + int start, + int end + ) + { + this(start, end, 1); + } + + public RangeIterable( + int start, + int end, + final int i + ) + { + this.start = start; + this.end = end; + this.increment = i; + } + + @Override + public Iterator iterator() + { + return new Iterator() + { + private int curr = start; + + @Override + public boolean hasNext() + { + return curr < end; + } + + @Override + public Integer next() + { + try { + return curr; + } + finally { + curr += increment; + } + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + }; + } +} From 9fc5cca663db94577a579a40cc8fac3758898eba Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 5 Mar 2014 13:58:38 -0800 Subject: [PATCH 12/66] add disable populate / use cache tests --- .../client/CachingClusteredClientTest.java | 215 ++++++++++++++---- 1 file changed, 168 insertions(+), 47 deletions(-) diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index 9a5154e5824..78cfa370df1 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Function; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; @@ -308,6 +309,56 @@ public class CachingClusteredClientTest ); } + @Test + public void testDisableUseCache() throws Exception + { + final Druids.TimeseriesQueryBuilder builder = Druids.newTimeseriesQueryBuilder() + .dataSource(DATA_SOURCE) + .intervals(SEG_SPEC) + .filters(DIM_FILTER) + .granularity(GRANULARITY) + .aggregators(AGGS) + .postAggregators(POST_AGGS); + + testQueryCaching( + 1, + true, + builder.context(ImmutableMap.of("useCache", "false", + "populateCache", "true")).build(), + new Interval("2011-01-01/2011-01-02"), makeTimeResults(new DateTime("2011-01-01"), 50, 5000) + ); + + Assert.assertEquals(1, cache.getStats().getNumEntries()); + Assert.assertEquals(0, cache.getStats().getNumHits()); + Assert.assertEquals(0, cache.getStats().getNumMisses()); + + cache.close("0_0"); + + testQueryCaching( + 1, + false, + builder.context(ImmutableMap.of("useCache", "false", + "populateCache", "false")).build(), + new Interval("2011-01-01/2011-01-02"), makeTimeResults(new DateTime("2011-01-01"), 50, 5000) + ); + + Assert.assertEquals(0, cache.getStats().getNumEntries()); + Assert.assertEquals(0, cache.getStats().getNumHits()); + Assert.assertEquals(0, cache.getStats().getNumMisses()); + + testQueryCaching( + 1, + false, + builder.context(ImmutableMap.of("useCache", "true", + "populateCache", "false")).build(), + new Interval("2011-01-01/2011-01-02"), makeTimeResults(new DateTime("2011-01-01"), 50, 5000) + ); + + Assert.assertEquals(0, cache.getStats().getNumEntries()); + Assert.assertEquals(0, cache.getStats().getNumHits()); + Assert.assertEquals(1, cache.getStats().getNumMisses()); + } + @Test @SuppressWarnings("unchecked") public void testTopNCaching() throws Exception @@ -528,9 +579,15 @@ public class CachingClusteredClientTest ); } + public void testQueryCaching(final Query query, Object... args) { + testQueryCaching(3, true, query, args); + } + @SuppressWarnings("unchecked") public void testQueryCaching( - final Query query, Object... args + final int numTimesToQuery, + boolean expectBySegment, + final Query query, Object... args // does this assume query intervals must be ordered? ) { if (args.length % 2 != 0) { @@ -553,9 +610,6 @@ public class CachingClusteredClientTest } for (int i = 0; i < queryIntervals.size(); ++i) { - timeline = new VersionedIntervalTimeline<>(Ordering.natural()); - final int numTimesToQuery = 3; - List mocks = Lists.newArrayList(); mocks.add(serverView); @@ -563,46 +617,12 @@ public class CachingClusteredClientTest queryIntervals.get(0).getStart(), queryIntervals.get(i).getEnd() ); - final List> serverExpectationList = Lists.newArrayList(); - - for (int k = 0; k < i + 1; ++k) { - final int numChunks = expectedResults.get(k).size(); - final TreeMap serverExpectations = Maps.newTreeMap(); - serverExpectationList.add(serverExpectations); - for (int j = 0; j < numChunks; ++j) { - DruidServer lastServer = servers[random.nextInt(servers.length)]; - if (!serverExpectations.containsKey(lastServer)) { - serverExpectations.put(lastServer, new ServerExpectations(lastServer, makeMock(mocks, QueryRunner.class))); - } - - ServerExpectation expectation = new ServerExpectation( - String.format("%s_%s", k, j), - queryIntervals.get(i), - makeMock(mocks, DataSegment.class), - expectedResults.get(k).get(j) - ); - serverExpectations.get(lastServer).addExpectation(expectation); - - ServerSelector selector = new ServerSelector(expectation.getSegment(), new RandomServerSelectorStrategy()); - selector.addServer(new QueryableDruidServer(lastServer, null)); - - final PartitionChunk chunk; - if (numChunks == 1) { - chunk = new SingleElementPartitionChunk<>(selector); - } else { - String start = null; - String end = null; - if (j > 0) { - start = String.valueOf(j - 1); - } - if (j + 1 < numChunks) { - end = String.valueOf(j); - } - chunk = new StringPartitionChunk<>(start, end, j, selector); - } - timeline.add(queryIntervals.get(k), String.valueOf(k), chunk); - } - } + final List> serverExpectationList = populateTimeline( + queryIntervals, + expectedResults, + i, + mocks + ); List queryCaptures = Lists.newArrayList(); final Map finalExpectation = serverExpectationList.get( @@ -612,7 +632,10 @@ public class CachingClusteredClientTest DruidServer server = entry.getKey(); ServerExpectations expectations = entry.getValue(); - EasyMock.expect(serverView.getQueryRunner(server)).andReturn(expectations.getQueryRunner()).once(); + + EasyMock.expect(serverView.getQueryRunner(server)) + .andReturn(expectations.getQueryRunner()) + .once(); final Capture capture = new Capture(); queryCaptures.add(capture); @@ -627,9 +650,11 @@ public class CachingClusteredClientTest intervals.add(expectation.getInterval()); results.add(expectation.getResults()); } + EasyMock.expect(queryable.run(EasyMock.capture(capture))) - .andReturn(toQueryableTimeseriesResults(segmentIds, intervals, results)) + .andReturn(toQueryableTimeseriesResults(expectBySegment, segmentIds, intervals, results)) .once(); + } else if (query instanceof TopNQuery) { List segmentIds = Lists.newArrayList(); List intervals = Lists.newArrayList(); @@ -729,19 +754,81 @@ public class CachingClusteredClientTest mocks.toArray() ); + // make sure all the queries were sent down as 'bySegment' for (Capture queryCapture : queryCaptures) { Query capturedQuery = (Query) queryCapture.getValue(); - Assert.assertEquals("true", capturedQuery.getContextValue("bySegment")); + if(expectBySegment) { + Assert.assertEquals("true", capturedQuery.getContextValue("bySegment")); + } + else { + Assert.assertTrue( + capturedQuery.getContextValue("bySegment") == null || + capturedQuery.getContextValue("bySegment").equals("false") + ); + } } } } + private List> populateTimeline( + List queryIntervals, + List>>> expectedResults, + int numQueryIntervals, + List mocks + ) + { + timeline = new VersionedIntervalTimeline<>(Ordering.natural()); + + final List> serverExpectationList = Lists.newArrayList(); + + for (int k = 0; k < numQueryIntervals + 1; ++k) { + final int numChunks = expectedResults.get(k).size(); + final TreeMap serverExpectations = Maps.newTreeMap(); + serverExpectationList.add(serverExpectations); + for (int j = 0; j < numChunks; ++j) { + DruidServer lastServer = servers[random.nextInt(servers.length)]; + if (!serverExpectations.containsKey(lastServer)) { + serverExpectations.put(lastServer, new ServerExpectations(lastServer, makeMock(mocks, QueryRunner.class))); + } + + ServerExpectation expectation = new ServerExpectation( + String.format("%s_%s", k, j), // interval/chunk + queryIntervals.get(numQueryIntervals), + makeMock(mocks, DataSegment.class), + expectedResults.get(k).get(j) + ); + serverExpectations.get(lastServer).addExpectation(expectation); + + ServerSelector selector = new ServerSelector(expectation.getSegment(), new RandomServerSelectorStrategy()); + selector.addServer(new QueryableDruidServer(lastServer, null)); + + final PartitionChunk chunk; + if (numChunks == 1) { + chunk = new SingleElementPartitionChunk<>(selector); + } else { + String start = null; + String end = null; + if (j > 0) { + start = String.valueOf(j - 1); + } + if (j + 1 < numChunks) { + end = String.valueOf(j); + } + chunk = new StringPartitionChunk<>(start, end, j, selector); + } + timeline.add(queryIntervals.get(k), String.valueOf(k), chunk); + } + } return serverExpectationList; + } + private Sequence> toQueryableTimeseriesResults( + boolean bySegment, Iterable segmentIds, Iterable intervals, Iterable>> results ) { + if(bySegment) { return Sequences.simple( FunctionalIterable .create(segmentIds) @@ -770,6 +857,9 @@ public class CachingClusteredClientTest } ) ); + } else { + return Sequences.simple(Iterables.concat(results)); + } } private Sequence> toQueryableTopNResults( @@ -903,6 +993,37 @@ public class CachingClusteredClientTest return retVal; } + private Iterable> makeBySegmentTimeResults + (Object... objects) + { + if (objects.length % 5 != 0) { + throw new ISE("makeTimeResults must be passed arguments in groups of 5, got[%d]", objects.length); + } + + List> retVal = Lists.newArrayListWithCapacity(objects.length / 5); + for (int i = 0; i < objects.length; i += 5) { + retVal.add( + new BySegmentResultValueClass( + Lists.newArrayList( + new TimeseriesResultValue( + ImmutableMap.of( + "rows", objects[i + 1], + "imps", objects[i + 2], + "impers", objects[i + 2], + "avg_imps_per_row", + ((Number) objects[i + 2]).doubleValue() / ((Number) objects[i + 1]).doubleValue() + ) + ) + ), + (String)objects[i+3], + (Interval)objects[i+4] + + ) + ); + } + return retVal; + } + private Iterable> makeRenamedTimeResults (Object... objects) { From 08138688e4b1e4a95c3a4a227dce13024d4eddcb Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 5 Mar 2014 14:19:38 -0800 Subject: [PATCH 13/66] port hyperunique to open source --- docs/content/Aggregations.md | 10 + docs/content/Post-aggregations.md | 27 +- .../io/druid/jackson/AggregatorsModule.java | 39 +- .../hyperloglog/ByteBitLookup.java | 269 ++++++ .../query/aggregation/hyperloglog/HLLCV0.java | 132 +++ .../query/aggregation/hyperloglog/HLLCV1.java | 145 ++++ .../hyperloglog/HyperLogLogCollector.java | 654 +++++++++++++++ .../HyperUniqueFinalizingPostAggregator.java | 50 ++ .../hyperloglog/HyperUniquesAggregator.java | 67 ++ .../HyperUniquesAggregatorFactory.java | 191 +++++ .../HyperUniquesBufferAggregator.java | 68 ++ .../hyperloglog/HyperUniquesSerde.java | 129 +++ .../io/druid/query/QueryRunnerTestHelper.java | 19 +- .../hyperloglog/HyperLogLogCollectorTest.java | 774 ++++++++++++++++++ ...perUniqueFinalizingPostAggregatorTest.java | 35 + .../HyperUniquesAggregatorFactoryTest.java | 29 + .../timeseries/TimeseriesQueryRunnerTest.java | 146 +++- .../druid/query/topn/TopNQueryRunnerTest.java | 598 ++++++++------ .../test/java/io/druid/segment/TestIndex.java | 23 +- 19 files changed, 3095 insertions(+), 310 deletions(-) create mode 100644 processing/src/main/java/io/druid/query/aggregation/hyperloglog/ByteBitLookup.java create mode 100644 processing/src/main/java/io/druid/query/aggregation/hyperloglog/HLLCV0.java create mode 100644 processing/src/main/java/io/druid/query/aggregation/hyperloglog/HLLCV1.java create mode 100644 processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java create mode 100644 processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregator.java create mode 100644 processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregator.java create mode 100644 processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java create mode 100644 processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java create mode 100644 processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java create mode 100644 processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollectorTest.java create mode 100644 processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregatorTest.java create mode 100644 processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactoryTest.java diff --git a/docs/content/Aggregations.md b/docs/content/Aggregations.md index 93bfb76c90e..74ad226ff81 100644 --- a/docs/content/Aggregations.md +++ b/docs/content/Aggregations.md @@ -82,3 +82,13 @@ All JavaScript functions must return numerical values. "fnReset" : "function() { return 10; }" } ``` + +### Complex aggregators + +#### `hyperUnique` aggregator + +`hyperUnique` uses [Hyperloglog](http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf) to compute the estimated cardinality of a dimension. + +```json +{ "type" : "hyperUnique", "name" : , "fieldName" : } +``` diff --git a/docs/content/Post-aggregations.md b/docs/content/Post-aggregations.md index 527d64e7971..4dce46ceff1 100644 --- a/docs/content/Post-aggregations.md +++ b/docs/content/Post-aggregations.md @@ -64,6 +64,31 @@ Example JavaScript aggregator: "function": "function(delta, total) { return 100 * Math.abs(delta) / total; }" } ``` +### `hyperUniqueCardinality` post-aggregator + +The hyperUniqueCardinality post aggregator is used to wrap a hyperUnique object such that it can be used in post aggregations. + +```json +{ "type" : "hyperUniqueCardinality", "fieldName" : } +``` + +It can be used in a sample calculation as so: + +```json + "aggregations" : [{ + {"type" : "count", "name" : "rows"}, + {"type" : "hyperUnique", "name" : "unique_users", "fieldName" : "uniques"} + }], + "postAggregations" : { + "type" : "arithmetic", + "name" : "average_users_per_row", + "fn" : "/", + "fields" : [ + { "type" : "hyperUniqueCardinality", "fieldName" : "unique_users" }, + { "type" : "fieldAccess", "name" : "rows", "fieldName" : "rows" } + ] + } +``` ### Example Usage @@ -98,4 +123,4 @@ The format of the query JSON is as follows: ... } -``` +``` \ No newline at end of file diff --git a/processing/src/main/java/io/druid/jackson/AggregatorsModule.java b/processing/src/main/java/io/druid/jackson/AggregatorsModule.java index 15a76639997..3029d2bcc4e 100644 --- a/processing/src/main/java/io/druid/jackson/AggregatorsModule.java +++ b/processing/src/main/java/io/druid/jackson/AggregatorsModule.java @@ -22,6 +22,7 @@ package io.druid.jackson; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.hash.Hashing; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -31,10 +32,14 @@ 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.hyperloglog.HyperUniqueFinalizingPostAggregator; +import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; 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.aggregation.post.JavaScriptPostAggregator; +import io.druid.segment.serde.ComplexMetrics; /** */ @@ -44,28 +49,38 @@ public class AggregatorsModule extends SimpleModule { super("AggregatorFactories"); + if (ComplexMetrics.getSerdeForType("hyperUnique") == null) { + ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde(Hashing.murmur3_128())); + } + setMixInAnnotation(AggregatorFactory.class, AggregatorFactoryMixin.class); setMixInAnnotation(PostAggregator.class, PostAggregatorMixin.class); } - @JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type") - @JsonSubTypes(value={ - @JsonSubTypes.Type(name="count", value=CountAggregatorFactory.class), - @JsonSubTypes.Type(name="longSum", value=LongSumAggregatorFactory.class), - @JsonSubTypes.Type(name="doubleSum", value=DoubleSumAggregatorFactory.class), - @JsonSubTypes.Type(name="max", value=MaxAggregatorFactory.class), - @JsonSubTypes.Type(name="min", value=MinAggregatorFactory.class), - @JsonSubTypes.Type(name="javascript", value=JavaScriptAggregatorFactory.class), - @JsonSubTypes.Type(name="histogram", value=HistogramAggregatorFactory.class) + @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") + @JsonSubTypes(value = { + @JsonSubTypes.Type(name = "count", value = CountAggregatorFactory.class), + @JsonSubTypes.Type(name = "longSum", value = LongSumAggregatorFactory.class), + @JsonSubTypes.Type(name = "doubleSum", value = DoubleSumAggregatorFactory.class), + @JsonSubTypes.Type(name = "max", value = MaxAggregatorFactory.class), + @JsonSubTypes.Type(name = "min", value = MinAggregatorFactory.class), + @JsonSubTypes.Type(name = "javascript", value = JavaScriptAggregatorFactory.class), + @JsonSubTypes.Type(name = "histogram", value = HistogramAggregatorFactory.class), + @JsonSubTypes.Type(name = "hyperUnique", value = HyperUniquesAggregatorFactory.class) }) - public static interface AggregatorFactoryMixin {} + public static interface AggregatorFactoryMixin + { + } @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @JsonSubTypes.Type(name = "arithmetic", value = ArithmeticPostAggregator.class), @JsonSubTypes.Type(name = "fieldAccess", value = FieldAccessPostAggregator.class), @JsonSubTypes.Type(name = "constant", value = ConstantPostAggregator.class), - @JsonSubTypes.Type(name = "javascript", value = JavaScriptPostAggregator.class) + @JsonSubTypes.Type(name = "javascript", value = JavaScriptPostAggregator.class), + @JsonSubTypes.Type(name = "hyperUniqueCardinality", value = HyperUniqueFinalizingPostAggregator.class) }) - public static interface PostAggregatorMixin {} + public static interface PostAggregatorMixin + { + } } diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/ByteBitLookup.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/ByteBitLookup.java new file mode 100644 index 00000000000..d5577ab046b --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/ByteBitLookup.java @@ -0,0 +1,269 @@ +package io.druid.query.aggregation.hyperloglog; + +/** + */ +public class ByteBitLookup +{ + public static final byte[] lookup; + + static { + lookup = new byte[256]; + + lookup[0] = 0; + lookup[1] = 1; + lookup[2] = 2; + lookup[3] = 1; + lookup[4] = 3; + lookup[5] = 1; + lookup[6] = 2; + lookup[7] = 1; + lookup[8] = 4; + lookup[9] = 1; + lookup[10] = 2; + lookup[11] = 1; + lookup[12] = 3; + lookup[13] = 1; + lookup[14] = 2; + lookup[15] = 1; + lookup[16] = 5; + lookup[17] = 1; + lookup[18] = 2; + lookup[19] = 1; + lookup[20] = 3; + lookup[21] = 1; + lookup[22] = 2; + lookup[23] = 1; + lookup[24] = 4; + lookup[25] = 1; + lookup[26] = 2; + lookup[27] = 1; + lookup[28] = 3; + lookup[29] = 1; + lookup[30] = 2; + lookup[31] = 1; + lookup[32] = 6; + lookup[33] = 1; + lookup[34] = 2; + lookup[35] = 1; + lookup[36] = 3; + lookup[37] = 1; + lookup[38] = 2; + lookup[39] = 1; + lookup[40] = 4; + lookup[41] = 1; + lookup[42] = 2; + lookup[43] = 1; + lookup[44] = 3; + lookup[45] = 1; + lookup[46] = 2; + lookup[47] = 1; + lookup[48] = 5; + lookup[49] = 1; + lookup[50] = 2; + lookup[51] = 1; + lookup[52] = 3; + lookup[53] = 1; + lookup[54] = 2; + lookup[55] = 1; + lookup[56] = 4; + lookup[57] = 1; + lookup[58] = 2; + lookup[59] = 1; + lookup[60] = 3; + lookup[61] = 1; + lookup[62] = 2; + lookup[63] = 1; + lookup[64] = 7; + lookup[65] = 1; + lookup[66] = 2; + lookup[67] = 1; + lookup[68] = 3; + lookup[69] = 1; + lookup[70] = 2; + lookup[71] = 1; + lookup[72] = 4; + lookup[73] = 1; + lookup[74] = 2; + lookup[75] = 1; + lookup[76] = 3; + lookup[77] = 1; + lookup[78] = 2; + lookup[79] = 1; + lookup[80] = 5; + lookup[81] = 1; + lookup[82] = 2; + lookup[83] = 1; + lookup[84] = 3; + lookup[85] = 1; + lookup[86] = 2; + lookup[87] = 1; + lookup[88] = 4; + lookup[89] = 1; + lookup[90] = 2; + lookup[91] = 1; + lookup[92] = 3; + lookup[93] = 1; + lookup[94] = 2; + lookup[95] = 1; + lookup[96] = 6; + lookup[97] = 1; + lookup[98] = 2; + lookup[99] = 1; + lookup[100] = 3; + lookup[101] = 1; + lookup[102] = 2; + lookup[103] = 1; + lookup[104] = 4; + lookup[105] = 1; + lookup[106] = 2; + lookup[107] = 1; + lookup[108] = 3; + lookup[109] = 1; + lookup[110] = 2; + lookup[111] = 1; + lookup[112] = 5; + lookup[113] = 1; + lookup[114] = 2; + lookup[115] = 1; + lookup[116] = 3; + lookup[117] = 1; + lookup[118] = 2; + lookup[119] = 1; + lookup[120] = 4; + lookup[121] = 1; + lookup[122] = 2; + lookup[123] = 1; + lookup[124] = 3; + lookup[125] = 1; + lookup[126] = 2; + lookup[127] = 1; + lookup[128] = 8; + lookup[129] = 1; + lookup[130] = 2; + lookup[131] = 1; + lookup[132] = 3; + lookup[133] = 1; + lookup[134] = 2; + lookup[135] = 1; + lookup[136] = 4; + lookup[137] = 1; + lookup[138] = 2; + lookup[139] = 1; + lookup[140] = 3; + lookup[141] = 1; + lookup[142] = 2; + lookup[143] = 1; + lookup[144] = 5; + lookup[145] = 1; + lookup[146] = 2; + lookup[147] = 1; + lookup[148] = 3; + lookup[149] = 1; + lookup[150] = 2; + lookup[151] = 1; + lookup[152] = 4; + lookup[153] = 1; + lookup[154] = 2; + lookup[155] = 1; + lookup[156] = 3; + lookup[157] = 1; + lookup[158] = 2; + lookup[159] = 1; + lookup[160] = 6; + lookup[161] = 1; + lookup[162] = 2; + lookup[163] = 1; + lookup[164] = 3; + lookup[165] = 1; + lookup[166] = 2; + lookup[167] = 1; + lookup[168] = 4; + lookup[169] = 1; + lookup[170] = 2; + lookup[171] = 1; + lookup[172] = 3; + lookup[173] = 1; + lookup[174] = 2; + lookup[175] = 1; + lookup[176] = 5; + lookup[177] = 1; + lookup[178] = 2; + lookup[179] = 1; + lookup[180] = 3; + lookup[181] = 1; + lookup[182] = 2; + lookup[183] = 1; + lookup[184] = 4; + lookup[185] = 1; + lookup[186] = 2; + lookup[187] = 1; + lookup[188] = 3; + lookup[189] = 1; + lookup[190] = 2; + lookup[191] = 1; + lookup[192] = 7; + lookup[193] = 1; + lookup[194] = 2; + lookup[195] = 1; + lookup[196] = 3; + lookup[197] = 1; + lookup[198] = 2; + lookup[199] = 1; + lookup[200] = 4; + lookup[201] = 1; + lookup[202] = 2; + lookup[203] = 1; + lookup[204] = 3; + lookup[205] = 1; + lookup[206] = 2; + lookup[207] = 1; + lookup[208] = 5; + lookup[209] = 1; + lookup[210] = 2; + lookup[211] = 1; + lookup[212] = 3; + lookup[213] = 1; + lookup[214] = 2; + lookup[215] = 1; + lookup[216] = 4; + lookup[217] = 1; + lookup[218] = 2; + lookup[219] = 1; + lookup[220] = 3; + lookup[221] = 1; + lookup[222] = 2; + lookup[223] = 1; + lookup[224] = 6; + lookup[225] = 1; + lookup[226] = 2; + lookup[227] = 1; + lookup[228] = 3; + lookup[229] = 1; + lookup[230] = 2; + lookup[231] = 1; + lookup[232] = 4; + lookup[233] = 1; + lookup[234] = 2; + lookup[235] = 1; + lookup[236] = 3; + lookup[237] = 1; + lookup[238] = 2; + lookup[239] = 1; + lookup[240] = 5; + lookup[241] = 1; + lookup[242] = 2; + lookup[243] = 1; + lookup[244] = 3; + lookup[245] = 1; + lookup[246] = 2; + lookup[247] = 1; + lookup[248] = 4; + lookup[249] = 1; + lookup[250] = 2; + lookup[251] = 1; + lookup[252] = 3; + lookup[253] = 1; + lookup[254] = 2; + lookup[255] = 1; + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HLLCV0.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HLLCV0.java new file mode 100644 index 00000000000..67aefd73f02 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HLLCV0.java @@ -0,0 +1,132 @@ +package io.druid.query.aggregation.hyperloglog; + +import java.nio.ByteBuffer; + +/** + */ +public class HLLCV0 extends HyperLogLogCollector +{ + /** + * Header: + * Byte 0: registerOffset + * Byte 1-2: numNonZeroRegisters + */ + public static final int NUM_NON_ZERO_REGISTERS_BYTE = 1; + public static final int HEADER_NUM_BYTES = 3; + public static final int NUM_BYTES_FOR_DENSE_STORAGE = NUM_BYTES_FOR_BUCKETS + HEADER_NUM_BYTES; + + private static final ByteBuffer defaultStorageBuffer = ByteBuffer.wrap(new byte[]{0, 0, 0}).asReadOnlyBuffer(); + + public HLLCV0() + { + super(defaultStorageBuffer); + } + + public HLLCV0(ByteBuffer buffer) + { + super(buffer); + } + + @Override + public byte getVersion() + { + return 0; + } + + @Override + public void setVersion(ByteBuffer buffer) + { + } + + @Override + public byte getRegisterOffset() + { + return getStorageBuffer().get(getInitPosition()); + } + + @Override + public void setRegisterOffset(byte registerOffset) + { + getStorageBuffer().put(getInitPosition(), registerOffset); + } + + @Override + public void setRegisterOffset(ByteBuffer buffer, byte registerOffset) + { + buffer.put(buffer.position(), registerOffset); + } + + @Override + public short getNumNonZeroRegisters() + { + return getStorageBuffer().getShort(getInitPosition() + NUM_NON_ZERO_REGISTERS_BYTE); + } + + @Override + public void setNumNonZeroRegisters(short numNonZeroRegisters) + { + getStorageBuffer().putShort(getInitPosition() + NUM_NON_ZERO_REGISTERS_BYTE, numNonZeroRegisters); + } + + @Override + public void setNumNonZeroRegisters(ByteBuffer buffer, short numNonZeroRegisters) + { + buffer.putShort(buffer.position() + NUM_NON_ZERO_REGISTERS_BYTE, numNonZeroRegisters); + } + + @Override + public byte getMaxOverflowValue() + { + return 0; + } + + @Override + public void setMaxOverflowValue(byte value) + { + } + + @Override + public void setMaxOverflowValue(ByteBuffer buffer, byte value) + { + } + + @Override + public short getMaxOverflowRegister() + { + return 0; + } + + @Override + public void setMaxOverflowRegister(short register) + { + } + + @Override + public void setMaxOverflowRegister(ByteBuffer buffer, short register) + { + } + + @Override + public int getNumHeaderBytes() + { + return HEADER_NUM_BYTES; + } + + @Override + public int getNumBytesForDenseStorage() + { + return NUM_BYTES_FOR_DENSE_STORAGE; + } + + @Override + public int getPayloadBytePosition() + { + return getInitPosition() + HEADER_NUM_BYTES; + } + + @Override + public int getPayloadBytePosition(ByteBuffer buffer) + { + return buffer.position() + HEADER_NUM_BYTES; + } +} \ No newline at end of file diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HLLCV1.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HLLCV1.java new file mode 100644 index 00000000000..421f10140de --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HLLCV1.java @@ -0,0 +1,145 @@ +package io.druid.query.aggregation.hyperloglog; + +import java.nio.ByteBuffer; + +/** + */ +public class HLLCV1 extends HyperLogLogCollector +{ + /** + * Header: + * Byte 0: version + * Byte 1: registerOffset + * Byte 2-3: numNonZeroRegisters + * Byte 4: maxOverflowValue + * Byte 5-6: maxOverflowRegister + */ + public static final byte VERSION = 0x1; + public static final int REGISTER_OFFSET_BYTE = 1; + public static final int NUM_NON_ZERO_REGISTERS_BYTE = 2; + public static final int MAX_OVERFLOW_VALUE_BYTE = 4; + public static final int MAX_OVERFLOW_REGISTER_BYTE = 5; + public static final int HEADER_NUM_BYTES = 7; + public static final int NUM_BYTES_FOR_DENSE_STORAGE = NUM_BYTES_FOR_BUCKETS + HEADER_NUM_BYTES; + + private static final ByteBuffer defaultStorageBuffer = ByteBuffer.wrap(new byte[]{VERSION, 0, 0, 0, 0, 0, 0}) + .asReadOnlyBuffer(); + + public HLLCV1() + { + super(defaultStorageBuffer); + } + + public HLLCV1(ByteBuffer buffer) + { + super(buffer); + } + + @Override + public byte getVersion() + { + return VERSION; + } + + @Override + public void setVersion(ByteBuffer buffer) + { + buffer.put(buffer.position(), VERSION); + } + + @Override + public byte getRegisterOffset() + { + return getStorageBuffer().get(getInitPosition() + REGISTER_OFFSET_BYTE); + } + + @Override + public void setRegisterOffset(byte registerOffset) + { + getStorageBuffer().put(getInitPosition() + REGISTER_OFFSET_BYTE, registerOffset); + } + + @Override + public void setRegisterOffset(ByteBuffer buffer, byte registerOffset) + { + buffer.put(buffer.position() + REGISTER_OFFSET_BYTE, registerOffset); + } + + @Override + public short getNumNonZeroRegisters() + { + return getStorageBuffer().getShort(getInitPosition() + NUM_NON_ZERO_REGISTERS_BYTE); + } + + @Override + public void setNumNonZeroRegisters(short numNonZeroRegisters) + { + getStorageBuffer().putShort(getInitPosition() + NUM_NON_ZERO_REGISTERS_BYTE, numNonZeroRegisters); + } + + @Override + public void setNumNonZeroRegisters(ByteBuffer buffer, short numNonZeroRegisters) + { + buffer.putShort(buffer.position() + NUM_NON_ZERO_REGISTERS_BYTE, numNonZeroRegisters); + } + + @Override + public byte getMaxOverflowValue() + { + return getStorageBuffer().get(getInitPosition() + MAX_OVERFLOW_VALUE_BYTE); + } + + @Override + public void setMaxOverflowValue(byte value) + { + getStorageBuffer().put(getInitPosition() + MAX_OVERFLOW_VALUE_BYTE, value); + } + + @Override + public void setMaxOverflowValue(ByteBuffer buffer, byte value) + { + buffer.put(buffer.position() + MAX_OVERFLOW_VALUE_BYTE, value); + } + + @Override + public short getMaxOverflowRegister() + { + return getStorageBuffer().getShort(getInitPosition() + MAX_OVERFLOW_REGISTER_BYTE); + } + + @Override + public void setMaxOverflowRegister(short register) + { + getStorageBuffer().putShort(getInitPosition() + MAX_OVERFLOW_REGISTER_BYTE, register); + } + + @Override + public void setMaxOverflowRegister(ByteBuffer buffer, short register) + { + buffer.putShort(buffer.position() + MAX_OVERFLOW_REGISTER_BYTE, register); + } + + @Override + public int getNumHeaderBytes() + { + return HEADER_NUM_BYTES; + } + + @Override + public int getNumBytesForDenseStorage() + { + return NUM_BYTES_FOR_DENSE_STORAGE; + } + + @Override + public int getPayloadBytePosition() + { + return getInitPosition() + HEADER_NUM_BYTES; + } + + @Override + public int getPayloadBytePosition(ByteBuffer buffer) + { + return buffer.position() + HEADER_NUM_BYTES; + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java new file mode 100644 index 00000000000..c056305f30c --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java @@ -0,0 +1,654 @@ +package io.druid.query.aggregation.hyperloglog; + +import com.fasterxml.jackson.annotation.JsonValue; +import com.google.common.primitives.UnsignedBytes; +import com.metamx.common.IAE; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; + +import java.nio.ByteBuffer; + +/** + * Implements the HyperLogLog cardinality estimator described in: + *

+ * http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf + *

+ * Run this code to see a simple indication of expected errors based on different m values: + *

+ * for (int i = 1; i < 20; ++i) { + * System.out.printf("i[%,d], val[%,d] => error[%f%%]%n", i, 2 << i, 104 / Math.sqrt(2 << i)); + * } + *

+ * This class is *not* multi-threaded. It can be passed among threads, but it is written with the assumption that + * only one thread is ever calling methods on it. + *

+ * If you have multiple threads calling methods on this concurrently, I hope you manage to get correct behavior + */ +public abstract class HyperLogLogCollector implements Comparable +{ + public static final int DENSE_THRESHOLD = 128; + public static final int BITS_FOR_BUCKETS = 11; + public static final int NUM_BUCKETS = 1 << BITS_FOR_BUCKETS; + public static final int NUM_BYTES_FOR_BUCKETS = NUM_BUCKETS / 2; + + private static final double TWO_TO_THE_SIXTY_FOUR = Math.pow(2, 64); + private static final double ALPHA = 0.7213 / (1 + 1.079 / NUM_BUCKETS); + + public static final double LOW_CORRECTION_THRESHOLD = (5 * NUM_BUCKETS) / 2.0d; + public static final double HIGH_CORRECTION_THRESHOLD = TWO_TO_THE_SIXTY_FOUR / 30.0d; + public static final double CORRECTION_PARAMETER = ALPHA * NUM_BUCKETS * NUM_BUCKETS; + + private static final Logger log = new Logger(HyperLogLogCollector.class); + private static final int bucketMask = 0x7ff; + private static final int minBytesRequired = 10; + private static final int bitsPerBucket = 4; + private static final int range = (int) Math.pow(2, bitsPerBucket) - 1; + + private final static double[][] minNumRegisterLookup = new double[64][256]; + + static { + for (int registerOffset = 0; registerOffset < 64; ++registerOffset) { + for (int register = 0; register < 256; ++register) { + final int upper = ((register & 0xf0) >> 4) + registerOffset; + final int lower = (register & 0x0f) + registerOffset; + minNumRegisterLookup[registerOffset][register] = 1.0d / Math.pow(2, upper) + 1.0d / Math.pow(2, lower); + } + } + } + + // we have to keep track of the number of zeroes in each of the two halves of the byte register (0, 1, or 2) + private final static int[] numZeroLookup = new int[256]; + + static { + for (int i = 0; i < numZeroLookup.length; ++i) { + numZeroLookup[i] = (((i & 0xf0) == 0) ? 1 : 0) + (((i & 0x0f) == 0) ? 1 : 0); + } + } + + // Methods to build the latest HLLC + public static HyperLogLogCollector makeLatestCollector() + { + return new HLLCV1(); + } + + public static HyperLogLogCollector makeCollector(ByteBuffer buffer) + { + int remaining = buffer.remaining(); + return (remaining % 3 == 0 || remaining == 1027) ? new HLLCV0(buffer) : new HLLCV1(buffer); + } + + public static int getLatestNumBytesForDenseStorage() + { + return HLLCV1.NUM_BYTES_FOR_DENSE_STORAGE; + } + + public static byte[] makeEmptyVersionedByteArray() + { + byte[] arr = new byte[getLatestNumBytesForDenseStorage()]; + arr[0] = HLLCV1.VERSION; + return arr; + } + + public static double applyCorrection(double e, int zeroCount) + { + e = CORRECTION_PARAMETER / e; + + if (e <= LOW_CORRECTION_THRESHOLD) { + return zeroCount == 0 ? e : NUM_BUCKETS * Math.log(NUM_BUCKETS / (double) zeroCount); + } + + if (e > HIGH_CORRECTION_THRESHOLD) { + final double ratio = e / TWO_TO_THE_SIXTY_FOUR; + if (ratio >= 1) { + // handle very unlikely case that value is > 2^64 + return Double.MAX_VALUE; + } else { + return -TWO_TO_THE_SIXTY_FOUR * Math.log(1 - ratio); + } + } + + return e; + } + + private static double estimateSparse( + final ByteBuffer buf, + final byte minNum, + final byte overflowValue, + final short overflowPosition, + final boolean isUpperNibble + ) + { + final ByteBuffer copy = buf.asReadOnlyBuffer(); + double e = 0.0d; + int zeroCount = NUM_BUCKETS - 2 * (buf.remaining() / 3); + while (copy.hasRemaining()) { + short position = copy.getShort(); + final int register = (int) copy.get() & 0xff; + if (overflowValue != 0 && position == overflowPosition) { + int upperNibble = ((register & 0xf0) >>> bitsPerBucket) + minNum; + int lowerNibble = (register & 0x0f) + minNum; + if (isUpperNibble) { + upperNibble = Math.max(upperNibble, overflowValue); + } else { + lowerNibble = Math.max(lowerNibble, overflowValue); + } + e += 1.0d / Math.pow(2, upperNibble) + 1.0d / Math.pow(2, lowerNibble); + zeroCount += (((upperNibble & 0xf0) == 0) ? 1 : 0) + (((lowerNibble & 0x0f) == 0) ? 1 : 0); + } else { + e += minNumRegisterLookup[minNum][register]; + zeroCount += numZeroLookup[register]; + } + } + + e += zeroCount; + return applyCorrection(e, zeroCount); + } + + private static double estimateDense( + final ByteBuffer buf, + final byte minNum, + final byte overflowValue, + final short overflowPosition, + final boolean isUpperNibble + ) + { + final ByteBuffer copy = buf.asReadOnlyBuffer(); + double e = 0.0d; + int zeroCount = 0; + int position = 0; + while (copy.hasRemaining()) { + final int register = (int) copy.get() & 0xff; + if (overflowValue != 0 && position == overflowPosition) { + int upperNibble = ((register & 0xf0) >>> bitsPerBucket) + minNum; + int lowerNibble = (register & 0x0f) + minNum; + if (isUpperNibble) { + upperNibble = Math.max(upperNibble, overflowValue); + } else { + lowerNibble = Math.max(lowerNibble, overflowValue); + } + e += 1.0d / Math.pow(2, upperNibble) + 1.0d / Math.pow(2, lowerNibble); + zeroCount += (((upperNibble & 0xf0) == 0) ? 1 : 0) + (((lowerNibble & 0x0f) == 0) ? 1 : 0); + } else { + e += minNumRegisterLookup[minNum][register]; + zeroCount += numZeroLookup[register]; + } + position++; + } + + return applyCorrection(e, zeroCount); + } + + private static boolean isSparse(ByteBuffer buffer) + { + return buffer.remaining() != NUM_BYTES_FOR_BUCKETS; + } + + private volatile ByteBuffer storageBuffer; + private volatile int initPosition; + private volatile Double estimatedCardinality; + + public HyperLogLogCollector(ByteBuffer byteBuffer) + { + storageBuffer = byteBuffer.duplicate(); + initPosition = byteBuffer.position(); + estimatedCardinality = null; + } + + public abstract byte getVersion(); + + public abstract void setVersion(ByteBuffer buffer); + + public abstract byte getRegisterOffset(); + + public abstract void setRegisterOffset(byte registerOffset); + + public abstract void setRegisterOffset(ByteBuffer buffer, byte registerOffset); + + public abstract short getNumNonZeroRegisters(); + + public abstract void setNumNonZeroRegisters(short numNonZeroRegisters); + + public abstract void setNumNonZeroRegisters(ByteBuffer buffer, short numNonZeroRegisters); + + public abstract byte getMaxOverflowValue(); + + public abstract void setMaxOverflowValue(byte value); + + public abstract void setMaxOverflowValue(ByteBuffer buffer, byte value); + + public abstract short getMaxOverflowRegister(); + + public abstract void setMaxOverflowRegister(short register); + + public abstract void setMaxOverflowRegister(ByteBuffer buffer, short register); + + public abstract int getNumHeaderBytes(); + + public abstract int getNumBytesForDenseStorage(); + + public abstract int getPayloadBytePosition(); + + public abstract int getPayloadBytePosition(ByteBuffer buffer); + + protected int getInitPosition() + { + return initPosition; + } + + protected ByteBuffer getStorageBuffer() + { + return storageBuffer; + } + + public void add(byte[] hashedValue) + { + if (hashedValue.length < minBytesRequired) { + throw new IAE("Insufficient bytes, need[%d] got [%d]", minBytesRequired, hashedValue.length); + } + + estimatedCardinality = null; + + final ByteBuffer buffer = ByteBuffer.wrap(hashedValue); + + short bucket = (short) (buffer.getShort(hashedValue.length - 2) & bucketMask); + + byte positionOf1 = 0; + + for (int i = 0; i < 8; ++i) { + byte lookupVal = ByteBitLookup.lookup[UnsignedBytes.toInt(hashedValue[i])]; + switch (lookupVal) { + case 0: + positionOf1 += 8; + continue; + default: + positionOf1 += lookupVal; + i = 8; + break; + } + } + + add(bucket, positionOf1); + } + + public void add(short bucket, byte positionOf1) + { + if (storageBuffer.isReadOnly()) { + convertToMutableByteBuffer(); + } + + byte registerOffset = getRegisterOffset(); + + // discard everything outside of the range we care about + if (positionOf1 <= registerOffset) { + return; + } else if (positionOf1 > (registerOffset + range)) { + byte currMax = getMaxOverflowValue(); + if (positionOf1 > currMax) { + setMaxOverflowValue(positionOf1); + setMaxOverflowRegister(bucket); + } + return; + } + + // whatever value we add must be stored in 4 bits + short numNonZeroRegisters = addNibbleRegister(bucket, (byte) ((0xff & positionOf1) - registerOffset)); + setNumNonZeroRegisters(numNonZeroRegisters); + if (numNonZeroRegisters == NUM_BUCKETS) { + setRegisterOffset(++registerOffset); + setNumNonZeroRegisters(decrementBuckets()); + } + } + + public HyperLogLogCollector fold(HyperLogLogCollector other) + { + if (other == null || other.storageBuffer.remaining() == 0) { + return this; + } + + if (storageBuffer.isReadOnly()) { + convertToMutableByteBuffer(); + } + + estimatedCardinality = null; + + if (getRegisterOffset() < other.getRegisterOffset()) { + // "Swap" the buffers so that we are folding into the one with the higher offset + ByteBuffer newStorage = ByteBuffer.allocate(other.storageBuffer.remaining()); + newStorage.put(other.storageBuffer.asReadOnlyBuffer()); + newStorage.clear(); + + other.storageBuffer = storageBuffer; + other.initPosition = initPosition; + storageBuffer = newStorage; + initPosition = 0; + } + + ByteBuffer otherBuffer = other.storageBuffer.asReadOnlyBuffer(); + byte otherOffset = other.getRegisterOffset(); + + if (storageBuffer.remaining() != getNumBytesForDenseStorage()) { + convertToDenseStorage(); + } + + byte myOffset = getRegisterOffset(); + short numNonZero = getNumNonZeroRegisters(); + + int offsetDiff = myOffset - otherOffset; + if (offsetDiff < 0) { + throw new ISE("offsetDiff[%d] < 0, shouldn't happen because of swap.", offsetDiff); + } + + byte otherOverflowValue = other.getMaxOverflowValue(); + short otherOverflowRegister = other.getMaxOverflowRegister(); + add(otherOverflowRegister, otherOverflowValue); + + int myPayloadStart = getPayloadBytePosition(); + otherBuffer.position(other.getPayloadBytePosition()); + if (isSparse(otherBuffer)) { + while (otherBuffer.hasRemaining()) { + short position = otherBuffer.getShort(); + int payloadStartPosition = position - other.getNumHeaderBytes(); + numNonZero += mergeAndStoreByteRegister( + myPayloadStart + payloadStartPosition, + offsetDiff, + otherBuffer.get() + ); + if (numNonZero == NUM_BUCKETS) { + myOffset += 1; + numNonZero = decrementBuckets(); + setRegisterOffset(myOffset); + setNumNonZeroRegisters(numNonZero); + + offsetDiff = myOffset - otherOffset; + } + } + } else { // dense + int position = getPayloadBytePosition(); + while (otherBuffer.hasRemaining()) { + numNonZero += mergeAndStoreByteRegister( + position, + offsetDiff, + otherBuffer.get() + ); + if (numNonZero == NUM_BUCKETS) { + myOffset += 1; + numNonZero = decrementBuckets(); + setRegisterOffset(myOffset); + setNumNonZeroRegisters(numNonZero); + + offsetDiff = myOffset - otherOffset; + } + position++; + } + } + + setRegisterOffset(myOffset); + setNumNonZeroRegisters(numNonZero); + + return this; + } + + public HyperLogLogCollector fold(ByteBuffer buffer) + { + return fold(makeCollector(buffer)); + } + + public ByteBuffer toByteBuffer() + { + short numNonZeroRegisters = getNumNonZeroRegisters(); + + // store sparsely + if (storageBuffer.remaining() == getNumBytesForDenseStorage() && numNonZeroRegisters < DENSE_THRESHOLD) { + ByteBuffer retVal = ByteBuffer.wrap(new byte[numNonZeroRegisters * 3 + getNumHeaderBytes()]); + setVersion(retVal); + setRegisterOffset(retVal, getRegisterOffset()); + setNumNonZeroRegisters(retVal, numNonZeroRegisters); + setMaxOverflowValue(retVal, getMaxOverflowValue()); + setMaxOverflowRegister(retVal, getMaxOverflowRegister()); + + int startPosition = getPayloadBytePosition(); + retVal.position(getPayloadBytePosition(retVal)); + for (int i = startPosition; i < startPosition + NUM_BYTES_FOR_BUCKETS; i++) { + if (storageBuffer.get(i) != 0) { + retVal.putShort((short) (0xffff & (i - initPosition))); + retVal.put(storageBuffer.get(i)); + } + } + retVal.rewind(); + return retVal.asReadOnlyBuffer(); + } + + return storageBuffer.asReadOnlyBuffer(); + } + + @JsonValue + public byte[] toByteArray() + { + final ByteBuffer buffer = toByteBuffer(); + byte[] theBytes = new byte[buffer.remaining()]; + buffer.get(theBytes); + + return theBytes; + } + + public double estimateCardinality() + { + if (estimatedCardinality == null) { + byte registerOffset = getRegisterOffset(); + byte overflowValue = getMaxOverflowValue(); + short overflowRegister = getMaxOverflowRegister(); + short overflowPosition = (short) (overflowRegister >>> 1); + boolean isUpperNibble = ((overflowRegister & 0x1) == 0); + + storageBuffer.position(getPayloadBytePosition()); + + if (isSparse(storageBuffer)) { + estimatedCardinality = estimateSparse( + storageBuffer, + registerOffset, + overflowValue, + overflowPosition, + isUpperNibble + ); + } else { + estimatedCardinality = estimateDense( + storageBuffer, + registerOffset, + overflowValue, + overflowPosition, + isUpperNibble + ); + } + + storageBuffer.position(initPosition); + } + return estimatedCardinality; + } + + public double estimateByteBuffer(ByteBuffer buf) + { + return makeCollector(buf).estimateCardinality(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + HyperLogLogCollector collector = (HyperLogLogCollector) o; + + if (storageBuffer != null ? !storageBuffer.equals(collector.storageBuffer) : collector.storageBuffer != null) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + int result = storageBuffer != null ? storageBuffer.hashCode() : 0; + result = 31 * result + initPosition; + return result; + } + + @Override + public String toString() + { + return "HyperLogLogCollector{" + + "initPosition=" + initPosition + + ", version=" + getVersion() + + ", registerOffset=" + getRegisterOffset() + + ", numNonZeroRegisters=" + getNumNonZeroRegisters() + + ", maxOverflowValue=" + getMaxOverflowValue() + + ", maxOverflowRegister=" + getMaxOverflowRegister() + + '}'; + } + + private short decrementBuckets() + { + short count = 0; + int startPosition = getPayloadBytePosition(); + for (int i = startPosition; i < startPosition + NUM_BYTES_FOR_BUCKETS; i++) { + byte val = (byte) (storageBuffer.get(i) - 0x11); + if ((val & 0xf0) != 0) { + count++; + } + if ((val & 0x0f) != 0) { + count++; + } + storageBuffer.put(i, val); + } + return count; + } + + private void convertToMutableByteBuffer() + { + ByteBuffer tmpBuffer = ByteBuffer.allocate(storageBuffer.remaining()); + tmpBuffer.put(storageBuffer.asReadOnlyBuffer()); + tmpBuffer.position(0); + storageBuffer = tmpBuffer; + initPosition = 0; + } + + private void convertToDenseStorage() + { + ByteBuffer tmpBuffer = ByteBuffer.wrap(new byte[getNumBytesForDenseStorage()]); + // put header + setVersion(tmpBuffer); + setRegisterOffset(tmpBuffer, getRegisterOffset()); + setNumNonZeroRegisters(tmpBuffer, getNumNonZeroRegisters()); + setMaxOverflowValue(tmpBuffer, getMaxOverflowValue()); + setMaxOverflowRegister(tmpBuffer, getMaxOverflowRegister()); + + storageBuffer.position(getPayloadBytePosition()); + tmpBuffer.position(getPayloadBytePosition(tmpBuffer)); + // put payload + while (storageBuffer.hasRemaining()) { + tmpBuffer.put(storageBuffer.getShort(), storageBuffer.get()); + } + tmpBuffer.rewind(); + storageBuffer = tmpBuffer; + initPosition = 0; + } + + private short addNibbleRegister(short bucket, byte positionOf1) + { + short numNonZeroRegs = getNumNonZeroRegisters(); + final short position = (short) (bucket >> 1); + final boolean isUpperNibble = ((bucket & 0x1) == 0); + + byte shiftedPositionOf1 = (isUpperNibble) ? (byte) (positionOf1 << bitsPerBucket) : positionOf1; + + if (storageBuffer.remaining() != getNumBytesForDenseStorage()) { + convertToDenseStorage(); + } + + byte origVal = storageBuffer.get(getPayloadBytePosition() + position); + byte newValueMask = (isUpperNibble) ? (byte) 0xf0 : (byte) 0x0f; + byte originalValueMask = (byte) (newValueMask ^ 0xff); + + // if something was at zero, we have to increase the numNonZeroRegisters + if ((origVal & newValueMask) == 0 && shiftedPositionOf1 != 0) { + numNonZeroRegs++; + } + + storageBuffer.put( + getPayloadBytePosition() + position, + (byte) (UnsignedBytes.max((byte) (origVal & newValueMask), shiftedPositionOf1) | (origVal & originalValueMask)) + ); + + return numNonZeroRegs; + } + + /** + * Returns the number of registers that are no longer zero after the value was added + * + * @param position The position into the byte buffer, this position represents two "registers" + * @param offsetDiff The difference in offset between the byteToAdd and the current HyperLogLogCollector + * @param byteToAdd The byte to merge into the current HyperLogLogCollector + * + * @return + */ + private int mergeAndStoreByteRegister( + int position, + int offsetDiff, + byte byteToAdd + ) + { + if (byteToAdd == 0) { + return 0; + } + + byte currVal = storageBuffer.get(position); + + int upperNibble = currVal & 0xf0; + int lowerNibble = currVal & 0x0f; + + // subtract the differences so that the nibbles align + int otherUpper = (byteToAdd & 0xf0) - (offsetDiff << bitsPerBucket); + int otherLower = (byteToAdd & 0x0f) - offsetDiff; + + final int newUpper = Math.max(upperNibble, otherUpper); + final int newLower = Math.max(lowerNibble, otherLower); + + int numNoLongerZero = 0; + if (upperNibble == 0 && newUpper > 0) { + ++numNoLongerZero; + } + + if (lowerNibble == 0 && newLower > 0) { + ++numNoLongerZero; + } + + storageBuffer.put(position, (byte) ((newUpper | newLower) & 0xff)); + + return numNoLongerZero; + } + + @Override + public int compareTo(HyperLogLogCollector other) + { + final int lhsOffset = (int) this.getRegisterOffset() & 0xffff; + final int rhsOffset = (int) other.getRegisterOffset() & 0xffff; + + if (lhsOffset == rhsOffset) { + final int lhsNumNonZero = (int) this.getNumNonZeroRegisters() & 0xff; + final int rhsNumNonZero = (int) this.getNumNonZeroRegisters() & 0xff; + int retVal = Double.compare(lhsNumNonZero, rhsNumNonZero); + + if (retVal == 0) { + retVal = Double.compare(this.estimateCardinality(), other.estimateCardinality()); + } + + return retVal; + } else { + return Double.compare(lhsOffset, rhsOffset); + } + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregator.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregator.java new file mode 100644 index 00000000000..db28ae33997 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregator.java @@ -0,0 +1,50 @@ +package io.druid.query.aggregation.hyperloglog; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Sets; +import io.druid.query.aggregation.PostAggregator; + +import java.util.Comparator; +import java.util.Map; +import java.util.Set; + +/** + */ +public class HyperUniqueFinalizingPostAggregator implements PostAggregator +{ + private final String fieldName; + + @JsonCreator + public HyperUniqueFinalizingPostAggregator( + @JsonProperty("fieldName") String fieldName + ) + { + this.fieldName = fieldName; + } + + @Override + public Set getDependentFields() + { + return Sets.newHashSet(fieldName); + } + + @Override + public Comparator getComparator() + { + throw new UnsupportedOperationException(); + } + + @Override + public Object compute(Map combinedAggregators) + { + return HyperUniquesAggregatorFactory.estimateCardinality(combinedAggregators.get(fieldName)); + } + + @Override + @JsonProperty("fieldName") + public String getName() + { + return fieldName; + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregator.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregator.java new file mode 100644 index 00000000000..9e174ef92a0 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregator.java @@ -0,0 +1,67 @@ +package io.druid.query.aggregation.hyperloglog; + +import io.druid.query.aggregation.Aggregator; +import io.druid.segment.ObjectColumnSelector; + +/** + */ +public class HyperUniquesAggregator implements Aggregator +{ + private final String name; + private final ObjectColumnSelector selector; + + private HyperLogLogCollector collector; + + public HyperUniquesAggregator( + String name, + ObjectColumnSelector selector + ) + { + this.name = name; + this.selector = selector; + + this.collector = HyperLogLogCollector.makeLatestCollector(); + } + + @Override + public void aggregate() + { + collector.fold((HyperLogLogCollector) selector.get()); + } + + @Override + public void reset() + { + collector = HyperLogLogCollector.makeLatestCollector(); + } + + @Override + public Object get() + { + return collector; + } + + @Override + public float getFloat() + { + throw new UnsupportedOperationException(); + } + + @Override + public String getName() + { + return name; + } + + @Override + public Aggregator clone() + { + return new HyperUniquesAggregator(name, selector); + } + + @Override + public void close() + { + // no resources to cleanup + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java new file mode 100644 index 00000000000..bd2f3183f65 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java @@ -0,0 +1,191 @@ +package io.druid.query.aggregation.hyperloglog; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; +import com.metamx.common.IAE; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.aggregation.NoopAggregator; +import io.druid.query.aggregation.NoopBufferAggregator; +import io.druid.segment.ColumnSelectorFactory; +import io.druid.segment.ObjectColumnSelector; +import org.apache.commons.codec.binary.Base64; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; + +/** + */ +public class HyperUniquesAggregatorFactory implements AggregatorFactory +{ + public static Object estimateCardinality(Object object) + { + if (object == null) { + return 0; + } + + return ((HyperLogLogCollector) object).estimateCardinality(); + } + + private static final byte CACHE_TYPE_ID = 0x5; + + private final String name; + private final String fieldName; + + @JsonCreator + public HyperUniquesAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") String fieldName + ) + { + this.name = name; + this.fieldName = fieldName.toLowerCase(); + } + + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName); + + if (selector == null) { + return new NoopAggregator(name); + } + + if (HyperLogLogCollector.class.isAssignableFrom(selector.classOfObject())) { + return new HyperUniquesAggregator(name, selector); + } + + throw new IAE( + "Incompatible type for metric[%s], expected a HyperUnique, got a %s", fieldName, selector.classOfObject() + ); + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName); + + if (selector == null) { + return new NoopBufferAggregator(); + } + + if (HyperLogLogCollector.class.isAssignableFrom(selector.classOfObject())) { + return new HyperUniquesBufferAggregator(selector); + } + + throw new IAE( + "Incompatible type for metric[%s], expected a HyperUnique, got a %s", fieldName, selector.classOfObject() + ); + } + + @Override + public Comparator getComparator() + { + return new Comparator() + { + @Override + public int compare(HyperLogLogCollector lhs, HyperLogLogCollector rhs) + { + return lhs.compareTo(rhs); + } + }; + } + + @Override + public Object combine(Object lhs, Object rhs) + { + if (rhs == null) { + return lhs; + } + if (lhs == null) { + return rhs; + } + return ((HyperLogLogCollector) lhs).fold((HyperLogLogCollector) rhs); + } + + @Override + public AggregatorFactory getCombiningFactory() + { + return new HyperUniquesAggregatorFactory(name, name); + } + + @Override + public Object deserialize(Object object) + { + if (object instanceof byte[]) { + return HyperLogLogCollector.makeCollector(ByteBuffer.wrap((byte[]) object)); + } else if (object instanceof ByteBuffer) { + return HyperLogLogCollector.makeCollector((ByteBuffer) object); + } else if (object instanceof String) { + return HyperLogLogCollector.makeCollector( + ByteBuffer.wrap(Base64.decodeBase64(((String) object).getBytes(Charsets.UTF_8))) + ); + } + return object; + } + + @Override + + public Object finalizeComputation(Object object) + { + return estimateCardinality(object); + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @Override + public List requiredFields() + { + return Arrays.asList(fieldName); + } + + @JsonProperty + public String getFieldName() + { + return fieldName; + } + + @Override + public byte[] getCacheKey() + { + byte[] fieldNameBytes = fieldName.getBytes(Charsets.UTF_8); + + return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array(); + } + + @Override + public String getTypeName() + { + return "hyperUnique"; + } + + @Override + public int getMaxIntermediateSize() + { + return HyperLogLogCollector.getLatestNumBytesForDenseStorage(); + } + + @Override + public Object getAggregatorStartValue() + { + return HyperLogLogCollector.makeLatestCollector(); + } + + @Override + public String toString() + { + return "HyperUniquesAggregatorFactory{" + + "name='" + name + '\'' + + ", fieldName='" + fieldName + '\'' + + '}'; + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java new file mode 100644 index 00000000000..feb70ccd682 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java @@ -0,0 +1,68 @@ +package io.druid.query.aggregation.hyperloglog; + +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.ObjectColumnSelector; + +import java.nio.ByteBuffer; + +/** + */ +public class HyperUniquesBufferAggregator implements BufferAggregator +{ + private static final byte[] EMPTY_BYTES = HyperLogLogCollector.makeEmptyVersionedByteArray(); + private final ObjectColumnSelector selector; + + public HyperUniquesBufferAggregator( + ObjectColumnSelector selector + ) + { + this.selector = selector; + } + + @Override + public void init(ByteBuffer buf, int position) + { + final ByteBuffer mutationBuffer = buf.duplicate(); + mutationBuffer.position(position); + mutationBuffer.put(EMPTY_BYTES); + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + HyperLogLogCollector collector = (HyperLogLogCollector) selector.get(); + + if (collector == null) { + return; + } + + HyperLogLogCollector.makeCollector( + (ByteBuffer) buf.duplicate().position(position).limit( + position + + HyperLogLogCollector.getLatestNumBytesForDenseStorage() + ) + ).fold(collector); + } + + @Override + public Object get(ByteBuffer buf, int position) + { + ByteBuffer dataCopyBuffer = ByteBuffer.allocate(HyperLogLogCollector.getLatestNumBytesForDenseStorage()); + ByteBuffer mutationBuffer = buf.duplicate(); + mutationBuffer.position(position); + mutationBuffer.get(dataCopyBuffer.array()); + return HyperLogLogCollector.makeCollector(dataCopyBuffer); + } + + @Override + public float getFloat(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException(); + } + + @Override + public void close() + { + // no resources to cleanup + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java new file mode 100644 index 00000000000..3ad31c1c45b --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java @@ -0,0 +1,129 @@ +package io.druid.query.aggregation.hyperloglog; + +import com.google.common.base.Charsets; +import com.google.common.collect.Ordering; +import com.google.common.hash.HashFunction; +import io.druid.data.input.InputRow; +import io.druid.segment.column.ColumnBuilder; +import io.druid.segment.data.GenericIndexed; +import io.druid.segment.data.ObjectStrategy; +import io.druid.segment.serde.ColumnPartSerde; +import io.druid.segment.serde.ComplexColumnPartSerde; +import io.druid.segment.serde.ComplexColumnPartSupplier; +import io.druid.segment.serde.ComplexMetricExtractor; +import io.druid.segment.serde.ComplexMetricSerde; + +import java.nio.ByteBuffer; +import java.util.List; + +/** + */ +public class HyperUniquesSerde extends ComplexMetricSerde +{ + private static Ordering comparator = new Ordering() + { + @Override + public int compare( + HyperLogLogCollector arg1, HyperLogLogCollector arg2 + ) + { + return arg1.toByteBuffer().compareTo(arg2.toByteBuffer()); + } + }.nullsFirst(); + + private final HashFunction hashFn; + + public HyperUniquesSerde( + HashFunction hashFn + ) + { + this.hashFn = hashFn; + } + + @Override + public String getTypeName() + { + return "hyperUnique"; + } + + @Override + public ComplexMetricExtractor getExtractor() + { + return new ComplexMetricExtractor() + { + @Override + public Class extractedClass() + { + return HyperLogLogCollector.class; + } + + @Override + public HyperLogLogCollector extractValue(InputRow inputRow, String metricName) + { + HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector(); + + List dimValues = inputRow.getDimension(metricName); + if (dimValues == null) { + return collector; + } + + for (String dimensionValue : dimValues) { + collector.add(hashFn.hashBytes(dimensionValue.getBytes(Charsets.UTF_8)).asBytes()); + } + return collector; + } + }; + } + + @Override + public ColumnPartSerde deserializeColumn( + ByteBuffer byteBuffer, ColumnBuilder columnBuilder + ) + { + final GenericIndexed column = GenericIndexed.read(byteBuffer, getObjectStrategy()); + + columnBuilder.setComplexColumn(new ComplexColumnPartSupplier(getTypeName(), column)); + + return new ComplexColumnPartSerde(column, getTypeName()); + } + + @Override + public ObjectStrategy getObjectStrategy() + { + return new ObjectStrategy() + { + @Override + public Class getClazz() + { + return HyperLogLogCollector.class; + } + + @Override + public HyperLogLogCollector fromByteBuffer(ByteBuffer buffer, int numBytes) + { + buffer.limit(buffer.position() + numBytes); + + int remaining = buffer.remaining(); + return (remaining % 3 == 0 || remaining == 1027) ? new HLLCV0(buffer) : new HLLCV1(buffer); + } + + @Override + public byte[] toBytes(HyperLogLogCollector collector) + { + if (collector == null) { + return new byte[]{}; + } + ByteBuffer val = collector.toByteBuffer(); + byte[] retVal = new byte[val.remaining()]; + val.asReadOnlyBuffer().get(retVal); + return retVal; + } + + @Override + public int compare(HyperLogLogCollector o1, HyperLogLogCollector o2) + { + return comparator.compare(o1, o2); + } + }; + } +} diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index 2f51162f076..d4058b86345 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -25,6 +25,7 @@ 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.hyperloglog.HyperUniquesAggregatorFactory; import io.druid.query.aggregation.post.ArithmeticPostAggregator; import io.druid.query.aggregation.post.ConstantPostAggregator; import io.druid.query.aggregation.post.FieldAccessPostAggregator; @@ -48,7 +49,7 @@ import java.util.List; */ public class QueryRunnerTestHelper { - public static final String segmentId= "testSegment"; + public static final String segmentId = "testSegment"; public static final String dataSource = "testing"; public static final QueryGranularity dayGran = QueryGranularity.DAY; public static final QueryGranularity allGran = QueryGranularity.ALL; @@ -57,9 +58,15 @@ public class QueryRunnerTestHelper public static final String placementDimension = "placement"; public static final String placementishDimension = "placementish"; public static final String indexMetric = "index"; + public static final String uniqueMetric = "uniques"; + public static final String addRowsIndexConstantMetric = "addRowsIndexConstant"; public static final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows"); public static final LongSumAggregatorFactory indexLongSum = new LongSumAggregatorFactory("index", "index"); public static final DoubleSumAggregatorFactory indexDoubleSum = new DoubleSumAggregatorFactory("index", "index"); + public static final HyperUniquesAggregatorFactory qualityUniques = new HyperUniquesAggregatorFactory( + "uniques", + "quality_uniques" + ); public static final ConstantPostAggregator constant = new ConstantPostAggregator("const", 1L); public static final FieldAccessPostAggregator rowsPostAgg = new FieldAccessPostAggregator("rows", "rows"); public static final FieldAccessPostAggregator indexPostAgg = new FieldAccessPostAggregator("index", "index"); @@ -67,7 +74,15 @@ public class QueryRunnerTestHelper new ArithmeticPostAggregator( "addRowsIndexConstant", "+", Lists.newArrayList(constant, rowsPostAgg, indexPostAgg) ); - public static final List commonAggregators = Arrays.asList(rowsCount, indexDoubleSum); + public static final List commonAggregators = Arrays.asList( + rowsCount, + indexDoubleSum, + qualityUniques + ); + + public static final double UNIQUES_9 = 9.019833517963864; + public static final double UNIQUES_2 = 2.000977198748901d; + public static final double UNIQUES_1 = 1.0002442201269182d; public static final String[] expectedFullOnIndexValues = new String[]{ "4500.0", "6077.949111938477", "4922.488838195801", "5726.140853881836", "4698.468170166016", diff --git a/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollectorTest.java b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollectorTest.java new file mode 100644 index 00000000000..4987b305585 --- /dev/null +++ b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollectorTest.java @@ -0,0 +1,774 @@ +package io.druid.query.aggregation.hyperloglog; + +import com.google.common.hash.HashFunction; +import com.google.common.hash.Hashing; +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.security.MessageDigest; +import java.util.Arrays; +import java.util.Comparator; +import java.util.Random; + +/** + */ +public class HyperLogLogCollectorTest +{ + + private final HashFunction fn = Hashing.murmur3_128(); + private final Random random = new Random(); + + @Test + public void testFolding() throws Exception + { + final int[] numValsToCheck = {10, 20, 50, 100, 1000, 2000}; + for (int numThings : numValsToCheck) { + HyperLogLogCollector allCombined = HyperLogLogCollector.makeLatestCollector(); + HyperLogLogCollector oneHalf = HyperLogLogCollector.makeLatestCollector(); + HyperLogLogCollector otherHalf = HyperLogLogCollector.makeLatestCollector(); + + for (int i = 0; i < numThings; ++i) { + byte[] hashedVal = fn.hashLong(random.nextLong()).asBytes(); + + allCombined.add(hashedVal); + if (i % 2 == 0) { + oneHalf.add(hashedVal); + } else { + otherHalf.add(hashedVal); + } + } + + HyperLogLogCollector folded = HyperLogLogCollector.makeLatestCollector(); + + folded.fold(oneHalf); + Assert.assertEquals(oneHalf, folded); + Assert.assertEquals(oneHalf.estimateCardinality(), folded.estimateCardinality(), 0.0d); + + folded.fold(otherHalf); + Assert.assertEquals(allCombined, folded); + Assert.assertEquals(allCombined.estimateCardinality(), folded.estimateCardinality(), 0.0d); + } + } + + // @Test + public void testHighCardinalityRollingFold() throws Exception + { + final HyperLogLogCollector rolling = HyperLogLogCollector.makeLatestCollector(); + final HyperLogLogCollector simple = HyperLogLogCollector.makeLatestCollector(); + + int count; + MessageDigest md = MessageDigest.getInstance("SHA-1"); + HyperLogLogCollector tmp = HyperLogLogCollector.makeLatestCollector(); + + for (count = 0; count < 5000000; ++count) { + md.update(Integer.toString(count).getBytes()); + + byte[] hashed = fn.hashBytes(md.digest()).asBytes(); + + tmp.add(hashed); + simple.add(hashed); + + if (count % 100 == 0) { + rolling.fold(tmp); + tmp = HyperLogLogCollector.makeLatestCollector(); + } + } + + int n = count; + + System.out.println("True cardinality " + n); + System.out.println("Rolling buffer cardinality " + rolling.estimateCardinality()); + System.out.println("Simple buffer cardinality " + simple.estimateCardinality()); + System.out.println( + String.format( + "Rolling cardinality estimate off by %4.1f%%", + 100 * (1 - rolling.estimateCardinality() / n) + ) + ); + + Assert.assertEquals(n, simple.estimateCardinality(), n * 0.05); + Assert.assertEquals(n, rolling.estimateCardinality(), n * 0.05); + } + + //@Test + public void testHighCardinalityRollingFold2() throws Exception + { + final HyperLogLogCollector rolling = HyperLogLogCollector.makeLatestCollector(); + int count; + long start = System.currentTimeMillis(); + + for (count = 0; count < 5000000; ++count) { + HyperLogLogCollector theCollector = HyperLogLogCollector.makeLatestCollector(); + theCollector.add(fn.hashLong(count).asBytes()); + rolling.fold(theCollector); + } + System.out.printf("testHighCardinalityRollingFold2 took %d ms%n", System.currentTimeMillis() - start); + + int n = count; + + System.out.println("True cardinality " + n); + System.out.println("Rolling buffer cardinality " + rolling.estimateCardinality()); + System.out.println( + String.format( + "Rolling cardinality estimate off by %4.1f%%", + 100 * (1 - rolling.estimateCardinality() / n) + ) + ); + + Assert.assertEquals(n, rolling.estimateCardinality(), n * 0.05); + } + + @Test + public void testFoldingByteBuffers() throws Exception + { + final int[] numValsToCheck = {10, 20, 50, 100, 1000, 2000}; + for (int numThings : numValsToCheck) { + HyperLogLogCollector allCombined = HyperLogLogCollector.makeLatestCollector(); + HyperLogLogCollector oneHalf = HyperLogLogCollector.makeLatestCollector(); + HyperLogLogCollector otherHalf = HyperLogLogCollector.makeLatestCollector(); + + for (int i = 0; i < numThings; ++i) { + byte[] hashedVal = fn.hashLong(random.nextLong()).asBytes(); + + allCombined.add(hashedVal); + if (i % 2 == 0) { + oneHalf.add(hashedVal); + } else { + otherHalf.add(hashedVal); + } + } + + HyperLogLogCollector folded = HyperLogLogCollector.makeLatestCollector(); + + folded.fold(oneHalf.toByteBuffer()); + Assert.assertEquals(oneHalf, folded); + Assert.assertEquals(oneHalf.estimateCardinality(), folded.estimateCardinality(), 0.0d); + + folded.fold(otherHalf.toByteBuffer()); + Assert.assertEquals(allCombined, folded); + Assert.assertEquals(allCombined.estimateCardinality(), folded.estimateCardinality(), 0.0d); + } + } + + @Test + public void testFoldingReadOnlyByteBuffers() throws Exception + { + final int[] numValsToCheck = {10, 20, 50, 100, 1000, 2000}; + for (int numThings : numValsToCheck) { + HyperLogLogCollector allCombined = HyperLogLogCollector.makeLatestCollector(); + HyperLogLogCollector oneHalf = HyperLogLogCollector.makeLatestCollector(); + HyperLogLogCollector otherHalf = HyperLogLogCollector.makeLatestCollector(); + + for (int i = 0; i < numThings; ++i) { + byte[] hashedVal = fn.hashLong(random.nextLong()).asBytes(); + + allCombined.add(hashedVal); + if (i % 2 == 0) { + oneHalf.add(hashedVal); + } else { + otherHalf.add(hashedVal); + } + } + + HyperLogLogCollector folded = HyperLogLogCollector.makeCollector( + ByteBuffer.wrap(HyperLogLogCollector.makeEmptyVersionedByteArray()) + .asReadOnlyBuffer() + ); + + folded.fold(oneHalf.toByteBuffer()); + Assert.assertEquals(oneHalf, folded); + Assert.assertEquals(oneHalf.estimateCardinality(), folded.estimateCardinality(), 0.0d); + + folded.fold(otherHalf.toByteBuffer()); + Assert.assertEquals(allCombined, folded); + Assert.assertEquals(allCombined.estimateCardinality(), folded.estimateCardinality(), 0.0d); + } + } + + @Test + public void testFoldingReadOnlyByteBuffersWithArbitraryPosition() throws Exception + { + final int[] numValsToCheck = {10, 20, 50, 100, 1000, 2000}; + for (int numThings : numValsToCheck) { + HyperLogLogCollector allCombined = HyperLogLogCollector.makeLatestCollector(); + HyperLogLogCollector oneHalf = HyperLogLogCollector.makeLatestCollector(); + HyperLogLogCollector otherHalf = HyperLogLogCollector.makeLatestCollector(); + + for (int i = 0; i < numThings; ++i) { + byte[] hashedVal = fn.hashLong(random.nextLong()).asBytes(); + + allCombined.add(hashedVal); + if (i % 2 == 0) { + oneHalf.add(hashedVal); + } else { + otherHalf.add(hashedVal); + } + } + + HyperLogLogCollector folded = HyperLogLogCollector.makeCollector( + shiftedBuffer( + ByteBuffer.wrap(HyperLogLogCollector.makeEmptyVersionedByteArray()) + .asReadOnlyBuffer(), + 17 + ) + ); + + folded.fold(oneHalf.toByteBuffer()); + Assert.assertEquals(oneHalf, folded); + Assert.assertEquals(oneHalf.estimateCardinality(), folded.estimateCardinality(), 0.0d); + + folded.fold(otherHalf.toByteBuffer()); + Assert.assertEquals(allCombined, folded); + Assert.assertEquals(allCombined.estimateCardinality(), folded.estimateCardinality(), 0.0d); + } + } + + @Test + public void testFoldWithDifferentOffsets1() throws Exception + { + ByteBuffer biggerOffset = makeCollectorBuffer(1, (byte) 0x00, 0x11); + ByteBuffer smallerOffset = makeCollectorBuffer(0, (byte) 0x20, 0x00); + + HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector(); + collector.fold(biggerOffset); + collector.fold(smallerOffset); + + ByteBuffer outBuffer = collector.toByteBuffer(); + + Assert.assertEquals(outBuffer.get(), collector.getVersion()); + Assert.assertEquals(outBuffer.get(), 1); + Assert.assertEquals(outBuffer.getShort(), 2047); + outBuffer.get(); + outBuffer.getShort(); + Assert.assertEquals(outBuffer.get(), 0x10); + while (outBuffer.hasRemaining()) { + Assert.assertEquals(outBuffer.get(), 0x11); + } + + collector = HyperLogLogCollector.makeLatestCollector(); + collector.fold(smallerOffset); + collector.fold(biggerOffset); + + outBuffer = collector.toByteBuffer(); + + Assert.assertEquals(outBuffer.get(), collector.getVersion()); + Assert.assertEquals(outBuffer.get(), 1); + Assert.assertEquals(outBuffer.getShort(), 2047); + Assert.assertEquals(outBuffer.get(), 0); + Assert.assertEquals(outBuffer.getShort(), 0); + Assert.assertEquals(outBuffer.get(), 0x10); + while (outBuffer.hasRemaining()) { + Assert.assertEquals(outBuffer.get(), 0x11); + } + } + + @Test + public void testFoldWithArbitraryInitialPositions() throws Exception + { + ByteBuffer biggerOffset = shiftedBuffer(makeCollectorBuffer(1, (byte) 0x00, 0x11), 10); + ByteBuffer smallerOffset = shiftedBuffer(makeCollectorBuffer(0, (byte) 0x20, 0x00), 15); + + HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector(); + collector.fold(biggerOffset); + collector.fold(smallerOffset); + + ByteBuffer outBuffer = collector.toByteBuffer(); + + Assert.assertEquals(outBuffer.get(), collector.getVersion()); + Assert.assertEquals(outBuffer.get(), 1); + Assert.assertEquals(outBuffer.getShort(), 2047); + outBuffer.get(); + outBuffer.getShort(); + Assert.assertEquals(outBuffer.get(), 0x10); + while (outBuffer.hasRemaining()) { + Assert.assertEquals(outBuffer.get(), 0x11); + } + + collector = HyperLogLogCollector.makeLatestCollector(); + collector.fold(smallerOffset); + collector.fold(biggerOffset); + + outBuffer = collector.toByteBuffer(); + + Assert.assertEquals(outBuffer.get(), collector.getVersion()); + Assert.assertEquals(outBuffer.get(), 1); + Assert.assertEquals(outBuffer.getShort(), 2047); + outBuffer.get(); + outBuffer.getShort(); + Assert.assertEquals(outBuffer.get(), 0x10); + while (outBuffer.hasRemaining()) { + Assert.assertEquals(outBuffer.get(), 0x11); + } + } + + protected ByteBuffer shiftedBuffer(ByteBuffer buf, int offset) + { + ByteBuffer shifted = ByteBuffer.allocate(buf.remaining() + offset); + shifted.position(offset); + shifted.put(buf); + shifted.position(offset); + return shifted; + } + + @Test + public void testFoldWithDifferentOffsets2() throws Exception + { + ByteBuffer biggerOffset = makeCollectorBuffer(1, (byte) 0x01, 0x11); + ByteBuffer smallerOffset = makeCollectorBuffer(0, (byte) 0x20, 0x00); + + HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector(); + collector.fold(biggerOffset); + collector.fold(smallerOffset); + + ByteBuffer outBuffer = collector.toByteBuffer(); + + Assert.assertEquals(outBuffer.get(), collector.getVersion()); + Assert.assertEquals(outBuffer.get(), 2); + Assert.assertEquals(outBuffer.getShort(), 0); + outBuffer.get(); + outBuffer.getShort(); + Assert.assertFalse(outBuffer.hasRemaining()); + + collector = HyperLogLogCollector.makeLatestCollector(); + collector.fold(smallerOffset); + collector.fold(biggerOffset); + + outBuffer = collector.toByteBuffer(); + + Assert.assertEquals(outBuffer.get(), collector.getVersion()); + Assert.assertEquals(outBuffer.get(), 2); + Assert.assertEquals(outBuffer.getShort(), 0); + outBuffer.get(); + outBuffer.getShort(); + Assert.assertFalse(outBuffer.hasRemaining()); + } + + @Test + public void testFoldWithUpperNibbleTriggersOffsetChange() throws Exception + { + byte[] arr1 = new byte[HyperLogLogCollector.getLatestNumBytesForDenseStorage()]; + Arrays.fill(arr1, (byte) 0x11); + ByteBuffer buffer1 = ByteBuffer.wrap(arr1); + buffer1.put(0, HLLCV1.VERSION); + buffer1.put(1, (byte) 0); + buffer1.putShort(2, (short) (2047)); + buffer1.put(HLLCV1.HEADER_NUM_BYTES, (byte) 0x1); + + byte[] arr2 = new byte[HyperLogLogCollector.getLatestNumBytesForDenseStorage()]; + Arrays.fill(arr2, (byte) 0x11); + ByteBuffer buffer2 = ByteBuffer.wrap(arr2); + buffer2.put(0, HLLCV1.VERSION); + buffer2.put(1, (byte) 0); + buffer2.putShort(2, (short) (2048)); + + HyperLogLogCollector collector = HyperLogLogCollector.makeCollector(buffer1); + collector.fold(buffer2); + + ByteBuffer outBuffer = collector.toByteBuffer(); + + Assert.assertEquals(outBuffer.get(), HLLCV1.VERSION); + Assert.assertEquals(outBuffer.get(), 1); + Assert.assertEquals(outBuffer.getShort(), 0); + outBuffer.get(); + outBuffer.getShort(); + Assert.assertFalse(outBuffer.hasRemaining()); + } + + @Test + public void testSparseFoldWithDifferentOffsets1() throws Exception + { + ByteBuffer biggerOffset = makeCollectorBuffer(1, new byte[]{0x11, 0x10}, 0x11); + ByteBuffer sparse = HyperLogLogCollector.makeCollector(makeCollectorBuffer(0, new byte[]{0x00, 0x02}, 0x00)) + .toByteBuffer(); + + HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector(); + collector.fold(biggerOffset); + collector.fold(sparse); + + ByteBuffer outBuffer = collector.toByteBuffer(); + + Assert.assertEquals(outBuffer.get(), collector.getVersion()); + Assert.assertEquals(outBuffer.get(), 2); + Assert.assertEquals(outBuffer.getShort(), 0); + Assert.assertEquals(outBuffer.get(), 0); + Assert.assertEquals(outBuffer.getShort(), 0); + Assert.assertFalse(outBuffer.hasRemaining()); + + collector = HyperLogLogCollector.makeLatestCollector(); + collector.fold(sparse); + collector.fold(biggerOffset); + + outBuffer = collector.toByteBuffer(); + + Assert.assertEquals(outBuffer.get(), collector.getVersion()); + Assert.assertEquals(outBuffer.get(), 2); + Assert.assertEquals(outBuffer.getShort(), 0); + Assert.assertEquals(outBuffer.get(), 0); + Assert.assertEquals(outBuffer.getShort(), 0); + Assert.assertFalse(outBuffer.hasRemaining()); + } + + private ByteBuffer makeCollectorBuffer(int offset, byte initialBytes, int remainingBytes) + { + return makeCollectorBuffer(offset, new byte[]{initialBytes}, remainingBytes); + } + + private ByteBuffer makeCollectorBuffer(int offset, byte[] initialBytes, int remainingBytes) + { + short numNonZero = 0; + for (byte initialByte : initialBytes) { + numNonZero += computeNumNonZero(initialByte); + } + + final short numNonZeroInRemaining = computeNumNonZero((byte) remainingBytes); + numNonZero += (HyperLogLogCollector.NUM_BYTES_FOR_BUCKETS - initialBytes.length) * numNonZeroInRemaining; + + ByteBuffer biggerOffset = ByteBuffer.allocate(HyperLogLogCollector.getLatestNumBytesForDenseStorage()); + biggerOffset.put(HLLCV1.VERSION); + biggerOffset.put((byte) offset); + biggerOffset.putShort(numNonZero); + biggerOffset.put((byte) 0); + biggerOffset.putShort((short) 0); + biggerOffset.put(initialBytes); + while (biggerOffset.hasRemaining()) { + biggerOffset.put((byte) remainingBytes); + } + biggerOffset.clear(); + return biggerOffset.asReadOnlyBuffer(); + } + + private short computeNumNonZero(byte theByte) + { + short retVal = 0; + if ((theByte & 0x0f) > 0) { + ++retVal; + } + if ((theByte & 0xf0) > 0) { + ++retVal; + } + return retVal; + } + + //@Test // This test can help when finding potential combinations that are weird, but it's non-deterministic + public void testFoldingwithDifferentOffsets() throws Exception + { + for (int j = 0; j < 10; j++) { + HyperLogLogCollector smallVals = HyperLogLogCollector.makeLatestCollector(); + HyperLogLogCollector bigVals = HyperLogLogCollector.makeLatestCollector(); + HyperLogLogCollector all = HyperLogLogCollector.makeLatestCollector(); + + int numThings = 500000; + for (int i = 0; i < numThings; i++) { + byte[] hashedVal = fn.hashLong(random.nextLong()).asBytes(); + + if (i < 1000) { + smallVals.add(hashedVal); + } else { + bigVals.add(hashedVal); + } + all.add(hashedVal); + } + + HyperLogLogCollector folded = HyperLogLogCollector.makeLatestCollector(); + folded.fold(smallVals); + folded.fold(bigVals); + final double expected = all.estimateCardinality(); + Assert.assertEquals(expected, folded.estimateCardinality(), expected * 0.025); + Assert.assertEquals(numThings, folded.estimateCardinality(), numThings * 0.05); + } + } + + //@Test + public void testFoldingwithDifferentOffsets2() throws Exception + { + MessageDigest md = MessageDigest.getInstance("SHA-1"); + + for (int j = 0; j < 1; j++) { + HyperLogLogCollector evenVals = HyperLogLogCollector.makeLatestCollector(); + HyperLogLogCollector oddVals = HyperLogLogCollector.makeLatestCollector(); + HyperLogLogCollector all = HyperLogLogCollector.makeLatestCollector(); + + int numThings = 500000; + for (int i = 0; i < numThings; i++) { + md.update(Integer.toString(random.nextInt()).getBytes()); + byte[] hashedVal = fn.hashBytes(md.digest()).asBytes(); + + if (i % 2 == 0) { + evenVals.add(hashedVal); + } else { + oddVals.add(hashedVal); + } + all.add(hashedVal); + } + + HyperLogLogCollector folded = HyperLogLogCollector.makeLatestCollector(); + folded.fold(evenVals); + folded.fold(oddVals); + final double expected = all.estimateCardinality(); + Assert.assertEquals(expected, folded.estimateCardinality(), expected * 0.025); + Assert.assertEquals(numThings, folded.estimateCardinality(), numThings * 0.05); + } + } + + @Test + public void testEstimation() throws Exception + { + Random random = new Random(0l); + + final int[] valsToCheck = {10, 20, 50, 100, 1000, 2000, 5000, 10000, 20000, 50000, 100000, 1000000, 2000000}; + final double[] expectedVals = { + 11.029647221949576, 21.108407720752034, 51.64575281885815, 100.42231726408892, + 981.8579991802412, 1943.1337257462792, 4946.192042635218, 9935.088157579434, + 20366.1486889433, 49433.56029693898, 100615.26273314281, 980831.624899156000, + 1982408.2608981386 + }; + + int valsToCheckIndex = 0; + HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector(); + for (int i = 0; i < valsToCheck[valsToCheck.length - 1]; ++i) { + collector.add(fn.hashLong(random.nextLong()).asBytes()); + if (i == valsToCheck[valsToCheckIndex]) { + Assert.assertEquals(expectedVals[valsToCheckIndex], collector.estimateCardinality(), 0.0d); + ++valsToCheckIndex; + } + } + Assert.assertEquals(expectedVals.length, valsToCheckIndex + 1); + Assert.assertEquals(expectedVals[valsToCheckIndex], collector.estimateCardinality(), 0.0d); + } + + @Test + public void testEstimationReadOnlyByteBuffers() throws Exception + { + Random random = new Random(0l); + + final int[] valsToCheck = {10, 20, 50, 100, 1000, 2000, 5000, 10000, 20000, 50000, 100000, 1000000, 2000000}; + final double[] expectedVals = { + 11.029647221949576, 21.108407720752034, 51.64575281885815, 100.42231726408892, + 981.8579991802412, 1943.1337257462792, 4946.192042635218, 9935.088157579434, + 20366.1486889433, 49433.56029693898, 100615.26273314281, 980831.624899156000, + 1982408.2608981386 + }; + + int valsToCheckIndex = 0; + HyperLogLogCollector collector = HyperLogLogCollector.makeCollector( + ByteBuffer.allocateDirect( + HyperLogLogCollector.getLatestNumBytesForDenseStorage() + ) + ); + for (int i = 0; i < valsToCheck[valsToCheck.length - 1]; ++i) { + collector.add(fn.hashLong(random.nextLong()).asBytes()); + if (i == valsToCheck[valsToCheckIndex]) { + Assert.assertEquals(expectedVals[valsToCheckIndex], collector.estimateCardinality(), 0.0d); + ++valsToCheckIndex; + } + } + Assert.assertEquals(expectedVals.length, valsToCheckIndex + 1); + Assert.assertEquals(expectedVals[valsToCheckIndex], collector.estimateCardinality(), 0.0d); + } + + @Test + public void testEstimationLimitDifferentFromCapacity() throws Exception + { + Random random = new Random(0l); + + final int[] valsToCheck = {10, 20, 50, 100, 1000, 2000, 5000, 10000, 20000, 50000, 100000, 1000000, 2000000}; + final double[] expectedVals = { + 11.029647221949576, 21.108407720752034, 51.64575281885815, 100.42231726408892, + 981.8579991802412, 1943.1337257462792, 4946.192042635218, 9935.088157579434, + 20366.1486889433, 49433.56029693898, 100615.26273314281, 980831.624899156000, + 1982408.2608981386 + }; + + int valsToCheckIndex = 0; + HyperLogLogCollector collector = HyperLogLogCollector.makeCollector( + (ByteBuffer) ByteBuffer.allocate(10000) + .position(0) + .limit(HyperLogLogCollector.getLatestNumBytesForDenseStorage()) + ); + for (int i = 0; i < valsToCheck[valsToCheck.length - 1]; ++i) { + collector.add(fn.hashLong(random.nextLong()).asBytes()); + if (i == valsToCheck[valsToCheckIndex]) { + Assert.assertEquals(expectedVals[valsToCheckIndex], collector.estimateCardinality(), 0.0d); + ++valsToCheckIndex; + } + } + Assert.assertEquals(expectedVals.length, valsToCheckIndex + 1); + Assert.assertEquals(expectedVals[valsToCheckIndex], collector.estimateCardinality(), 0.0d); + } + + @Test + public void testSparseEstimation() throws Exception + { + HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector(); + + for (int i = 0; i < 100; ++i) { + collector.add(fn.hashLong(random.nextLong()).asBytes()); + } + + Assert.assertEquals( + collector.estimateCardinality(), collector.estimateByteBuffer(collector.toByteBuffer()), 0.0d + ); + } + + @Test + public void testHighBits() throws Exception + { + HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector(); + + // fill up all the buckets so we reach a registerOffset of 49 + fillBuckets(collector, (byte) 0, (byte) 49); + + // highest possible bit position is 64 + collector.add(new byte[]{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}); + Assert.assertEquals(8.5089685793441677E17, collector.estimateCardinality(), 1000); + + // this might happen once in a million years if you hash a billion values a second + fillBuckets(collector, (byte) 0, (byte) 63); + collector.add(new byte[]{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}); + + Assert.assertEquals(Double.MAX_VALUE, collector.estimateCardinality(), 1000); + } + + @Test + public void testCompare1() throws Exception + { + HyperLogLogCollector collector1 = HyperLogLogCollector.makeLatestCollector(); + HyperLogLogCollector collector2 = HyperLogLogCollector.makeLatestCollector(); + collector1.add(fn.hashLong(0).asBytes()); + HyperUniquesAggregatorFactory factory = new HyperUniquesAggregatorFactory("foo", "bar"); + Comparator comparator = factory.getComparator(); + for (int i = 1; i < 100; i = i + 2) { + collector1.add(fn.hashLong(i).asBytes()); + collector2.add(fn.hashLong(i + 1).asBytes()); + Assert.assertEquals(1, comparator.compare(collector1, collector2)); + Assert.assertEquals(1, Double.compare(collector1.estimateCardinality(), collector2.estimateCardinality())); + } + } + + @Test + public void testCompare2() throws Exception + { + Random rand = new Random(0); + HyperUniquesAggregatorFactory factory = new HyperUniquesAggregatorFactory("foo", "bar"); + Comparator comparator = factory.getComparator(); + for (int i = 1; i < 1000; ++i) { + HyperLogLogCollector collector1 = HyperLogLogCollector.makeLatestCollector(); + int j = rand.nextInt(50); + for (int l = 0; l < j; ++l) { + collector1.add(fn.hashLong(rand.nextLong()).asBytes()); + } + + HyperLogLogCollector collector2 = HyperLogLogCollector.makeLatestCollector(); + int k = j + 1 + rand.nextInt(5); + for (int l = 0; l < k; ++l) { + collector2.add(fn.hashLong(rand.nextLong()).asBytes()); + } + + Assert.assertEquals( + Double.compare(collector1.estimateCardinality(), collector2.estimateCardinality()), + comparator.compare(collector1, collector2) + ); + } + + for (int i = 1; i < 100; ++i) { + HyperLogLogCollector collector1 = HyperLogLogCollector.makeLatestCollector(); + int j = rand.nextInt(500); + for (int l = 0; l < j; ++l) { + collector1.add(fn.hashLong(rand.nextLong()).asBytes()); + } + + HyperLogLogCollector collector2 = HyperLogLogCollector.makeLatestCollector(); + int k = j + 2 + rand.nextInt(5); + for (int l = 0; l < k; ++l) { + collector2.add(fn.hashLong(rand.nextLong()).asBytes()); + } + + Assert.assertEquals( + Double.compare(collector1.estimateCardinality(), collector2.estimateCardinality()), + comparator.compare(collector1, collector2) + ); + } + + for (int i = 1; i < 10; ++i) { + HyperLogLogCollector collector1 = HyperLogLogCollector.makeLatestCollector(); + int j = rand.nextInt(100000); + for (int l = 0; l < j; ++l) { + collector1.add(fn.hashLong(rand.nextLong()).asBytes()); + } + + HyperLogLogCollector collector2 = HyperLogLogCollector.makeLatestCollector(); + int k = j + 20000 + rand.nextInt(100000); + for (int l = 0; l < k; ++l) { + collector2.add(fn.hashLong(rand.nextLong()).asBytes()); + } + + Assert.assertEquals( + Double.compare(collector1.estimateCardinality(), collector2.estimateCardinality()), + comparator.compare(collector1, collector2) + ); + } + } + + + private static void fillBuckets(HyperLogLogCollector collector, byte startOffset, byte endOffset) + { + byte offset = startOffset; + while (offset <= endOffset) { + // fill buckets to shift registerOffset + for (short bucket = 0; bucket < 2048; ++bucket) { + collector.add(bucket, offset); + } + offset++; + } + } + + // Provides a nice printout of error rates as a function of cardinality + //@Test + public void showErrorRate() throws Exception + { + HashFunction fn = Hashing.murmur3_128(); + Random random = new Random(); + + double error = 0.0d; + int count = 0; + + final int[] valsToCheck = { + 10, 20, 50, 100, 1000, 2000, 5000, 10000, 20000, 50000, 100000, 1000000, 2000000, 10000000, Integer.MAX_VALUE + }; + + for (int numThings : valsToCheck) { + long startTime = System.currentTimeMillis(); + HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector(); + + for (int i = 0; i < numThings; ++i) { + if (i != 0 && i % 100000000 == 0) { + ++count; + error = computeError(error, count, i, startTime, collector); + } + collector.add(fn.hashLong(random.nextLong()).asBytes()); + } + + ++count; + error = computeError(error, count, numThings, startTime, collector); + } + } + + private double computeError(double error, int count, int numThings, long startTime, HyperLogLogCollector collector) + { + final double estimatedValue = collector.estimateCardinality(); + final double errorThisTime = Math.abs((double) numThings - estimatedValue) / numThings; + + error += errorThisTime; + + System.out.printf( + "%,d ==? %,f in %,d millis. actual error[%,f%%], avg. error [%,f%%]%n", + numThings, + estimatedValue, + System.currentTimeMillis() - startTime, + 100 * errorThisTime, + (error / count) * 100 + ); + return error; + } +} diff --git a/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregatorTest.java new file mode 100644 index 00000000000..eb8a0c23d00 --- /dev/null +++ b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregatorTest.java @@ -0,0 +1,35 @@ +package io.druid.query.aggregation.hyperloglog; + +import com.google.common.collect.ImmutableMap; +import com.google.common.hash.HashFunction; +import com.google.common.hash.Hashing; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Random; + +/** + */ +public class HyperUniqueFinalizingPostAggregatorTest +{ + private final HashFunction fn = Hashing.murmur3_128(); + + @Test + public void testCompute() throws Exception + { + Random random = new Random(0l); + HyperUniqueFinalizingPostAggregator postAggregator = new HyperUniqueFinalizingPostAggregator( + "uniques" + ); + HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector(); + + for (int i = 0; i < 100; ++i) { + byte[] hashedVal = fn.hashLong(random.nextLong()).asBytes(); + collector.add(hashedVal); + } + + double cardinality = (Double) postAggregator.compute(ImmutableMap.of("uniques", collector)); + + Assert.assertTrue(cardinality == 99.37233005831612); + } +} diff --git a/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactoryTest.java b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactoryTest.java new file mode 100644 index 00000000000..56efc884148 --- /dev/null +++ b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactoryTest.java @@ -0,0 +1,29 @@ +package io.druid.query.aggregation.hyperloglog; + +import junit.framework.Assert; +import org.junit.Test; + +public class HyperUniquesAggregatorFactoryTest +{ + final static HyperUniquesAggregatorFactory aggregatorFactory = new HyperUniquesAggregatorFactory( + "hyperUnique", + "uniques" + ); + final static String V0_BASE64 = "AAYbEyQwFyQVASMCVFEQQgEQIxIhM4ISAQMhUkICEDFDIBMhMgFQFAFAMjAAEhEREyVAEiUBAhIjISATMCECMiERIRIiVRFRAyIAEgFCQSMEJAITATAAEAMQgCEBEjQiAyUTAyEQASJyAGURAAISAwISATETQhAREBYDIVIlFTASAzJgERIgRCcmUyAwNAMyEJMjIhQXQhEWECABQDETATEREjIRAgEyIiMxMBQiAkBBMDYAMEQQACMzMhIkMTQSkYIRABIBADMBAhIEISAENkEBQDAxETMAIEEwEzQiQSEVQSFBBAQDICIiAVIAMTAQIQYBIRABADMDEzEAQSMkEiAYFBAQI0AmECEyQSARRTIVMhEkMiKAMCUBxUghAkIBI3EmMAQiACEAJDJCAAADOzESEDBCRjMgEUQQETQwEWIhA6MlAiAAZDI1AgEIIDUyFDIHMQEEAwIRBRABBStCZCQhAgJSMQIiQEEURTBmM1MxACIAETGhMgQnBRICNiIREyIUNAEAAkABAwQSEBJBIhIhIRERAiIRACUhEUAVMkQGEVMjECYjACBwEQQSIRIgAAEyExQUFSEAIBJCIDIDYTAgMiNBIUADUiETADMoFEADETMCIwUEQkIAESMSIzIABDERIXEhIiACQgUSEgJiQCAUARIRAREDQiEUAkQgAgQiIEAzIxRCARIgBAAVAzMAECEwE0Qh8gAAASEhEiAiMhUxcRImIVABATYyUBAwIoE1QhRDIiYBIBEBEiQSQyERAAADMAARAEACFYUwQSQBIRIgURITARFSEzEHEBACOTMREBIAMjIgEhU0cxEQIRIhIi1wEgMRUBEgMQIRAnAVASURMHQBAiEyBSAAEBQTAWQ5EQA0IUMSISAUEiASIjIhMhMFJBBSEjEAECEwACASEQFBAjARITEQIgYTEKEAeAAiMkEyARowARFBAicRISIBIxAQAgEBARMCIRQgMSIVIAkjMxIAIEMyADASMgFRIjEyKjEjBBIEQCUAARYBEQMxMCIBACNCACRCMlEzUUAAUDM1MhAjEgAxAAISAVFQECAhQAMBMhEzEgASNxAhFRIxECMRJBQAERAToBgQMhJSRQFAEhAwMiIhMQAwAgQiBQJiIGMQQhEiQxR1MiAjIAIEEiAkARECEzQlMjECIRATBgIhEBQAIQAEATEjBCMwAgMBMhAhIyFBIxQAARI1AAEABCIDFBIRUzMBIgAgEiARQCASMQQDQCFBAQAUJwMUElAyIAIRBSIRITICEAIxMAEUBEYTcBMBEEIxMREwIRIDAGIAEgYxBAEANCAhBAI2UhIiIgIRABIEVRAwNEIQERQgEFMhFCQSIAEhQDMTEQMiAjJyEQ=="; + + @Test + public void testDeserializeV0() throws Exception + { + Object v0 = aggregatorFactory.deserialize(V0_BASE64); + Assert.assertEquals("deserialized value is HLLCV0", HLLCV0.class, v0.getClass()); + } + + @Test + public void testCombineStartValueV0() throws Exception + { + Object combined = aggregatorFactory.getAggregatorStartValue(); + aggregatorFactory.combine(combined, aggregatorFactory.deserialize(V0_BASE64)); + } + + +} diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java index 00aaca6f9e5..a8d626ae044 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -85,7 +85,8 @@ public class TimeseriesQueryRunnerTest .aggregators( Arrays.asList( QueryRunnerTestHelper.rowsCount, - QueryRunnerTestHelper.indexDoubleSum + QueryRunnerTestHelper.indexDoubleSum, + QueryRunnerTestHelper.qualityUniques ) ) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) @@ -128,6 +129,11 @@ public class TimeseriesQueryRunnerTest value.getDoubleMetric("addRowsIndexConstant"), 0.0 ); + Assert.assertEquals( + value.getDoubleMetric("uniques"), + QueryRunnerTestHelper.skippedDay.equals(result.getTimestamp()) ? 0.0d : 9.0d, + 0.02 + ); expectedEarliest = gran.toDateTime(gran.next(expectedEarliest.getMillis())); ++count; @@ -182,7 +188,12 @@ public class TimeseriesQueryRunnerTest .granularity(QueryRunnerTestHelper.dayGran) .filters(QueryRunnerTestHelper.providerDimension, "upfront") .intervals(QueryRunnerTestHelper.fullOnInterval) - .aggregators(Arrays.asList(QueryRunnerTestHelper.rowsCount)) + .aggregators( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + QueryRunnerTestHelper.qualityUniques + ) + ) .build(); Assert.assertEquals( @@ -215,6 +226,14 @@ public class TimeseriesQueryRunnerTest QueryRunnerTestHelper.skippedDay.equals(result.getTimestamp()) ? 0L : 2L, value.getLongMetric("rows").longValue() ); + Assert.assertEquals( + result.toString(), + QueryRunnerTestHelper.skippedDay.equals(result.getTimestamp()) ? 0.0d : 2.0d, + value.getDoubleMetric( + "uniques" + ), + 0.01 + ); expectedEarliest = gran.toDateTime(gran.next(expectedEarliest.getMillis())); } @@ -233,7 +252,8 @@ public class TimeseriesQueryRunnerTest new LongSumAggregatorFactory( "idx", "index" - ) + ), + QueryRunnerTestHelper.qualityUniques ) ) .build(); @@ -242,13 +262,13 @@ public class TimeseriesQueryRunnerTest new Result( new DateTime("2011-04-01"), new TimeseriesResultValue( - ImmutableMap.of("rows", 13L, "idx", 6619L) + ImmutableMap.of("rows", 13L, "idx", 6619L, "uniques", QueryRunnerTestHelper.UNIQUES_9) ) ), new Result( new DateTime("2011-04-02"), new TimeseriesResultValue( - ImmutableMap.of("rows", 13L, "idx", 5827L) + ImmutableMap.of("rows", 13L, "idx", 5827L, "uniques", QueryRunnerTestHelper.UNIQUES_9) ) ) ); @@ -327,7 +347,8 @@ public class TimeseriesQueryRunnerTest new LongSumAggregatorFactory( "idx", "index" - ) + ), + QueryRunnerTestHelper.qualityUniques ) ) .build(); @@ -336,7 +357,7 @@ public class TimeseriesQueryRunnerTest new Result( new DateTime("2011-04-01"), new TimeseriesResultValue( - ImmutableMap.of("rows", 13L, "idx", 5827L) + ImmutableMap.of("rows", 13L, "idx", 5827L, "uniques", QueryRunnerTestHelper.UNIQUES_9) ) ) ); @@ -363,7 +384,8 @@ public class TimeseriesQueryRunnerTest new LongSumAggregatorFactory( "idx", "index" - ) + ), + QueryRunnerTestHelper.qualityUniques ) ) .build(); @@ -372,7 +394,7 @@ public class TimeseriesQueryRunnerTest new Result( new DateTime("2011-04-02"), new TimeseriesResultValue( - ImmutableMap.of("rows", 13L, "idx", 5827L) + ImmutableMap.of("rows", 13L, "idx", 5827L, "uniques", QueryRunnerTestHelper.UNIQUES_9) ) ) ); @@ -457,7 +479,8 @@ public class TimeseriesQueryRunnerTest new LongSumAggregatorFactory( "idx", "index" - ) + ), + QueryRunnerTestHelper.qualityUniques ) ) .build(); @@ -466,7 +489,7 @@ public class TimeseriesQueryRunnerTest new Result( new DateTime("2011-04-01"), new TimeseriesResultValue( - ImmutableMap.of("rows", 13L, "idx", 5827L) + ImmutableMap.of("rows", 13L, "idx", 5827L, "uniques", QueryRunnerTestHelper.UNIQUES_9) ) ) ); @@ -494,7 +517,8 @@ public class TimeseriesQueryRunnerTest new LongSumAggregatorFactory( "idx", "index" - ) + ), + QueryRunnerTestHelper.qualityUniques ) ) .build(); @@ -503,7 +527,7 @@ public class TimeseriesQueryRunnerTest new Result( new DateTime("2011-04-02"), new TimeseriesResultValue( - ImmutableMap.of("rows", 13L, "idx", 5827L) + ImmutableMap.of("rows", 13L, "idx", 5827L, "uniques", QueryRunnerTestHelper.UNIQUES_9) ) ) ); @@ -561,7 +585,8 @@ public class TimeseriesQueryRunnerTest .aggregators( Arrays.asList( QueryRunnerTestHelper.rowsCount, - QueryRunnerTestHelper.indexLongSum + QueryRunnerTestHelper.indexLongSum, + QueryRunnerTestHelper.qualityUniques ) ) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) @@ -574,7 +599,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 13L, "index", 6619L, - "addRowsIndexConstant", 6633.0 + "addRowsIndexConstant", 6633.0, + "uniques", QueryRunnerTestHelper.UNIQUES_9 ) ) ), @@ -584,7 +610,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 13L, "index", 5827L, - "addRowsIndexConstant", 5841.0 + "addRowsIndexConstant", 5841.0, + "uniques", QueryRunnerTestHelper.UNIQUES_9 ) ) ) @@ -608,7 +635,8 @@ public class TimeseriesQueryRunnerTest .aggregators( Arrays.asList( QueryRunnerTestHelper.rowsCount, - QueryRunnerTestHelper.indexLongSum + QueryRunnerTestHelper.indexLongSum, + QueryRunnerTestHelper.qualityUniques ) ) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) @@ -621,7 +649,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 11L, "index", 3783L, - "addRowsIndexConstant", 3795.0 + "addRowsIndexConstant", 3795.0, + "uniques", QueryRunnerTestHelper.UNIQUES_9 ) ) ), @@ -631,7 +660,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 11L, "index", 3313L, - "addRowsIndexConstant", 3325.0 + "addRowsIndexConstant", 3325.0, + "uniques", QueryRunnerTestHelper.UNIQUES_9 ) ) ) @@ -655,7 +685,8 @@ public class TimeseriesQueryRunnerTest .aggregators( Arrays.asList( QueryRunnerTestHelper.rowsCount, - QueryRunnerTestHelper.indexLongSum + QueryRunnerTestHelper.indexLongSum, + QueryRunnerTestHelper.qualityUniques ) ) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) @@ -668,7 +699,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 9L, "index", 1102L, - "addRowsIndexConstant", 1112.0 + "addRowsIndexConstant", 1112.0, + "uniques", QueryRunnerTestHelper.UNIQUES_9 ) ) ), @@ -678,7 +710,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 9L, "index", 1120L, - "addRowsIndexConstant", 1130.0 + "addRowsIndexConstant", 1130.0, + "uniques", QueryRunnerTestHelper.UNIQUES_9 ) ) ) @@ -702,7 +735,8 @@ public class TimeseriesQueryRunnerTest .aggregators( Arrays.asList( QueryRunnerTestHelper.rowsCount, - QueryRunnerTestHelper.indexLongSum + QueryRunnerTestHelper.indexLongSum, + QueryRunnerTestHelper.qualityUniques ) ) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) @@ -715,7 +749,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 2L, "index", 2681L, - "addRowsIndexConstant", 2684.0 + "addRowsIndexConstant", 2684.0, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ) ) ), @@ -725,7 +760,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 2L, "index", 2193L, - "addRowsIndexConstant", 2196.0 + "addRowsIndexConstant", 2196.0, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ) ) ) @@ -749,7 +785,8 @@ public class TimeseriesQueryRunnerTest .aggregators( Arrays.asList( QueryRunnerTestHelper.rowsCount, - QueryRunnerTestHelper.indexLongSum + QueryRunnerTestHelper.indexLongSum, + QueryRunnerTestHelper.qualityUniques ) ) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) @@ -762,7 +799,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 2L, "index", 2836L, - "addRowsIndexConstant", 2839.0 + "addRowsIndexConstant", 2839.0, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ) ) ), @@ -772,7 +810,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 2L, "index", 2514L, - "addRowsIndexConstant", 2517.0 + "addRowsIndexConstant", 2517.0, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ) ) ) @@ -818,7 +857,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 2L, "index", 254.4554443359375D, - "addRowsIndexConstant", 257.4554443359375D + "addRowsIndexConstant", 257.4554443359375D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ) ) ), @@ -828,7 +868,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 2L, "index", 260.4129638671875D, - "addRowsIndexConstant", 263.4129638671875D + "addRowsIndexConstant", 263.4129638671875D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ) ) ) @@ -874,7 +915,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 1L, "index", new Float(135.885094).doubleValue(), - "addRowsIndexConstant", new Float(137.885094).doubleValue() + "addRowsIndexConstant", new Float(137.885094).doubleValue(), + "uniques", QueryRunnerTestHelper.UNIQUES_1 ) ) ), @@ -884,7 +926,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 1L, "index", new Float(147.425935).doubleValue(), - "addRowsIndexConstant", new Float(149.425935).doubleValue() + "addRowsIndexConstant", new Float(149.425935).doubleValue(), + "uniques", QueryRunnerTestHelper.UNIQUES_1 ) ) ) @@ -930,7 +973,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 1L, "index", new Float(118.570340).doubleValue(), - "addRowsIndexConstant", new Float(120.570340).doubleValue() + "addRowsIndexConstant", new Float(120.570340).doubleValue(), + "uniques", QueryRunnerTestHelper.UNIQUES_1 ) ) ), @@ -940,7 +984,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 1L, "index", new Float(112.987027).doubleValue(), - "addRowsIndexConstant", new Float(114.987027).doubleValue() + "addRowsIndexConstant", new Float(114.987027).doubleValue(), + "uniques", QueryRunnerTestHelper.UNIQUES_1 ) ) ) @@ -970,7 +1015,8 @@ public class TimeseriesQueryRunnerTest .aggregators( Arrays.asList( QueryRunnerTestHelper.rowsCount, - QueryRunnerTestHelper.indexLongSum + QueryRunnerTestHelper.indexLongSum, + QueryRunnerTestHelper.qualityUniques ) ) .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) @@ -983,7 +1029,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 13L, "index", 6619L, - "addRowsIndexConstant", 6633.0 + "addRowsIndexConstant", 6633.0, + "uniques", QueryRunnerTestHelper.UNIQUES_9 ) ) ), @@ -993,7 +1040,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 13L, "index", 5827L, - "addRowsIndexConstant", 5841.0 + "addRowsIndexConstant", 5841.0, + "uniques", QueryRunnerTestHelper.UNIQUES_9 ) ) ) @@ -1043,7 +1091,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 2L, "index", 254.4554443359375D, - "addRowsIndexConstant", 257.4554443359375D + "addRowsIndexConstant", 257.4554443359375D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ) ) ), @@ -1053,7 +1102,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 2L, "index", 260.4129638671875D, - "addRowsIndexConstant", 263.4129638671875D + "addRowsIndexConstant", 263.4129638671875D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ) ) ) @@ -1085,7 +1135,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 0L, "index", 0.0, - "addRowsIndexConstant", 1.0 + "addRowsIndexConstant", 1.0, + "uniques", 0.0 ) ) ), @@ -1095,7 +1146,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 0L, "index", 0.0, - "addRowsIndexConstant", 1.0 + "addRowsIndexConstant", 1.0, + "uniques", 0.0 ) ) ) @@ -1127,7 +1179,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 0L, "index", 0.0, - "addRowsIndexConstant", 1.0 + "addRowsIndexConstant", 1.0, + "uniques", 0.0 ) ) ), @@ -1137,7 +1190,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 0L, "index", 0.0, - "addRowsIndexConstant", 1.0 + "addRowsIndexConstant", 1.0, + "uniques", 0.0 ) ) ) @@ -1183,7 +1237,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 0L, "index", 0.0, - "addRowsIndexConstant", 1.0 + "addRowsIndexConstant", 1.0, + "uniques", 0.0 ) ) ), @@ -1193,7 +1248,8 @@ public class TimeseriesQueryRunnerTest ImmutableMap.of( "rows", 0L, "index", 0.0, - "addRowsIndexConstant", 1.0 + "addRowsIndexConstant", 1.0, + "uniques", 0.0 ) ) ) diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java index 839f82c3cbc..c938ef49259 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java @@ -25,27 +25,20 @@ 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.QueryRunnerTestHelper; 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; @@ -108,73 +101,22 @@ public class TopNQueryRunnerTest 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")) - ); - + private static final String providerDimension = "provider"; @Test public void testFullOnTopN() { TopNQuery query = new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) .dimension(providerDimension) - .metric(indexMetric) + .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) - .intervals(fullOnInterval) + .intervals(QueryRunnerTestHelper.fullOnInterval) .aggregators( Lists.newArrayList( Iterables.concat( - commonAggregators, + QueryRunnerTestHelper.commonAggregators, Lists.newArrayList( new MaxAggregatorFactory("maxIndex", "index"), new MinAggregatorFactory("minIndex", "index") @@ -182,7 +124,7 @@ public class TopNQueryRunnerTest ) ) ) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); List> expectedResults = Arrays.asList( @@ -195,6 +137,7 @@ public class TopNQueryRunnerTest .put("rows", 186L) .put("index", 215679.82879638672D) .put("addRowsIndexConstant", 215866.82879638672D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_2) .put("maxIndex", 1743.9217529296875D) .put("minIndex", 792.3260498046875D) .build(), @@ -203,6 +146,7 @@ public class TopNQueryRunnerTest .put("rows", 186L) .put("index", 192046.1060180664D) .put("addRowsIndexConstant", 192233.1060180664D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_2) .put("maxIndex", 1870.06103515625D) .put("minIndex", 545.9906005859375D) .build(), @@ -211,6 +155,7 @@ public class TopNQueryRunnerTest .put("rows", 837L) .put("index", 95606.57232284546D) .put("addRowsIndexConstant", 96444.57232284546D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_9) .put("maxIndex", 277.2735290527344D) .put("minIndex", 59.02102279663086D) .build() @@ -226,16 +171,16 @@ public class TopNQueryRunnerTest public void testFullOnTopNOverPostAggs() { TopNQuery query = new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) .dimension(providerDimension) - .metric(addRowsIndexConstantMetric) + .metric(QueryRunnerTestHelper.addRowsIndexConstantMetric) .threshold(4) - .intervals(fullOnInterval) + .intervals(QueryRunnerTestHelper.fullOnInterval) .aggregators( Lists.newArrayList( Iterables.concat( - commonAggregators, + QueryRunnerTestHelper.commonAggregators, Lists.newArrayList( new MaxAggregatorFactory("maxIndex", "index"), new MinAggregatorFactory("minIndex", "index") @@ -243,7 +188,7 @@ public class TopNQueryRunnerTest ) ) ) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); List> expectedResults = Arrays.asList( @@ -256,6 +201,7 @@ public class TopNQueryRunnerTest .put("rows", 186L) .put("index", 215679.82879638672D) .put("addRowsIndexConstant", 215866.82879638672D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_2) .put("maxIndex", 1743.9217529296875D) .put("minIndex", 792.3260498046875D) .build(), @@ -264,6 +210,7 @@ public class TopNQueryRunnerTest .put("rows", 186L) .put("index", 192046.1060180664D) .put("addRowsIndexConstant", 192233.1060180664D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_2) .put("maxIndex", 1870.06103515625D) .put("minIndex", 545.9906005859375D) .build(), @@ -272,6 +219,7 @@ public class TopNQueryRunnerTest .put("rows", 837L) .put("index", 95606.57232284546D) .put("addRowsIndexConstant", 96444.57232284546D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_9) .put("maxIndex", 277.2735290527344D) .put("minIndex", 59.02102279663086D) .build() @@ -283,18 +231,84 @@ public class TopNQueryRunnerTest TestHelper.assertExpectedResults(expectedResults, runner.run(query)); } + + @Test + public void testFullOnTopNOverUniques() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(providerDimension) + .metric(QueryRunnerTestHelper.uniqueMetric) + .threshold(3) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new MaxAggregatorFactory("maxIndex", "index"), + new MinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 837L) + .put("index", 95606.57232284546D) + .put("addRowsIndexConstant", 96444.57232284546D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_9) + .put("maxIndex", 277.2735290527344D) + .put("minIndex", 59.02102279663086D) + .build(), + ImmutableMap.builder() + .put("provider", "total_market") + .put("rows", 186L) + .put("index", 215679.82879638672D) + .put("addRowsIndexConstant", 215866.82879638672D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_2) + .put("maxIndex", 1743.9217529296875D) + .put("minIndex", 792.3260498046875D) + .build(), + ImmutableMap.builder() + .put("provider", "upfront") + .put("rows", 186L) + .put("index", 192046.1060180664D) + .put("addRowsIndexConstant", 192233.1060180664D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_2) + .put("maxIndex", 1870.06103515625D) + .put("minIndex", 545.9906005859375D) + .build() + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test public void testTopN() { TopNQuery query = new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) .dimension(providerDimension) - .metric(indexMetric) + .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) - .intervals(firstToThird) - .aggregators(commonAggregators) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); @@ -307,19 +321,71 @@ public class TopNQueryRunnerTest "provider", "total_market", "rows", 4L, "index", 5351.814697265625D, - "addRowsIndexConstant", 5356.814697265625D + "addRowsIndexConstant", 5356.814697265625D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( "provider", "upfront", "rows", 4L, "index", 4875.669677734375D, - "addRowsIndexConstant", 4880.669677734375D + "addRowsIndexConstant", 4880.669677734375D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( "provider", "spot", "rows", 18L, "index", 2231.8768157958984D, - "addRowsIndexConstant", 2250.8768157958984D + "addRowsIndexConstant", 2250.8768157958984D, + "uniques", QueryRunnerTestHelper.UNIQUES_9 + ) + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTopNByUniques() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(providerDimension) + .metric(new NumericTopNMetricSpec("uniques")) + .threshold(4) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) + .build(); + + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-04-01T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.of( + "provider", "spot", + "rows", 18L, + "index", 2231.8768157958984D, + "addRowsIndexConstant", 2250.8768157958984D, + "uniques", QueryRunnerTestHelper.UNIQUES_9 + ), + ImmutableMap.of( + "provider", "total_market", + "rows", 4L, + "index", 5351.814697265625D, + "addRowsIndexConstant", 5356.814697265625D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 + ), + ImmutableMap.of( + "provider", "upfront", + "rows", 4L, + "index", 4875.669677734375D, + "addRowsIndexConstant", 4880.669677734375D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ) ) ) @@ -333,15 +399,15 @@ public class TopNQueryRunnerTest public void testTopNWithOrFilter1() { TopNQuery query = new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) .filters(providerDimension, "total_market", "upfront", "spot") .dimension(providerDimension) - .metric(indexMetric) + .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) - .intervals(firstToThird) - .aggregators(commonAggregators) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); List> expectedResults = Arrays.asList( @@ -353,19 +419,22 @@ public class TopNQueryRunnerTest "provider", "total_market", "rows", 4L, "index", 5351.814697265625D, - "addRowsIndexConstant", 5356.814697265625D + "addRowsIndexConstant", 5356.814697265625D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( "provider", "upfront", "rows", 4L, "index", 4875.669677734375D, - "addRowsIndexConstant", 4880.669677734375D + "addRowsIndexConstant", 4880.669677734375D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( "provider", "spot", "rows", 18L, "index", 2231.8768157958984D, - "addRowsIndexConstant", 2250.8768157958984D + "addRowsIndexConstant", 2250.8768157958984D, + "uniques", QueryRunnerTestHelper.UNIQUES_9 ) ) ) @@ -379,15 +448,15 @@ public class TopNQueryRunnerTest public void testTopNWithOrFilter2() { TopNQuery query = new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) .filters(providerDimension, "total_market", "upfront") .dimension(providerDimension) - .metric(indexMetric) + .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) - .intervals(firstToThird) - .aggregators(commonAggregators) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); List> expectedResults = Arrays.asList( @@ -399,13 +468,15 @@ public class TopNQueryRunnerTest "provider", "total_market", "rows", 4L, "index", 5351.814697265625D, - "addRowsIndexConstant", 5356.814697265625D + "addRowsIndexConstant", 5356.814697265625D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( "provider", "upfront", "rows", 4L, "index", 4875.669677734375D, - "addRowsIndexConstant", 4880.669677734375D + "addRowsIndexConstant", 4880.669677734375D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ) ) ) @@ -419,15 +490,15 @@ public class TopNQueryRunnerTest public void testTopNWithFilter1() { TopNQuery query = new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) .filters(providerDimension, "upfront") .dimension(providerDimension) - .metric(indexMetric) + .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) - .intervals(firstToThird) - .aggregators(commonAggregators) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); List> expectedResults = Arrays.asList( @@ -439,7 +510,8 @@ public class TopNQueryRunnerTest "provider", "upfront", "rows", 4L, "index", 4875.669677734375D, - "addRowsIndexConstant", 4880.669677734375D + "addRowsIndexConstant", 4880.669677734375D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ) ) ) @@ -453,15 +525,15 @@ public class TopNQueryRunnerTest public void testTopNWithFilter2() { TopNQuery query = new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) - .filters(qualityDimension, "mezzanine") + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .filters(QueryRunnerTestHelper.qualityDimension, "mezzanine") .dimension(providerDimension) - .metric(indexMetric) + .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) - .intervals(firstToThird) - .aggregators(commonAggregators) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); List> expectedResults = Arrays.asList( @@ -473,19 +545,22 @@ public class TopNQueryRunnerTest "provider", "upfront", "rows", 2L, "index", 2591.68359375D, - "addRowsIndexConstant", 2594.68359375D + "addRowsIndexConstant", 2594.68359375D, + "uniques", QueryRunnerTestHelper.UNIQUES_1 ), ImmutableMap.of( "provider", "total_market", "rows", 2L, "index", 2508.39599609375D, - "addRowsIndexConstant", 2511.39599609375D + "addRowsIndexConstant", 2511.39599609375D, + "uniques", QueryRunnerTestHelper.UNIQUES_1 ), ImmutableMap.of( "provider", "spot", "rows", 2L, "index", 220.63774871826172D, - "addRowsIndexConstant", 223.63774871826172D + "addRowsIndexConstant", 223.63774871826172D, + "uniques", QueryRunnerTestHelper.UNIQUES_1 ) ) ) @@ -499,19 +574,19 @@ public class TopNQueryRunnerTest public void testTopNWithFilter2OneDay() { TopNQuery query = new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) - .filters(qualityDimension, "mezzanine") + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .filters(QueryRunnerTestHelper.qualityDimension, "mezzanine") .dimension(providerDimension) - .metric(indexMetric) + .metric(QueryRunnerTestHelper.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)) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); List> expectedResults = Arrays.asList( @@ -523,19 +598,22 @@ public class TopNQueryRunnerTest "provider", "upfront", "rows", 1L, "index", new Float(1447.341160).doubleValue(), - "addRowsIndexConstant", new Float(1449.341160).doubleValue() + "addRowsIndexConstant", new Float(1449.341160).doubleValue(), + "uniques", QueryRunnerTestHelper.UNIQUES_1 ), ImmutableMap.of( "provider", "total_market", "rows", 1L, "index", new Float(1314.839715).doubleValue(), - "addRowsIndexConstant", new Float(1316.839715).doubleValue() + "addRowsIndexConstant", new Float(1316.839715).doubleValue(), + "uniques", QueryRunnerTestHelper.UNIQUES_1 ), ImmutableMap.of( "provider", "spot", "rows", 1L, "index", new Float(109.705815).doubleValue(), - "addRowsIndexConstant", new Float(111.705815).doubleValue() + "addRowsIndexConstant", new Float(111.705815).doubleValue(), + "uniques", QueryRunnerTestHelper.UNIQUES_1 ) ) ) @@ -549,15 +627,15 @@ public class TopNQueryRunnerTest public void testTopNWithNonExistentFilterInOr() { TopNQuery query = new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) .filters(providerDimension, "total_market", "upfront", "billyblank") .dimension(providerDimension) - .metric(indexMetric) + .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) - .intervals(firstToThird) - .aggregators(commonAggregators) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); List> expectedResults = Arrays.asList( @@ -569,13 +647,15 @@ public class TopNQueryRunnerTest "provider", "total_market", "rows", 4L, "index", 5351.814697265625D, - "addRowsIndexConstant", 5356.814697265625D + "addRowsIndexConstant", 5356.814697265625D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( "provider", "upfront", "rows", 4L, "index", 4875.669677734375D, - "addRowsIndexConstant", 4880.669677734375D + "addRowsIndexConstant", 4880.669677734375D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ) ) ) @@ -589,15 +669,15 @@ public class TopNQueryRunnerTest public void testTopNWithNonExistentFilter() { TopNQuery query = new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) .filters(providerDimension, "billyblank") .dimension(providerDimension) - .metric(indexMetric) + .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) - .intervals(firstToThird) - .aggregators(commonAggregators) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); TestHelper.assertExpectedResults( @@ -622,21 +702,21 @@ public class TopNQueryRunnerTest .value("billyblank") .build(), Druids.newSelectorDimFilterBuilder() - .dimension(qualityDimension) + .dimension(QueryRunnerTestHelper.qualityDimension) .value("mezzanine") .build() ) ).build(); TopNQuery query = new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) .filters(andDimFilter) .dimension(providerDimension) - .metric(indexMetric) + .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) - .intervals(firstToThird) - .aggregators(commonAggregators) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); TestHelper.assertExpectedResults( @@ -654,30 +734,30 @@ public class TopNQueryRunnerTest public void testTopNWithMultiValueDimFilter1() { TopNQuery query = new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) - .filters(placementishDimension, "m") + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .filters(QueryRunnerTestHelper.placementishDimension, "m") .dimension(providerDimension) - .metric(indexMetric) + .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) - .intervals(firstToThird) - .aggregators(commonAggregators) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); TestHelper.assertExpectedResults( Sequences.toList( runner.run( new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) - .filters(qualityDimension, "mezzanine") + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .filters(QueryRunnerTestHelper.qualityDimension, "mezzanine") .dimension(providerDimension) - .metric(indexMetric) + .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) - .intervals(firstToThird) - .aggregators(commonAggregators) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build() ), Lists.>newArrayList() ), runner.run(query) @@ -688,30 +768,30 @@ public class TopNQueryRunnerTest public void testTopNWithMultiValueDimFilter2() { TopNQuery query = new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) - .filters(placementishDimension, "m", "a", "b") - .dimension(qualityDimension) - .metric(indexMetric) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .filters(QueryRunnerTestHelper.placementishDimension, "m", "a", "b") + .dimension(QueryRunnerTestHelper.qualityDimension) + .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) - .intervals(firstToThird) - .aggregators(commonAggregators) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); TestHelper.assertExpectedResults( Sequences.toList( runner.run( new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) - .filters(qualityDimension, "mezzanine", "automotive", "business") - .dimension(qualityDimension) - .metric(indexMetric) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .filters(QueryRunnerTestHelper.qualityDimension, "mezzanine", "automotive", "business") + .dimension(QueryRunnerTestHelper.qualityDimension) + .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) - .intervals(firstToThird) - .aggregators(commonAggregators) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build() ), Lists.>newArrayList() ) @@ -723,15 +803,15 @@ public class TopNQueryRunnerTest public void testTopNWithMultiValueDimFilter3() { TopNQuery query = new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) - .filters(placementishDimension, "a") - .dimension(placementishDimension) - .metric(indexMetric) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .filters(QueryRunnerTestHelper.placementishDimension, "a") + .dimension(QueryRunnerTestHelper.placementishDimension) + .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) - .intervals(firstToThird) - .aggregators(commonAggregators) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); final ArrayList> expectedResults = Lists.newArrayList( @@ -743,13 +823,15 @@ public class TopNQueryRunnerTest "placementish", "a", "rows", 2L, "index", 283.31103515625D, - "addRowsIndexConstant", 286.31103515625D + "addRowsIndexConstant", 286.31103515625D, + "uniques", QueryRunnerTestHelper.UNIQUES_1 ), ImmutableMap.of( "placementish", "preferred", "rows", 2L, "index", 283.31103515625D, - "addRowsIndexConstant", 286.31103515625D + "addRowsIndexConstant", 286.31103515625D, + "uniques", QueryRunnerTestHelper.UNIQUES_1 ) ) ) @@ -763,15 +845,15 @@ public class TopNQueryRunnerTest public void testTopNWithMultiValueDimFilter4() { TopNQuery query = new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) - .filters(placementishDimension, "a", "b") - .dimension(placementishDimension) - .metric(indexMetric) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .filters(QueryRunnerTestHelper.placementishDimension, "a", "b") + .dimension(QueryRunnerTestHelper.placementishDimension) + .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) - .intervals(firstToThird) - .aggregators(commonAggregators) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); final ArrayList> expectedResults = Lists.newArrayList( @@ -783,19 +865,22 @@ public class TopNQueryRunnerTest "placementish", "preferred", "rows", 4L, "index", 514.868408203125D, - "addRowsIndexConstant", 519.868408203125D + "addRowsIndexConstant", 519.868408203125D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( "placementish", "a", "rows", 2L, "index", 283.31103515625D, - "addRowsIndexConstant", 286.31103515625D + "addRowsIndexConstant", 286.31103515625D, + "uniques", QueryRunnerTestHelper.UNIQUES_1 ), ImmutableMap.of( "placementish", "b", "rows", 2L, "index", 231.557373046875D, - "addRowsIndexConstant", 234.557373046875D + "addRowsIndexConstant", 234.557373046875D, + "uniques", QueryRunnerTestHelper.UNIQUES_1 ) ) ) @@ -809,15 +894,15 @@ public class TopNQueryRunnerTest public void testTopNWithMultiValueDimFilter5() { TopNQuery query = new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) - .filters(placementishDimension, "preferred") - .dimension(placementishDimension) - .metric(indexMetric) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .filters(QueryRunnerTestHelper.placementishDimension, "preferred") + .dimension(QueryRunnerTestHelper.placementishDimension) + .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) - .intervals(firstToThird) - .aggregators(commonAggregators) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); final ArrayList> expectedResults = Lists.newArrayList( @@ -829,25 +914,29 @@ public class TopNQueryRunnerTest "placementish", "preferred", "rows", 26L, "index", 12459.361190795898D, - "addRowsIndexConstant", 12486.361190795898D + "addRowsIndexConstant", 12486.361190795898D, + "uniques", QueryRunnerTestHelper.UNIQUES_9 ), ImmutableMap.of( "placementish", "p", "rows", 6L, "index", 5407.213653564453D, - "addRowsIndexConstant", 5414.213653564453D + "addRowsIndexConstant", 5414.213653564453D, + "uniques", QueryRunnerTestHelper.UNIQUES_1 ), ImmutableMap.of( "placementish", "m", "rows", 6L, "index", 5320.717338562012D, - "addRowsIndexConstant", 5327.717338562012D + "addRowsIndexConstant", 5327.717338562012D, + "uniques", QueryRunnerTestHelper.UNIQUES_1 ), ImmutableMap.of( "placementish", "t", "rows", 4L, "index", 422.3440856933594D, - "addRowsIndexConstant", 427.3440856933594D + "addRowsIndexConstant", 427.3440856933594D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ) ) ) @@ -861,14 +950,14 @@ public class TopNQueryRunnerTest public void testTopNLexicographic() { TopNQuery query = new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) .dimension(providerDimension) .metric(new LexicographicTopNMetricSpec("")) .threshold(4) - .intervals(firstToThird) - .aggregators(commonAggregators) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); List> expectedResults = Arrays.asList( @@ -880,19 +969,22 @@ public class TopNQueryRunnerTest "provider", "spot", "rows", 18L, "index", 2231.8768157958984D, - "addRowsIndexConstant", 2250.8768157958984D + "addRowsIndexConstant", 2250.8768157958984D, + "uniques", QueryRunnerTestHelper.UNIQUES_9 ), ImmutableMap.of( "provider", "total_market", "rows", 4L, "index", 5351.814697265625D, - "addRowsIndexConstant", 5356.814697265625D + "addRowsIndexConstant", 5356.814697265625D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( "provider", "upfront", "rows", 4L, "index", 4875.669677734375D, - "addRowsIndexConstant", 4880.669677734375D + "addRowsIndexConstant", 4880.669677734375D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ) ) ) @@ -906,14 +998,14 @@ public class TopNQueryRunnerTest public void testTopNLexicographicWithPreviousStop() { TopNQuery query = new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) .dimension(providerDimension) .metric(new LexicographicTopNMetricSpec("spot")) .threshold(4) - .intervals(firstToThird) - .aggregators(commonAggregators) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); List> expectedResults = Arrays.asList( @@ -925,13 +1017,15 @@ public class TopNQueryRunnerTest "provider", "total_market", "rows", 4L, "index", 5351.814697265625D, - "addRowsIndexConstant", 5356.814697265625D + "addRowsIndexConstant", 5356.814697265625D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( "provider", "upfront", "rows", 4L, "index", 4875.669677734375D, - "addRowsIndexConstant", 4880.669677734375D + "addRowsIndexConstant", 4880.669677734375D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ) ) ) @@ -945,14 +1039,14 @@ public class TopNQueryRunnerTest public void testTopNLexicographicWithNonExistingPreviousStop() { TopNQuery query = new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) .dimension(providerDimension) .metric(new LexicographicTopNMetricSpec("t")) .threshold(4) - .intervals(firstToThird) - .aggregators(commonAggregators) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); List> expectedResults = Arrays.asList( @@ -964,13 +1058,15 @@ public class TopNQueryRunnerTest "provider", "total_market", "rows", 4L, "index", 5351.814697265625D, - "addRowsIndexConstant", 5356.814697265625D + "addRowsIndexConstant", 5356.814697265625D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( "provider", "upfront", "rows", 4L, "index", 4875.669677734375D, - "addRowsIndexConstant", 4880.669677734375D + "addRowsIndexConstant", 4880.669677734375D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ) ) ) @@ -984,8 +1080,8 @@ public class TopNQueryRunnerTest public void testTopNDimExtraction() { TopNQuery query = new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) .dimension( new ExtractionDimensionSpec( providerDimension, providerDimension, new RegexDimExtractionFn("(.)") @@ -993,9 +1089,9 @@ public class TopNQueryRunnerTest ) .metric("rows") .threshold(4) - .intervals(firstToThird) - .aggregators(commonAggregators) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); List> expectedResults = Arrays.asList( @@ -1007,19 +1103,22 @@ public class TopNQueryRunnerTest "provider", "s", "rows", 18L, "index", 2231.8768157958984D, - "addRowsIndexConstant", 2250.8768157958984D + "addRowsIndexConstant", 2250.8768157958984D, + "uniques", QueryRunnerTestHelper.UNIQUES_9 ), ImmutableMap.of( "provider", "t", "rows", 4L, "index", 5351.814697265625D, - "addRowsIndexConstant", 5356.814697265625D + "addRowsIndexConstant", 5356.814697265625D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( "provider", "u", "rows", 4L, "index", 4875.669677734375D, - "addRowsIndexConstant", 4880.669677734375D + "addRowsIndexConstant", 4880.669677734375D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ) ) ) @@ -1034,14 +1133,14 @@ public class TopNQueryRunnerTest { TopNQuery query = new TopNQueryBuilder() - .dataSource(dataSource) - .granularity(allGran) + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) .dimension(providerDimension) - .metric(new InvertedTopNMetricSpec(new NumericTopNMetricSpec(indexMetric))) + .metric(new InvertedTopNMetricSpec(new NumericTopNMetricSpec(QueryRunnerTestHelper.indexMetric))) .threshold(3) - .intervals(firstToThird) - .aggregators(commonAggregators) - .postAggregators(Arrays.asList(addRowsIndexConstant)) + .intervals(QueryRunnerTestHelper.firstToThird) + .aggregators(QueryRunnerTestHelper.commonAggregators) + .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); List> expectedResults = Arrays.asList( @@ -1053,19 +1152,22 @@ public class TopNQueryRunnerTest "provider", "spot", "rows", 18L, "index", 2231.8768157958984D, - "addRowsIndexConstant", 2250.8768157958984D + "addRowsIndexConstant", 2250.8768157958984D, + "uniques", QueryRunnerTestHelper.UNIQUES_9 ), ImmutableMap.of( "provider", "upfront", "rows", 4L, "index", 4875.669677734375D, - "addRowsIndexConstant", 4880.669677734375D + "addRowsIndexConstant", 4880.669677734375D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ), ImmutableMap.of( "provider", "total_market", "rows", 4L, "index", 5351.814697265625D, - "addRowsIndexConstant", 5356.814697265625D + "addRowsIndexConstant", 5356.814697265625D, + "uniques", QueryRunnerTestHelper.UNIQUES_2 ) ) ) @@ -1074,4 +1176,4 @@ public class TopNQueryRunnerTest TestHelper.assertExpectedResults(expectedResults, runner.run(query)); } -} \ No newline at end of file +} diff --git a/processing/src/test/java/io/druid/segment/TestIndex.java b/processing/src/test/java/io/druid/segment/TestIndex.java index 0fd53185727..aba067d6f93 100644 --- a/processing/src/test/java/io/druid/segment/TestIndex.java +++ b/processing/src/test/java/io/druid/segment/TestIndex.java @@ -21,6 +21,7 @@ package io.druid.segment; import com.google.common.base.Charsets; import com.google.common.base.Throwables; +import com.google.common.hash.Hashing; import com.google.common.io.CharStreams; import com.google.common.io.InputSupplier; import com.google.common.io.LineProcessor; @@ -31,7 +32,10 @@ import io.druid.data.input.impl.TimestampSpec; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.serde.ComplexMetrics; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -52,14 +56,29 @@ public class TestIndex private static QueryableIndex mmappedIndex = null; private static QueryableIndex mergedRealtime = null; - public static final String[] COLUMNS = new String[]{"ts", "provider", "quALIty", "plAcEmEnT", "pLacementish", "iNdEx"}; + public static final String[] COLUMNS = new String[]{ + "ts", + "provider", + "quALIty", + "plAcEmEnT", + "pLacementish", + "iNdEx", + "qualiTy_Uniques" + }; public static final String[] DIMENSIONS = new String[]{"provider", "quALIty", "plAcEmEnT", "pLacementish"}; public static final String[] METRICS = new String[]{"iNdEx"}; private static final Interval DATA_INTERVAL = new Interval("2011-01-12T00:00:00.000Z/2011-04-16T00:00:00.000Z"); private static final AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ - new DoubleSumAggregatorFactory(METRICS[0], METRICS[0]) + new DoubleSumAggregatorFactory(METRICS[0], METRICS[0]), + new HyperUniquesAggregatorFactory("quality_uniques", "quality") }; + static { + if (ComplexMetrics.getSerdeForType("hyperUnique") == null) { + ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde(Hashing.murmur3_128())); + } + } + public static IncrementalIndex getIncrementalTestIndex() { synchronized (log) { From 5826d9594cc95f5a6a2b51b60bf66c974f03a755 Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 5 Mar 2014 14:29:00 -0800 Subject: [PATCH 14/66] add headers --- .../hyperloglog/ByteBitLookup.java | 19 ++++++++++++++++++ .../query/aggregation/hyperloglog/HLLCV0.java | 20 +++++++++++++++++++ .../query/aggregation/hyperloglog/HLLCV1.java | 19 ++++++++++++++++++ .../hyperloglog/HyperLogLogCollector.java | 19 ++++++++++++++++++ .../HyperUniqueFinalizingPostAggregator.java | 19 ++++++++++++++++++ .../hyperloglog/HyperUniquesAggregator.java | 19 ++++++++++++++++++ .../HyperUniquesAggregatorFactory.java | 19 ++++++++++++++++++ .../HyperUniquesBufferAggregator.java | 19 ++++++++++++++++++ .../hyperloglog/HyperUniquesSerde.java | 19 ++++++++++++++++++ .../hyperloglog/HyperLogLogCollectorTest.java | 19 ++++++++++++++++++ ...perUniqueFinalizingPostAggregatorTest.java | 19 ++++++++++++++++++ .../HyperUniquesAggregatorFactoryTest.java | 19 ++++++++++++++++++ 12 files changed, 229 insertions(+) diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/ByteBitLookup.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/ByteBitLookup.java index d5577ab046b..aebc908be9a 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/ByteBitLookup.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/ByteBitLookup.java @@ -1,3 +1,22 @@ +/* + * 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.aggregation.hyperloglog; /** diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HLLCV0.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HLLCV0.java index 67aefd73f02..281a19fc9ef 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HLLCV0.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HLLCV0.java @@ -1,9 +1,29 @@ +/* + * 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.aggregation.hyperloglog; import java.nio.ByteBuffer; /** */ +@Deprecated public class HLLCV0 extends HyperLogLogCollector { /** diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HLLCV1.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HLLCV1.java index 421f10140de..3080275ec82 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HLLCV1.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HLLCV1.java @@ -1,3 +1,22 @@ +/* + * 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.aggregation.hyperloglog; import java.nio.ByteBuffer; diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java index c056305f30c..bf32d85886c 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java @@ -1,3 +1,22 @@ +/* + * 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.aggregation.hyperloglog; import com.fasterxml.jackson.annotation.JsonValue; diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregator.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregator.java index db28ae33997..0286ddd70d9 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregator.java @@ -1,3 +1,22 @@ +/* + * 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.aggregation.hyperloglog; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregator.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregator.java index 9e174ef92a0..1aa8f6fd6d2 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregator.java @@ -1,3 +1,22 @@ +/* + * 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.aggregation.hyperloglog; import io.druid.query.aggregation.Aggregator; diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java index bd2f3183f65..99ec241699a 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java @@ -1,3 +1,22 @@ +/* + * 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.aggregation.hyperloglog; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java index feb70ccd682..290fe35a370 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java @@ -1,3 +1,22 @@ +/* + * 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.aggregation.hyperloglog; import io.druid.query.aggregation.BufferAggregator; diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java index 3ad31c1c45b..b720184fc9e 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java @@ -1,3 +1,22 @@ +/* + * 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.aggregation.hyperloglog; import com.google.common.base.Charsets; diff --git a/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollectorTest.java b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollectorTest.java index 4987b305585..8968c9a8675 100644 --- a/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollectorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollectorTest.java @@ -1,3 +1,22 @@ +/* + * 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.aggregation.hyperloglog; import com.google.common.hash.HashFunction; diff --git a/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregatorTest.java index eb8a0c23d00..5a0f0de6617 100644 --- a/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregatorTest.java @@ -1,3 +1,22 @@ +/* + * 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.aggregation.hyperloglog; import com.google.common.collect.ImmutableMap; diff --git a/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactoryTest.java b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactoryTest.java index 56efc884148..44162e8db51 100644 --- a/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactoryTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactoryTest.java @@ -1,3 +1,22 @@ +/* + * 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.aggregation.hyperloglog; import junit.framework.Assert; From 4ec1959c30d1b1fe8cab2faf33e33e726d98548c Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Fri, 7 Mar 2014 00:06:40 +0530 Subject: [PATCH 15/66] Use druid implementation of HyperLogLog remove dependency on clear spring analytics --- indexing-hadoop/pom.xml | 5 --- .../indexer/DetermineHashedPartitionsJob.java | 37 ++++++++----------- 2 files changed, 15 insertions(+), 27 deletions(-) diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index e164043244e..8eb8dabccd4 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -97,11 +97,6 @@ junit test - - com.clearspring.analytics - stream - 2.5.2 - diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java index 407cf84dec3..ae2d61a9a93 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java @@ -19,8 +19,6 @@ package io.druid.indexer; -import com.clearspring.analytics.stream.cardinality.CardinalityMergeException; -import com.clearspring.analytics.stream.cardinality.HyperLogLog; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Optional; import com.google.common.base.Throwables; @@ -36,6 +34,7 @@ import io.druid.data.input.InputRow; import io.druid.data.input.Rows; import io.druid.granularity.QueryGranularity; import io.druid.indexer.granularity.UniformGranularitySpec; +import io.druid.query.aggregation.hyperloglog.HyperLogLogCollector; import io.druid.timeline.partition.HashBasedNumberedShardSpec; import io.druid.timeline.partition.NoneShardSpec; import org.apache.hadoop.conf.Configuration; @@ -56,6 +55,7 @@ import org.joda.time.Interval; import java.io.IOException; import java.io.OutputStream; +import java.nio.ByteBuffer; import java.util.List; import java.util.Map; import java.util.Set; @@ -67,7 +67,6 @@ public class DetermineHashedPartitionsJob implements Jobby { private static final int MAX_SHARDS = 128; private static final Logger log = new Logger(DetermineHashedPartitionsJob.class); - private static final int HYPER_LOG_LOG_BIT_SIZE = 20; private final HadoopDruidIndexerConfig config; public DetermineHashedPartitionsJob( @@ -99,8 +98,8 @@ public class DetermineHashedPartitionsJob implements Jobby groupByJob.setOutputKeyClass(NullWritable.class); groupByJob.setOutputValueClass(NullWritable.class); groupByJob.setOutputFormatClass(SequenceFileOutputFormat.class); - if(!config.getSegmentGranularIntervals().isPresent()){ - groupByJob.setNumReduceTasks(1); + if (!config.getSegmentGranularIntervals().isPresent()) { + groupByJob.setNumReduceTasks(1); } JobHelper.setupClasspath(config, groupByJob); @@ -194,7 +193,7 @@ public class DetermineHashedPartitionsJob implements Jobby { private static HashFunction hashFunction = Hashing.murmur3_128(); private QueryGranularity rollupGranularity = null; - private Map hyperLogLogs; + private Map hyperLogLogs; private HadoopDruidIndexerConfig config; private boolean determineIntervals; @@ -208,9 +207,9 @@ public class DetermineHashedPartitionsJob implements Jobby Optional> intervals = config.getSegmentGranularIntervals(); if (intervals.isPresent()) { determineIntervals = false; - final ImmutableMap.Builder builder = ImmutableMap.builder(); + final ImmutableMap.Builder builder = ImmutableMap.builder(); for (final Interval bucketInterval : intervals.get()) { - builder.put(bucketInterval, new HyperLogLog(HYPER_LOG_LOG_BIT_SIZE)); + builder.put(bucketInterval, HyperLogLogCollector.makeLatestCollector()); } hyperLogLogs = builder.build(); } else { @@ -236,7 +235,7 @@ public class DetermineHashedPartitionsJob implements Jobby interval = config.getGranularitySpec().getGranularity().bucket(new DateTime(inputRow.getTimestampFromEpoch())); if (!hyperLogLogs.containsKey(interval)) { - hyperLogLogs.put(interval, new HyperLogLog(HYPER_LOG_LOG_BIT_SIZE)); + hyperLogLogs.put(interval, HyperLogLogCollector.makeLatestCollector()); } } else { final Optional maybeInterval = config.getGranularitySpec() @@ -248,9 +247,9 @@ public class DetermineHashedPartitionsJob implements Jobby interval = maybeInterval.get(); } hyperLogLogs.get(interval) - .offerHashed( + .add( hashFunction.hashBytes(HadoopDruidIndexerConfig.jsonMapper.writeValueAsBytes(groupKey)) - .asLong() + .asBytes() ); } @@ -263,10 +262,10 @@ public class DetermineHashedPartitionsJob implements Jobby map(context.getCurrentKey(), context.getCurrentValue(), context); } - for (Map.Entry entry : hyperLogLogs.entrySet()) { + for (Map.Entry entry : hyperLogLogs.entrySet()) { context.write( new LongWritable(entry.getKey().getStartMillis()), - new BytesWritable(entry.getValue().getBytes()) + new BytesWritable(entry.getValue().toByteArray()) ); } cleanup(context); @@ -294,15 +293,9 @@ public class DetermineHashedPartitionsJob implements Jobby Context context ) throws IOException, InterruptedException { - HyperLogLog aggregate = new HyperLogLog(HYPER_LOG_LOG_BIT_SIZE); + HyperLogLogCollector aggregate = HyperLogLogCollector.makeLatestCollector(); for (BytesWritable value : values) { - HyperLogLog logValue = HyperLogLog.Builder.build(getDataBytes(value)); - try { - aggregate.addAll(logValue); - } - catch (CardinalityMergeException e) { - e.printStackTrace(); // TODO: check for better handling - } + aggregate.fold(ByteBuffer.wrap(value.getBytes(), 0, value.getLength())); } Interval interval = config.getGranularitySpec().getGranularity().bucket(new DateTime(key.get())); intervals.add(interval); @@ -318,7 +311,7 @@ public class DetermineHashedPartitionsJob implements Jobby } ).writeValue( out, - aggregate.cardinality() + new Double(aggregate.estimateCardinality()).longValue() ); } finally { From ae38c9249125677c061b3096632bf23ca1eb62ac Mon Sep 17 00:00:00 2001 From: Tugdual Saunier Date: Thu, 6 Mar 2014 14:53:54 +0000 Subject: [PATCH 16/66] Fixed segments SQL queries for PostgreSQL compatibility --- .../io/druid/indexing/overlord/IndexerDBCoordinator.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java b/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java index dc02ee9d4ef..2afa4ed0dd5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java @@ -90,7 +90,7 @@ public class IndexerDBCoordinator final ResultIterator> dbSegments = handle.createQuery( String.format( - "SELECT payload FROM %s WHERE used = 1 AND dataSource = :dataSource", + "SELECT payload FROM %s WHERE used = true AND dataSource = :dataSource", dbTables.getSegmentsTable() ) ) @@ -304,8 +304,8 @@ public class IndexerDBCoordinator return handle.createQuery( String.format( DbConnector.isPostgreSQL(handle)? - "SELECT payload FROM %s WHERE dataSource = :dataSource and start >= :start and \"end\" <= :end and used = 0": - "SELECT payload FROM %s WHERE dataSource = :dataSource and start >= :start and end <= :end and used = 0", + "SELECT payload FROM %s WHERE dataSource = :dataSource and start >= :start and \"end\" <= :end and used = false": + "SELECT payload FROM %s WHERE dataSource = :dataSource and start >= :start and end <= :end and used = false", dbTables.getSegmentsTable() ) ) From 1e9e61f78676b1971055dab580b6eeeb3decad1c Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 6 Mar 2014 11:57:08 -0800 Subject: [PATCH 17/66] fix some broken coordinator endpoints --- .../server/coordinator/DruidCoordinator.java | 63 ++++++++++++------- .../server/http/CoordinatorResource.java | 56 ++++++++++++++++- .../java/io/druid/server/http/DBResource.java | 1 - .../server/http/DatasourcesResource.java | 44 ++++++++----- .../io/druid/server/http/TiersResource.java | 5 +- 5 files changed, 126 insertions(+), 43 deletions(-) diff --git a/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java index f5f202b5904..4b241060b0a 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java @@ -81,7 +81,6 @@ import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; /** @@ -95,10 +94,6 @@ public class DruidCoordinator private final Object lock = new Object(); - private volatile boolean started = false; - private volatile int leaderCounter = 0; - private volatile boolean leader = false; - private final DruidCoordinatorConfig config; private final ZkPathsConfig zkPaths; private final JacksonConfigManager configManager; @@ -115,6 +110,12 @@ public class DruidCoordinator private final ServiceAnnouncer serviceAnnouncer; private final DruidNode self; + private volatile boolean started = false; + private volatile int leaderCounter = 0; + private volatile boolean leader = false; + private volatile SegmentReplicantLookup segmentReplicantLookup = null; + + @Inject public DruidCoordinator( DruidCoordinatorConfig config, @@ -197,39 +198,55 @@ public class DruidCoordinator return loadManagementPeons; } - public Map getReplicationStatus() + public Map> getReplicationStatus() { - // find expected load per datasource - final CountingMap expectedSegmentsInCluster = new CountingMap<>(); + final Map> retVal = Maps.newHashMap(); + + if (segmentReplicantLookup == null) { + return retVal; + } + final DateTime now = new DateTime(); for (DataSegment segment : getAvailableDataSegments()) { List rules = databaseRuleManager.getRulesWithDefault(segment.getDataSource()); for (Rule rule : rules) { if (rule instanceof LoadRule && rule.appliesTo(segment, now)) { - for (Integer numReplicants : ((LoadRule) rule).getTieredReplicants().values()) { - expectedSegmentsInCluster.add(segment.getDataSource(), numReplicants); + for (Map.Entry entry : ((LoadRule) rule).getTieredReplicants().entrySet()) { + CountingMap dataSourceMap = retVal.get(entry.getKey()); + if (dataSourceMap == null) { + dataSourceMap = new CountingMap<>(); + retVal.put(entry.getKey(), dataSourceMap); + } + + int diff = Math.max( + entry.getValue() - segmentReplicantLookup.getTotalReplicants(segment.getIdentifier(), entry.getKey()), + 0 + ); + dataSourceMap.add(segment.getDataSource(), diff); } break; } } } - // find segments currently loaded per datasource - CountingMap segmentsInCluster = new CountingMap<>(); - for (DruidServer druidServer : serverInventoryView.getInventory()) { - for (DataSegment segment : druidServer.getSegments().values()) { - segmentsInCluster.add(segment.getDataSource(), 1); - } + return retVal; + } + + + public CountingMap getSegmentAvailability() + { + final CountingMap retVal = new CountingMap<>(); + + if (segmentReplicantLookup == null) { + return retVal; } - // compare available segments with currently loaded - Map loadStatus = Maps.newHashMap(); - for (Map.Entry entry : expectedSegmentsInCluster.entrySet()) { - Long actual = segmentsInCluster.get(entry.getKey()).get(); - loadStatus.put(entry.getKey(), 100 * (actual == null ? 0.0D : (double) actual) / entry.getValue().get()); + for (DataSegment segment : getAvailableDataSegments()) { + int available = (segmentReplicantLookup.getTotalReplicants(segment.getIdentifier()) == 0) ? 0 : 1; + retVal.add(segment.getDataSource(), 1 - available); } - return loadStatus; + return retVal; } public Map getLoadStatus() @@ -808,7 +825,7 @@ public class DruidCoordinator cluster.add(new ServerHolder(server, loadManagementPeons.get(server.getName()))); } - SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(cluster); + segmentReplicantLookup = SegmentReplicantLookup.make(cluster); // Stop peons for servers that aren't there anymore. final Set disappeared = Sets.newHashSet(loadManagementPeons.keySet()); diff --git a/server/src/main/java/io/druid/server/http/CoordinatorResource.java b/server/src/main/java/io/druid/server/http/CoordinatorResource.java index aea61681183..218cd81904b 100644 --- a/server/src/main/java/io/druid/server/http/CoordinatorResource.java +++ b/server/src/main/java/io/druid/server/http/CoordinatorResource.java @@ -20,6 +20,7 @@ package io.druid.server.http; import com.google.common.base.Function; +import com.google.common.collect.Collections2; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.inject.Inject; @@ -60,9 +61,14 @@ public class CoordinatorResource @Path("/loadstatus") @Produces("application/json") public Response getLoadStatus( + @QueryParam("simple") String simple, @QueryParam("full") String full ) { + if (simple != null) { + return Response.ok(coordinator.getSegmentAvailability()).build(); + } + if (full != null) { return Response.ok(coordinator.getReplicationStatus()).build(); } @@ -73,7 +79,8 @@ public class CoordinatorResource @Path("/loadqueue") @Produces("application/json") public Response getLoadQueue( - @QueryParam("simple") String simple + @QueryParam("simple") String simple, + @QueryParam("simple") String full ) { if (simple != null) { @@ -106,6 +113,51 @@ public class CoordinatorResource ) ).build(); } - return Response.ok(coordinator.getLoadManagementPeons()).build(); + + if (full != null) { + return Response.ok(coordinator.getLoadManagementPeons()).build(); + } + + return Response.ok( + Maps.transformValues( + coordinator.getLoadManagementPeons(), + new Function() + { + @Override + public Object apply(LoadQueuePeon input) + { + return new ImmutableMap.Builder<>() + .put( + "segmentsToLoad", + Collections2.transform( + input.getSegmentsToLoad(), + new Function() + { + @Override + public String apply(DataSegment segment) + { + return segment.getIdentifier(); + } + } + ) + ) + .put( + "segmentsToDrop", Collections2.transform( + input.getSegmentsToDrop(), + new Function() + { + @Override + public String apply(DataSegment segment) + { + return segment.getIdentifier(); + } + } + ) + ) + .build(); + } + } + ) + ).build(); } } \ No newline at end of file diff --git a/server/src/main/java/io/druid/server/http/DBResource.java b/server/src/main/java/io/druid/server/http/DBResource.java index f979b76961b..02277f9e79d 100644 --- a/server/src/main/java/io/druid/server/http/DBResource.java +++ b/server/src/main/java/io/druid/server/http/DBResource.java @@ -52,7 +52,6 @@ public class DBResource this.databaseSegmentManager = databaseSegmentManager; } - @GET @Path("/datasources") @Produces("application/json") diff --git a/server/src/main/java/io/druid/server/http/DatasourcesResource.java b/server/src/main/java/io/druid/server/http/DatasourcesResource.java index c77e6fee9bb..3a1a316e658 100644 --- a/server/src/main/java/io/druid/server/http/DatasourcesResource.java +++ b/server/src/main/java/io/druid/server/http/DatasourcesResource.java @@ -31,7 +31,6 @@ import io.druid.client.DruidServer; import io.druid.client.InventoryView; import io.druid.client.indexing.IndexingServiceClient; import io.druid.db.DatabaseSegmentManager; -import io.druid.segment.IndexGranularity; import io.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -125,6 +124,35 @@ public class DatasourcesResource ).build(); } + @GET + @Path("/{dataSourceName}") + @Consumes("application/json") + public Response getTheDataSource( + @PathParam("dataSourceName") final String dataSourceName + ) + { + DruidDataSource dataSource = getDataSource(dataSourceName.toLowerCase()); + if (dataSource == null) { + return Response.status(Response.Status.NOT_FOUND).build(); + } + + return Response.ok(dataSource).build(); + } + + @POST + @Path("/{dataSourceName}") + @Consumes("application/json") + public Response enableDataSource( + @PathParam("dataSourceName") final String dataSourceName + ) + { + if (!databaseSegmentManager.enableDatasource(dataSourceName)) { + return Response.status(Response.Status.NOT_FOUND).build(); + } + + return Response.status(Response.Status.OK).build(); + } + @DELETE @Path("/{dataSourceName}") @Produces("application/json") @@ -160,20 +188,6 @@ public class DatasourcesResource return Response.status(Response.Status.OK).build(); } - @POST - @Path("/{dataSourceName}") - @Consumes("application/json") - public Response enableDataSource( - @PathParam("dataSourceName") final String dataSourceName - ) - { - if (!databaseSegmentManager.enableDatasource(dataSourceName)) { - return Response.status(Response.Status.NOT_FOUND).build(); - } - - return Response.status(Response.Status.OK).build(); - } - @GET @Path("/{dataSourceName}/segments") @Produces("application/json") diff --git a/server/src/main/java/io/druid/server/http/TiersResource.java b/server/src/main/java/io/druid/server/http/TiersResource.java index 2ec84d79e32..81698e4ff3d 100644 --- a/server/src/main/java/io/druid/server/http/TiersResource.java +++ b/server/src/main/java/io/druid/server/http/TiersResource.java @@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; import com.google.common.collect.Table; import com.google.inject.Inject; +import com.metamx.common.MapUtils; import io.druid.client.DruidServer; import io.druid.client.InventoryView; @@ -70,10 +71,10 @@ public class TiersResource } Long currSize = tierMetadata.get("currSize"); - tierMetadata.put("currSize", (currSize == null) ? 0 : currSize + druidServer.getCurrSize()); + tierMetadata.put("currSize", ((currSize == null) ? 0 : currSize) + druidServer.getCurrSize()); Long maxSize = tierMetadata.get("maxSize"); - tierMetadata.put("maxSize", (maxSize == null) ? 0 : maxSize + druidServer.getMaxSize()); + tierMetadata.put("maxSize", ((maxSize == null) ? 0 : maxSize) + druidServer.getMaxSize()); } return builder.entity(metadata).build(); } From 14a514941982ec6a7bc3257773b23abf1215607f Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 6 Mar 2014 12:43:28 -0800 Subject: [PATCH 18/66] minor javascript problem --- server/src/main/resources/static/js/rules-0.0.2.js | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/server/src/main/resources/static/js/rules-0.0.2.js b/server/src/main/resources/static/js/rules-0.0.2.js index 78ff948484a..e7268620a43 100644 --- a/server/src/main/resources/static/js/rules-0.0.2.js +++ b/server/src/main/resources/static/js/rules-0.0.2.js @@ -140,6 +140,10 @@ function makeJSON() { function makeTiersDropdown(selTier) { var retVal = "tier