From 4c8495662f672f146bd1f7eb64cd3bd3096a7aed Mon Sep 17 00:00:00 2001 From: Yuval Oren Date: Wed, 15 Jan 2014 15:38:57 -0800 Subject: [PATCH 001/217] 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 6e880f8b24947bc5b775f7bb18f34f60f236f15b Mon Sep 17 00:00:00 2001 From: Igal Levy Date: Thu, 16 Jan 2014 15:16:04 -0800 Subject: [PATCH 002/217] tweaked naming --- docs/content/toc.textile | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/content/toc.textile b/docs/content/toc.textile index 23b4023edaf..e430534c42d 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -38,15 +38,15 @@ h2. Querying ** "Granularities":./Granularities.html ** "DimensionSpecs":./DimensionSpecs.html * Query Types -** "GroupByQuery":./GroupByQuery.html +** "GroupBy":./GroupByQuery.html *** "OrderBy":./OrderBy.html *** "Having":./Having.html -** "SearchQuery":./SearchQuery.html +** "Search":./SearchQuery.html *** "SearchQuerySpec":./SearchQuerySpec.html -** "SegmentMetadataQuery":./SegmentMetadataQuery.html -** "TimeBoundaryQuery":./TimeBoundaryQuery.html -** "TimeseriesQuery":./TimeseriesQuery.html -** "TopNQuery":./TopNQuery.html +** "Segment Metadata":./SegmentMetadataQuery.html +** "Time Boundary":./TimeBoundaryQuery.html +** "Timeseries":./TimeseriesQuery.html +** "TopN":./TopNQuery.html *** "TopNMetricSpec":./TopNMetricSpec.html h2. Architecture From 53b6392dce0b9436a325508c0e3a90a2dffbafbd Mon Sep 17 00:00:00 2001 From: Igal Levy Date: Thu, 16 Jan 2014 15:37:07 -0800 Subject: [PATCH 003/217] added titles; fixed a few typos --- docs/content/About-Experimental-Features.md | 3 ++- docs/content/Aggregations.md | 1 + docs/content/Deep-Storage.md | 1 + docs/content/DimensionSpecs.md | 4 +++- docs/content/Filters.md | 3 ++- docs/content/GeographicQueries.md | 1 + docs/content/Granularities.md | 1 + docs/content/GroupByQuery.md | 3 ++- docs/content/Having.md | 1 + docs/content/Indexing-Service.md | 1 + docs/content/MySQL.md | 1 + docs/content/OrderBy.md | 1 + docs/content/Post-aggregations.md | 1 + docs/content/Rule-Configuration.md | 1 + docs/content/SearchQuery.md | 1 + docs/content/SearchQuerySpec.md | 1 + docs/content/SegmentMetadataQuery.md | 1 + docs/content/Tasks.md | 1 + docs/content/TimeBoundaryQuery.md | 1 + docs/content/Versioning.md | 1 + docs/content/ZooKeeper.md | 1 + 21 files changed, 26 insertions(+), 4 deletions(-) diff --git a/docs/content/About-Experimental-Features.md b/docs/content/About-Experimental-Features.md index 72244df48e1..5fe74527bc3 100644 --- a/docs/content/About-Experimental-Features.md +++ b/docs/content/About-Experimental-Features.md @@ -1,4 +1,5 @@ --- layout: doc_page --- -Experimental features are features we have developed but have not fully tested in a production environment. If you choose to try them out, there will likely to edge cases that we have not covered. We would love feedback on any of these features, whether they are bug reports, suggestions for improvement, or letting us know they work as intended. \ No newline at end of file +# About Experimental Features +Experimental features are features we have developed but have not fully tested in a production environment. If you choose to try them out, there will likely be edge cases that we have not covered. We would love feedback on any of these features, whether they are bug reports, suggestions for improvement, or letting us know they work as intended. diff --git a/docs/content/Aggregations.md b/docs/content/Aggregations.md index 3beec02a73f..93bfb76c90e 100644 --- a/docs/content/Aggregations.md +++ b/docs/content/Aggregations.md @@ -1,6 +1,7 @@ --- layout: doc_page --- +# Aggregations Aggregations are specifications of processing over metrics available in Druid. Available aggregations are: diff --git a/docs/content/Deep-Storage.md b/docs/content/Deep-Storage.md index 5e8f668fe36..bf01571c6e9 100644 --- a/docs/content/Deep-Storage.md +++ b/docs/content/Deep-Storage.md @@ -1,6 +1,7 @@ --- layout: doc_page --- +# Deep Storage Deep storage is where segments are stored. It is a storage mechanism that Druid does not provide. This deep storage infrastructure defines the level of durability of your data, as long as Druid nodes can see this storage infrastructure and get at the segments stored on it, you will not lose data no matter how many Druid nodes you lose. If segments disappear from this storage layer, then you will lose whatever data those segments represented. The currently supported types of deep storage follow. diff --git a/docs/content/DimensionSpecs.md b/docs/content/DimensionSpecs.md index bb1dda63221..c9cb351343e 100644 --- a/docs/content/DimensionSpecs.md +++ b/docs/content/DimensionSpecs.md @@ -1,6 +1,8 @@ --- layout: doc_page --- +# Transforming Dimension Values +The following JSON fields can be used in a query to operate on dimension values. ## DimensionSpec @@ -8,7 +10,7 @@ layout: doc_page ### DefaultDimensionSpec -Returns dimension values as is and optionally renames renames the dimension. +Returns dimension values as is and optionally renames the dimension. ```json { "type" : "default", "dimension" : , "outputName": } diff --git a/docs/content/Filters.md b/docs/content/Filters.md index 8564b84e6c5..0fe931dbb96 100644 --- a/docs/content/Filters.md +++ b/docs/content/Filters.md @@ -1,6 +1,7 @@ --- layout: doc_page --- +#Query Filters A filter is a JSON object indicating which rows of data should be included in the computation for a query. It’s essentially the equivalent of the WHERE clause in SQL. Druid supports the following types of filters. ### Selector filter @@ -78,4 +79,4 @@ The following matches any dimension values for the dimension `name` between `'ba "dimension" : "name", "function" : "function(x) { return(x >= 'bar' && x <= 'foo') }" } -``` \ No newline at end of file +``` diff --git a/docs/content/GeographicQueries.md b/docs/content/GeographicQueries.md index 165f1d4fe40..f85fccf3531 100644 --- a/docs/content/GeographicQueries.md +++ b/docs/content/GeographicQueries.md @@ -1,6 +1,7 @@ --- layout: doc_page --- +# Geographic Queries Druid supports filtering specially spatially indexed columns based on an origin and a bound. # Spatial Indexing diff --git a/docs/content/Granularities.md b/docs/content/Granularities.md index b71a5b1a8d8..ae0ffe2105d 100644 --- a/docs/content/Granularities.md +++ b/docs/content/Granularities.md @@ -1,6 +1,7 @@ --- layout: doc_page --- +# Aggregation Granularity The granularity field determines how data gets bucketed across the time dimension, i.e how it gets aggregated by hour, day, minute, etc. It can be specified either as a string for simple granularities or as an object for arbitrary granularities. diff --git a/docs/content/GroupByQuery.md b/docs/content/GroupByQuery.md index dd7f49f7179..9edca5d2861 100644 --- a/docs/content/GroupByQuery.md +++ b/docs/content/GroupByQuery.md @@ -1,6 +1,7 @@ --- layout: doc_page --- +# groupBy Queries These types of queries take a groupBy query object and return an array of JSON objects where each object represents a grouping asked for by the query. Note: If you only want to do straight aggreagates for some time range, we highly recommend using [TimeseriesQueries](TimeseriesQuery.html) instead. The performance will be substantially better. An example groupBy query object is shown below: @@ -86,4 +87,4 @@ To pull it all together, the above query would return *n\*m* data points, up to }, ... ] -``` \ No newline at end of file +``` diff --git a/docs/content/Having.md b/docs/content/Having.md index 9b37fdb4ba1..e3528be48b7 100644 --- a/docs/content/Having.md +++ b/docs/content/Having.md @@ -1,6 +1,7 @@ --- layout: doc_page --- +# Filter groupBy Query Results A having clause is a JSON object identifying which rows from a groupBy query should be returned, by specifying conditions on aggregated values. It is essentially the equivalent of the HAVING clause in SQL. diff --git a/docs/content/Indexing-Service.md b/docs/content/Indexing-Service.md index 2f15b200025..6c6deb3fee3 100644 --- a/docs/content/Indexing-Service.md +++ b/docs/content/Indexing-Service.md @@ -1,6 +1,7 @@ --- layout: doc_page --- +# Druid Indexing Service The indexing service is a highly-available, distributed service that runs indexing related tasks. Indexing service [tasks](Tasks.html) create (and sometimes destroy) Druid [segments](Segments.html). The indexing service has a master/slave like architecture. The indexing service is composed of three main components: a peon component that can run a single task, a [Middle Manager](Middlemanager.html) component that manages peons, and an overlord component that manages task distribution to middle managers. diff --git a/docs/content/MySQL.md b/docs/content/MySQL.md index bb352b5780d..c2b542a50dd 100644 --- a/docs/content/MySQL.md +++ b/docs/content/MySQL.md @@ -1,6 +1,7 @@ --- layout: doc_page --- +# MySQL Database MySQL is an external dependency of Druid. We use it to store various metadata about the system, but not to store the actual data. There are a number of tables used for various purposes described below. Segments Table diff --git a/docs/content/OrderBy.md b/docs/content/OrderBy.md index 7b8f88edfb1..97f999cad83 100644 --- a/docs/content/OrderBy.md +++ b/docs/content/OrderBy.md @@ -1,6 +1,7 @@ --- layout: doc_page --- +# Sort groupBy Query Results The orderBy field provides the functionality to sort and limit the set of results from a groupBy query. If you group by a single dimension and are ordering by a single metric, we highly recommend using [TopN Queries](TopNQuery.html) instead. The performance will be substantially better. Available options are: ### DefaultLimitSpec diff --git a/docs/content/Post-aggregations.md b/docs/content/Post-aggregations.md index 7fd91d0ea99..527d64e7971 100644 --- a/docs/content/Post-aggregations.md +++ b/docs/content/Post-aggregations.md @@ -1,6 +1,7 @@ --- layout: doc_page --- +# Post-Aggregations Post-aggregations are specifications of processing that should happen on aggregated values as they come out of Druid. If you include a post aggregation as part of a query, make sure to include all aggregators the post-aggregator requires. There are several post-aggregators available. diff --git a/docs/content/Rule-Configuration.md b/docs/content/Rule-Configuration.md index cba6bdd3924..bf8b8a9792d 100644 --- a/docs/content/Rule-Configuration.md +++ b/docs/content/Rule-Configuration.md @@ -1,6 +1,7 @@ --- layout: doc_page --- +# Configuring Rules for Coordinator Nodes Note: It is recommended that the coordinator console is used to configure rules. However, the coordinator node does have HTTP endpoints to programmatically configure rules. Load Rules diff --git a/docs/content/SearchQuery.md b/docs/content/SearchQuery.md index 2cfc726d60b..b0c55c57e94 100644 --- a/docs/content/SearchQuery.md +++ b/docs/content/SearchQuery.md @@ -1,6 +1,7 @@ --- layout: doc_page --- +# Search Queries A search query returns dimension values that match the search specification. ```json diff --git a/docs/content/SearchQuerySpec.md b/docs/content/SearchQuerySpec.md index 46589554075..bb2e782e93f 100644 --- a/docs/content/SearchQuerySpec.md +++ b/docs/content/SearchQuerySpec.md @@ -1,6 +1,7 @@ --- layout: doc_page --- +# Refining Search Queries Search query specs define how a "match" is defined between a search value and a dimension value. The available search query specs are: InsensitiveContainsSearchQuerySpec diff --git a/docs/content/SegmentMetadataQuery.md b/docs/content/SegmentMetadataQuery.md index c65e6957d43..7efe37d519a 100644 --- a/docs/content/SegmentMetadataQuery.md +++ b/docs/content/SegmentMetadataQuery.md @@ -1,6 +1,7 @@ --- layout: doc_page --- +# Segment Metadata Queries Segment metadata queries return per segment information about: * Cardinality of all columns in the segment diff --git a/docs/content/Tasks.md b/docs/content/Tasks.md index 5ad56b55d06..eafb01d0244 100644 --- a/docs/content/Tasks.md +++ b/docs/content/Tasks.md @@ -1,6 +1,7 @@ --- layout: doc_page --- +# Tasks Tasks are run on middle managers and always operate on a single data source. There are several different types of tasks. diff --git a/docs/content/TimeBoundaryQuery.md b/docs/content/TimeBoundaryQuery.md index dd07169b103..03c692b9474 100644 --- a/docs/content/TimeBoundaryQuery.md +++ b/docs/content/TimeBoundaryQuery.md @@ -1,6 +1,7 @@ --- layout: doc_page --- +# Time Boundary Queries Time boundary queries return the earliest and latest data points of a data set. The grammar is: ```json diff --git a/docs/content/Versioning.md b/docs/content/Versioning.md index 74b3acf8aa3..09df6b5930d 100644 --- a/docs/content/Versioning.md +++ b/docs/content/Versioning.md @@ -1,6 +1,7 @@ --- layout: doc_page --- +# Versioning Druid This page discusses how we do versioning and provides information on our stable releases. Versioning Strategy diff --git a/docs/content/ZooKeeper.md b/docs/content/ZooKeeper.md index 303e96f351f..4730f4b5043 100644 --- a/docs/content/ZooKeeper.md +++ b/docs/content/ZooKeeper.md @@ -1,6 +1,7 @@ --- layout: doc_page --- +# ZooKeeper Druid uses [ZooKeeper](http://zookeeper.apache.org/) (ZK) for management of current cluster state. The operations that happen over ZK are 1. [Coordinator](Coordinator.html) leader election From 53f69c57919f0a80cc338f1e60472ea71c86de72 Mon Sep 17 00:00:00 2001 From: Igal Levy Date: Mon, 20 Jan 2014 17:12:36 -0800 Subject: [PATCH 004/217] updated to not refer to old wiki --- docs/content/Tutorial:-A-First-Look-at-Druid.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index ffcc29e8c32..0854b715ae2 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -330,6 +330,6 @@ Druid is even more fun if you load your own data into it! To learn how to load y Additional Information ---------------------- -This tutorial is merely showcasing a small fraction of what Druid can do. If you are interested in more information about Druid, including setting up a more sophisticated Druid cluster, please read the other links in our wiki. +This tutorial is merely showcasing a small fraction of what Druid can do. If you are interested in more information about Druid, including setting up a more sophisticated Druid cluster, read more of the Druid documentation and the blogs found on druid.io. And thus concludes our journey! Hopefully you learned a thing or two about Druid real-time ingestion, querying Druid, and how Druid can be used to solve problems. If you have additional questions, feel free to post in our [google groups page](https://groups.google.com/forum/#!forum/druid-development). From 010abf2552b1ae8b91b2f436c7f458e214e2038f Mon Sep 17 00:00:00 2001 From: Igal Levy Date: Tue, 21 Jan 2014 10:05:03 -0800 Subject: [PATCH 005/217] fixed typo --- docs/content/Tutorial:-A-First-Look-at-Druid.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index 0854b715ae2..ab9d091ae5a 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -323,7 +323,7 @@ Feel free to tweak other query parameters to answer other questions you may have Next Steps ---------- -What to know even more information about the Druid Cluster? Check out [The Druid Cluster](Tutorial%3A-The-Druid-Cluster.html) +What to know even more information about the Druid Cluster? Check out [The Druid Cluster](Tutorial%3A-The-Druid-Cluster.html). Druid is even more fun if you load your own data into it! To learn how to load your data, see [Loading Your Data](Tutorial%3A-Loading-Your-Data-Part-1.html). From 0c16f24ba166834877c88cbd5dc32ea0c7f748ac Mon Sep 17 00:00:00 2001 From: Igal Levy Date: Tue, 21 Jan 2014 11:03:07 -0800 Subject: [PATCH 006/217] added note about slow start first time around due to fetching of dependencies --- docs/content/Tutorial:-A-First-Look-at-Druid.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index ab9d091ae5a..6b6557bd7e0 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -80,7 +80,7 @@ Let's start doing stuff. You can start a Druid [Realtime](Realtime.html) node by Select "wikipedia". -Once the node starts up you will see a bunch of logs about setting up properties and connecting to the data source. If everything was successful, you should see messages of the form shown below. +Note that the first time you start the example, it may take some extra time due to its fetching various dependencies. Once the node starts up you will see a bunch of logs about setting up properties and connecting to the data source. If everything was successful, you should see messages of the form shown below. ``` 2013-09-04 19:33:11,922 INFO [main] org.eclipse.jetty.server.AbstractConnector - Started SelectChannelConnector@0.0.0.0:8083 @@ -118,7 +118,7 @@ Select "wikipedia" once again. This script issues [GroupByQueries](GroupByQuery. This is a **groupBy** query, which you may be familiar with from SQL. We are grouping, or aggregating, via the `dimensions` field: `["page"]`. We are **filtering** via the `namespace` dimension, to only look at edits on `articles`. Our **aggregations** are what we are calculating: a count of the number of data rows, and a count of the number of edits that have occurred. -The result looks something like this: +The result looks something like this (when it's prettified): ```json [ From 5aacfe72e1e0715b7c90093575540669a05ec58e Mon Sep 17 00:00:00 2001 From: Igal Levy Date: Fri, 24 Jan 2014 12:26:38 -0800 Subject: [PATCH 007/217] minor corrections and edits for flow --- docs/content/Tutorial:-The-Druid-Cluster.md | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index b4ddb2b043b..d76e01e097b 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -7,15 +7,13 @@ Welcome back! In our first [tutorial](Tutorial%3A-A-First-Look-at-Druid.html), w This tutorial will hopefully answer these questions! -In this tutorial, we will set up other types of Druid nodes as well as and external dependencies for a fully functional Druid cluster. The architecture of Druid is very much like the [Megazord](http://www.youtube.com/watch?v=7mQuHh1X4H4) from the popular 90s show Mighty Morphin' Power Rangers. Each Druid node has a specific purpose and the nodes come together to form a fully functional system. +In this tutorial, we will set up other types of Druid nodes and external dependencies for a fully functional Druid cluster. The architecture of Druid is very much like the [Megazord](http://www.youtube.com/watch?v=7mQuHh1X4H4) from the popular 90s show Mighty Morphin' Power Rangers. Each Druid node has a specific purpose and the nodes come together to form a fully functional system. ## Downloading Druid If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first. -You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.51-bin.tar.gz) - -and untar the contents within by issuing: +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.51-bin.tar.gz) and untar the contents within by issuing: ```bash tar -zxvf druid-services-*-bin.tar.gz @@ -32,9 +30,9 @@ For deep storage, we have made a public S3 bucket (static.druid.io) available wh #### Setting up MySQL -1. If you don't already have it, download MySQL Community Server here: [http://dev.mysql.com/downloads/mysql/](http://dev.mysql.com/downloads/mysql/) -2. Install MySQL -3. Create a druid user and database +1. If you don't already have it, download MySQL Community Server here: [http://dev.mysql.com/downloads/mysql/](http://dev.mysql.com/downloads/mysql/). +2. Install MySQL. +3. Create a druid user and database. ```bash mysql -u root @@ -88,7 +86,7 @@ Metrics (things to aggregate over): ## The Cluster -Let's start up a few nodes and download our data. First things though, let's make sure we have config directory where we will store configs for our various nodes: +Let's start up a few nodes and download our data. First, let's make sure we have configs in the config directory for our various nodes. Issue the following from the Druid home directory: ``` ls config From 3301d5d510b3ba203901d5ac435f8b0844c1440c Mon Sep 17 00:00:00 2001 From: Igal Levy Date: Fri, 24 Jan 2014 13:14:56 -0800 Subject: [PATCH 008/217] added a little padding to table cells to improve look and readablity --- docs/content/css/docs.css | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/content/css/docs.css b/docs/content/css/docs.css index da63bc09095..c59923a98ca 100644 --- a/docs/content/css/docs.css +++ b/docs/content/css/docs.css @@ -53,4 +53,8 @@ .doc-content table code { background-color: transparent; -} \ No newline at end of file +} + +td, th { + padding: 5px; +} From f2c1f798b9858cb96b3094529a8ec479e19dd01b Mon Sep 17 00:00:00 2001 From: Yuval Oren Date: Fri, 24 Jan 2014 15:03:48 -0800 Subject: [PATCH 009/217] 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 010/217] 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 011/217] 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 06b73b366a6d0b12d4bd2bb18023e8ce2bcdfff5 Mon Sep 17 00:00:00 2001 From: Igal Levy Date: Fri, 24 Jan 2014 16:48:19 -0800 Subject: [PATCH 012/217] minor rewriting for flow; fixed some wrong links --- docs/content/Tasks.md | 25 +++---------------------- 1 file changed, 3 insertions(+), 22 deletions(-) diff --git a/docs/content/Tasks.md b/docs/content/Tasks.md index eafb01d0244..5e5ab3f60c7 100644 --- a/docs/content/Tasks.md +++ b/docs/content/Tasks.md @@ -159,34 +159,15 @@ The indexing service can also run real-time tasks. These tasks effectively trans } ``` -Id: -The ID of the task. Not required. - -Resource: -A JSON object used for high availability purposes. Not required. - |Field|Type|Description|Required| |-----|----|-----------|--------| +|id|String|The ID of the task.|No| +|Resource|JSON object|Used for high availability purposes.|No| |availabilityGroup|String|An uniqueness identifier for the task. Tasks with the same availability group will always run on different middle managers. Used mainly for replication. |yes| |requiredCapacity|Integer|How much middle manager capacity this task will take.|yes| -Schema: -See [Schema](Realtime.html). +For schema, fireDepartmentConfig, windowPeriod, segmentGranularity, and rejectionPolicy, see the [realtime-ingestion doc](Realtime-ingestion.html). For firehose configuration, see [Firehose](Firehose.html). -Fire Department Config: -See [Config](Realtime.html). - -Firehose: -See [Firehose](Firehose.html). - -Window Period: -See [Realtime](Realtime.html). - -Segment Granularity: -See [Realtime](Realtime.html). - -Rejection Policy: -See [Realtime](Realtime.html). Segment Merging Tasks --------------------- From 1bca1c47b7da7cbb63c7430905e5b09688f36c6c Mon Sep 17 00:00:00 2001 From: Igal Levy Date: Mon, 27 Jan 2014 17:29:05 -0800 Subject: [PATCH 013/217] creating a doc to cover ingestion of csv and tsv formats --- docs/content/Data_formats.md | 86 ++++++++++++++++++++++++++++++++++++ 1 file changed, 86 insertions(+) create mode 100644 docs/content/Data_formats.md diff --git a/docs/content/Data_formats.md b/docs/content/Data_formats.md new file mode 100644 index 00000000000..da02289e80c --- /dev/null +++ b/docs/content/Data_formats.md @@ -0,0 +1,86 @@ +--- +layout: doc_page +--- + +# Data Formats for Ingestion +Druid can ingest data in JSON, CSV, or TSV. While most examples in the documentation use data in JSON format, it is not difficult to configure Druid to ingest CSV or TSV data. + +## Formatting the Data +The following are three samples of the data used in the [Wikipedia example](Tutorial:-Loading-Your-Data-Part-1.html). + +_JSON_ + +``` +{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143} +{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} +{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} +{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} +{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "cancer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} +``` + +_CSV_ + +``` +2013-08-31T01:02:33Z,"Gypsy Danger","en","nuclear","true","true","false","false","article","North America","United States","Bay Area","San Francisco",57,200,-143 +2013-08-31T03:32:45Z,"Striker Eureka","en","speed","false","true","true","false","wikipedia","Australia","Australia","Cantebury","Syndey",459,129,330 +2013-08-31T07:11:21Z,"Cherno Alpha","ru","masterYi","false","true","true","false","article","Asia","Russia","Oblast","Moscow",123,12,111 +2013-08-31T11:58:39Z,"Crimson Typhoon","zh","triplets","true","false","true","false","wikipedia","Asia","China","Shanxi","Taiyuan",905,5,900 +2013-08-31T12:41:27Z,"Coyote Tango","ja","cancer","true","false","true","false","wikipedia","Asia","Japan","Kanto","Tokyo",1,10,-9 +``` + +_TSV_ + +``` +2013-08-31T01:02:33Z "Gypsy Danger" "en" "nuclear" "true" "true" "false" "false" "article" "North America" "United States" "Bay Area" "San Francisco" 57 200 -143 +2013-08-31T03:32:45Z "Striker Eureka" "en" "speed" "false" "true" "true" "false" "wikipedia" "Australia" "Australia" "Cantebury" "Syndey" 459 129 330 +2013-08-31T07:11:21Z "Cherno Alpha" "ru" "masterYi" "false" "true" "true" "false" "article" "Asia" "Russia" "Oblast" "Moscow" 123 12 111 +2013-08-31T11:58:39Z "Crimson Typhoon" "zh" "triplets" "true" "false" "true" "false" "wikipedia" "Asia" "China" "Shanxi" "Taiyuan" 905 5 900 +2013-08-31T12:41:27Z "Coyote Tango" "ja" "cancer" "true" "false" "true" "false" "wikipedia" "Asia" "Japan" "Kanto" "Tokyo" 1 10 -9 +``` + +Note that the CSV and TSV data do not contain column heads. This becomes important when you specify the data for ingesting. + +## Configuring Ingestion For the Indexing Service +If you use the [indexing service](Indexing-Service.html) for ingesting the data, a [task](Tasks.html) must be configured and submitted. Tasks are configured with a JSON object which, among other things, specifies the data source and type. In the Wikipedia example, JSON data was read from a local file. The task spec contains a firehose element to specify this: + + ... + "firehose" : { + "type" : "local", + "baseDir" : "examples/indexing", + "filter" : "wikipedia_data.json", + "parser" : { + "timestampSpec" : { + "column" : "timestamp" + }, + "data" : { + "format" : "json", + "dimensions" : ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"] + } + } + } + ... + +Specified here are the location of the datafile, the timestamp column, the format of the data, and the columns that will become dimensions in Druid. + +Since the CSV data does not contain the column names, they will have to be added before that data can be processed: + + ... + "firehose" : { + "type" : "local", + "baseDir" : "examples/indexing/", + "filter" : "wikipedia_data.csv", + "parser" : { + "timestampSpec" : { + "column" : "timestamp" + }, + "data" : { + "type" : "csv", + "columns" : ["timestamp","page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"], + "dimensions" : ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"] + } + } + } + ... + +Note also that the filename extension and the data type were changed to "csv". For the TSV data, the same changes are made but with "tsv" for the filename extension and the data type. + From 01e8d9e0a0f33cdd77fde650260f7b443f2238a2 Mon Sep 17 00:00:00 2001 From: Igal Levy Date: Mon, 27 Jan 2014 17:29:49 -0800 Subject: [PATCH 014/217] fixed typo --- docs/content/Segments.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/Segments.md b/docs/content/Segments.md index 644fea72579..eeefd5b80d3 100644 --- a/docs/content/Segments.md +++ b/docs/content/Segments.md @@ -19,7 +19,7 @@ datasource_intervalStart_intervalEnd_version_partitionNum Segment Components ------------------ -A segment is compromised of several files, listed below. +A segment is comprised of several files, listed below. * `version.bin` From e9cf9355186381a543d8fc8f1b8aea156509fef1 Mon Sep 17 00:00:00 2001 From: Yuval Oren Date: Mon, 27 Jan 2014 18:58:25 -0800 Subject: [PATCH 015/217] 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 7ca87d59dff7af608f3a583aa5419bb79c349b23 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Fri, 31 Jan 2014 00:49:11 +0530 Subject: [PATCH 016/217] Determine partitions using cardinality --- indexing-hadoop/pom.xml | 5 + ...eterminePartitionsUsingCardinalityJob.java | 306 ++++++++++++++++++ .../io/druid/indexer/IndexGeneratorJob.java | 7 +- .../java/io/druid/guice/ServerModule.java | 4 +- .../partition/HashBasedNumberedShardSpec.java | 93 ++++++ .../shard/HashBasedNumberedShardSpecTest.java | 129 ++++++++ 6 files changed, 537 insertions(+), 7 deletions(-) create mode 100644 indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsUsingCardinalityJob.java create mode 100644 server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java create mode 100644 server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 726a106e47c..1b0eb508fa6 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -97,6 +97,11 @@ junit test + + com.clearspring.analytics + stream + 2.5.2 + diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsUsingCardinalityJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsUsingCardinalityJob.java new file mode 100644 index 00000000000..505d47b0e45 --- /dev/null +++ b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsUsingCardinalityJob.java @@ -0,0 +1,306 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 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.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; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSortedSet; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.hash.HashFunction; +import com.google.common.hash.Hashing; +import com.google.common.io.Closeables; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; +import io.druid.data.input.InputRow; +import io.druid.granularity.QueryGranularity; +import io.druid.timeline.partition.HashBasedNumberedShardSpec; +import io.druid.timeline.partition.NoneShardSpec; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.Reducer; +import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; +import org.joda.time.DateTime; +import org.joda.time.DateTimeComparator; +import org.joda.time.Interval; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Determines appropriate ShardSpecs for a job by determining approximate cardinality of data set + */ +public class DeterminePartitionsUsingCardinalityJob implements Jobby +{ + private static final int MAX_SHARDS = 128; + private static final Logger log = new Logger(DeterminePartitionsUsingCardinalityJob.class); + private final HadoopDruidIndexerConfig config; + + public DeterminePartitionsUsingCardinalityJob( + HadoopDruidIndexerConfig config + ) + { + this.config = config; + } + + public static void injectSystemProperties(Job job) + { + final Configuration conf = job.getConfiguration(); + for (String propName : System.getProperties().stringPropertyNames()) { + if (propName.startsWith("hadoop.")) { + conf.set(propName.substring("hadoop.".length()), System.getProperty(propName)); + } + } + } + + public boolean run() + { + try { + /* + * Group by (timestamp, dimensions) so we can correctly count dimension values as they would appear + * in the final segment. + */ + long startTime = System.currentTimeMillis(); + final Job groupByJob = new Job( + new Configuration(), + String.format("%s-determine_cardinality_grouped-%s", config.getDataSource(), config.getIntervals()) + ); + + injectSystemProperties(groupByJob); + groupByJob.setInputFormatClass(TextInputFormat.class); + groupByJob.setMapperClass(DetermineCardinalityMapper.class); + groupByJob.setMapOutputKeyClass(LongWritable.class); + groupByJob.setMapOutputValueClass(BytesWritable.class); + groupByJob.setReducerClass(DetermineCardinalityReducer.class); + groupByJob.setOutputKeyClass(NullWritable.class); + groupByJob.setOutputValueClass(NullWritable.class); + groupByJob.setOutputFormatClass(SequenceFileOutputFormat.class); + JobHelper.setupClasspath(config, groupByJob); + + config.addInputPaths(groupByJob); + config.intoConfiguration(groupByJob); + FileOutputFormat.setOutputPath(groupByJob, config.makeGroupedDataDir()); + + groupByJob.submit(); + log.info("Job %s submitted, status available at: %s", groupByJob.getJobName(), groupByJob.getTrackingURL()); + + if (!groupByJob.waitForCompletion(true)) { + log.error("Job failed: %s", groupByJob.getJobID()); + return false; + } + + /* + * Load partitions determined by the previous job. + */ + + log.info("Job completed, loading up partitions for intervals[%s].", config.getSegmentGranularIntervals()); + FileSystem fileSystem = null; + Map> shardSpecs = Maps.newTreeMap(DateTimeComparator.getInstance()); + for (Interval segmentGranularity : config.getSegmentGranularIntervals()) { + DateTime bucket = segmentGranularity.getStart(); + + final Path partitionInfoPath = config.makeSegmentPartitionInfoPath(new Bucket(0, bucket, 0)); + if (fileSystem == null) { + fileSystem = partitionInfoPath.getFileSystem(groupByJob.getConfiguration()); + } + if (fileSystem.exists(partitionInfoPath)) { + Long cardinality = config.jsonMapper.readValue( + Utils.openInputStream(groupByJob, partitionInfoPath), new TypeReference() + { + } + ); + int numberOfShards = (int) Math.ceil((double) cardinality / config.getTargetPartitionSize()); + + if (numberOfShards > MAX_SHARDS) { + numberOfShards = MAX_SHARDS; + } + + List actualSpecs = Lists.newArrayListWithExpectedSize(numberOfShards); + if (numberOfShards == 1) { + actualSpecs.add(new HadoopyShardSpec(new NoneShardSpec(), 0)); + } else { + int shardCount = 0; + for (int i = 0; i < numberOfShards; ++i) { + actualSpecs.add(new HadoopyShardSpec(new HashBasedNumberedShardSpec(i, numberOfShards), shardCount++)); + log.info("DateTime[%s], partition[%d], spec[%s]", bucket, i, actualSpecs.get(i)); + } + + shardSpecs.put(bucket, actualSpecs); + } + } else { + log.info("Path[%s] didn't exist!?", partitionInfoPath); + } + } + config.setShardSpecs(shardSpecs); + log.info("Determine partitions Using cardinality took %d millis", (System.currentTimeMillis() - startTime)); + + return true; + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + public static class DetermineCardinalityMapper extends HadoopDruidIndexerMapper + { + private static HashFunction hashFunction = null; + private QueryGranularity rollupGranularity = null; + private Map hyperLogLogs; + private HadoopDruidIndexerConfig config; + + @Override + protected void setup(Context context) + throws IOException, InterruptedException + { + super.setup(context); + rollupGranularity = getConfig().getRollupSpec().getRollupGranularity(); + hashFunction = Hashing.murmur3_128(); + config = HadoopDruidIndexerConfigBuilder.fromConfiguration(context.getConfiguration()); + + final ImmutableMap.Builder builder = ImmutableMap.builder(); + for (final Interval bucketInterval : config.getGranularitySpec().bucketIntervals()) { + builder.put(bucketInterval, new HyperLogLog(20)); + } + hyperLogLogs = builder.build(); + } + + @Override + protected void innerMap( + InputRow inputRow, + Text text, + Context context + ) throws IOException, InterruptedException + { + // Create group key, there are probably more efficient ways of doing this + final Map> dims = Maps.newTreeMap(); + for (final String dim : inputRow.getDimensions()) { + final Set dimValues = ImmutableSortedSet.copyOf(inputRow.getDimension(dim)); + if (dimValues.size() > 0) { + dims.put(dim, dimValues); + } + } + + final List groupKey = ImmutableList.of( + rollupGranularity.truncate(inputRow.getTimestampFromEpoch()), + dims + ); + final Optional maybeInterval = config.getGranularitySpec() + .bucketInterval(new DateTime(inputRow.getTimestampFromEpoch())); + + if (!maybeInterval.isPresent()) { + throw new ISE("WTF?! No bucket found for timestamp: %s", inputRow.getTimestampFromEpoch()); + } + hyperLogLogs.get(maybeInterval.get()) + .offerHashed( + hashFunction.hashBytes(HadoopDruidIndexerConfig.jsonMapper.writeValueAsBytes(groupKey)) + .asLong() + ); + } + + @Override + public void run(Context context) throws IOException, InterruptedException + { + setup(context); + + while (context.nextKeyValue()) { + map(context.getCurrentKey(), context.getCurrentValue(), context); + } + + for (Map.Entry entry : hyperLogLogs.entrySet()) { + context.write( + new LongWritable(entry.getKey().getStartMillis()), + new BytesWritable(entry.getValue().getBytes()) + ); + } + cleanup(context); + } + + } + + public static class DetermineCardinalityReducer + extends Reducer + { + protected HadoopDruidIndexerConfig config = null; + + @Override + protected void setup(Context context) + throws IOException, InterruptedException + { + config = HadoopDruidIndexerConfigBuilder.fromConfiguration(context.getConfiguration()); + } + + @Override + protected void reduce( + LongWritable key, + Iterable values, + Context context + ) throws IOException, InterruptedException + { + HyperLogLog aggregate = new HyperLogLog(20); + for (BytesWritable value : values) { + HyperLogLog logValue = HyperLogLog.Builder.build(value.getBytes()); + try { + aggregate.addAll(logValue); + } + catch (CardinalityMergeException e) { + e.printStackTrace(); // TODO: check for better handling + } + } + + final Path outPath = config.makeSegmentPartitionInfoPath(new Bucket(0, new DateTime(key.get()), 0)); + new Exception("creating output path" + outPath).printStackTrace(); + final OutputStream out = Utils.makePathAndOutputStream( + context, outPath, config.isOverwriteFiles() + ); + + try { + HadoopDruidIndexerConfig.jsonMapper.writerWithType( + new TypeReference() + { + } + ).writeValue( + out, + aggregate.cardinality() + ); + } + finally { + Closeables.close(out, false); + } + } + } + + +} diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java index fde7161d8a4..e896ad89c97 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -225,12 +225,7 @@ public class IndexGeneratorJob implements Jobby final ByteBuffer bytes = ByteBuffer.wrap(bytesWritable.getBytes()); bytes.position(4); // Skip length added by SortableBytes int shardNum = bytes.getInt(); - - if (shardNum >= numPartitions) { - throw new ISE("Not enough partitions, shard[%,d] >= numPartitions[%,d]", shardNum, numPartitions); - } - - return shardNum; + return shardNum % numPartitions; } } diff --git a/server/src/main/java/io/druid/guice/ServerModule.java b/server/src/main/java/io/druid/guice/ServerModule.java index 3417ef3d82e..d0ead89825d 100644 --- a/server/src/main/java/io/druid/guice/ServerModule.java +++ b/server/src/main/java/io/druid/guice/ServerModule.java @@ -30,6 +30,7 @@ import io.druid.guice.annotations.Self; import io.druid.initialization.DruidModule; import io.druid.server.DruidNode; import io.druid.server.initialization.ZkPathsConfig; +import io.druid.timeline.partition.HashBasedNumberedShardSpec; import io.druid.timeline.partition.LinearShardSpec; import io.druid.timeline.partition.NumberedShardSpec; import io.druid.timeline.partition.SingleDimensionShardSpec; @@ -63,7 +64,8 @@ public class ServerModule implements DruidModule .registerSubtypes( new NamedType(SingleDimensionShardSpec.class, "single"), new NamedType(LinearShardSpec.class, "linear"), - new NamedType(NumberedShardSpec.class, "numbered") + new NamedType(NumberedShardSpec.class, "numbered"), + new NamedType(HashBasedNumberedShardSpec.class, "hashed") ) ); } diff --git a/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java b/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java new file mode 100644 index 00000000000..7720462e808 --- /dev/null +++ b/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java @@ -0,0 +1,93 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 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.timeline.partition; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.api.client.repackaged.com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSortedSet; +import com.google.common.collect.Maps; +import com.google.common.hash.HashFunction; +import com.google.common.hash.Hashing; +import io.druid.data.input.InputRow; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class HashBasedNumberedShardSpec extends NumberedShardSpec +{ + + private static HashFunction hashFunction = null; + @JacksonInject + public ObjectMapper jsonMapper; + + @JsonCreator + public HashBasedNumberedShardSpec( + @JsonProperty("partitionNum") int partitionNum, + @JsonProperty("partitions") int partitions + ) + { + super(partitionNum, partitions); + hashFunction = Hashing.murmur3_32(); + } + + @Override + public boolean isInChunk(InputRow inputRow) + { + return Math.abs(hash(inputRow)) % getPartitions() == getPartitionNum(); + } + + protected int hash(InputRow inputRow) + { + final Map> dims = Maps.newTreeMap(); + for (final String dim : inputRow.getDimensions()) { + final Set dimValues = ImmutableSortedSet.copyOf(inputRow.getDimension(dim)); + if (dimValues.size() > 0) { + dims.put(dim, dimValues); + } + } + final List groupKey = ImmutableList.of( + inputRow.getTimestampFromEpoch(), + dims + ); + + try { + return hashFunction.hashBytes(jsonMapper.writeValueAsBytes(groupKey)).asInt(); + } + catch (JsonProcessingException e) { + throw Throwables.propagate(e); + } + } + + @Override + public String toString() + { + return "HashBasedNumberedShardSpec{" + + "partitionNum=" + getPartitionNum() + + ", partitions=" + getPartitions() + + '}'; + } + +} \ No newline at end of file diff --git a/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java new file mode 100644 index 00000000000..59fa866beb4 --- /dev/null +++ b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java @@ -0,0 +1,129 @@ +/* + * 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.server.shard; + +import com.fasterxml.jackson.databind.BeanProperty; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.InjectableValues; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.metamx.common.ISE; +import io.druid.TestUtil; +import io.druid.timeline.partition.HashBasedNumberedShardSpec; +import io.druid.timeline.partition.PartitionChunk; +import io.druid.timeline.partition.ShardSpec; +import junit.framework.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.List; + +public class HashBasedNumberedShardSpecTest +{ + @Before + public void setup() + { + TestUtil.MAPPER.setInjectableValues( + new InjectableValues() + { + @Override + public Object findInjectableValue( + Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance + ) + { + if (valueId.equals("com.fasterxml.jackson.databind.ObjectMapper")) { + return TestUtil.MAPPER; + } + throw new ISE("No Injectable value found"); + } + } + ); + } + + @Test + public void testSerdeRoundTrip() throws Exception + { + + final ShardSpec spec = TestUtil.MAPPER.readValue( + TestUtil.MAPPER.writeValueAsBytes(new HashBasedNumberedShardSpec(1, 2)), + ShardSpec.class + ); + Assert.assertEquals(1, spec.getPartitionNum()); + Assert.assertEquals(2, ((HashBasedNumberedShardSpec) spec).getPartitions()); + } + + @Test + public void testSerdeBackwardsCompat() throws Exception + { + final ShardSpec spec = TestUtil.MAPPER.readValue( + "{\"type\": \"hashed\", \"partitions\": 2, \"partitionNum\": 1}", + ShardSpec.class + ); + Assert.assertEquals(1, spec.getPartitionNum()); + Assert.assertEquals(2, ((HashBasedNumberedShardSpec) spec).getPartitions()); + } + + @Test + public void testPartitionChunks() + { + final List specs = ImmutableList.of( + new HashBasedNumberedShardSpec(0, 3), + new HashBasedNumberedShardSpec(1, 3), + new HashBasedNumberedShardSpec(2, 3) + ); + + final List> chunks = Lists.transform( + specs, + new Function>() + { + @Override + public PartitionChunk apply(ShardSpec shardSpec) + { + return shardSpec.createChunk("rofl"); + } + } + ); + + Assert.assertEquals(0, chunks.get(0).getChunkNumber()); + Assert.assertEquals(1, chunks.get(1).getChunkNumber()); + Assert.assertEquals(2, chunks.get(2).getChunkNumber()); + + Assert.assertTrue(chunks.get(0).isStart()); + Assert.assertFalse(chunks.get(1).isStart()); + Assert.assertFalse(chunks.get(2).isStart()); + + Assert.assertFalse(chunks.get(0).isEnd()); + Assert.assertFalse(chunks.get(1).isEnd()); + Assert.assertTrue(chunks.get(2).isEnd()); + + Assert.assertTrue(chunks.get(0).abuts(chunks.get(1))); + Assert.assertTrue(chunks.get(1).abuts(chunks.get(2))); + + Assert.assertFalse(chunks.get(0).abuts(chunks.get(0))); + Assert.assertFalse(chunks.get(0).abuts(chunks.get(2))); + Assert.assertFalse(chunks.get(1).abuts(chunks.get(0))); + Assert.assertFalse(chunks.get(1).abuts(chunks.get(1))); + Assert.assertFalse(chunks.get(2).abuts(chunks.get(0))); + Assert.assertFalse(chunks.get(2).abuts(chunks.get(1))); + Assert.assertFalse(chunks.get(2).abuts(chunks.get(2))); + } + +} From 18c6a070ca281579c5e7cbdc71d4d09a83dddc43 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 30 Jan 2014 11:36:34 -0800 Subject: [PATCH 017/217] fix default locale for tests - fixes #374 --- pom.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pom.xml b/pom.xml index 990a1a75e5c..f18a852268c 100644 --- a/pom.xml +++ b/pom.xml @@ -513,6 +513,8 @@ maven-surefire-plugin 2.12.2 + + -Duser.language=en -Duser.country=US UTC From 5fd76067cdad60f283517b3512e854eac672a335 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Fri, 31 Jan 2014 13:51:38 +0530 Subject: [PATCH 018/217] remove logging and use new determine partition job --- .../indexer/DeterminePartitionsUsingCardinalityJob.java | 1 - .../java/io/druid/indexer/HadoopDruidIndexerConfig.java | 2 +- .../main/java/io/druid/indexer/HadoopDruidIndexerJob.java | 8 ++++++-- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsUsingCardinalityJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsUsingCardinalityJob.java index 505d47b0e45..1fad084fde6 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsUsingCardinalityJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsUsingCardinalityJob.java @@ -281,7 +281,6 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby } final Path outPath = config.makeSegmentPartitionInfoPath(new Bucket(0, new DateTime(key.get()), 0)); - new Exception("creating output path" + outPath).printStackTrace(); final OutputStream out = Utils.makePathAndOutputStream( context, outPath, config.isOverwriteFiles() ); diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java index 19586c14113..9addd6135f8 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java @@ -388,7 +388,7 @@ public class HadoopDruidIndexerConfig return partitionsSpec.getPartitionDimension(); } - public boolean partitionByDimension() + public boolean isDeterminingPartitions() { return partitionsSpec.isDeterminingPartitions(); } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java index 2c593af68a3..a471e09acf9 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java @@ -72,8 +72,12 @@ public class HadoopDruidIndexerJob implements Jobby ensurePaths(); - if (config.partitionByDimension()) { - jobs.add(new DeterminePartitionsJob(config)); + if (config.isDeterminingPartitions()) { + if(config.getPartitionDimension() == null){ + jobs.add(new DeterminePartitionsUsingCardinalityJob(config)); + } else { + jobs.add(new DeterminePartitionsJob(config)); + } } else { Map> shardSpecs = Maps.newTreeMap(DateTimeComparator.getInstance()); From 97e5d68635589027937216d3ed8c97917b5c90fc Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Fri, 31 Jan 2014 19:04:52 +0530 Subject: [PATCH 019/217] determine intervals working with determine partitions --- .../druid/indexer/DeterminePartitionsJob.java | 10 +- ...eterminePartitionsUsingCardinalityJob.java | 107 ++++++++++++++---- .../indexer/HadoopDruidIndexerConfig.java | 17 ++- .../indexer/HadoopDruidIndexerMapper.java | 4 +- .../granularity/UniformGranularitySpec.java | 11 +- .../indexing/common/task/HadoopIndexTask.java | 71 ++++++++---- .../indexing/common/task/TaskSerdeTest.java | 2 - 7 files changed, 158 insertions(+), 64 deletions(-) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java index 30f9471e445..4af000ed859 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java @@ -217,9 +217,7 @@ public class DeterminePartitionsJob implements Jobby Map> shardSpecs = Maps.newTreeMap(DateTimeComparator.getInstance()); int shardCount = 0; for (Interval segmentGranularity : config.getSegmentGranularIntervals()) { - DateTime bucket = segmentGranularity.getStart(); - - final Path partitionInfoPath = config.makeSegmentPartitionInfoPath(new Bucket(0, bucket, 0)); + final Path partitionInfoPath = config.makeSegmentPartitionInfoPath(segmentGranularity); if (fileSystem == null) { fileSystem = partitionInfoPath.getFileSystem(dimSelectionJob.getConfiguration()); } @@ -233,10 +231,10 @@ public class DeterminePartitionsJob implements Jobby List actualSpecs = Lists.newArrayListWithExpectedSize(specs.size()); for (int i = 0; i < specs.size(); ++i) { actualSpecs.add(new HadoopyShardSpec(specs.get(i), shardCount++)); - log.info("DateTime[%s], partition[%d], spec[%s]", bucket, i, actualSpecs.get(i)); + log.info("DateTime[%s], partition[%d], spec[%s]", segmentGranularity, i, actualSpecs.get(i)); } - shardSpecs.put(bucket, actualSpecs); + shardSpecs.put(segmentGranularity.getStart(), actualSpecs); } else { log.info("Path[%s] didn't exist!?", partitionInfoPath); } @@ -741,7 +739,7 @@ public class DeterminePartitionsJob implements Jobby } final OutputStream out = Utils.makePathAndOutputStream( - context, config.makeSegmentPartitionInfoPath(new Bucket(0, bucket, 0)), config.isOverwriteFiles() + context, config.makeSegmentPartitionInfoPath(config.getGranularitySpec().getGranularity().bucket(bucket)), config.isOverwriteFiles() ); final DimPartitions chosenPartitions = maxCardinality > HIGH_CARDINALITY_THRESHOLD diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsUsingCardinalityJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsUsingCardinalityJob.java index 1fad084fde6..bde1dfb4955 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsUsingCardinalityJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsUsingCardinalityJob.java @@ -36,6 +36,7 @@ import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import io.druid.data.input.InputRow; import io.druid.granularity.QueryGranularity; +import io.druid.indexer.granularity.UniformGranularitySpec; import io.druid.timeline.partition.HashBasedNumberedShardSpec; import io.druid.timeline.partition.NoneShardSpec; import org.apache.hadoop.conf.Configuration; @@ -108,6 +109,7 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby groupByJob.setOutputKeyClass(NullWritable.class); groupByJob.setOutputValueClass(NullWritable.class); groupByJob.setOutputFormatClass(SequenceFileOutputFormat.class); + groupByJob.setNumReduceTasks(1); JobHelper.setupClasspath(config, groupByJob); config.addInputPaths(groupByJob); @@ -123,16 +125,30 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby } /* - * Load partitions determined by the previous job. + * Load partitions and intervals determined by the previous job. */ log.info("Job completed, loading up partitions for intervals[%s].", config.getSegmentGranularIntervals()); FileSystem fileSystem = null; + if (config.getSegmentGranularIntervals().isEmpty()) { + final Path intervalInfoPath = config.makeIntervalInfoPath(); + fileSystem = intervalInfoPath.getFileSystem(groupByJob.getConfiguration()); + if (!fileSystem.exists(intervalInfoPath)) { + throw new ISE("Path[%s] didn't exist!?", intervalInfoPath); + } + List intervals = config.jsonMapper.readValue( + Utils.openInputStream(groupByJob, intervalInfoPath), new TypeReference>() + { + } + ); + config.setGranularitySpec(new UniformGranularitySpec(config.getGranularitySpec().getGranularity(), intervals)); + log.info("Determined Intervals for Job [%s]" + config.getSegmentGranularIntervals()); + } Map> shardSpecs = Maps.newTreeMap(DateTimeComparator.getInstance()); for (Interval segmentGranularity : config.getSegmentGranularIntervals()) { DateTime bucket = segmentGranularity.getStart(); - final Path partitionInfoPath = config.makeSegmentPartitionInfoPath(new Bucket(0, bucket, 0)); + final Path partitionInfoPath = config.makeSegmentPartitionInfoPath(segmentGranularity); if (fileSystem == null) { fileSystem = partitionInfoPath.getFileSystem(groupByJob.getConfiguration()); } @@ -157,15 +173,20 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby actualSpecs.add(new HadoopyShardSpec(new HashBasedNumberedShardSpec(i, numberOfShards), shardCount++)); log.info("DateTime[%s], partition[%d], spec[%s]", bucket, i, actualSpecs.get(i)); } - - shardSpecs.put(bucket, actualSpecs); } + + shardSpecs.put(bucket, actualSpecs); + } else { log.info("Path[%s] didn't exist!?", partitionInfoPath); } } config.setShardSpecs(shardSpecs); - log.info("Determine partitions Using cardinality took %d millis", (System.currentTimeMillis() - startTime)); + log.info( + "Determine partitions Using cardinality took %d millis shardSpecs %s", + (System.currentTimeMillis() - startTime), + shardSpecs + ); return true; } @@ -176,10 +197,11 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby public static class DetermineCardinalityMapper extends HadoopDruidIndexerMapper { - private static HashFunction hashFunction = null; + private static HashFunction hashFunction = Hashing.murmur3_128(); private QueryGranularity rollupGranularity = null; private Map hyperLogLogs; private HadoopDruidIndexerConfig config; + private boolean determineIntervals; @Override protected void setup(Context context) @@ -187,14 +209,19 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby { super.setup(context); rollupGranularity = getConfig().getRollupSpec().getRollupGranularity(); - hashFunction = Hashing.murmur3_128(); config = HadoopDruidIndexerConfigBuilder.fromConfiguration(context.getConfiguration()); - final ImmutableMap.Builder builder = ImmutableMap.builder(); - for (final Interval bucketInterval : config.getGranularitySpec().bucketIntervals()) { - builder.put(bucketInterval, new HyperLogLog(20)); + if (config.getSegmentGranularIntervals().isEmpty()) { + determineIntervals = true; + hyperLogLogs = Maps.newHashMap(); + } else { + determineIntervals = false; + final ImmutableMap.Builder builder = ImmutableMap.builder(); + for (final Interval bucketInterval : config.getGranularitySpec().bucketIntervals()) { + builder.put(bucketInterval, new HyperLogLog(20)); + } + hyperLogLogs = builder.build(); } - hyperLogLogs = builder.build(); } @Override @@ -217,13 +244,23 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby rollupGranularity.truncate(inputRow.getTimestampFromEpoch()), dims ); - final Optional maybeInterval = config.getGranularitySpec() - .bucketInterval(new DateTime(inputRow.getTimestampFromEpoch())); + Interval interval; + if (determineIntervals) { + interval = config.getGranularitySpec().getGranularity().bucket(new DateTime(inputRow.getTimestampFromEpoch())); - if (!maybeInterval.isPresent()) { - throw new ISE("WTF?! No bucket found for timestamp: %s", inputRow.getTimestampFromEpoch()); + if (!hyperLogLogs.containsKey(interval)) { + hyperLogLogs.put(interval, new HyperLogLog(20)); + } + } else { + final Optional maybeInterval = config.getGranularitySpec() + .bucketInterval(new DateTime(inputRow.getTimestampFromEpoch())); + + if (!maybeInterval.isPresent()) { + throw new ISE("WTF?! No bucket found for timestamp: %s", inputRow.getTimestampFromEpoch()); + } + interval = maybeInterval.get(); } - hyperLogLogs.get(maybeInterval.get()) + hyperLogLogs.get(interval) .offerHashed( hashFunction.hashBytes(HadoopDruidIndexerConfig.jsonMapper.writeValueAsBytes(groupKey)) .asLong() @@ -253,6 +290,7 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby public static class DetermineCardinalityReducer extends Reducer { + private final List intervals = Lists.newArrayList(); protected HadoopDruidIndexerConfig config = null; @Override @@ -279,8 +317,9 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby e.printStackTrace(); // TODO: check for better handling } } - - final Path outPath = config.makeSegmentPartitionInfoPath(new Bucket(0, new DateTime(key.get()), 0)); + Interval interval = config.getGranularitySpec().getGranularity().bucket(new DateTime(key.get())); + intervals.add(interval); + final Path outPath = config.makeSegmentPartitionInfoPath(interval); final OutputStream out = Utils.makePathAndOutputStream( context, outPath, config.isOverwriteFiles() ); @@ -299,7 +338,35 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby Closeables.close(out, false); } } + + @Override + public void run(Context context) + throws IOException, InterruptedException + { + super.run(context); + if (config.getSegmentGranularIntervals().isEmpty()) { + final Path outPath = config.makeIntervalInfoPath(); + final OutputStream out = Utils.makePathAndOutputStream( + context, outPath, config.isOverwriteFiles() + ); + + try { + HadoopDruidIndexerConfig.jsonMapper.writerWithType( + new TypeReference>() + { + } + ).writeValue( + out, + intervals + ); + } + finally { + Closeables.close(out, false); + } + } + } } - - } + + + diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java index 9addd6135f8..8826b16184e 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java @@ -537,10 +537,8 @@ public class HadoopDruidIndexerConfig return new Path(String.format("%s/%s/%s", getWorkingPath(), getDataSource(), getVersion().replace(":", ""))); } - public Path makeSegmentPartitionInfoPath(Bucket bucket) + public Path makeSegmentPartitionInfoPath(Interval bucketInterval) { - final Interval bucketInterval = getGranularitySpec().bucketInterval(bucket.time).get(); - return new Path( String.format( "%s/%s_%s/partitions.json", @@ -551,6 +549,16 @@ public class HadoopDruidIndexerConfig ); } + public Path makeIntervalInfoPath() + { + return new Path( + String.format( + "%s/intervals.json", + makeIntermediatePath() + ) + ); + } + public Path makeDescriptorInfoDir() { return new Path(makeIntermediatePath(), "segmentDescriptorInfo"); @@ -625,8 +633,5 @@ public class HadoopDruidIndexerConfig Preconditions.checkNotNull(segmentOutputPath, "segmentOutputPath"); Preconditions.checkNotNull(version, "version"); Preconditions.checkNotNull(rollupSpec, "rollupSpec"); - - final int nIntervals = getIntervals().size(); - Preconditions.checkArgument(nIntervals > 0, "intervals.size()[%s] <= 0", nIntervals); } } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerMapper.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerMapper.java index 2eedaf76d31..7ea5ea7ba59 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerMapper.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerMapper.java @@ -71,7 +71,9 @@ public abstract class HadoopDruidIndexerMapper extends Mapper< } } - if(config.getGranularitySpec().bucketInterval(new DateTime(inputRow.getTimestampFromEpoch())).isPresent()) { + if (config.getGranularitySpec().bucketIntervals().isEmpty() || config.getGranularitySpec() + .bucketInterval(new DateTime(inputRow.getTimestampFromEpoch())) + .isPresent()) { innerMap(inputRow, value, context); } } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/granularity/UniformGranularitySpec.java b/indexing-hadoop/src/main/java/io/druid/indexer/granularity/UniformGranularitySpec.java index 726426c2a14..221e9dc2bdd 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/granularity/UniformGranularitySpec.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/granularity/UniformGranularitySpec.java @@ -29,6 +29,7 @@ import com.metamx.common.Granularity; import org.joda.time.DateTime; import org.joda.time.Interval; +import java.util.Collections; import java.util.List; import java.util.SortedSet; @@ -45,13 +46,15 @@ public class UniformGranularitySpec implements GranularitySpec ) { List granularIntervals = Lists.newArrayList(); - - for (Interval inputInterval : inputIntervals) { - Iterables.addAll(granularIntervals, granularity.getIterable(inputInterval)); + if (inputIntervals != null) { + for (Interval inputInterval : inputIntervals) { + Iterables.addAll(granularIntervals, granularity.getIterable(inputInterval)); + } } + this.granularity = granularity; - this.inputIntervals = ImmutableList.copyOf(inputIntervals); + this.inputIntervals = inputIntervals == null ? Collections.EMPTY_LIST : ImmutableList.copyOf(inputIntervals); this.wrappedSpec = new ArbitraryGranularitySpec(granularIntervals); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java index 233714f5c71..820052acf72 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java @@ -24,14 +24,9 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import com.google.api.client.util.Lists; -import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; -import com.google.common.collect.Maps; -import com.google.common.collect.Multimap; -import com.google.common.collect.Multimaps; import com.metamx.common.logger.Logger; import io.druid.common.utils.JodaUtils; import io.druid.indexer.HadoopDruidIndexerConfig; @@ -42,7 +37,6 @@ import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.LockTryAcquireAction; -import io.druid.indexing.common.actions.SegmentInsertAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.initialization.Initialization; import io.druid.server.initialization.ExtensionsConfig; @@ -51,30 +45,25 @@ import io.tesla.aether.internal.DefaultTeslaAether; import org.joda.time.DateTime; import org.joda.time.Interval; -import javax.annotation.Nullable; import java.io.File; import java.lang.reflect.Method; import java.net.URL; import java.net.URLClassLoader; import java.util.Arrays; -import java.util.Collection; import java.util.List; -import java.util.Map; -public class HadoopIndexTask extends AbstractFixedIntervalTask +public class HadoopIndexTask extends AbstractTask { private static final Logger log = new Logger(HadoopIndexTask.class); - private static String defaultHadoopCoordinates = "org.apache.hadoop:hadoop-core:1.0.3"; - private static final ExtensionsConfig extensionsConfig; static { extensionsConfig = Initialization.makeStartupInjector().getInstance(ExtensionsConfig.class); } + private static String defaultHadoopCoordinates = "org.apache.hadoop:hadoop-core:1.0.3"; @JsonIgnore private final HadoopDruidIndexerSchema schema; - @JsonIgnore private final String hadoopCoordinates; @@ -97,13 +86,7 @@ public class HadoopIndexTask extends AbstractFixedIntervalTask { super( id != null ? id : String.format("index_hadoop_%s_%s", schema.getDataSource(), new DateTime()), - schema.getDataSource(), - JodaUtils.umbrellaInterval( - JodaUtils.condenseIntervals( - schema.getGranularitySpec() - .bucketIntervals() - ) - ) + schema.getDataSource() ); // Some HadoopDruidIndexerSchema stuff doesn't make sense in the context of the indexing service @@ -121,6 +104,22 @@ public class HadoopIndexTask extends AbstractFixedIntervalTask return "index_hadoop"; } + @Override + public boolean isReady(TaskActionClient taskActionClient) throws Exception + { + if (!schema.getGranularitySpec().bucketIntervals().isEmpty()) { + Interval interval = JodaUtils.umbrellaInterval( + JodaUtils.condenseIntervals( + schema.getGranularitySpec() + .bucketIntervals() + ) + ); + return taskActionClient.submit(new LockTryAcquireAction(interval)).isPresent(); + } else { + return true; + } + } + @JsonProperty("config") public HadoopDruidIndexerSchema getSchema() { @@ -171,13 +170,22 @@ public class HadoopIndexTask extends AbstractFixedIntervalTask final Class mainClass = loader.loadClass(HadoopIndexTaskInnerProcessing.class.getName()); final Method mainMethod = mainClass.getMethod("runTask", String[].class); - // We should have a lock from before we started running - final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox)); - log.info("Setting version to: %s", myLock.getVersion()); + // We should have a lock from before we started running only if interval was specified + boolean determineIntervals = schema.getGranularitySpec().bucketIntervals().isEmpty(); + final String version; + if (determineIntervals) { + version = new DateTime().toString(); + } else { + Iterable locks = getTaskLocks(toolbox); + final TaskLock myLock = Iterables.getOnlyElement(locks); + version = myLock.getVersion(); + log.info("Setting version to: %s", myLock.getVersion()); + } + String[] args = new String[]{ toolbox.getObjectMapper().writeValueAsString(schema), - myLock.getVersion(), + version, toolbox.getConfig().getHadoopWorkingPath(), toolbox.getSegmentPusher().getPathForHadoop(getDataSource()), }; @@ -186,10 +194,23 @@ public class HadoopIndexTask extends AbstractFixedIntervalTask if (segments != null) { + List publishedSegments = toolbox.getObjectMapper().readValue( segments, - new TypeReference>() {} + new TypeReference>() + { + } ); + + // What if we cannot take the lock ?? + if (determineIntervals) { + List intervals = Lists.newArrayList(); + for (DataSegment segment : publishedSegments) { + intervals.add(segment.getInterval()); + } + Interval interval = JodaUtils.umbrellaInterval(JodaUtils.condenseIntervals(intervals)); + toolbox.getTaskActionClient().submit(new LockTryAcquireAction(interval)); + } toolbox.pushSegments(publishedSegments); return TaskStatus.success(getId()); } else { 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 77ae0af4b52..5813f8a78d3 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 @@ -381,11 +381,9 @@ public class TaskSerdeTest final HadoopIndexTask task2 = (HadoopIndexTask) jsonMapper.readValue(json, Task.class); Assert.assertEquals("foo", task.getDataSource()); - Assert.assertEquals(new Interval("2010-01-01/P1D"), task.getInterval()); Assert.assertEquals(task.getId(), task2.getId()); Assert.assertEquals(task.getGroupId(), task2.getGroupId()); Assert.assertEquals(task.getDataSource(), task2.getDataSource()); - Assert.assertEquals(task.getInterval(), task2.getInterval()); } } From 82b748ad43b306d2ad1f564d15f55142ffc866b2 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Fri, 31 Jan 2014 20:19:33 +0530 Subject: [PATCH 020/217] review comments --- ...java => DetermineHashedPartitionsJob.java} | 44 +++++----- .../druid/indexer/DeterminePartitionsJob.java | 14 +-- .../druid/indexer/HadoopDruidIndexerJob.java | 9 +- .../io/druid/indexer/IndexGeneratorJob.java | 85 ++++++++++--------- .../main/java/io/druid/indexer/JobHelper.java | 10 +++ .../partition/HashBasedNumberedShardSpec.java | 7 +- server/src/test/java/io/druid/TestUtil.java | 19 +++++ .../shard/HashBasedNumberedShardSpecTest.java | 19 ----- 8 files changed, 98 insertions(+), 109 deletions(-) rename indexing-hadoop/src/main/java/io/druid/indexer/{DeterminePartitionsUsingCardinalityJob.java => DetermineHashedPartitionsJob.java} (91%) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsUsingCardinalityJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java similarity index 91% rename from indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsUsingCardinalityJob.java rename to indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java index bde1dfb4955..3b824d70794 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsUsingCardinalityJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java @@ -62,31 +62,22 @@ import java.util.Map; import java.util.Set; /** - * Determines appropriate ShardSpecs for a job by determining approximate cardinality of data set + * Determines appropriate ShardSpecs for a job by determining approximate cardinality of data set using HyperLogLog */ -public class DeterminePartitionsUsingCardinalityJob implements Jobby +public class DetermineHashedPartitionsJob implements Jobby { private static final int MAX_SHARDS = 128; - private static final Logger log = new Logger(DeterminePartitionsUsingCardinalityJob.class); + private static final Logger log = new Logger(DetermineHashedPartitionsJob.class); private final HadoopDruidIndexerConfig config; + private static final int HYPER_LOG_LOG_BIT_SIZE = 20; - public DeterminePartitionsUsingCardinalityJob( + public DetermineHashedPartitionsJob( HadoopDruidIndexerConfig config ) { this.config = config; } - public static void injectSystemProperties(Job job) - { - final Configuration conf = job.getConfiguration(); - for (String propName : System.getProperties().stringPropertyNames()) { - if (propName.startsWith("hadoop.")) { - conf.set(propName.substring("hadoop.".length()), System.getProperty(propName)); - } - } - } - public boolean run() { try { @@ -97,10 +88,10 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby long startTime = System.currentTimeMillis(); final Job groupByJob = new Job( new Configuration(), - String.format("%s-determine_cardinality_grouped-%s", config.getDataSource(), config.getIntervals()) + String.format("%s-determine_partitions_hashed-%s", config.getDataSource(), config.getIntervals()) ); - injectSystemProperties(groupByJob); + JobHelper.injectSystemProperties(groupByJob); groupByJob.setInputFormatClass(TextInputFormat.class); groupByJob.setMapperClass(DetermineCardinalityMapper.class); groupByJob.setMapOutputKeyClass(LongWritable.class); @@ -145,6 +136,7 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby log.info("Determined Intervals for Job [%s]" + config.getSegmentGranularIntervals()); } Map> shardSpecs = Maps.newTreeMap(DateTimeComparator.getInstance()); + int shardCount = 0; for (Interval segmentGranularity : config.getSegmentGranularIntervals()) { DateTime bucket = segmentGranularity.getStart(); @@ -161,14 +153,17 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby int numberOfShards = (int) Math.ceil((double) cardinality / config.getTargetPartitionSize()); if (numberOfShards > MAX_SHARDS) { - numberOfShards = MAX_SHARDS; + throw new ISE( + "Number of shards [%d] exceed the maximum limit of [%d], either targetPartitionSize is too low or data volume is too high", + numberOfShards, + MAX_SHARDS + ); } List actualSpecs = Lists.newArrayListWithExpectedSize(numberOfShards); if (numberOfShards == 1) { - actualSpecs.add(new HadoopyShardSpec(new NoneShardSpec(), 0)); + actualSpecs.add(new HadoopyShardSpec(new NoneShardSpec(), shardCount++)); } else { - int shardCount = 0; for (int i = 0; i < numberOfShards; ++i) { actualSpecs.add(new HadoopyShardSpec(new HashBasedNumberedShardSpec(i, numberOfShards), shardCount++)); log.info("DateTime[%s], partition[%d], spec[%s]", bucket, i, actualSpecs.get(i)); @@ -183,9 +178,8 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby } config.setShardSpecs(shardSpecs); log.info( - "Determine partitions Using cardinality took %d millis shardSpecs %s", - (System.currentTimeMillis() - startTime), - shardSpecs + "DetermineHashedPartitionsJob took %d millis", + (System.currentTimeMillis() - startTime) ); return true; @@ -218,7 +212,7 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby determineIntervals = false; final ImmutableMap.Builder builder = ImmutableMap.builder(); for (final Interval bucketInterval : config.getGranularitySpec().bucketIntervals()) { - builder.put(bucketInterval, new HyperLogLog(20)); + builder.put(bucketInterval, new HyperLogLog(HYPER_LOG_LOG_BIT_SIZE)); } hyperLogLogs = builder.build(); } @@ -249,7 +243,7 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby interval = config.getGranularitySpec().getGranularity().bucket(new DateTime(inputRow.getTimestampFromEpoch())); if (!hyperLogLogs.containsKey(interval)) { - hyperLogLogs.put(interval, new HyperLogLog(20)); + hyperLogLogs.put(interval, new HyperLogLog(HYPER_LOG_LOG_BIT_SIZE)); } } else { final Optional maybeInterval = config.getGranularitySpec() @@ -307,7 +301,7 @@ public class DeterminePartitionsUsingCardinalityJob implements Jobby Context context ) throws IOException, InterruptedException { - HyperLogLog aggregate = new HyperLogLog(20); + HyperLogLog aggregate = new HyperLogLog(HYPER_LOG_LOG_BIT_SIZE); for (BytesWritable value : values) { HyperLogLog logValue = HyperLogLog.Builder.build(value.getBytes()); try { diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java index 4af000ed859..38486c6a16a 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java @@ -107,16 +107,6 @@ public class DeterminePartitionsJob implements Jobby this.config = config; } - public static void injectSystemProperties(Job job) - { - final Configuration conf = job.getConfiguration(); - for (String propName : System.getProperties().stringPropertyNames()) { - if (propName.startsWith("hadoop.")) { - conf.set(propName.substring("hadoop.".length()), System.getProperty(propName)); - } - } - } - public boolean run() { try { @@ -131,7 +121,7 @@ public class DeterminePartitionsJob implements Jobby String.format("%s-determine_partitions_groupby-%s", config.getDataSource(), config.getIntervals()) ); - injectSystemProperties(groupByJob); + JobHelper.injectSystemProperties(groupByJob); groupByJob.setInputFormatClass(TextInputFormat.class); groupByJob.setMapperClass(DeterminePartitionsGroupByMapper.class); groupByJob.setMapOutputKeyClass(BytesWritable.class); @@ -168,7 +158,7 @@ public class DeterminePartitionsJob implements Jobby dimSelectionJob.getConfiguration().set("io.sort.record.percent", "0.19"); - injectSystemProperties(dimSelectionJob); + JobHelper.injectSystemProperties(dimSelectionJob); if (!config.getPartitionsSpec().isAssumeGrouped()) { // Read grouped data from the groupByJob. diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java index a471e09acf9..53160cb4f1d 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java @@ -74,7 +74,7 @@ public class HadoopDruidIndexerJob implements Jobby if (config.isDeterminingPartitions()) { if(config.getPartitionDimension() == null){ - jobs.add(new DeterminePartitionsUsingCardinalityJob(config)); + jobs.add(new DetermineHashedPartitionsJob(config)); } else { jobs.add(new DeterminePartitionsJob(config)); } @@ -143,12 +143,7 @@ public class HadoopDruidIndexerJob implements Jobby ); job.getConfiguration().set("io.sort.record.percent", "0.19"); - for (String propName : System.getProperties().stringPropertyNames()) { - Configuration conf = job.getConfiguration(); - if (propName.startsWith("hadoop.")) { - conf.set(propName.substring("hadoop.".length()), System.getProperty(propName)); - } - } + JobHelper.injectSystemProperties(job); config.addInputPaths(job); } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java index e896ad89c97..39ad77fe8a4 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -84,9 +84,7 @@ import java.util.zip.ZipOutputStream; public class IndexGeneratorJob implements Jobby { private static final Logger log = new Logger(IndexGeneratorJob.class); - private final HadoopDruidIndexerConfig config; - private IndexGeneratorStats jobStats; public IndexGeneratorJob( @@ -97,6 +95,39 @@ public class IndexGeneratorJob implements Jobby this.jobStats = new IndexGeneratorStats(); } + public static List getPublishedSegments(HadoopDruidIndexerConfig config) + { + + final Configuration conf = new Configuration(); + final ObjectMapper jsonMapper = HadoopDruidIndexerConfig.jsonMapper; + + ImmutableList.Builder publishedSegmentsBuilder = ImmutableList.builder(); + + for (String propName : System.getProperties().stringPropertyNames()) { + if (propName.startsWith("hadoop.")) { + conf.set(propName.substring("hadoop.".length()), System.getProperty(propName)); + } + } + + final Path descriptorInfoDir = config.makeDescriptorInfoDir(); + + try { + FileSystem fs = descriptorInfoDir.getFileSystem(conf); + + for (FileStatus status : fs.listStatus(descriptorInfoDir)) { + final DataSegment segment = jsonMapper.readValue(fs.open(status.getPath()), DataSegment.class); + publishedSegmentsBuilder.add(segment); + log.info("Adding segment %s to the list of published segments", segment.getIdentifier()); + } + } + catch (IOException e) { + throw Throwables.propagate(e); + } + List publishedSegments = publishedSegmentsBuilder.build(); + + return publishedSegments; + } + public IndexGeneratorStats getJobStats() { return jobStats; @@ -112,12 +143,7 @@ public class IndexGeneratorJob implements Jobby job.getConfiguration().set("io.sort.record.percent", "0.23"); - for (String propName : System.getProperties().stringPropertyNames()) { - Configuration conf = job.getConfiguration(); - if (propName.startsWith("hadoop.")) { - conf.set(propName.substring("hadoop.".length()), System.getProperty(propName)); - } - } + JobHelper.injectSystemProperties(job); job.setInputFormatClass(TextInputFormat.class); @@ -156,39 +182,6 @@ public class IndexGeneratorJob implements Jobby } } - public static List getPublishedSegments(HadoopDruidIndexerConfig config) - { - - final Configuration conf = new Configuration(); - final ObjectMapper jsonMapper = HadoopDruidIndexerConfig.jsonMapper; - - ImmutableList.Builder publishedSegmentsBuilder = ImmutableList.builder(); - - for (String propName : System.getProperties().stringPropertyNames()) { - if (propName.startsWith("hadoop.")) { - conf.set(propName.substring("hadoop.".length()), System.getProperty(propName)); - } - } - - final Path descriptorInfoDir = config.makeDescriptorInfoDir(); - - try { - FileSystem fs = descriptorInfoDir.getFileSystem(conf); - - for (FileStatus status : fs.listStatus(descriptorInfoDir)) { - final DataSegment segment = jsonMapper.readValue(fs.open(status.getPath()), DataSegment.class); - publishedSegmentsBuilder.add(segment); - log.info("Adding segment %s to the list of published segments", segment.getIdentifier()); - } - } - catch (IOException e) { - throw Throwables.propagate(e); - } - List publishedSegments = publishedSegmentsBuilder.build(); - - return publishedSegments; - } - public static class IndexGeneratorMapper extends HadoopDruidIndexerMapper { @@ -225,7 +218,15 @@ public class IndexGeneratorJob implements Jobby final ByteBuffer bytes = ByteBuffer.wrap(bytesWritable.getBytes()); bytes.position(4); // Skip length added by SortableBytes int shardNum = bytes.getInt(); - return shardNum % numPartitions; + if (System.getProperty("mapred.job.tracker").equals("local")) { + return shardNum % numPartitions; + } else { + if (shardNum >= numPartitions) { + throw new ISE("Not enough partitions, shard[%,d] >= numPartitions[%,d]", shardNum, numPartitions); + } + return shardNum; + + } } } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java b/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java index 654f70b5b4d..3d61538cab2 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java @@ -94,4 +94,14 @@ public class JobHelper } } } + + public static void injectSystemProperties(Job job) + { + final Configuration conf = job.getConfiguration(); + for (String propName : System.getProperties().stringPropertyNames()) { + if (propName.startsWith("hadoop.")) { + conf.set(propName.substring("hadoop.".length()), System.getProperty(propName)); + } + } + } } diff --git a/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java b/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java index 7720462e808..a7e193f1dcf 100644 --- a/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java @@ -39,9 +39,9 @@ import java.util.Set; public class HashBasedNumberedShardSpec extends NumberedShardSpec { - private static HashFunction hashFunction = null; + private static final HashFunction hashFunction = Hashing.murmur3_32(); @JacksonInject - public ObjectMapper jsonMapper; + private ObjectMapper jsonMapper; @JsonCreator public HashBasedNumberedShardSpec( @@ -50,7 +50,6 @@ public class HashBasedNumberedShardSpec extends NumberedShardSpec ) { super(partitionNum, partitions); - hashFunction = Hashing.murmur3_32(); } @Override @@ -90,4 +89,4 @@ public class HashBasedNumberedShardSpec extends NumberedShardSpec '}'; } -} \ No newline at end of file +} diff --git a/server/src/test/java/io/druid/TestUtil.java b/server/src/test/java/io/druid/TestUtil.java index e184c77998f..9f47f2183fb 100644 --- a/server/src/test/java/io/druid/TestUtil.java +++ b/server/src/test/java/io/druid/TestUtil.java @@ -19,8 +19,12 @@ package io.druid; +import com.fasterxml.jackson.databind.BeanProperty; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.common.ISE; import io.druid.guice.ServerModule; import io.druid.jackson.DefaultObjectMapper; @@ -37,5 +41,20 @@ public class TestUtil for (Module module : list) { MAPPER.registerModule(module); } + MAPPER.setInjectableValues( + new InjectableValues() + { + @Override + public Object findInjectableValue( + Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance + ) + { + if (valueId.equals("com.fasterxml.jackson.databind.ObjectMapper")) { + return TestUtil.MAPPER; + } + throw new ISE("No Injectable value found"); + } + } + ); } } diff --git a/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java index 59fa866beb4..9b30768ca40 100644 --- a/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java +++ b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java @@ -38,25 +38,6 @@ import java.util.List; public class HashBasedNumberedShardSpecTest { - @Before - public void setup() - { - TestUtil.MAPPER.setInjectableValues( - new InjectableValues() - { - @Override - public Object findInjectableValue( - Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance - ) - { - if (valueId.equals("com.fasterxml.jackson.databind.ObjectMapper")) { - return TestUtil.MAPPER; - } - throw new ISE("No Injectable value found"); - } - } - ); - } @Test public void testSerdeRoundTrip() throws Exception From 569452121e11e784b34b5cc36d30bd3f2b38f4f3 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Fri, 31 Jan 2014 21:59:17 +0530 Subject: [PATCH 021/217] fix partitioner for loca mode --- .../io/druid/indexer/IndexGeneratorJob.java | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java index 39ad77fe8a4..c0107a18aa0 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -44,6 +44,7 @@ import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.timeline.DataSegment; import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; @@ -209,8 +210,9 @@ public class IndexGeneratorJob implements Jobby } } - public static class IndexGeneratorPartitioner extends Partitioner + public static class IndexGeneratorPartitioner extends Partitioner implements Configurable { + private Configuration config; @Override public int getPartition(BytesWritable bytesWritable, Text text, int numPartitions) @@ -218,7 +220,7 @@ public class IndexGeneratorJob implements Jobby final ByteBuffer bytes = ByteBuffer.wrap(bytesWritable.getBytes()); bytes.position(4); // Skip length added by SortableBytes int shardNum = bytes.getInt(); - if (System.getProperty("mapred.job.tracker").equals("local")) { + if (config.get("mapred.job.tracker").equals("local")) { return shardNum % numPartitions; } else { if (shardNum >= numPartitions) { @@ -228,6 +230,18 @@ public class IndexGeneratorJob implements Jobby } } + + @Override + public Configuration getConf() + { + return config; + } + + @Override + public void setConf(Configuration config) + { + this.config = config; + } } public static class IndexGeneratorReducer extends Reducer From 019be5c3b0093cebff54d09ce6aaf8c6bc2ddfce Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 31 Jan 2014 12:37:37 -0800 Subject: [PATCH 022/217] update jquery --- .../resources/indexer_static/console.html | 2 +- .../java/io/druid/db/DatabaseRuleManager.java | 3 +- .../server/http/DatasourcesResource.java | 23 +- .../io/druid/server/http/RulesResource.java | 18 + server/src/main/resources/static/cluster.html | 2 +- server/src/main/resources/static/config.html | 4 +- server/src/main/resources/static/enable.html | 2 +- .../js/{config-0.0.1.js => config-0.0.2.js} | 6 +- .../main/resources/static/js/enable-0.0.1.js | 7 +- .../{handlers-0.0.1.js => handlers-0.0.2.js} | 10 +- .../resources/static/js/jquery-1.11.0.min.js | 4 + .../main/resources/static/js/jquery-1.8.3.js | 9472 ----------------- .../main/resources/static/js/kill-0.0.1.js | 4 +- .../js/{rules-0.0.1.js => rules-0.0.2.js} | 12 +- server/src/main/resources/static/kill.html | 2 +- server/src/main/resources/static/rules.html | 4 +- server/src/main/resources/static/view.html | 4 +- 17 files changed, 70 insertions(+), 9509 deletions(-) rename server/src/main/resources/static/js/{config-0.0.1.js => config-0.0.2.js} (94%) rename server/src/main/resources/static/js/{handlers-0.0.1.js => handlers-0.0.2.js} (92%) create mode 100644 server/src/main/resources/static/js/jquery-1.11.0.min.js delete mode 100644 server/src/main/resources/static/js/jquery-1.8.3.js rename server/src/main/resources/static/js/{rules-0.0.1.js => rules-0.0.2.js} (95%) diff --git a/indexing-service/src/main/resources/indexer_static/console.html b/indexing-service/src/main/resources/indexer_static/console.html index e8221aa287e..9b9c254996b 100644 --- a/indexing-service/src/main/resources/indexer_static/console.html +++ b/indexing-service/src/main/resources/indexer_static/console.html @@ -29,7 +29,7 @@ - + diff --git a/server/src/main/java/io/druid/db/DatabaseRuleManager.java b/server/src/main/java/io/druid/db/DatabaseRuleManager.java index dd4084d3ed5..9a1c014b36f 100644 --- a/server/src/main/java/io/druid/db/DatabaseRuleManager.java +++ b/server/src/main/java/io/druid/db/DatabaseRuleManager.java @@ -263,7 +263,8 @@ public class DatabaseRuleManager public List getRules(final String dataSource) { - return rules.get().get(dataSource); + List retVal = rules.get().get(dataSource); + return retVal == null ? Lists.newArrayList() : retVal; } public List getRulesWithDefault(final String dataSource) 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 c5718782d77..c77e6fee9bb 100644 --- a/server/src/main/java/io/druid/server/http/DatasourcesResource.java +++ b/server/src/main/java/io/druid/server/http/DatasourcesResource.java @@ -84,8 +84,7 @@ public class DatasourcesResource @Produces("application/json") public Response getQueryableDataSources( @QueryParam("full") String full, - @QueryParam("simple") String simple, - @QueryParam("gran") String gran + @QueryParam("simple") String simple ) { Response.ResponseBuilder builder = Response.status(Response.Status.OK); @@ -107,9 +106,6 @@ public class DatasourcesResource ) ) ).build(); - } else if (gran != null) { - IndexGranularity granularity = IndexGranularity.fromString(gran); - // TODO } return builder.entity( @@ -131,6 +127,7 @@ public class DatasourcesResource @DELETE @Path("/{dataSourceName}") + @Produces("application/json") public Response deleteDataSource( @PathParam("dataSourceName") final String dataSourceName, @QueryParam("kill") final String kill, @@ -138,10 +135,22 @@ public class DatasourcesResource ) { if (indexingServiceClient == null) { - return Response.status(Response.Status.OK).entity(ImmutableMap.of("error", "no indexing service found")).build(); + return Response.ok().entity(ImmutableMap.of("error", "no indexing service found")).build(); } if (kill != null && Boolean.valueOf(kill)) { - indexingServiceClient.killSegments(dataSourceName, new Interval(interval)); + try { + indexingServiceClient.killSegments(dataSourceName, new Interval(interval)); + } + catch (Exception e) { + return Response.status(Response.Status.NOT_FOUND) + .entity( + ImmutableMap.of( + "error", + "Exception occurred. Are you sure you have an indexing service?" + ) + ) + .build(); + } } else { if (!databaseSegmentManager.removeDatasource(dataSourceName)) { return Response.status(Response.Status.NOT_FOUND).build(); diff --git a/server/src/main/java/io/druid/server/http/RulesResource.java b/server/src/main/java/io/druid/server/http/RulesResource.java index bd6d98e925a..d19c39e36f8 100644 --- a/server/src/main/java/io/druid/server/http/RulesResource.java +++ b/server/src/main/java/io/druid/server/http/RulesResource.java @@ -21,13 +21,17 @@ package io.druid.server.http; import com.google.inject.Inject; import io.druid.db.DatabaseRuleManager; +import io.druid.server.coordinator.rules.Rule; +import javax.ws.rs.Consumes; import javax.ws.rs.GET; +import javax.ws.rs.POST; import javax.ws.rs.Path; import javax.ws.rs.PathParam; import javax.ws.rs.Produces; import javax.ws.rs.QueryParam; import javax.ws.rs.core.Response; +import java.util.List; /** */ @@ -67,4 +71,18 @@ public class RulesResource return Response.ok(databaseRuleManager.getRules(dataSourceName)) .build(); } + + @POST + @Path("/{dataSourceName}") + @Consumes("application/json") + public Response setDatasourceRules( + @PathParam("dataSourceName") final String dataSourceName, + final List rules + ) + { + if (databaseRuleManager.overrideRule(dataSourceName, rules)) { + return Response.ok().build(); + } + return Response.status(Response.Status.INTERNAL_SERVER_ERROR).build(); + } } diff --git a/server/src/main/resources/static/cluster.html b/server/src/main/resources/static/cluster.html index 3f4512b8713..10a1ebd4acd 100644 --- a/server/src/main/resources/static/cluster.html +++ b/server/src/main/resources/static/cluster.html @@ -29,7 +29,7 @@ - + diff --git a/server/src/main/resources/static/config.html b/server/src/main/resources/static/config.html index bf6ddca9722..37734e2a7cf 100644 --- a/server/src/main/resources/static/config.html +++ b/server/src/main/resources/static/config.html @@ -30,9 +30,9 @@ - + - +
diff --git a/server/src/main/resources/static/enable.html b/server/src/main/resources/static/enable.html index ae080b0a995..f760b2c5d79 100644 --- a/server/src/main/resources/static/enable.html +++ b/server/src/main/resources/static/enable.html @@ -30,7 +30,7 @@ - + diff --git a/server/src/main/resources/static/js/config-0.0.1.js b/server/src/main/resources/static/js/config-0.0.2.js similarity index 94% rename from server/src/main/resources/static/js/config-0.0.1.js rename to server/src/main/resources/static/js/config-0.0.2.js index 5b092a4e76f..3f6b3a918e6 100644 --- a/server/src/main/resources/static/js/config-0.0.1.js +++ b/server/src/main/resources/static/js/config-0.0.2.js @@ -20,7 +20,7 @@ function domToConfig(configDiv) { } function getConfigs() { - $.getJSON("/coordinator/config", function(data) { + $.getJSON("/druid/coordinator/v1/config", function(data) { $('#config_list').empty(); $.each(data, function (key, value) { @@ -72,10 +72,10 @@ $(document).ready(function() { $.ajax({ type: 'POST', - url:'/coordinator/config', + url:'/druid/coordinator/v1/config', data: JSON.stringify(configs), contentType:"application/json; charset=utf-8", - dataType:"json", + dataType:"text", error: function(xhr, status, error) { $("#update_dialog").dialog("close"); $("#error_dialog").html(xhr.responseText); diff --git a/server/src/main/resources/static/js/enable-0.0.1.js b/server/src/main/resources/static/js/enable-0.0.1.js index 65fd75fb1fa..6c37ac6fd98 100644 --- a/server/src/main/resources/static/js/enable-0.0.1.js +++ b/server/src/main/resources/static/js/enable-0.0.1.js @@ -24,7 +24,7 @@ $(document).ready(function() { url:'/druid/coordinator/v1/datasources/' + selected, data: JSON.stringify(selected), contentType:"application/json; charset=utf-8", - dataType:"json", + dataType:"text", error: function(xhr, status, error) { $("#enable_dialog").dialog("close"); $("#error_dialog").html(xhr.responseText); @@ -53,7 +53,7 @@ $(document).ready(function() { url:'/druid/coordinator/v1/datasources/' + selected, data: JSON.stringify(selected), contentType:"application/json; charset=utf-8", - dataType:"json", + dataType:"text", error: function(xhr, status, error) { $("#disable_dialog").dialog("close"); $("#error_dialog").html(xhr.responseText); @@ -81,12 +81,11 @@ $(document).ready(function() { $('#disabled_datasources').append($('
  • ' + datasource + '
  • ')); }); $.each(db_datasources, function(index, datasource) { - $('#datasources').append($('').attr("value", datasource).text(datasource)); + $('#datasources').append($('').val(datasource).text(datasource)); }); }); }); - $("#enable").click(function() { $("#enable_dialog").dialog("open"); }); diff --git a/server/src/main/resources/static/js/handlers-0.0.1.js b/server/src/main/resources/static/js/handlers-0.0.2.js similarity index 92% rename from server/src/main/resources/static/js/handlers-0.0.1.js rename to server/src/main/resources/static/js/handlers-0.0.2.js index 1cc16adfe4a..8441fe9c561 100644 --- a/server/src/main/resources/static/js/handlers-0.0.1.js +++ b/server/src/main/resources/static/js/handlers-0.0.2.js @@ -3,11 +3,12 @@ $(document).ready(function() { var basePath = "/druid/coordinator/v1/"; - var type = $('#select_type').attr('value') + ''; - var view = $('#select_view').attr('value') + ''; + var type = $('#select_type').val() + ''; + var view = $('#select_view').val() + ''; function handleTable(dontDisplay) { + console.log(type); $.get(basePath + type + '?full', function(data) { buildTable(data, $('#result_table'), dontDisplay); @@ -75,8 +76,9 @@ $(document).ready(function() { } $('#view_button').click(function() { - type = $('#select_type').attr('value') + ''; - view = $('#select_view').attr('value') + ''; + console.log("here"); + type = $('#select_type').val() + ''; + view = $('#select_view').val() + ''; resetViews(); diff --git a/server/src/main/resources/static/js/jquery-1.11.0.min.js b/server/src/main/resources/static/js/jquery-1.11.0.min.js new file mode 100644 index 00000000000..046e93aa15e --- /dev/null +++ b/server/src/main/resources/static/js/jquery-1.11.0.min.js @@ -0,0 +1,4 @@ +/*! jQuery v1.11.0 | (c) 2005, 2014 jQuery Foundation, Inc. | jquery.org/license */ +!function(a,b){"object"==typeof module&&"object"==typeof module.exports?module.exports=a.document?b(a,!0):function(a){if(!a.document)throw new Error("jQuery requires a window with a document");return b(a)}:b(a)}("undefined"!=typeof window?window:this,function(a,b){var c=[],d=c.slice,e=c.concat,f=c.push,g=c.indexOf,h={},i=h.toString,j=h.hasOwnProperty,k="".trim,l={},m="1.11.0",n=function(a,b){return new n.fn.init(a,b)},o=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,p=/^-ms-/,q=/-([\da-z])/gi,r=function(a,b){return b.toUpperCase()};n.fn=n.prototype={jquery:m,constructor:n,selector:"",length:0,toArray:function(){return d.call(this)},get:function(a){return null!=a?0>a?this[a+this.length]:this[a]:d.call(this)},pushStack:function(a){var b=n.merge(this.constructor(),a);return b.prevObject=this,b.context=this.context,b},each:function(a,b){return n.each(this,a,b)},map:function(a){return this.pushStack(n.map(this,function(b,c){return a.call(b,c,b)}))},slice:function(){return this.pushStack(d.apply(this,arguments))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(a){var b=this.length,c=+a+(0>a?b:0);return this.pushStack(c>=0&&b>c?[this[c]]:[])},end:function(){return this.prevObject||this.constructor(null)},push:f,sort:c.sort,splice:c.splice},n.extend=n.fn.extend=function(){var a,b,c,d,e,f,g=arguments[0]||{},h=1,i=arguments.length,j=!1;for("boolean"==typeof g&&(j=g,g=arguments[h]||{},h++),"object"==typeof g||n.isFunction(g)||(g={}),h===i&&(g=this,h--);i>h;h++)if(null!=(e=arguments[h]))for(d in e)a=g[d],c=e[d],g!==c&&(j&&c&&(n.isPlainObject(c)||(b=n.isArray(c)))?(b?(b=!1,f=a&&n.isArray(a)?a:[]):f=a&&n.isPlainObject(a)?a:{},g[d]=n.extend(j,f,c)):void 0!==c&&(g[d]=c));return g},n.extend({expando:"jQuery"+(m+Math.random()).replace(/\D/g,""),isReady:!0,error:function(a){throw new Error(a)},noop:function(){},isFunction:function(a){return"function"===n.type(a)},isArray:Array.isArray||function(a){return"array"===n.type(a)},isWindow:function(a){return null!=a&&a==a.window},isNumeric:function(a){return a-parseFloat(a)>=0},isEmptyObject:function(a){var b;for(b in a)return!1;return!0},isPlainObject:function(a){var b;if(!a||"object"!==n.type(a)||a.nodeType||n.isWindow(a))return!1;try{if(a.constructor&&!j.call(a,"constructor")&&!j.call(a.constructor.prototype,"isPrototypeOf"))return!1}catch(c){return!1}if(l.ownLast)for(b in a)return j.call(a,b);for(b in a);return void 0===b||j.call(a,b)},type:function(a){return null==a?a+"":"object"==typeof a||"function"==typeof a?h[i.call(a)]||"object":typeof a},globalEval:function(b){b&&n.trim(b)&&(a.execScript||function(b){a.eval.call(a,b)})(b)},camelCase:function(a){return a.replace(p,"ms-").replace(q,r)},nodeName:function(a,b){return a.nodeName&&a.nodeName.toLowerCase()===b.toLowerCase()},each:function(a,b,c){var d,e=0,f=a.length,g=s(a);if(c){if(g){for(;f>e;e++)if(d=b.apply(a[e],c),d===!1)break}else for(e in a)if(d=b.apply(a[e],c),d===!1)break}else if(g){for(;f>e;e++)if(d=b.call(a[e],e,a[e]),d===!1)break}else for(e in a)if(d=b.call(a[e],e,a[e]),d===!1)break;return a},trim:k&&!k.call("\ufeff\xa0")?function(a){return null==a?"":k.call(a)}:function(a){return null==a?"":(a+"").replace(o,"")},makeArray:function(a,b){var c=b||[];return null!=a&&(s(Object(a))?n.merge(c,"string"==typeof a?[a]:a):f.call(c,a)),c},inArray:function(a,b,c){var d;if(b){if(g)return g.call(b,a,c);for(d=b.length,c=c?0>c?Math.max(0,d+c):c:0;d>c;c++)if(c in b&&b[c]===a)return c}return-1},merge:function(a,b){var c=+b.length,d=0,e=a.length;while(c>d)a[e++]=b[d++];if(c!==c)while(void 0!==b[d])a[e++]=b[d++];return a.length=e,a},grep:function(a,b,c){for(var d,e=[],f=0,g=a.length,h=!c;g>f;f++)d=!b(a[f],f),d!==h&&e.push(a[f]);return e},map:function(a,b,c){var d,f=0,g=a.length,h=s(a),i=[];if(h)for(;g>f;f++)d=b(a[f],f,c),null!=d&&i.push(d);else for(f in a)d=b(a[f],f,c),null!=d&&i.push(d);return e.apply([],i)},guid:1,proxy:function(a,b){var c,e,f;return"string"==typeof b&&(f=a[b],b=a,a=f),n.isFunction(a)?(c=d.call(arguments,2),e=function(){return a.apply(b||this,c.concat(d.call(arguments)))},e.guid=a.guid=a.guid||n.guid++,e):void 0},now:function(){return+new Date},support:l}),n.each("Boolean Number String Function Array Date RegExp Object Error".split(" "),function(a,b){h["[object "+b+"]"]=b.toLowerCase()});function s(a){var b=a.length,c=n.type(a);return"function"===c||n.isWindow(a)?!1:1===a.nodeType&&b?!0:"array"===c||0===b||"number"==typeof b&&b>0&&b-1 in a}var t=function(a){var b,c,d,e,f,g,h,i,j,k,l,m,n,o,p,q,r,s="sizzle"+-new Date,t=a.document,u=0,v=0,w=eb(),x=eb(),y=eb(),z=function(a,b){return a===b&&(j=!0),0},A="undefined",B=1<<31,C={}.hasOwnProperty,D=[],E=D.pop,F=D.push,G=D.push,H=D.slice,I=D.indexOf||function(a){for(var b=0,c=this.length;c>b;b++)if(this[b]===a)return b;return-1},J="checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|ismap|loop|multiple|open|readonly|required|scoped",K="[\\x20\\t\\r\\n\\f]",L="(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",M=L.replace("w","w#"),N="\\["+K+"*("+L+")"+K+"*(?:([*^$|!~]?=)"+K+"*(?:(['\"])((?:\\\\.|[^\\\\])*?)\\3|("+M+")|)|)"+K+"*\\]",O=":("+L+")(?:\\(((['\"])((?:\\\\.|[^\\\\])*?)\\3|((?:\\\\.|[^\\\\()[\\]]|"+N.replace(3,8)+")*)|.*)\\)|)",P=new RegExp("^"+K+"+|((?:^|[^\\\\])(?:\\\\.)*)"+K+"+$","g"),Q=new RegExp("^"+K+"*,"+K+"*"),R=new RegExp("^"+K+"*([>+~]|"+K+")"+K+"*"),S=new RegExp("="+K+"*([^\\]'\"]*?)"+K+"*\\]","g"),T=new RegExp(O),U=new RegExp("^"+M+"$"),V={ID:new RegExp("^#("+L+")"),CLASS:new RegExp("^\\.("+L+")"),TAG:new RegExp("^("+L.replace("w","w*")+")"),ATTR:new RegExp("^"+N),PSEUDO:new RegExp("^"+O),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+K+"*(even|odd|(([+-]|)(\\d*)n|)"+K+"*(?:([+-]|)"+K+"*(\\d+)|))"+K+"*\\)|)","i"),bool:new RegExp("^(?:"+J+")$","i"),needsContext:new RegExp("^"+K+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+K+"*((?:-\\d)?\\d*)"+K+"*\\)|)(?=[^-]|$)","i")},W=/^(?:input|select|textarea|button)$/i,X=/^h\d$/i,Y=/^[^{]+\{\s*\[native \w/,Z=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,$=/[+~]/,_=/'|\\/g,ab=new RegExp("\\\\([\\da-f]{1,6}"+K+"?|("+K+")|.)","ig"),bb=function(a,b,c){var d="0x"+b-65536;return d!==d||c?b:0>d?String.fromCharCode(d+65536):String.fromCharCode(d>>10|55296,1023&d|56320)};try{G.apply(D=H.call(t.childNodes),t.childNodes),D[t.childNodes.length].nodeType}catch(cb){G={apply:D.length?function(a,b){F.apply(a,H.call(b))}:function(a,b){var c=a.length,d=0;while(a[c++]=b[d++]);a.length=c-1}}}function db(a,b,d,e){var f,g,h,i,j,m,p,q,u,v;if((b?b.ownerDocument||b:t)!==l&&k(b),b=b||l,d=d||[],!a||"string"!=typeof a)return d;if(1!==(i=b.nodeType)&&9!==i)return[];if(n&&!e){if(f=Z.exec(a))if(h=f[1]){if(9===i){if(g=b.getElementById(h),!g||!g.parentNode)return d;if(g.id===h)return d.push(g),d}else if(b.ownerDocument&&(g=b.ownerDocument.getElementById(h))&&r(b,g)&&g.id===h)return d.push(g),d}else{if(f[2])return G.apply(d,b.getElementsByTagName(a)),d;if((h=f[3])&&c.getElementsByClassName&&b.getElementsByClassName)return G.apply(d,b.getElementsByClassName(h)),d}if(c.qsa&&(!o||!o.test(a))){if(q=p=s,u=b,v=9===i&&a,1===i&&"object"!==b.nodeName.toLowerCase()){m=ob(a),(p=b.getAttribute("id"))?q=p.replace(_,"\\$&"):b.setAttribute("id",q),q="[id='"+q+"'] ",j=m.length;while(j--)m[j]=q+pb(m[j]);u=$.test(a)&&mb(b.parentNode)||b,v=m.join(",")}if(v)try{return G.apply(d,u.querySelectorAll(v)),d}catch(w){}finally{p||b.removeAttribute("id")}}}return xb(a.replace(P,"$1"),b,d,e)}function eb(){var a=[];function b(c,e){return a.push(c+" ")>d.cacheLength&&delete b[a.shift()],b[c+" "]=e}return b}function fb(a){return a[s]=!0,a}function gb(a){var b=l.createElement("div");try{return!!a(b)}catch(c){return!1}finally{b.parentNode&&b.parentNode.removeChild(b),b=null}}function hb(a,b){var c=a.split("|"),e=a.length;while(e--)d.attrHandle[c[e]]=b}function ib(a,b){var c=b&&a,d=c&&1===a.nodeType&&1===b.nodeType&&(~b.sourceIndex||B)-(~a.sourceIndex||B);if(d)return d;if(c)while(c=c.nextSibling)if(c===b)return-1;return a?1:-1}function jb(a){return function(b){var c=b.nodeName.toLowerCase();return"input"===c&&b.type===a}}function kb(a){return function(b){var c=b.nodeName.toLowerCase();return("input"===c||"button"===c)&&b.type===a}}function lb(a){return fb(function(b){return b=+b,fb(function(c,d){var e,f=a([],c.length,b),g=f.length;while(g--)c[e=f[g]]&&(c[e]=!(d[e]=c[e]))})})}function mb(a){return a&&typeof a.getElementsByTagName!==A&&a}c=db.support={},f=db.isXML=function(a){var b=a&&(a.ownerDocument||a).documentElement;return b?"HTML"!==b.nodeName:!1},k=db.setDocument=function(a){var b,e=a?a.ownerDocument||a:t,g=e.defaultView;return e!==l&&9===e.nodeType&&e.documentElement?(l=e,m=e.documentElement,n=!f(e),g&&g!==g.top&&(g.addEventListener?g.addEventListener("unload",function(){k()},!1):g.attachEvent&&g.attachEvent("onunload",function(){k()})),c.attributes=gb(function(a){return a.className="i",!a.getAttribute("className")}),c.getElementsByTagName=gb(function(a){return a.appendChild(e.createComment("")),!a.getElementsByTagName("*").length}),c.getElementsByClassName=Y.test(e.getElementsByClassName)&&gb(function(a){return a.innerHTML="
    ",a.firstChild.className="i",2===a.getElementsByClassName("i").length}),c.getById=gb(function(a){return m.appendChild(a).id=s,!e.getElementsByName||!e.getElementsByName(s).length}),c.getById?(d.find.ID=function(a,b){if(typeof b.getElementById!==A&&n){var c=b.getElementById(a);return c&&c.parentNode?[c]:[]}},d.filter.ID=function(a){var b=a.replace(ab,bb);return function(a){return a.getAttribute("id")===b}}):(delete d.find.ID,d.filter.ID=function(a){var b=a.replace(ab,bb);return function(a){var c=typeof a.getAttributeNode!==A&&a.getAttributeNode("id");return c&&c.value===b}}),d.find.TAG=c.getElementsByTagName?function(a,b){return typeof b.getElementsByTagName!==A?b.getElementsByTagName(a):void 0}:function(a,b){var c,d=[],e=0,f=b.getElementsByTagName(a);if("*"===a){while(c=f[e++])1===c.nodeType&&d.push(c);return d}return f},d.find.CLASS=c.getElementsByClassName&&function(a,b){return typeof b.getElementsByClassName!==A&&n?b.getElementsByClassName(a):void 0},p=[],o=[],(c.qsa=Y.test(e.querySelectorAll))&&(gb(function(a){a.innerHTML="",a.querySelectorAll("[t^='']").length&&o.push("[*^$]="+K+"*(?:''|\"\")"),a.querySelectorAll("[selected]").length||o.push("\\["+K+"*(?:value|"+J+")"),a.querySelectorAll(":checked").length||o.push(":checked")}),gb(function(a){var b=e.createElement("input");b.setAttribute("type","hidden"),a.appendChild(b).setAttribute("name","D"),a.querySelectorAll("[name=d]").length&&o.push("name"+K+"*[*^$|!~]?="),a.querySelectorAll(":enabled").length||o.push(":enabled",":disabled"),a.querySelectorAll("*,:x"),o.push(",.*:")})),(c.matchesSelector=Y.test(q=m.webkitMatchesSelector||m.mozMatchesSelector||m.oMatchesSelector||m.msMatchesSelector))&&gb(function(a){c.disconnectedMatch=q.call(a,"div"),q.call(a,"[s!='']:x"),p.push("!=",O)}),o=o.length&&new RegExp(o.join("|")),p=p.length&&new RegExp(p.join("|")),b=Y.test(m.compareDocumentPosition),r=b||Y.test(m.contains)?function(a,b){var c=9===a.nodeType?a.documentElement:a,d=b&&b.parentNode;return a===d||!(!d||1!==d.nodeType||!(c.contains?c.contains(d):a.compareDocumentPosition&&16&a.compareDocumentPosition(d)))}:function(a,b){if(b)while(b=b.parentNode)if(b===a)return!0;return!1},z=b?function(a,b){if(a===b)return j=!0,0;var d=!a.compareDocumentPosition-!b.compareDocumentPosition;return d?d:(d=(a.ownerDocument||a)===(b.ownerDocument||b)?a.compareDocumentPosition(b):1,1&d||!c.sortDetached&&b.compareDocumentPosition(a)===d?a===e||a.ownerDocument===t&&r(t,a)?-1:b===e||b.ownerDocument===t&&r(t,b)?1:i?I.call(i,a)-I.call(i,b):0:4&d?-1:1)}:function(a,b){if(a===b)return j=!0,0;var c,d=0,f=a.parentNode,g=b.parentNode,h=[a],k=[b];if(!f||!g)return a===e?-1:b===e?1:f?-1:g?1:i?I.call(i,a)-I.call(i,b):0;if(f===g)return ib(a,b);c=a;while(c=c.parentNode)h.unshift(c);c=b;while(c=c.parentNode)k.unshift(c);while(h[d]===k[d])d++;return d?ib(h[d],k[d]):h[d]===t?-1:k[d]===t?1:0},e):l},db.matches=function(a,b){return db(a,null,null,b)},db.matchesSelector=function(a,b){if((a.ownerDocument||a)!==l&&k(a),b=b.replace(S,"='$1']"),!(!c.matchesSelector||!n||p&&p.test(b)||o&&o.test(b)))try{var d=q.call(a,b);if(d||c.disconnectedMatch||a.document&&11!==a.document.nodeType)return d}catch(e){}return db(b,l,null,[a]).length>0},db.contains=function(a,b){return(a.ownerDocument||a)!==l&&k(a),r(a,b)},db.attr=function(a,b){(a.ownerDocument||a)!==l&&k(a);var e=d.attrHandle[b.toLowerCase()],f=e&&C.call(d.attrHandle,b.toLowerCase())?e(a,b,!n):void 0;return void 0!==f?f:c.attributes||!n?a.getAttribute(b):(f=a.getAttributeNode(b))&&f.specified?f.value:null},db.error=function(a){throw new Error("Syntax error, unrecognized expression: "+a)},db.uniqueSort=function(a){var b,d=[],e=0,f=0;if(j=!c.detectDuplicates,i=!c.sortStable&&a.slice(0),a.sort(z),j){while(b=a[f++])b===a[f]&&(e=d.push(f));while(e--)a.splice(d[e],1)}return i=null,a},e=db.getText=function(a){var b,c="",d=0,f=a.nodeType;if(f){if(1===f||9===f||11===f){if("string"==typeof a.textContent)return a.textContent;for(a=a.firstChild;a;a=a.nextSibling)c+=e(a)}else if(3===f||4===f)return a.nodeValue}else while(b=a[d++])c+=e(b);return c},d=db.selectors={cacheLength:50,createPseudo:fb,match:V,attrHandle:{},find:{},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(a){return a[1]=a[1].replace(ab,bb),a[3]=(a[4]||a[5]||"").replace(ab,bb),"~="===a[2]&&(a[3]=" "+a[3]+" "),a.slice(0,4)},CHILD:function(a){return a[1]=a[1].toLowerCase(),"nth"===a[1].slice(0,3)?(a[3]||db.error(a[0]),a[4]=+(a[4]?a[5]+(a[6]||1):2*("even"===a[3]||"odd"===a[3])),a[5]=+(a[7]+a[8]||"odd"===a[3])):a[3]&&db.error(a[0]),a},PSEUDO:function(a){var b,c=!a[5]&&a[2];return V.CHILD.test(a[0])?null:(a[3]&&void 0!==a[4]?a[2]=a[4]:c&&T.test(c)&&(b=ob(c,!0))&&(b=c.indexOf(")",c.length-b)-c.length)&&(a[0]=a[0].slice(0,b),a[2]=c.slice(0,b)),a.slice(0,3))}},filter:{TAG:function(a){var b=a.replace(ab,bb).toLowerCase();return"*"===a?function(){return!0}:function(a){return a.nodeName&&a.nodeName.toLowerCase()===b}},CLASS:function(a){var b=w[a+" "];return b||(b=new RegExp("(^|"+K+")"+a+"("+K+"|$)"))&&w(a,function(a){return b.test("string"==typeof a.className&&a.className||typeof a.getAttribute!==A&&a.getAttribute("class")||"")})},ATTR:function(a,b,c){return function(d){var e=db.attr(d,a);return null==e?"!="===b:b?(e+="","="===b?e===c:"!="===b?e!==c:"^="===b?c&&0===e.indexOf(c):"*="===b?c&&e.indexOf(c)>-1:"$="===b?c&&e.slice(-c.length)===c:"~="===b?(" "+e+" ").indexOf(c)>-1:"|="===b?e===c||e.slice(0,c.length+1)===c+"-":!1):!0}},CHILD:function(a,b,c,d,e){var f="nth"!==a.slice(0,3),g="last"!==a.slice(-4),h="of-type"===b;return 1===d&&0===e?function(a){return!!a.parentNode}:function(b,c,i){var j,k,l,m,n,o,p=f!==g?"nextSibling":"previousSibling",q=b.parentNode,r=h&&b.nodeName.toLowerCase(),t=!i&&!h;if(q){if(f){while(p){l=b;while(l=l[p])if(h?l.nodeName.toLowerCase()===r:1===l.nodeType)return!1;o=p="only"===a&&!o&&"nextSibling"}return!0}if(o=[g?q.firstChild:q.lastChild],g&&t){k=q[s]||(q[s]={}),j=k[a]||[],n=j[0]===u&&j[1],m=j[0]===u&&j[2],l=n&&q.childNodes[n];while(l=++n&&l&&l[p]||(m=n=0)||o.pop())if(1===l.nodeType&&++m&&l===b){k[a]=[u,n,m];break}}else if(t&&(j=(b[s]||(b[s]={}))[a])&&j[0]===u)m=j[1];else while(l=++n&&l&&l[p]||(m=n=0)||o.pop())if((h?l.nodeName.toLowerCase()===r:1===l.nodeType)&&++m&&(t&&((l[s]||(l[s]={}))[a]=[u,m]),l===b))break;return m-=e,m===d||m%d===0&&m/d>=0}}},PSEUDO:function(a,b){var c,e=d.pseudos[a]||d.setFilters[a.toLowerCase()]||db.error("unsupported pseudo: "+a);return e[s]?e(b):e.length>1?(c=[a,a,"",b],d.setFilters.hasOwnProperty(a.toLowerCase())?fb(function(a,c){var d,f=e(a,b),g=f.length;while(g--)d=I.call(a,f[g]),a[d]=!(c[d]=f[g])}):function(a){return e(a,0,c)}):e}},pseudos:{not:fb(function(a){var b=[],c=[],d=g(a.replace(P,"$1"));return d[s]?fb(function(a,b,c,e){var f,g=d(a,null,e,[]),h=a.length;while(h--)(f=g[h])&&(a[h]=!(b[h]=f))}):function(a,e,f){return b[0]=a,d(b,null,f,c),!c.pop()}}),has:fb(function(a){return function(b){return db(a,b).length>0}}),contains:fb(function(a){return function(b){return(b.textContent||b.innerText||e(b)).indexOf(a)>-1}}),lang:fb(function(a){return U.test(a||"")||db.error("unsupported lang: "+a),a=a.replace(ab,bb).toLowerCase(),function(b){var c;do if(c=n?b.lang:b.getAttribute("xml:lang")||b.getAttribute("lang"))return c=c.toLowerCase(),c===a||0===c.indexOf(a+"-");while((b=b.parentNode)&&1===b.nodeType);return!1}}),target:function(b){var c=a.location&&a.location.hash;return c&&c.slice(1)===b.id},root:function(a){return a===m},focus:function(a){return a===l.activeElement&&(!l.hasFocus||l.hasFocus())&&!!(a.type||a.href||~a.tabIndex)},enabled:function(a){return a.disabled===!1},disabled:function(a){return a.disabled===!0},checked:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&!!a.checked||"option"===b&&!!a.selected},selected:function(a){return a.parentNode&&a.parentNode.selectedIndex,a.selected===!0},empty:function(a){for(a=a.firstChild;a;a=a.nextSibling)if(a.nodeType<6)return!1;return!0},parent:function(a){return!d.pseudos.empty(a)},header:function(a){return X.test(a.nodeName)},input:function(a){return W.test(a.nodeName)},button:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&"button"===a.type||"button"===b},text:function(a){var b;return"input"===a.nodeName.toLowerCase()&&"text"===a.type&&(null==(b=a.getAttribute("type"))||"text"===b.toLowerCase())},first:lb(function(){return[0]}),last:lb(function(a,b){return[b-1]}),eq:lb(function(a,b,c){return[0>c?c+b:c]}),even:lb(function(a,b){for(var c=0;b>c;c+=2)a.push(c);return a}),odd:lb(function(a,b){for(var c=1;b>c;c+=2)a.push(c);return a}),lt:lb(function(a,b,c){for(var d=0>c?c+b:c;--d>=0;)a.push(d);return a}),gt:lb(function(a,b,c){for(var d=0>c?c+b:c;++db;b++)d+=a[b].value;return d}function qb(a,b,c){var d=b.dir,e=c&&"parentNode"===d,f=v++;return b.first?function(b,c,f){while(b=b[d])if(1===b.nodeType||e)return a(b,c,f)}:function(b,c,g){var h,i,j=[u,f];if(g){while(b=b[d])if((1===b.nodeType||e)&&a(b,c,g))return!0}else while(b=b[d])if(1===b.nodeType||e){if(i=b[s]||(b[s]={}),(h=i[d])&&h[0]===u&&h[1]===f)return j[2]=h[2];if(i[d]=j,j[2]=a(b,c,g))return!0}}}function rb(a){return a.length>1?function(b,c,d){var e=a.length;while(e--)if(!a[e](b,c,d))return!1;return!0}:a[0]}function sb(a,b,c,d,e){for(var f,g=[],h=0,i=a.length,j=null!=b;i>h;h++)(f=a[h])&&(!c||c(f,d,e))&&(g.push(f),j&&b.push(h));return g}function tb(a,b,c,d,e,f){return d&&!d[s]&&(d=tb(d)),e&&!e[s]&&(e=tb(e,f)),fb(function(f,g,h,i){var j,k,l,m=[],n=[],o=g.length,p=f||wb(b||"*",h.nodeType?[h]:h,[]),q=!a||!f&&b?p:sb(p,m,a,h,i),r=c?e||(f?a:o||d)?[]:g:q;if(c&&c(q,r,h,i),d){j=sb(r,n),d(j,[],h,i),k=j.length;while(k--)(l=j[k])&&(r[n[k]]=!(q[n[k]]=l))}if(f){if(e||a){if(e){j=[],k=r.length;while(k--)(l=r[k])&&j.push(q[k]=l);e(null,r=[],j,i)}k=r.length;while(k--)(l=r[k])&&(j=e?I.call(f,l):m[k])>-1&&(f[j]=!(g[j]=l))}}else r=sb(r===g?r.splice(o,r.length):r),e?e(null,g,r,i):G.apply(g,r)})}function ub(a){for(var b,c,e,f=a.length,g=d.relative[a[0].type],i=g||d.relative[" "],j=g?1:0,k=qb(function(a){return a===b},i,!0),l=qb(function(a){return I.call(b,a)>-1},i,!0),m=[function(a,c,d){return!g&&(d||c!==h)||((b=c).nodeType?k(a,c,d):l(a,c,d))}];f>j;j++)if(c=d.relative[a[j].type])m=[qb(rb(m),c)];else{if(c=d.filter[a[j].type].apply(null,a[j].matches),c[s]){for(e=++j;f>e;e++)if(d.relative[a[e].type])break;return tb(j>1&&rb(m),j>1&&pb(a.slice(0,j-1).concat({value:" "===a[j-2].type?"*":""})).replace(P,"$1"),c,e>j&&ub(a.slice(j,e)),f>e&&ub(a=a.slice(e)),f>e&&pb(a))}m.push(c)}return rb(m)}function vb(a,b){var c=b.length>0,e=a.length>0,f=function(f,g,i,j,k){var m,n,o,p=0,q="0",r=f&&[],s=[],t=h,v=f||e&&d.find.TAG("*",k),w=u+=null==t?1:Math.random()||.1,x=v.length;for(k&&(h=g!==l&&g);q!==x&&null!=(m=v[q]);q++){if(e&&m){n=0;while(o=a[n++])if(o(m,g,i)){j.push(m);break}k&&(u=w)}c&&((m=!o&&m)&&p--,f&&r.push(m))}if(p+=q,c&&q!==p){n=0;while(o=b[n++])o(r,s,g,i);if(f){if(p>0)while(q--)r[q]||s[q]||(s[q]=E.call(j));s=sb(s)}G.apply(j,s),k&&!f&&s.length>0&&p+b.length>1&&db.uniqueSort(j)}return k&&(u=w,h=t),r};return c?fb(f):f}g=db.compile=function(a,b){var c,d=[],e=[],f=y[a+" "];if(!f){b||(b=ob(a)),c=b.length;while(c--)f=ub(b[c]),f[s]?d.push(f):e.push(f);f=y(a,vb(e,d))}return f};function wb(a,b,c){for(var d=0,e=b.length;e>d;d++)db(a,b[d],c);return c}function xb(a,b,e,f){var h,i,j,k,l,m=ob(a);if(!f&&1===m.length){if(i=m[0]=m[0].slice(0),i.length>2&&"ID"===(j=i[0]).type&&c.getById&&9===b.nodeType&&n&&d.relative[i[1].type]){if(b=(d.find.ID(j.matches[0].replace(ab,bb),b)||[])[0],!b)return e;a=a.slice(i.shift().value.length)}h=V.needsContext.test(a)?0:i.length;while(h--){if(j=i[h],d.relative[k=j.type])break;if((l=d.find[k])&&(f=l(j.matches[0].replace(ab,bb),$.test(i[0].type)&&mb(b.parentNode)||b))){if(i.splice(h,1),a=f.length&&pb(i),!a)return G.apply(e,f),e;break}}}return g(a,m)(f,b,!n,e,$.test(a)&&mb(b.parentNode)||b),e}return c.sortStable=s.split("").sort(z).join("")===s,c.detectDuplicates=!!j,k(),c.sortDetached=gb(function(a){return 1&a.compareDocumentPosition(l.createElement("div"))}),gb(function(a){return a.innerHTML="","#"===a.firstChild.getAttribute("href")})||hb("type|href|height|width",function(a,b,c){return c?void 0:a.getAttribute(b,"type"===b.toLowerCase()?1:2)}),c.attributes&&gb(function(a){return a.innerHTML="",a.firstChild.setAttribute("value",""),""===a.firstChild.getAttribute("value")})||hb("value",function(a,b,c){return c||"input"!==a.nodeName.toLowerCase()?void 0:a.defaultValue}),gb(function(a){return null==a.getAttribute("disabled")})||hb(J,function(a,b,c){var d;return c?void 0:a[b]===!0?b.toLowerCase():(d=a.getAttributeNode(b))&&d.specified?d.value:null}),db}(a);n.find=t,n.expr=t.selectors,n.expr[":"]=n.expr.pseudos,n.unique=t.uniqueSort,n.text=t.getText,n.isXMLDoc=t.isXML,n.contains=t.contains;var u=n.expr.match.needsContext,v=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,w=/^.[^:#\[\.,]*$/;function x(a,b,c){if(n.isFunction(b))return n.grep(a,function(a,d){return!!b.call(a,d,a)!==c});if(b.nodeType)return n.grep(a,function(a){return a===b!==c});if("string"==typeof b){if(w.test(b))return n.filter(b,a,c);b=n.filter(b,a)}return n.grep(a,function(a){return n.inArray(a,b)>=0!==c})}n.filter=function(a,b,c){var d=b[0];return c&&(a=":not("+a+")"),1===b.length&&1===d.nodeType?n.find.matchesSelector(d,a)?[d]:[]:n.find.matches(a,n.grep(b,function(a){return 1===a.nodeType}))},n.fn.extend({find:function(a){var b,c=[],d=this,e=d.length;if("string"!=typeof a)return this.pushStack(n(a).filter(function(){for(b=0;e>b;b++)if(n.contains(d[b],this))return!0}));for(b=0;e>b;b++)n.find(a,d[b],c);return c=this.pushStack(e>1?n.unique(c):c),c.selector=this.selector?this.selector+" "+a:a,c},filter:function(a){return this.pushStack(x(this,a||[],!1))},not:function(a){return this.pushStack(x(this,a||[],!0))},is:function(a){return!!x(this,"string"==typeof a&&u.test(a)?n(a):a||[],!1).length}});var y,z=a.document,A=/^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]*))$/,B=n.fn.init=function(a,b){var c,d;if(!a)return this;if("string"==typeof a){if(c="<"===a.charAt(0)&&">"===a.charAt(a.length-1)&&a.length>=3?[null,a,null]:A.exec(a),!c||!c[1]&&b)return!b||b.jquery?(b||y).find(a):this.constructor(b).find(a);if(c[1]){if(b=b instanceof n?b[0]:b,n.merge(this,n.parseHTML(c[1],b&&b.nodeType?b.ownerDocument||b:z,!0)),v.test(c[1])&&n.isPlainObject(b))for(c in b)n.isFunction(this[c])?this[c](b[c]):this.attr(c,b[c]);return this}if(d=z.getElementById(c[2]),d&&d.parentNode){if(d.id!==c[2])return y.find(a);this.length=1,this[0]=d}return this.context=z,this.selector=a,this}return a.nodeType?(this.context=this[0]=a,this.length=1,this):n.isFunction(a)?"undefined"!=typeof y.ready?y.ready(a):a(n):(void 0!==a.selector&&(this.selector=a.selector,this.context=a.context),n.makeArray(a,this))};B.prototype=n.fn,y=n(z);var C=/^(?:parents|prev(?:Until|All))/,D={children:!0,contents:!0,next:!0,prev:!0};n.extend({dir:function(a,b,c){var d=[],e=a[b];while(e&&9!==e.nodeType&&(void 0===c||1!==e.nodeType||!n(e).is(c)))1===e.nodeType&&d.push(e),e=e[b];return d},sibling:function(a,b){for(var c=[];a;a=a.nextSibling)1===a.nodeType&&a!==b&&c.push(a);return c}}),n.fn.extend({has:function(a){var b,c=n(a,this),d=c.length;return this.filter(function(){for(b=0;d>b;b++)if(n.contains(this,c[b]))return!0})},closest:function(a,b){for(var c,d=0,e=this.length,f=[],g=u.test(a)||"string"!=typeof a?n(a,b||this.context):0;e>d;d++)for(c=this[d];c&&c!==b;c=c.parentNode)if(c.nodeType<11&&(g?g.index(c)>-1:1===c.nodeType&&n.find.matchesSelector(c,a))){f.push(c);break}return this.pushStack(f.length>1?n.unique(f):f)},index:function(a){return a?"string"==typeof a?n.inArray(this[0],n(a)):n.inArray(a.jquery?a[0]:a,this):this[0]&&this[0].parentNode?this.first().prevAll().length:-1},add:function(a,b){return this.pushStack(n.unique(n.merge(this.get(),n(a,b))))},addBack:function(a){return this.add(null==a?this.prevObject:this.prevObject.filter(a))}});function E(a,b){do a=a[b];while(a&&1!==a.nodeType);return a}n.each({parent:function(a){var b=a.parentNode;return b&&11!==b.nodeType?b:null},parents:function(a){return n.dir(a,"parentNode")},parentsUntil:function(a,b,c){return n.dir(a,"parentNode",c)},next:function(a){return E(a,"nextSibling")},prev:function(a){return E(a,"previousSibling")},nextAll:function(a){return n.dir(a,"nextSibling")},prevAll:function(a){return n.dir(a,"previousSibling")},nextUntil:function(a,b,c){return n.dir(a,"nextSibling",c)},prevUntil:function(a,b,c){return n.dir(a,"previousSibling",c)},siblings:function(a){return n.sibling((a.parentNode||{}).firstChild,a)},children:function(a){return n.sibling(a.firstChild)},contents:function(a){return n.nodeName(a,"iframe")?a.contentDocument||a.contentWindow.document:n.merge([],a.childNodes)}},function(a,b){n.fn[a]=function(c,d){var e=n.map(this,b,c);return"Until"!==a.slice(-5)&&(d=c),d&&"string"==typeof d&&(e=n.filter(d,e)),this.length>1&&(D[a]||(e=n.unique(e)),C.test(a)&&(e=e.reverse())),this.pushStack(e)}});var F=/\S+/g,G={};function H(a){var b=G[a]={};return n.each(a.match(F)||[],function(a,c){b[c]=!0}),b}n.Callbacks=function(a){a="string"==typeof a?G[a]||H(a):n.extend({},a);var b,c,d,e,f,g,h=[],i=!a.once&&[],j=function(l){for(c=a.memory&&l,d=!0,f=g||0,g=0,e=h.length,b=!0;h&&e>f;f++)if(h[f].apply(l[0],l[1])===!1&&a.stopOnFalse){c=!1;break}b=!1,h&&(i?i.length&&j(i.shift()):c?h=[]:k.disable())},k={add:function(){if(h){var d=h.length;!function f(b){n.each(b,function(b,c){var d=n.type(c);"function"===d?a.unique&&k.has(c)||h.push(c):c&&c.length&&"string"!==d&&f(c)})}(arguments),b?e=h.length:c&&(g=d,j(c))}return this},remove:function(){return h&&n.each(arguments,function(a,c){var d;while((d=n.inArray(c,h,d))>-1)h.splice(d,1),b&&(e>=d&&e--,f>=d&&f--)}),this},has:function(a){return a?n.inArray(a,h)>-1:!(!h||!h.length)},empty:function(){return h=[],e=0,this},disable:function(){return h=i=c=void 0,this},disabled:function(){return!h},lock:function(){return i=void 0,c||k.disable(),this},locked:function(){return!i},fireWith:function(a,c){return!h||d&&!i||(c=c||[],c=[a,c.slice?c.slice():c],b?i.push(c):j(c)),this},fire:function(){return k.fireWith(this,arguments),this},fired:function(){return!!d}};return k},n.extend({Deferred:function(a){var b=[["resolve","done",n.Callbacks("once memory"),"resolved"],["reject","fail",n.Callbacks("once memory"),"rejected"],["notify","progress",n.Callbacks("memory")]],c="pending",d={state:function(){return c},always:function(){return e.done(arguments).fail(arguments),this},then:function(){var a=arguments;return n.Deferred(function(c){n.each(b,function(b,f){var g=n.isFunction(a[b])&&a[b];e[f[1]](function(){var a=g&&g.apply(this,arguments);a&&n.isFunction(a.promise)?a.promise().done(c.resolve).fail(c.reject).progress(c.notify):c[f[0]+"With"](this===d?c.promise():this,g?[a]:arguments)})}),a=null}).promise()},promise:function(a){return null!=a?n.extend(a,d):d}},e={};return d.pipe=d.then,n.each(b,function(a,f){var g=f[2],h=f[3];d[f[1]]=g.add,h&&g.add(function(){c=h},b[1^a][2].disable,b[2][2].lock),e[f[0]]=function(){return e[f[0]+"With"](this===e?d:this,arguments),this},e[f[0]+"With"]=g.fireWith}),d.promise(e),a&&a.call(e,e),e},when:function(a){var b=0,c=d.call(arguments),e=c.length,f=1!==e||a&&n.isFunction(a.promise)?e:0,g=1===f?a:n.Deferred(),h=function(a,b,c){return function(e){b[a]=this,c[a]=arguments.length>1?d.call(arguments):e,c===i?g.notifyWith(b,c):--f||g.resolveWith(b,c)}},i,j,k;if(e>1)for(i=new Array(e),j=new Array(e),k=new Array(e);e>b;b++)c[b]&&n.isFunction(c[b].promise)?c[b].promise().done(h(b,k,c)).fail(g.reject).progress(h(b,j,i)):--f;return f||g.resolveWith(k,c),g.promise()}});var I;n.fn.ready=function(a){return n.ready.promise().done(a),this},n.extend({isReady:!1,readyWait:1,holdReady:function(a){a?n.readyWait++:n.ready(!0)},ready:function(a){if(a===!0?!--n.readyWait:!n.isReady){if(!z.body)return setTimeout(n.ready);n.isReady=!0,a!==!0&&--n.readyWait>0||(I.resolveWith(z,[n]),n.fn.trigger&&n(z).trigger("ready").off("ready"))}}});function J(){z.addEventListener?(z.removeEventListener("DOMContentLoaded",K,!1),a.removeEventListener("load",K,!1)):(z.detachEvent("onreadystatechange",K),a.detachEvent("onload",K))}function K(){(z.addEventListener||"load"===event.type||"complete"===z.readyState)&&(J(),n.ready())}n.ready.promise=function(b){if(!I)if(I=n.Deferred(),"complete"===z.readyState)setTimeout(n.ready);else if(z.addEventListener)z.addEventListener("DOMContentLoaded",K,!1),a.addEventListener("load",K,!1);else{z.attachEvent("onreadystatechange",K),a.attachEvent("onload",K);var c=!1;try{c=null==a.frameElement&&z.documentElement}catch(d){}c&&c.doScroll&&!function e(){if(!n.isReady){try{c.doScroll("left")}catch(a){return setTimeout(e,50)}J(),n.ready()}}()}return I.promise(b)};var L="undefined",M;for(M in n(l))break;l.ownLast="0"!==M,l.inlineBlockNeedsLayout=!1,n(function(){var a,b,c=z.getElementsByTagName("body")[0];c&&(a=z.createElement("div"),a.style.cssText="border:0;width:0;height:0;position:absolute;top:0;left:-9999px;margin-top:1px",b=z.createElement("div"),c.appendChild(a).appendChild(b),typeof b.style.zoom!==L&&(b.style.cssText="border:0;margin:0;width:1px;padding:1px;display:inline;zoom:1",(l.inlineBlockNeedsLayout=3===b.offsetWidth)&&(c.style.zoom=1)),c.removeChild(a),a=b=null)}),function(){var a=z.createElement("div");if(null==l.deleteExpando){l.deleteExpando=!0;try{delete a.test}catch(b){l.deleteExpando=!1}}a=null}(),n.acceptData=function(a){var b=n.noData[(a.nodeName+" ").toLowerCase()],c=+a.nodeType||1;return 1!==c&&9!==c?!1:!b||b!==!0&&a.getAttribute("classid")===b};var N=/^(?:\{[\w\W]*\}|\[[\w\W]*\])$/,O=/([A-Z])/g;function P(a,b,c){if(void 0===c&&1===a.nodeType){var d="data-"+b.replace(O,"-$1").toLowerCase();if(c=a.getAttribute(d),"string"==typeof c){try{c="true"===c?!0:"false"===c?!1:"null"===c?null:+c+""===c?+c:N.test(c)?n.parseJSON(c):c}catch(e){}n.data(a,b,c)}else c=void 0}return c}function Q(a){var b;for(b in a)if(("data"!==b||!n.isEmptyObject(a[b]))&&"toJSON"!==b)return!1;return!0}function R(a,b,d,e){if(n.acceptData(a)){var f,g,h=n.expando,i=a.nodeType,j=i?n.cache:a,k=i?a[h]:a[h]&&h;if(k&&j[k]&&(e||j[k].data)||void 0!==d||"string"!=typeof b)return k||(k=i?a[h]=c.pop()||n.guid++:h),j[k]||(j[k]=i?{}:{toJSON:n.noop}),("object"==typeof b||"function"==typeof b)&&(e?j[k]=n.extend(j[k],b):j[k].data=n.extend(j[k].data,b)),g=j[k],e||(g.data||(g.data={}),g=g.data),void 0!==d&&(g[n.camelCase(b)]=d),"string"==typeof b?(f=g[b],null==f&&(f=g[n.camelCase(b)])):f=g,f +}}function S(a,b,c){if(n.acceptData(a)){var d,e,f=a.nodeType,g=f?n.cache:a,h=f?a[n.expando]:n.expando;if(g[h]){if(b&&(d=c?g[h]:g[h].data)){n.isArray(b)?b=b.concat(n.map(b,n.camelCase)):b in d?b=[b]:(b=n.camelCase(b),b=b in d?[b]:b.split(" ")),e=b.length;while(e--)delete d[b[e]];if(c?!Q(d):!n.isEmptyObject(d))return}(c||(delete g[h].data,Q(g[h])))&&(f?n.cleanData([a],!0):l.deleteExpando||g!=g.window?delete g[h]:g[h]=null)}}}n.extend({cache:{},noData:{"applet ":!0,"embed ":!0,"object ":"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000"},hasData:function(a){return a=a.nodeType?n.cache[a[n.expando]]:a[n.expando],!!a&&!Q(a)},data:function(a,b,c){return R(a,b,c)},removeData:function(a,b){return S(a,b)},_data:function(a,b,c){return R(a,b,c,!0)},_removeData:function(a,b){return S(a,b,!0)}}),n.fn.extend({data:function(a,b){var c,d,e,f=this[0],g=f&&f.attributes;if(void 0===a){if(this.length&&(e=n.data(f),1===f.nodeType&&!n._data(f,"parsedAttrs"))){c=g.length;while(c--)d=g[c].name,0===d.indexOf("data-")&&(d=n.camelCase(d.slice(5)),P(f,d,e[d]));n._data(f,"parsedAttrs",!0)}return e}return"object"==typeof a?this.each(function(){n.data(this,a)}):arguments.length>1?this.each(function(){n.data(this,a,b)}):f?P(f,a,n.data(f,a)):void 0},removeData:function(a){return this.each(function(){n.removeData(this,a)})}}),n.extend({queue:function(a,b,c){var d;return a?(b=(b||"fx")+"queue",d=n._data(a,b),c&&(!d||n.isArray(c)?d=n._data(a,b,n.makeArray(c)):d.push(c)),d||[]):void 0},dequeue:function(a,b){b=b||"fx";var c=n.queue(a,b),d=c.length,e=c.shift(),f=n._queueHooks(a,b),g=function(){n.dequeue(a,b)};"inprogress"===e&&(e=c.shift(),d--),e&&("fx"===b&&c.unshift("inprogress"),delete f.stop,e.call(a,g,f)),!d&&f&&f.empty.fire()},_queueHooks:function(a,b){var c=b+"queueHooks";return n._data(a,c)||n._data(a,c,{empty:n.Callbacks("once memory").add(function(){n._removeData(a,b+"queue"),n._removeData(a,c)})})}}),n.fn.extend({queue:function(a,b){var c=2;return"string"!=typeof a&&(b=a,a="fx",c--),arguments.lengthh;h++)b(a[h],c,g?d:d.call(a[h],h,b(a[h],c)));return e?a:j?b.call(a):i?b(a[0],c):f},X=/^(?:checkbox|radio)$/i;!function(){var a=z.createDocumentFragment(),b=z.createElement("div"),c=z.createElement("input");if(b.setAttribute("className","t"),b.innerHTML="
    a",l.leadingWhitespace=3===b.firstChild.nodeType,l.tbody=!b.getElementsByTagName("tbody").length,l.htmlSerialize=!!b.getElementsByTagName("link").length,l.html5Clone="<:nav>"!==z.createElement("nav").cloneNode(!0).outerHTML,c.type="checkbox",c.checked=!0,a.appendChild(c),l.appendChecked=c.checked,b.innerHTML="",l.noCloneChecked=!!b.cloneNode(!0).lastChild.defaultValue,a.appendChild(b),b.innerHTML="",l.checkClone=b.cloneNode(!0).cloneNode(!0).lastChild.checked,l.noCloneEvent=!0,b.attachEvent&&(b.attachEvent("onclick",function(){l.noCloneEvent=!1}),b.cloneNode(!0).click()),null==l.deleteExpando){l.deleteExpando=!0;try{delete b.test}catch(d){l.deleteExpando=!1}}a=b=c=null}(),function(){var b,c,d=z.createElement("div");for(b in{submit:!0,change:!0,focusin:!0})c="on"+b,(l[b+"Bubbles"]=c in a)||(d.setAttribute(c,"t"),l[b+"Bubbles"]=d.attributes[c].expando===!1);d=null}();var Y=/^(?:input|select|textarea)$/i,Z=/^key/,$=/^(?:mouse|contextmenu)|click/,_=/^(?:focusinfocus|focusoutblur)$/,ab=/^([^.]*)(?:\.(.+)|)$/;function bb(){return!0}function cb(){return!1}function db(){try{return z.activeElement}catch(a){}}n.event={global:{},add:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,o,p,q,r=n._data(a);if(r){c.handler&&(i=c,c=i.handler,e=i.selector),c.guid||(c.guid=n.guid++),(g=r.events)||(g=r.events={}),(k=r.handle)||(k=r.handle=function(a){return typeof n===L||a&&n.event.triggered===a.type?void 0:n.event.dispatch.apply(k.elem,arguments)},k.elem=a),b=(b||"").match(F)||[""],h=b.length;while(h--)f=ab.exec(b[h])||[],o=q=f[1],p=(f[2]||"").split(".").sort(),o&&(j=n.event.special[o]||{},o=(e?j.delegateType:j.bindType)||o,j=n.event.special[o]||{},l=n.extend({type:o,origType:q,data:d,handler:c,guid:c.guid,selector:e,needsContext:e&&n.expr.match.needsContext.test(e),namespace:p.join(".")},i),(m=g[o])||(m=g[o]=[],m.delegateCount=0,j.setup&&j.setup.call(a,d,p,k)!==!1||(a.addEventListener?a.addEventListener(o,k,!1):a.attachEvent&&a.attachEvent("on"+o,k))),j.add&&(j.add.call(a,l),l.handler.guid||(l.handler.guid=c.guid)),e?m.splice(m.delegateCount++,0,l):m.push(l),n.event.global[o]=!0);a=null}},remove:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,o,p,q,r=n.hasData(a)&&n._data(a);if(r&&(k=r.events)){b=(b||"").match(F)||[""],j=b.length;while(j--)if(h=ab.exec(b[j])||[],o=q=h[1],p=(h[2]||"").split(".").sort(),o){l=n.event.special[o]||{},o=(d?l.delegateType:l.bindType)||o,m=k[o]||[],h=h[2]&&new RegExp("(^|\\.)"+p.join("\\.(?:.*\\.|)")+"(\\.|$)"),i=f=m.length;while(f--)g=m[f],!e&&q!==g.origType||c&&c.guid!==g.guid||h&&!h.test(g.namespace)||d&&d!==g.selector&&("**"!==d||!g.selector)||(m.splice(f,1),g.selector&&m.delegateCount--,l.remove&&l.remove.call(a,g));i&&!m.length&&(l.teardown&&l.teardown.call(a,p,r.handle)!==!1||n.removeEvent(a,o,r.handle),delete k[o])}else for(o in k)n.event.remove(a,o+b[j],c,d,!0);n.isEmptyObject(k)&&(delete r.handle,n._removeData(a,"events"))}},trigger:function(b,c,d,e){var f,g,h,i,k,l,m,o=[d||z],p=j.call(b,"type")?b.type:b,q=j.call(b,"namespace")?b.namespace.split("."):[];if(h=l=d=d||z,3!==d.nodeType&&8!==d.nodeType&&!_.test(p+n.event.triggered)&&(p.indexOf(".")>=0&&(q=p.split("."),p=q.shift(),q.sort()),g=p.indexOf(":")<0&&"on"+p,b=b[n.expando]?b:new n.Event(p,"object"==typeof b&&b),b.isTrigger=e?2:3,b.namespace=q.join("."),b.namespace_re=b.namespace?new RegExp("(^|\\.)"+q.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,b.result=void 0,b.target||(b.target=d),c=null==c?[b]:n.makeArray(c,[b]),k=n.event.special[p]||{},e||!k.trigger||k.trigger.apply(d,c)!==!1)){if(!e&&!k.noBubble&&!n.isWindow(d)){for(i=k.delegateType||p,_.test(i+p)||(h=h.parentNode);h;h=h.parentNode)o.push(h),l=h;l===(d.ownerDocument||z)&&o.push(l.defaultView||l.parentWindow||a)}m=0;while((h=o[m++])&&!b.isPropagationStopped())b.type=m>1?i:k.bindType||p,f=(n._data(h,"events")||{})[b.type]&&n._data(h,"handle"),f&&f.apply(h,c),f=g&&h[g],f&&f.apply&&n.acceptData(h)&&(b.result=f.apply(h,c),b.result===!1&&b.preventDefault());if(b.type=p,!e&&!b.isDefaultPrevented()&&(!k._default||k._default.apply(o.pop(),c)===!1)&&n.acceptData(d)&&g&&d[p]&&!n.isWindow(d)){l=d[g],l&&(d[g]=null),n.event.triggered=p;try{d[p]()}catch(r){}n.event.triggered=void 0,l&&(d[g]=l)}return b.result}},dispatch:function(a){a=n.event.fix(a);var b,c,e,f,g,h=[],i=d.call(arguments),j=(n._data(this,"events")||{})[a.type]||[],k=n.event.special[a.type]||{};if(i[0]=a,a.delegateTarget=this,!k.preDispatch||k.preDispatch.call(this,a)!==!1){h=n.event.handlers.call(this,a,j),b=0;while((f=h[b++])&&!a.isPropagationStopped()){a.currentTarget=f.elem,g=0;while((e=f.handlers[g++])&&!a.isImmediatePropagationStopped())(!a.namespace_re||a.namespace_re.test(e.namespace))&&(a.handleObj=e,a.data=e.data,c=((n.event.special[e.origType]||{}).handle||e.handler).apply(f.elem,i),void 0!==c&&(a.result=c)===!1&&(a.preventDefault(),a.stopPropagation()))}return k.postDispatch&&k.postDispatch.call(this,a),a.result}},handlers:function(a,b){var c,d,e,f,g=[],h=b.delegateCount,i=a.target;if(h&&i.nodeType&&(!a.button||"click"!==a.type))for(;i!=this;i=i.parentNode||this)if(1===i.nodeType&&(i.disabled!==!0||"click"!==a.type)){for(e=[],f=0;h>f;f++)d=b[f],c=d.selector+" ",void 0===e[c]&&(e[c]=d.needsContext?n(c,this).index(i)>=0:n.find(c,this,null,[i]).length),e[c]&&e.push(d);e.length&&g.push({elem:i,handlers:e})}return h]","i"),ib=/^\s+/,jb=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:]+)[^>]*)\/>/gi,kb=/<([\w:]+)/,lb=/\s*$/g,sb={option:[1,""],legend:[1,"
    ","
    "],area:[1,"",""],param:[1,"",""],thead:[1,"","
    "],tr:[2,"","
    "],col:[2,"","
    "],td:[3,"","
    "],_default:l.htmlSerialize?[0,"",""]:[1,"X
    ","
    "]},tb=eb(z),ub=tb.appendChild(z.createElement("div"));sb.optgroup=sb.option,sb.tbody=sb.tfoot=sb.colgroup=sb.caption=sb.thead,sb.th=sb.td;function vb(a,b){var c,d,e=0,f=typeof a.getElementsByTagName!==L?a.getElementsByTagName(b||"*"):typeof a.querySelectorAll!==L?a.querySelectorAll(b||"*"):void 0;if(!f)for(f=[],c=a.childNodes||a;null!=(d=c[e]);e++)!b||n.nodeName(d,b)?f.push(d):n.merge(f,vb(d,b));return void 0===b||b&&n.nodeName(a,b)?n.merge([a],f):f}function wb(a){X.test(a.type)&&(a.defaultChecked=a.checked)}function xb(a,b){return n.nodeName(a,"table")&&n.nodeName(11!==b.nodeType?b:b.firstChild,"tr")?a.getElementsByTagName("tbody")[0]||a.appendChild(a.ownerDocument.createElement("tbody")):a}function yb(a){return a.type=(null!==n.find.attr(a,"type"))+"/"+a.type,a}function zb(a){var b=qb.exec(a.type);return b?a.type=b[1]:a.removeAttribute("type"),a}function Ab(a,b){for(var c,d=0;null!=(c=a[d]);d++)n._data(c,"globalEval",!b||n._data(b[d],"globalEval"))}function Bb(a,b){if(1===b.nodeType&&n.hasData(a)){var c,d,e,f=n._data(a),g=n._data(b,f),h=f.events;if(h){delete g.handle,g.events={};for(c in h)for(d=0,e=h[c].length;e>d;d++)n.event.add(b,c,h[c][d])}g.data&&(g.data=n.extend({},g.data))}}function Cb(a,b){var c,d,e;if(1===b.nodeType){if(c=b.nodeName.toLowerCase(),!l.noCloneEvent&&b[n.expando]){e=n._data(b);for(d in e.events)n.removeEvent(b,d,e.handle);b.removeAttribute(n.expando)}"script"===c&&b.text!==a.text?(yb(b).text=a.text,zb(b)):"object"===c?(b.parentNode&&(b.outerHTML=a.outerHTML),l.html5Clone&&a.innerHTML&&!n.trim(b.innerHTML)&&(b.innerHTML=a.innerHTML)):"input"===c&&X.test(a.type)?(b.defaultChecked=b.checked=a.checked,b.value!==a.value&&(b.value=a.value)):"option"===c?b.defaultSelected=b.selected=a.defaultSelected:("input"===c||"textarea"===c)&&(b.defaultValue=a.defaultValue)}}n.extend({clone:function(a,b,c){var d,e,f,g,h,i=n.contains(a.ownerDocument,a);if(l.html5Clone||n.isXMLDoc(a)||!hb.test("<"+a.nodeName+">")?f=a.cloneNode(!0):(ub.innerHTML=a.outerHTML,ub.removeChild(f=ub.firstChild)),!(l.noCloneEvent&&l.noCloneChecked||1!==a.nodeType&&11!==a.nodeType||n.isXMLDoc(a)))for(d=vb(f),h=vb(a),g=0;null!=(e=h[g]);++g)d[g]&&Cb(e,d[g]);if(b)if(c)for(h=h||vb(a),d=d||vb(f),g=0;null!=(e=h[g]);g++)Bb(e,d[g]);else Bb(a,f);return d=vb(f,"script"),d.length>0&&Ab(d,!i&&vb(a,"script")),d=h=e=null,f},buildFragment:function(a,b,c,d){for(var e,f,g,h,i,j,k,m=a.length,o=eb(b),p=[],q=0;m>q;q++)if(f=a[q],f||0===f)if("object"===n.type(f))n.merge(p,f.nodeType?[f]:f);else if(mb.test(f)){h=h||o.appendChild(b.createElement("div")),i=(kb.exec(f)||["",""])[1].toLowerCase(),k=sb[i]||sb._default,h.innerHTML=k[1]+f.replace(jb,"<$1>")+k[2],e=k[0];while(e--)h=h.lastChild;if(!l.leadingWhitespace&&ib.test(f)&&p.push(b.createTextNode(ib.exec(f)[0])),!l.tbody){f="table"!==i||lb.test(f)?""!==k[1]||lb.test(f)?0:h:h.firstChild,e=f&&f.childNodes.length;while(e--)n.nodeName(j=f.childNodes[e],"tbody")&&!j.childNodes.length&&f.removeChild(j)}n.merge(p,h.childNodes),h.textContent="";while(h.firstChild)h.removeChild(h.firstChild);h=o.lastChild}else p.push(b.createTextNode(f));h&&o.removeChild(h),l.appendChecked||n.grep(vb(p,"input"),wb),q=0;while(f=p[q++])if((!d||-1===n.inArray(f,d))&&(g=n.contains(f.ownerDocument,f),h=vb(o.appendChild(f),"script"),g&&Ab(h),c)){e=0;while(f=h[e++])pb.test(f.type||"")&&c.push(f)}return h=null,o},cleanData:function(a,b){for(var d,e,f,g,h=0,i=n.expando,j=n.cache,k=l.deleteExpando,m=n.event.special;null!=(d=a[h]);h++)if((b||n.acceptData(d))&&(f=d[i],g=f&&j[f])){if(g.events)for(e in g.events)m[e]?n.event.remove(d,e):n.removeEvent(d,e,g.handle);j[f]&&(delete j[f],k?delete d[i]:typeof d.removeAttribute!==L?d.removeAttribute(i):d[i]=null,c.push(f))}}}),n.fn.extend({text:function(a){return W(this,function(a){return void 0===a?n.text(this):this.empty().append((this[0]&&this[0].ownerDocument||z).createTextNode(a))},null,a,arguments.length)},append:function(){return this.domManip(arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=xb(this,a);b.appendChild(a)}})},prepend:function(){return this.domManip(arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=xb(this,a);b.insertBefore(a,b.firstChild)}})},before:function(){return this.domManip(arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this)})},after:function(){return this.domManip(arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this.nextSibling)})},remove:function(a,b){for(var c,d=a?n.filter(a,this):this,e=0;null!=(c=d[e]);e++)b||1!==c.nodeType||n.cleanData(vb(c)),c.parentNode&&(b&&n.contains(c.ownerDocument,c)&&Ab(vb(c,"script")),c.parentNode.removeChild(c));return this},empty:function(){for(var a,b=0;null!=(a=this[b]);b++){1===a.nodeType&&n.cleanData(vb(a,!1));while(a.firstChild)a.removeChild(a.firstChild);a.options&&n.nodeName(a,"select")&&(a.options.length=0)}return this},clone:function(a,b){return a=null==a?!1:a,b=null==b?a:b,this.map(function(){return n.clone(this,a,b)})},html:function(a){return W(this,function(a){var b=this[0]||{},c=0,d=this.length;if(void 0===a)return 1===b.nodeType?b.innerHTML.replace(gb,""):void 0;if(!("string"!=typeof a||nb.test(a)||!l.htmlSerialize&&hb.test(a)||!l.leadingWhitespace&&ib.test(a)||sb[(kb.exec(a)||["",""])[1].toLowerCase()])){a=a.replace(jb,"<$1>");try{for(;d>c;c++)b=this[c]||{},1===b.nodeType&&(n.cleanData(vb(b,!1)),b.innerHTML=a);b=0}catch(e){}}b&&this.empty().append(a)},null,a,arguments.length)},replaceWith:function(){var a=arguments[0];return this.domManip(arguments,function(b){a=this.parentNode,n.cleanData(vb(this)),a&&a.replaceChild(b,this)}),a&&(a.length||a.nodeType)?this:this.remove()},detach:function(a){return this.remove(a,!0)},domManip:function(a,b){a=e.apply([],a);var c,d,f,g,h,i,j=0,k=this.length,m=this,o=k-1,p=a[0],q=n.isFunction(p);if(q||k>1&&"string"==typeof p&&!l.checkClone&&ob.test(p))return this.each(function(c){var d=m.eq(c);q&&(a[0]=p.call(this,c,d.html())),d.domManip(a,b)});if(k&&(i=n.buildFragment(a,this[0].ownerDocument,!1,this),c=i.firstChild,1===i.childNodes.length&&(i=c),c)){for(g=n.map(vb(i,"script"),yb),f=g.length;k>j;j++)d=i,j!==o&&(d=n.clone(d,!0,!0),f&&n.merge(g,vb(d,"script"))),b.call(this[j],d,j);if(f)for(h=g[g.length-1].ownerDocument,n.map(g,zb),j=0;f>j;j++)d=g[j],pb.test(d.type||"")&&!n._data(d,"globalEval")&&n.contains(h,d)&&(d.src?n._evalUrl&&n._evalUrl(d.src):n.globalEval((d.text||d.textContent||d.innerHTML||"").replace(rb,"")));i=c=null}return this}}),n.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"after",replaceAll:"replaceWith"},function(a,b){n.fn[a]=function(a){for(var c,d=0,e=[],g=n(a),h=g.length-1;h>=d;d++)c=d===h?this:this.clone(!0),n(g[d])[b](c),f.apply(e,c.get());return this.pushStack(e)}});var Db,Eb={};function Fb(b,c){var d=n(c.createElement(b)).appendTo(c.body),e=a.getDefaultComputedStyle?a.getDefaultComputedStyle(d[0]).display:n.css(d[0],"display");return d.detach(),e}function Gb(a){var b=z,c=Eb[a];return c||(c=Fb(a,b),"none"!==c&&c||(Db=(Db||n("