From 5822deadb9787bc1728ab94dcbffc9ac62eae290 Mon Sep 17 00:00:00 2001 From: Colin Goodheart-Smithe Date: Tue, 15 Aug 2017 11:14:45 +0100 Subject: [PATCH 01/24] Adds a new auto-interval date histogram This change adds a new type of histogram aggregation called `auto_date_histogram` where you can specify the target number of buckets you require and it will find an appropriate interval for the returned buckets. The aggregation works by first collecting documents in buckets at second interval, when it has created more than the target number of buckets it merges these buckets into minute interval bucket and continues collecting until it reaches the target number of buckets again. It will keep merging buckets when it exceeds the target until either collection is finished or the highest interval (currently years) is reached. A similar process happens at reduce time. This aggregation intentionally does not support min_doc_count, offest and extended_bounds to keep the already complex logic from becoming more complex. The aggregation accepts sub-aggregations but will always operate in `breadth_first` mode deferring the computation of sub-aggregations until the final buckets from the shard are known. min_doc_count is effectively hard-coded to zero meaning that we will insert empty buckets where necessary. Closes #9572 --- .../client/RestHighLevelClient.java | 3 + .../MergingBucketsDeferringCollector.java | 217 +++++++ .../AutoDateHistogramAggregationBuilder.java | 137 +++++ .../AutoDateHistogramAggregator.java | 196 +++++++ .../AutoDateHistogramAggregatorFactory.java | 72 +++ .../histogram/InternalAutoDateHistogram.java | 527 +++++++++++++++++ .../histogram/ParsedAutoDateHistogram.java | 91 +++ .../bucket/AutoDateHistogramTests.java | 44 ++ .../AutoDateHistogramAggregatorTests.java | 536 ++++++++++++++++++ .../InternalAutoDateHistogramTests.java | 153 +++++ .../elasticsearch/search/SearchModule.java | 4 + .../bucket/BucketsAggregator.java | 13 + .../histogram/DateHistogramAggregator.java | 4 +- .../histogram/InternalDateHistogram.java | 2 +- .../aggregations/AggregationsTests.java | 2 + .../test/InternalAggregationTestCase.java | 3 + ...nternalMultiBucketAggregationTestCase.java | 3 +- 17 files changed, 2003 insertions(+), 4 deletions(-) create mode 100644 core/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java create mode 100644 core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java create mode 100644 core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java create mode 100644 core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java create mode 100644 core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java create mode 100644 core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/ParsedAutoDateHistogram.java create mode 100644 core/src/test/java/org/elasticsearch/search/aggregations/bucket/AutoDateHistogramTests.java create mode 100644 core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java create mode 100644 core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java index bf80aa7720741..e345db2a005a0 100755 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java @@ -73,8 +73,10 @@ import org.elasticsearch.search.aggregations.bucket.geogrid.ParsedGeoHashGrid; import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.global.ParsedGlobal; +import org.elasticsearch.search.aggregations.bucket.histogram.AutoDateHistogramAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.histogram.ParsedAutoDateHistogram; import org.elasticsearch.search.aggregations.bucket.histogram.ParsedDateHistogram; import org.elasticsearch.search.aggregations.bucket.histogram.ParsedHistogram; import org.elasticsearch.search.aggregations.bucket.missing.MissingAggregationBuilder; @@ -683,6 +685,7 @@ static List getDefaultNamedXContents() { map.put(GeoCentroidAggregationBuilder.NAME, (p, c) -> ParsedGeoCentroid.fromXContent(p, (String) c)); map.put(HistogramAggregationBuilder.NAME, (p, c) -> ParsedHistogram.fromXContent(p, (String) c)); map.put(DateHistogramAggregationBuilder.NAME, (p, c) -> ParsedDateHistogram.fromXContent(p, (String) c)); + map.put(AutoDateHistogramAggregationBuilder.NAME, (p, c) -> ParsedAutoDateHistogram.fromXContent(p, (String) c)); map.put(StringTerms.NAME, (p, c) -> ParsedStringTerms.fromXContent(p, (String) c)); map.put(LongTerms.NAME, (p, c) -> ParsedLongTerms.fromXContent(p, (String) c)); map.put(DoubleTerms.NAME, (p, c) -> ParsedDoubleTerms.fromXContent(p, (String) c)); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java new file mode 100644 index 0000000000000..72653440c800c --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java @@ -0,0 +1,217 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.bucket; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.Weight; +import org.apache.lucene.util.packed.PackedInts; +import org.apache.lucene.util.packed.PackedLongValues; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.LongHash; +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.BucketCollector; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +public class MergingBucketsDeferringCollector extends DeferringBucketCollector { + + List entries = new ArrayList<>(); + BucketCollector collector; + final SearchContext searchContext; + LeafReaderContext context; + PackedLongValues.Builder docDeltas; + PackedLongValues.Builder buckets; + long maxBucket = -1; + boolean finished = false; + LongHash selectedBuckets; + + public MergingBucketsDeferringCollector(SearchContext context) { + this.searchContext = context; + } + + @Override + public void setDeferredCollector(Iterable deferredCollectors) { + this.collector = BucketCollector.wrap(deferredCollectors); + } + + @Override + public boolean needsScores() { + if (collector == null) { + throw new IllegalStateException(); + } + return collector.needsScores(); + } + + @Override + public void preCollection() throws IOException { + collector.preCollection(); + } + + private void finishLeaf() { + if (context != null) { + entries.add(new Entry(context, docDeltas.build(), buckets.build())); + } + context = null; + docDeltas = null; + buckets = null; + } + + @Override + public LeafBucketCollector getLeafCollector(LeafReaderContext ctx) throws IOException { + finishLeaf(); + + context = ctx; + docDeltas = PackedLongValues.packedBuilder(PackedInts.DEFAULT); + buckets = PackedLongValues.packedBuilder(PackedInts.DEFAULT); + + return new LeafBucketCollector() { + int lastDoc = 0; + + @Override + public void collect(int doc, long bucket) throws IOException { + docDeltas.add(doc - lastDoc); + buckets.add(bucket); + lastDoc = doc; + maxBucket = Math.max(maxBucket, bucket); + } + }; + } + + public void mergeBuckets(long[] mergeMap) { + List newEntries = new ArrayList<>(); + for (Entry sourceEntry : entries) { + PackedLongValues.Builder newBuckets = PackedLongValues.packedBuilder(PackedInts.DEFAULT); + for (PackedLongValues.Iterator itr = sourceEntry.buckets.iterator(); itr.hasNext();) { + long bucket = itr.next(); + newBuckets.add(mergeMap[(int) bucket]); + } + newEntries.add(new Entry(sourceEntry.context, sourceEntry.docDeltas, newBuckets.build())); + } + entries = newEntries; + } + + @Override + public void postCollection() throws IOException { + finishLeaf(); + finished = true; + } + + /** + * Replay the wrapped collector, but only on a selection of buckets. + */ + @Override + public void prepareSelectedBuckets(long... selectedBuckets) throws IOException { + if (!finished) { + throw new IllegalStateException("Cannot replay yet, collection is not finished: postCollect() has not been called"); + } + if (this.selectedBuckets != null) { + throw new IllegalStateException("Already been replayed"); + } + + final LongHash hash = new LongHash(selectedBuckets.length, BigArrays.NON_RECYCLING_INSTANCE); + for (long bucket : selectedBuckets) { + hash.add(bucket); + } + this.selectedBuckets = hash; + + boolean needsScores = collector.needsScores(); + Weight weight = null; + if (needsScores) { + weight = searchContext.searcher().createNormalizedWeight(searchContext.query(), true); + } + for (Entry entry : entries) { + final LeafBucketCollector leafCollector = collector.getLeafCollector(entry.context); + DocIdSetIterator docIt = null; + if (needsScores && entry.docDeltas.size() > 0) { + Scorer scorer = weight.scorer(entry.context); + // We don't need to check if the scorer is null + // since we are sure that there are documents to replay + // (entry.docDeltas it not empty). + docIt = scorer.iterator(); + leafCollector.setScorer(scorer); + } + final PackedLongValues.Iterator docDeltaIterator = entry.docDeltas.iterator(); + final PackedLongValues.Iterator buckets = entry.buckets.iterator(); + int doc = 0; + for (long i = 0, end = entry.docDeltas.size(); i < end; ++i) { + doc += docDeltaIterator.next(); + final long bucket = buckets.next(); + final long rebasedBucket = hash.find(bucket); + if (rebasedBucket != -1) { + if (needsScores) { + if (docIt.docID() < doc) { + docIt.advance(doc); + } + // aggregations should only be replayed on matching + // documents + assert docIt.docID() == doc; + } + leafCollector.collect(doc, rebasedBucket); + } + } + } + + collector.postCollection(); + } + + /** + * Wrap the provided aggregator so that it behaves (almost) as if it had + * been collected directly. + */ + @Override + public Aggregator wrap(final Aggregator in) { + + return new WrappedAggregator(in) { + + @Override + public InternalAggregation buildAggregation(long bucket) throws IOException { + if (selectedBuckets == null) { + throw new IllegalStateException("Collection has not been replayed yet."); + } + final long rebasedBucket = selectedBuckets.find(bucket); + if (rebasedBucket == -1) { + throw new IllegalStateException("Cannot build for a bucket which has not been collected"); + } + return in.buildAggregation(rebasedBucket); + } + + }; + } + + private static class Entry { + final LeafReaderContext context; + final PackedLongValues docDeltas; + final PackedLongValues buckets; + + Entry(LeafReaderContext context, PackedLongValues docDeltas, PackedLongValues buckets) { + this.context = context; + this.docDeltas = docDeltas; + this.buckets = buckets; + } + } + +} diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java new file mode 100644 index 0000000000000..0194ca6789f45 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java @@ -0,0 +1,137 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.bucket.histogram; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.rounding.DateTimeUnit; +import org.elasticsearch.common.rounding.Rounding; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; +import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.support.ValueType; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; +import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; +import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper; +import org.elasticsearch.search.aggregations.support.ValuesSourceType; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.Objects; + +public class AutoDateHistogramAggregationBuilder + extends ValuesSourceAggregationBuilder { + + public static final String NAME = "auto_date_histogram"; + + public static final ParseField NUM_BUCKETS_FIELD = new ParseField("buckets"); + + private static final ObjectParser PARSER; + static { + PARSER = new ObjectParser<>(AutoDateHistogramAggregationBuilder.NAME); + ValuesSourceParserHelper.declareNumericFields(PARSER, true, true, true); + + PARSER.declareInt(AutoDateHistogramAggregationBuilder::setNumBuckets, NUM_BUCKETS_FIELD); + } + + public static AutoDateHistogramAggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException { + return PARSER.parse(parser, new AutoDateHistogramAggregationBuilder(aggregationName), null); + } + + private int numBuckets = 10; + + /** Create a new builder with the given name. */ + public AutoDateHistogramAggregationBuilder(String name) { + super(name, ValuesSourceType.NUMERIC, ValueType.DATE); + } + + /** Read from a stream, for internal use only. */ + public AutoDateHistogramAggregationBuilder(StreamInput in) throws IOException { + super(in, ValuesSourceType.NUMERIC, ValueType.DATE); + numBuckets = in.readVInt(); + } + + @Override + protected void innerWriteTo(StreamOutput out) throws IOException { + out.writeVInt(numBuckets); + } + + @Override + public String getType() { + return NAME; + } + + public AutoDateHistogramAggregationBuilder setNumBuckets(int numBuckets) { + if (numBuckets <= 0) { + throw new IllegalArgumentException(NUM_BUCKETS_FIELD.getPreferredName() + " must be greater than 0 for [" + name + "]"); + } + this.numBuckets = numBuckets; + return this; + } + + public int getNumBuckets() { + return numBuckets; + } + + @Override + protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + Rounding[] roundings = new Rounding[6]; + roundings[0] = createRounding(DateTimeUnit.SECOND_OF_MINUTE); + roundings[1] = createRounding(DateTimeUnit.MINUTES_OF_HOUR); + roundings[2] = createRounding(DateTimeUnit.HOUR_OF_DAY); + roundings[3] = createRounding(DateTimeUnit.DAY_OF_MONTH); + roundings[4] = createRounding(DateTimeUnit.MONTH_OF_YEAR); + roundings[5] = createRounding(DateTimeUnit.YEAR_OF_CENTURY); + return new AutoDateHistogramAggregatorFactory(name, config, numBuckets, roundings, context, parent, subFactoriesBuilder, metaData); + } + + private Rounding createRounding(DateTimeUnit interval) { + Rounding.Builder tzRoundingBuilder = Rounding.builder(interval); + if (timeZone() != null) { + tzRoundingBuilder.timeZone(timeZone()); + } + Rounding rounding = tzRoundingBuilder.build(); + return rounding; + } + + @Override + protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + builder.field(NUM_BUCKETS_FIELD.getPreferredName(), numBuckets); + return builder; + } + + @Override + protected int innerHashCode() { + return Objects.hash(numBuckets); + } + + @Override + protected boolean innerEquals(Object obj) { + AutoDateHistogramAggregationBuilder other = (AutoDateHistogramAggregationBuilder) obj; + return Objects.equals(numBuckets, other.numBuckets); + } +} diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java new file mode 100644 index 0000000000000..293ee2fcbee74 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java @@ -0,0 +1,196 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.search.aggregations.bucket.histogram; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.util.CollectionUtil; +import org.elasticsearch.common.inject.internal.Nullable; +import org.elasticsearch.common.lease.Releasables; +import org.elasticsearch.common.rounding.Rounding; +import org.elasticsearch.common.util.LongHash; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.BucketOrder; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; +import org.elasticsearch.search.aggregations.bucket.DeferableBucketAggregator; +import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector; +import org.elasticsearch.search.aggregations.bucket.MergingBucketsDeferringCollector; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * An aggregator for date values. Every date is rounded down using a configured + * {@link Rounding}. + * + * @see Rounding + */ +class AutoDateHistogramAggregator extends DeferableBucketAggregator { + + private final ValuesSource.Numeric valuesSource; + private final DocValueFormat formatter; + private final Rounding[] roundings; + private int roundingIdx = 0; + + private LongHash bucketOrds; + private int targetBuckets; + private MergingBucketsDeferringCollector deferringCollector; + + AutoDateHistogramAggregator(String name, AggregatorFactories factories, int numBuckets, Rounding[] roundings, + @Nullable ValuesSource.Numeric valuesSource, DocValueFormat formatter, SearchContext aggregationContext, Aggregator parent, + List pipelineAggregators, Map metaData) throws IOException { + + super(name, factories, aggregationContext, parent, pipelineAggregators, metaData); + this.targetBuckets = numBuckets; + this.valuesSource = valuesSource; + this.formatter = formatter; + this.roundings = roundings; + + bucketOrds = new LongHash(1, aggregationContext.bigArrays()); + + } + + @Override + public boolean needsScores() { + return (valuesSource != null && valuesSource.needsScores()) || super.needsScores(); + } + + @Override + protected boolean shouldDefer(Aggregator aggregator) { + return true; + } + + @Override + public DeferringBucketCollector getDeferringCollector() { + deferringCollector = new MergingBucketsDeferringCollector(context); + return deferringCollector; + } + + @Override + public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, + final LeafBucketCollector sub) throws IOException { + if (valuesSource == null) { + return LeafBucketCollector.NO_OP_COLLECTOR; + } + final SortedNumericDocValues values = valuesSource.longValues(ctx); + return new LeafBucketCollectorBase(sub, values) { + @Override + public void collect(int doc, long bucket) throws IOException { + assert bucket == 0; + if (values.advanceExact(doc)) { + final int valuesCount = values.docValueCount(); + + long previousRounded = Long.MIN_VALUE; + for (int i = 0; i < valuesCount; ++i) { + long value = values.nextValue(); + long rounded = roundings[roundingIdx].round(value); + assert rounded >= previousRounded; + if (rounded == previousRounded) { + continue; + } + long bucketOrd = bucketOrds.add(rounded); + if (bucketOrd < 0) { // already seen + bucketOrd = -1 - bucketOrd; + collectExistingBucket(sub, doc, bucketOrd); + } else { + collectBucket(sub, doc, bucketOrd); + while (bucketOrds.size() > targetBuckets) { + increaseRounding(); + } + } + previousRounded = rounded; + } + } + } + + private void increaseRounding() { + try (LongHash oldBucketOrds = bucketOrds) { + LongHash newBucketOrds = new LongHash(1, context.bigArrays()); + long[] mergeMap = new long[(int) oldBucketOrds.size()]; + Rounding newRounding = roundings[++roundingIdx]; + for (int i = 0; i < oldBucketOrds.size(); i++) { + long oldKey = oldBucketOrds.get(i); + long newKey = newRounding.round(oldKey); + long newBucketOrd = newBucketOrds.add(newKey); + if (newBucketOrd >= 0) { + mergeMap[i] = newBucketOrd; + } else { + mergeMap[i] = -1 - newBucketOrd; + } + } + mergeBuckets(mergeMap, newBucketOrds.size()); + if (deferringCollector != null) { + deferringCollector.mergeBuckets(mergeMap); + } + bucketOrds = newBucketOrds; + } + } + }; + } + + @Override + public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { + assert owningBucketOrdinal == 0; + + long[] bucketOrdArray = new long[(int) bucketOrds.size()]; + for (int i = 0; i < bucketOrds.size(); i++) { + bucketOrdArray[i] = bucketOrds.get(i); + } + + runDeferredCollections(bucketOrdArray); + + List buckets = new ArrayList<>((int) bucketOrds.size()); + for (long i = 0; i < bucketOrds.size(); i++) { + buckets.add(new InternalAutoDateHistogram.Bucket(bucketOrds.get(i), bucketDocCount(i), formatter, bucketAggregations(i))); + } + + // the contract of the histogram aggregation is that shards must return + // buckets ordered by key in ascending order + CollectionUtil.introSort(buckets, BucketOrder.key(true).comparator(this)); + + // value source will be null for unmapped fields + InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundings, roundingIdx, + buildEmptySubAggregations()); + + return new InternalAutoDateHistogram(name, buckets, targetBuckets, emptyBucketInfo, formatter, pipelineAggregators(), metaData()); + } + + @Override + public InternalAggregation buildEmptyAggregation() { + InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundings, roundingIdx, + buildEmptySubAggregations()); + return new InternalAutoDateHistogram(name, Collections.emptyList(), targetBuckets, emptyBucketInfo, formatter, + pipelineAggregators(), metaData()); + } + + @Override + public void doClose() { + Releasables.close(bucketOrds); + } +} diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java new file mode 100644 index 0000000000000..18ce727204151 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java @@ -0,0 +1,72 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.bucket.histogram; + +import org.elasticsearch.common.rounding.Rounding; +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; +import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +public final class AutoDateHistogramAggregatorFactory + extends ValuesSourceAggregatorFactory { + + private final int numBuckets; + private Rounding[] roundings; + + public AutoDateHistogramAggregatorFactory(String name, ValuesSourceConfig config, int numBuckets, Rounding[] roundings, + SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, + Map metaData) throws IOException { + super(name, config, context, parent, subFactoriesBuilder, metaData); + this.numBuckets = numBuckets; + this.roundings = roundings; + } + + @Override + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, Aggregator parent, boolean collectsFromSingleBucket, + List pipelineAggregators, Map metaData) throws IOException { + if (collectsFromSingleBucket == false) { + return asMultiBucketAggregator(this, context, parent); + } + return createAggregator(valuesSource, parent, pipelineAggregators, metaData); + } + + private Aggregator createAggregator(ValuesSource.Numeric valuesSource, Aggregator parent, List pipelineAggregators, + Map metaData) throws IOException { + return new AutoDateHistogramAggregator(name, factories, numBuckets, roundings, valuesSource, config.format(), context, parent, + pipelineAggregators, + metaData); + } + + @Override + protected Aggregator createUnmapped(Aggregator parent, List pipelineAggregators, Map metaData) + throws IOException { + return createAggregator(null, parent, pipelineAggregators, metaData); + } +} diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java new file mode 100644 index 0000000000000..4b17da4283d16 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java @@ -0,0 +1,527 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.search.aggregations.bucket.histogram; + +import org.apache.lucene.util.PriorityQueue; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.rounding.Rounding; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.Aggregations; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; +import org.elasticsearch.search.aggregations.KeyComparable; +import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.ListIterator; +import java.util.Map; +import java.util.Objects; + +/** + * Implementation of {@link Histogram}. + */ +public final class InternalAutoDateHistogram extends + InternalMultiBucketAggregation implements Histogram, HistogramFactory { + + public static class Bucket extends InternalMultiBucketAggregation.InternalBucket implements Histogram.Bucket, KeyComparable { + + final long key; + final long docCount; + final InternalAggregations aggregations; + protected final transient DocValueFormat format; + + public Bucket(long key, long docCount, DocValueFormat format, + InternalAggregations aggregations) { + this.format = format; + this.key = key; + this.docCount = docCount; + this.aggregations = aggregations; + } + + /** + * Read from a stream. + */ + public Bucket(StreamInput in, DocValueFormat format) throws IOException { + this.format = format; + key = in.readLong(); + docCount = in.readVLong(); + aggregations = InternalAggregations.readAggregations(in); + } + + @Override + public boolean equals(Object obj) { + if (obj == null || obj.getClass() != InternalAutoDateHistogram.Bucket.class) { + return false; + } + InternalAutoDateHistogram.Bucket that = (InternalAutoDateHistogram.Bucket) obj; + // No need to take the keyed and format parameters into account, + // they are already stored and tested on the InternalDateHistogram object + return key == that.key + && docCount == that.docCount + && Objects.equals(aggregations, that.aggregations); + } + + @Override + public int hashCode() { + return Objects.hash(getClass(), key, docCount, aggregations); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeLong(key); + out.writeVLong(docCount); + aggregations.writeTo(out); + } + + @Override + public String getKeyAsString() { + return format.format(key); + } + + @Override + public Object getKey() { + return new DateTime(key, DateTimeZone.UTC); + } + + @Override + public long getDocCount() { + return docCount; + } + + @Override + public Aggregations getAggregations() { + return aggregations; + } + + Bucket reduce(List buckets, Rounding rounding, ReduceContext context) { + List aggregations = new ArrayList<>(buckets.size()); + long docCount = 0; + for (Bucket bucket : buckets) { + docCount += bucket.docCount; + aggregations.add((InternalAggregations) bucket.getAggregations()); + } + InternalAggregations aggs = InternalAggregations.reduce(aggregations, context); + return new InternalAutoDateHistogram.Bucket(rounding.round(key), docCount, format, aggs); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + String keyAsString = format.format(key); + builder.startObject(); + if (format != DocValueFormat.RAW) { + builder.field(CommonFields.KEY_AS_STRING.getPreferredName(), keyAsString); + } + builder.field(CommonFields.KEY.getPreferredName(), key); + builder.field(CommonFields.DOC_COUNT.getPreferredName(), docCount); + aggregations.toXContentInternal(builder, params); + builder.endObject(); + return builder; + } + + @Override + public int compareKey(Bucket other) { + return Long.compare(key, other.key); + } + + public DocValueFormat getFormatter() { + return format; + } + } + + static class BucketInfo { + + final Rounding[] roundings; + final int roundingIdx; + final InternalAggregations emptySubAggregations; + + BucketInfo(Rounding[] roundings, int roundingIdx, InternalAggregations subAggregations) { + this.roundings = roundings; + this.roundingIdx = roundingIdx; + this.emptySubAggregations = subAggregations; + } + + BucketInfo(StreamInput in) throws IOException { + int size = in.readVInt(); + roundings = new Rounding[size]; + for (int i = 0; i < size; i++) { + roundings[i] = Rounding.Streams.read(in); + } + roundingIdx = in.readVInt(); + emptySubAggregations = InternalAggregations.readAggregations(in); + } + + void writeTo(StreamOutput out) throws IOException { + out.writeVInt(roundings.length); + for (Rounding rounding : roundings) { + Rounding.Streams.write(rounding, out); + } + out.writeVInt(roundingIdx); + emptySubAggregations.writeTo(out); + } + + @Override + public boolean equals(Object obj) { + if (obj == null || getClass() != obj.getClass()) { + return false; + } + BucketInfo that = (BucketInfo) obj; + return Objects.deepEquals(roundings, that.roundings) + && Objects.equals(roundingIdx, that.roundingIdx) + && Objects.equals(emptySubAggregations, that.emptySubAggregations); + } + + @Override + public int hashCode() { + return Objects.hash(getClass(), Arrays.hashCode(roundings), roundingIdx, emptySubAggregations); + } + } + + private final List buckets; + private final DocValueFormat format; + private final BucketInfo bucketInfo; + private final int targetBuckets; + + + InternalAutoDateHistogram(String name, List buckets, int targetBuckets, BucketInfo emptyBucketInfo, DocValueFormat formatter, + List pipelineAggregators, Map metaData) { + super(name, pipelineAggregators, metaData); + this.buckets = buckets; + this.bucketInfo = emptyBucketInfo; + this.format = formatter; + this.targetBuckets = targetBuckets; + } + + /** + * Stream from a stream. + */ + public InternalAutoDateHistogram(StreamInput in) throws IOException { + super(in); + bucketInfo = new BucketInfo(in); + format = in.readNamedWriteable(DocValueFormat.class); + buckets = in.readList(stream -> new Bucket(stream, format)); + this.targetBuckets = in.readVInt(); + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + bucketInfo.writeTo(out); + out.writeNamedWriteable(format); + out.writeList(buckets); + out.writeVInt(targetBuckets); + } + + @Override + public String getWriteableName() { + return AutoDateHistogramAggregationBuilder.NAME; + } + + @Override + public List getBuckets() { + return Collections.unmodifiableList(buckets); + } + + DocValueFormat getFormatter() { + return format; + } + + public int getTargetBuckets() { + return targetBuckets; + } + + public BucketInfo getBucketInfo() { + return bucketInfo; + } + + @Override + public InternalAutoDateHistogram create(List buckets) { + return new InternalAutoDateHistogram(name, buckets, targetBuckets, bucketInfo, format, pipelineAggregators(), metaData); + } + + @Override + public Bucket createBucket(InternalAggregations aggregations, Bucket prototype) { + return new Bucket(prototype.key, prototype.docCount, prototype.format, aggregations); + } + + private static class IteratorAndCurrent { + + private final Iterator iterator; + private Bucket current; + + IteratorAndCurrent(Iterator iterator) { + this.iterator = iterator; + current = iterator.next(); + } + + } + + /** + * This method works almost exactly the same as + * InternalDateHistogram#reduceBuckets(List, ReduceContext), the different + * here is that we need to round all the keys we see using the highest level + * rounding returned across all the shards so the resolution of the buckets + * is the same and they can be reduced together. + */ + private BucketReduceResult reduceBuckets(List aggregations, ReduceContext reduceContext) { + + // First we need to find the highest level rounding used across all the + // shards + int reduceRoundingIdx = 0; + for (InternalAggregation aggregation : aggregations) { + int aggRoundingIdx = ((InternalAutoDateHistogram) aggregation).bucketInfo.roundingIdx; + if (aggRoundingIdx > reduceRoundingIdx) { + reduceRoundingIdx = aggRoundingIdx; + } + } + // This rounding will be used to reduce all the buckets + Rounding reduceRounding = bucketInfo.roundings[reduceRoundingIdx]; + + final PriorityQueue pq = new PriorityQueue(aggregations.size()) { + @Override + protected boolean lessThan(IteratorAndCurrent a, IteratorAndCurrent b) { + return a.current.key < b.current.key; + } + }; + for (InternalAggregation aggregation : aggregations) { + InternalAutoDateHistogram histogram = (InternalAutoDateHistogram) aggregation; + if (histogram.buckets.isEmpty() == false) { + pq.add(new IteratorAndCurrent(histogram.buckets.iterator())); + } + } + + List reducedBuckets = new ArrayList<>(); + if (pq.size() > 0) { + // list of buckets coming from different shards that have the same key + List currentBuckets = new ArrayList<>(); + double key = reduceRounding.round(pq.top().current.key); + + do { + final IteratorAndCurrent top = pq.top(); + + if (reduceRounding.round(top.current.key) != key) { + // the key changes, reduce what we already buffered and reset the buffer for current buckets + final Bucket reduced = currentBuckets.get(0).reduce(currentBuckets, reduceRounding, reduceContext); + reducedBuckets.add(reduced); + currentBuckets.clear(); + key = reduceRounding.round(top.current.key); + } + + currentBuckets.add(top.current); + + if (top.iterator.hasNext()) { + final Bucket next = top.iterator.next(); + assert next.key > top.current.key : "shards must return data sorted by key"; + top.current = next; + pq.updateTop(); + } else { + pq.pop(); + } + } while (pq.size() > 0); + + if (currentBuckets.isEmpty() == false) { + final Bucket reduced = currentBuckets.get(0).reduce(currentBuckets, reduceRounding, reduceContext); + reducedBuckets.add(reduced); + } + } + + return mergeBucketsIfNeeded(reducedBuckets, reduceRoundingIdx, reduceRounding, reduceContext); + } + + private BucketReduceResult mergeBucketsIfNeeded(List reducedBuckets, int reduceRoundingIdx, Rounding reduceRounding, + ReduceContext reduceContext) { + while (reducedBuckets.size() > targetBuckets && reduceRoundingIdx < bucketInfo.roundings.length - 1) { + reduceRoundingIdx++; + reduceRounding = bucketInfo.roundings[reduceRoundingIdx]; + reducedBuckets = mergeBuckets(reducedBuckets, reduceRounding, reduceContext); + } + return new BucketReduceResult(reducedBuckets, reduceRounding, reduceRoundingIdx); + } + + private List mergeBuckets(List reducedBuckets, Rounding reduceRounding, ReduceContext reduceContext) { + List mergedBuckets = new ArrayList<>(); + + List sameKeyedBuckets = new ArrayList<>(); + double key = Double.NaN; + for (Bucket bucket : reducedBuckets) { + long roundedBucketKey = reduceRounding.round(bucket.key); + if (Double.isNaN(key)) { + key = roundedBucketKey; + sameKeyedBuckets.add(createBucket(key, bucket.docCount, bucket.aggregations)); + } else if (roundedBucketKey == key) { + sameKeyedBuckets.add(createBucket(key, bucket.docCount, bucket.aggregations)); + } else { + mergedBuckets.add(sameKeyedBuckets.get(0).reduce(sameKeyedBuckets, reduceRounding, reduceContext)); + sameKeyedBuckets.clear(); + key = roundedBucketKey; + sameKeyedBuckets.add(createBucket(key, bucket.docCount, bucket.aggregations)); + } + } + if (sameKeyedBuckets.isEmpty() == false) { + mergedBuckets.add(sameKeyedBuckets.get(0).reduce(sameKeyedBuckets, reduceRounding, reduceContext)); + } + reducedBuckets = mergedBuckets; + return reducedBuckets; + } + + private static class BucketReduceResult { + List buckets; + Rounding rounding; + int roundingIdx; + + BucketReduceResult(List buckets, Rounding rounding, int roundingIdx) { + this.buckets = buckets; + this.rounding = rounding; + this.roundingIdx = roundingIdx; + + } + } + + private BucketReduceResult addEmptyBuckets(BucketReduceResult currentResult, ReduceContext reduceContext) { + List list = currentResult.buckets; + if (list.isEmpty()) { + return currentResult; + } + int roundingIdx = getAppropriateRounding(list.get(0).key, list.get(list.size() - 1).key, currentResult.roundingIdx, + bucketInfo.roundings); + Rounding rounding = bucketInfo.roundings[roundingIdx]; + // merge buckets using the new rounding + list = mergeBuckets(list, rounding, reduceContext); + + Bucket lastBucket = null; + ListIterator iter = list.listIterator(); + InternalAggregations reducedEmptySubAggs = InternalAggregations.reduce(Collections.singletonList(bucketInfo.emptySubAggregations), + reduceContext); + + // Add the empty buckets within the data, + // e.g. if the data series is [1,2,3,7] there're 3 empty buckets that will be created for 4,5,6 + while (iter.hasNext()) { + Bucket nextBucket = list.get(iter.nextIndex()); + if (lastBucket != null) { + long key = rounding.nextRoundingValue(lastBucket.key); + while (key < nextBucket.key) { + iter.add(new InternalAutoDateHistogram.Bucket(key, 0, format, reducedEmptySubAggs)); + key = rounding.nextRoundingValue(key); + } + assert key == nextBucket.key : "key: " + key + ", nextBucket.key: " + nextBucket.key; + } + lastBucket = iter.next(); + } + return new BucketReduceResult(list, rounding, roundingIdx); + } + + private int getAppropriateRounding(long minKey, long maxKey, int roundingIdx, Rounding[] roundings) { + if (roundingIdx == roundings.length - 1) { + return roundingIdx; + } + int currentRoundingIdx = roundingIdx; + int requiredBuckets = 0; + do { + Rounding currentRounding = roundings[currentRoundingIdx]; + long currentKey = minKey; + requiredBuckets = 0; + while (currentKey < maxKey) { + requiredBuckets++; + currentKey = currentRounding.nextRoundingValue(currentKey); + } + currentRoundingIdx++; + } while (requiredBuckets > targetBuckets && currentRoundingIdx < roundings.length); + // The loop will increase past the correct rounding index here so we + // need to subtract one to get the rounding index we need + return currentRoundingIdx - 1; + } + + @Override + public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { + BucketReduceResult reducedBucketsResult = reduceBuckets(aggregations, reduceContext); + + // adding empty buckets if needed + reducedBucketsResult = addEmptyBuckets(reducedBucketsResult, reduceContext); + + // Adding empty buckets may have tipped us over the target so merge the buckets again if needed + reducedBucketsResult = mergeBucketsIfNeeded(reducedBucketsResult.buckets, reducedBucketsResult.roundingIdx, + reducedBucketsResult.rounding, reduceContext); + + BucketInfo bucketInfo = new BucketInfo(this.bucketInfo.roundings, reducedBucketsResult.roundingIdx, + this.bucketInfo.emptySubAggregations); + + return new InternalAutoDateHistogram(getName(), reducedBucketsResult.buckets, targetBuckets, bucketInfo, format, + pipelineAggregators(), getMetaData()); + } + + @Override + public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + builder.startArray(CommonFields.BUCKETS.getPreferredName()); + for (Bucket bucket : buckets) { + bucket.toXContent(builder, params); + } + builder.endArray(); + return builder; + } + + // HistogramFactory method impls + + @Override + public Number getKey(MultiBucketsAggregation.Bucket bucket) { + return ((Bucket) bucket).key; + } + + @Override + public Number nextKey(Number key) { + return bucketInfo.roundings[bucketInfo.roundingIdx].nextRoundingValue(key.longValue()); + } + + @Override + public InternalAggregation createAggregation(List buckets) { + // convert buckets to the right type + List buckets2 = new ArrayList<>(buckets.size()); + for (Object b : buckets) { + buckets2.add((Bucket) b); + } + buckets2 = Collections.unmodifiableList(buckets2); + return new InternalAutoDateHistogram(name, buckets2, targetBuckets, bucketInfo, format, pipelineAggregators(), getMetaData()); + } + + @Override + public Bucket createBucket(Number key, long docCount, InternalAggregations aggregations) { + return new Bucket(key.longValue(), docCount, format, aggregations); + } + + @Override + protected boolean doEquals(Object obj) { + InternalAutoDateHistogram that = (InternalAutoDateHistogram) obj; + return Objects.equals(buckets, that.buckets) + && Objects.equals(format, that.format) + && Objects.equals(bucketInfo, that.bucketInfo); + } + + @Override + protected int doHashCode() { + return Objects.hash(buckets, format, bucketInfo); + } +} diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/ParsedAutoDateHistogram.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/ParsedAutoDateHistogram.java new file mode 100644 index 0000000000000..caca44f9f2ea7 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/ParsedAutoDateHistogram.java @@ -0,0 +1,91 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.bucket.histogram; + +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.search.aggregations.ParsedMultiBucketAggregation; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; + +import java.io.IOException; +import java.util.List; + +public class ParsedAutoDateHistogram extends ParsedMultiBucketAggregation implements Histogram { + + @Override + public String getType() { + return AutoDateHistogramAggregationBuilder.NAME; + } + + @Override + public List getBuckets() { + return buckets; + } + + private static ObjectParser PARSER = + new ObjectParser<>(ParsedAutoDateHistogram.class.getSimpleName(), true, ParsedAutoDateHistogram::new); + static { + declareMultiBucketAggregationFields(PARSER, + parser -> ParsedBucket.fromXContent(parser, false), + parser -> ParsedBucket.fromXContent(parser, true)); + } + + public static ParsedAutoDateHistogram fromXContent(XContentParser parser, String name) throws IOException { + ParsedAutoDateHistogram aggregation = PARSER.parse(parser, null); + aggregation.setName(name); + return aggregation; + } + + public static class ParsedBucket extends ParsedMultiBucketAggregation.ParsedBucket implements Histogram.Bucket { + + private Long key; + + @Override + public Object getKey() { + if (key != null) { + return new DateTime(key, DateTimeZone.UTC); + } + return null; + } + + @Override + public String getKeyAsString() { + String keyAsString = super.getKeyAsString(); + if (keyAsString != null) { + return keyAsString; + } + if (key != null) { + return Long.toString(key); + } + return null; + } + + @Override + protected XContentBuilder keyToXContent(XContentBuilder builder) throws IOException { + return builder.field(CommonFields.KEY.getPreferredName(), key); + } + + static ParsedBucket fromXContent(XContentParser parser, boolean keyed) throws IOException { + return parseXContent(parser, keyed, ParsedBucket::new, (p, bucket) -> bucket.key = p.longValue()); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/AutoDateHistogramTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/AutoDateHistogramTests.java new file mode 100644 index 0000000000000..3a10edf183376 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/AutoDateHistogramTests.java @@ -0,0 +1,44 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.bucket; + +import org.elasticsearch.search.aggregations.BaseAggregationTestCase; +import org.elasticsearch.search.aggregations.bucket.histogram.AutoDateHistogramAggregationBuilder; + +public class AutoDateHistogramTests extends BaseAggregationTestCase { + + @Override + protected AutoDateHistogramAggregationBuilder createTestAggregatorBuilder() { + AutoDateHistogramAggregationBuilder builder = new AutoDateHistogramAggregationBuilder(randomAlphaOfLengthBetween(1, 10)); + builder.field(INT_FIELD_NAME); + builder.setNumBuckets(randomIntBetween(1, 100000)); + if (randomBoolean()) { + builder.format("###.##"); + } + if (randomBoolean()) { + builder.missing(randomIntBetween(0, 10)); + } + if (randomBoolean()) { + builder.timeZone(randomDateTimeZone()); + } + return builder; + } + +} diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java new file mode 100644 index 0000000000000..16cf4e392bc01 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java @@ -0,0 +1,536 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.bucket.histogram; + +import org.apache.lucene.document.Document; +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.MatchNoDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.store.Directory; +import org.elasticsearch.index.mapper.DateFieldMapper; +import org.elasticsearch.search.aggregations.AggregatorTestCase; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.function.Consumer; + +public class AutoDateHistogramAggregatorTests extends AggregatorTestCase { + + private static final String DATE_FIELD = "date"; + private static final String INSTANT_FIELD = "instant"; + + private static final List dataset = Arrays.asList( + "2010-03-12T01:07:45", + "2010-04-27T03:43:34", + "2012-05-18T04:11:00", + "2013-05-29T05:11:31", + "2013-10-31T08:24:05", + "2015-02-13T13:09:32", + "2015-06-24T13:47:43", + "2015-11-13T16:14:34", + "2016-03-04T17:09:50", + "2017-12-12T22:55:46"); + + public void testMatchNoDocs() throws IOException { + testBothCases(new MatchNoDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(10).field(DATE_FIELD), + histogram -> assertEquals(0, histogram.getBuckets().size()) + ); + } + + public void testMatchAllDocs() throws IOException { + Query query = new MatchAllDocsQuery(); + + testSearchCase(query, dataset, + aggregation -> aggregation.setNumBuckets(6).field(DATE_FIELD), + histogram -> assertEquals(6, histogram.getBuckets().size()) + ); + testSearchAndReduceCase(query, dataset, + aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD), + histogram -> assertEquals(8, histogram.getBuckets().size()) + ); + } + + public void testNoDocs() throws IOException { + Query query = new MatchNoDocsQuery(); + List dates = Collections.emptyList(); + Consumer aggregation = agg -> agg.setNumBuckets(10).field(DATE_FIELD); + + testSearchCase(query, dates, aggregation, + histogram -> assertEquals(0, histogram.getBuckets().size()) + ); + testSearchAndReduceCase(query, dates, aggregation, + histogram -> assertNull(histogram) + ); + } + + public void testAggregateWrongField() throws IOException { + testBothCases(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(10).field("wrong_field"), + histogram -> assertEquals(0, histogram.getBuckets().size()) + ); + } + + public void testIntervalYear() throws IOException { + testBothCases(LongPoint.newRangeQuery(INSTANT_FIELD, asLong("2015-01-01"), asLong("2017-12-31")), dataset, + aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(3, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2015-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2016-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + } + ); + } + + public void testIntervalMonth() throws IOException { + testBothCases(new MatchAllDocsQuery(), + Arrays.asList("2017-01-01", "2017-02-02", "2017-02-03", "2017-03-04", "2017-03-05", "2017-03-06"), + aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(3, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-03-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + } + ); + } + + public void testIntervalDay() throws IOException { + testSearchCase(new MatchAllDocsQuery(), + Arrays.asList("2017-02-01", "2017-02-02", "2017-02-02", "2017-02-03", "2017-02-03", "2017-02-03", "2017-02-05"), + aggregation -> aggregation.setNumBuckets(5).field(DATE_FIELD), histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(4, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-02-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-02T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-03T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + + bucket = buckets.get(3); + assertEquals("2017-02-05T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), + Arrays.asList( + "2017-02-01", + "2017-02-02", + "2017-02-02", + "2017-02-03", + "2017-02-03", + "2017-02-03", + "2017-02-05" + ), + aggregation -> aggregation.setNumBuckets(5).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(5, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-02-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-02T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-03T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + + bucket = buckets.get(3); + assertEquals("2017-02-04T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(4); + assertEquals("2017-02-05T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + } + ); + } + + public void testIntervalHour() throws IOException { + testSearchCase(new MatchAllDocsQuery(), + Arrays.asList( + "2017-02-01T09:02:00.000Z", + "2017-02-01T09:35:00.000Z", + "2017-02-01T10:15:00.000Z", + "2017-02-01T13:06:00.000Z", + "2017-02-01T14:04:00.000Z", + "2017-02-01T14:05:00.000Z", + "2017-02-01T15:59:00.000Z", + "2017-02-01T16:06:00.000Z", + "2017-02-01T16:48:00.000Z", + "2017-02-01T16:59:00.000Z" + ), + aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(6, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-02-01T09:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-01T10:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-01T13:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(3); + assertEquals("2017-02-01T14:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(4); + assertEquals("2017-02-01T15:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(5); + assertEquals("2017-02-01T16:00:00.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + } + ); + testSearchAndReduceCase(new MatchAllDocsQuery(), + Arrays.asList( + "2017-02-01T09:02:00.000Z", + "2017-02-01T09:35:00.000Z", + "2017-02-01T10:15:00.000Z", + "2017-02-01T13:06:00.000Z", + "2017-02-01T14:04:00.000Z", + "2017-02-01T14:05:00.000Z", + "2017-02-01T15:59:00.000Z", + "2017-02-01T16:06:00.000Z", + "2017-02-01T16:48:00.000Z", + "2017-02-01T16:59:00.000Z" + ), + aggregation -> aggregation.setNumBuckets(10).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(8, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-02-01T09:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-01T10:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-01T11:00:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(3); + assertEquals("2017-02-01T12:00:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(4); + assertEquals("2017-02-01T13:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(5); + assertEquals("2017-02-01T14:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(6); + assertEquals("2017-02-01T15:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(7); + assertEquals("2017-02-01T16:00:00.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + } + ); + } + + public void testIntervalMinute() throws IOException { + testSearchCase(new MatchAllDocsQuery(), + Arrays.asList( + "2017-02-01T09:02:35.000Z", + "2017-02-01T09:02:59.000Z", + "2017-02-01T09:15:37.000Z", + "2017-02-01T09:16:04.000Z", + "2017-02-01T09:16:42.000Z" + ), + aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(3, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-02-01T09:02:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-01T09:15:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-01T09:16:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + } + ); + testSearchAndReduceCase(new MatchAllDocsQuery(), + Arrays.asList( + "2017-02-01T09:02:35.000Z", + "2017-02-01T09:02:59.000Z", + "2017-02-01T09:15:37.000Z", + "2017-02-01T09:16:04.000Z", + "2017-02-01T09:16:42.000Z" + ), + aggregation -> aggregation.setNumBuckets(15).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(15, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-02-01T09:02:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-01T09:03:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-01T09:04:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(3); + assertEquals("2017-02-01T09:05:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(4); + assertEquals("2017-02-01T09:06:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(5); + assertEquals("2017-02-01T09:07:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(6); + assertEquals("2017-02-01T09:08:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(7); + assertEquals("2017-02-01T09:09:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(8); + assertEquals("2017-02-01T09:10:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(9); + assertEquals("2017-02-01T09:11:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(10); + assertEquals("2017-02-01T09:12:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(11); + assertEquals("2017-02-01T09:13:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(12); + assertEquals("2017-02-01T09:14:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(13); + assertEquals("2017-02-01T09:15:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(14); + assertEquals("2017-02-01T09:16:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + } + ); + } + + public void testIntervalSecond() throws IOException { + testSearchCase(new MatchAllDocsQuery(), + Arrays.asList("2017-02-01T00:00:05.015Z", "2017-02-01T00:00:07.299Z", "2017-02-01T00:00:07.074Z", + "2017-02-01T00:00:11.688Z", "2017-02-01T00:00:11.210Z", "2017-02-01T00:00:11.380Z"), + aggregation -> aggregation.setNumBuckets(7).field(DATE_FIELD), histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(3, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-02-01T00:00:05.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-01T00:00:07.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-01T00:00:11.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), + Arrays.asList( + "2017-02-01T00:00:05.015Z", + "2017-02-01T00:00:07.299Z", + "2017-02-01T00:00:07.074Z", + "2017-02-01T00:00:11.688Z", + "2017-02-01T00:00:11.210Z", + "2017-02-01T00:00:11.380Z" + ), + aggregation -> aggregation.setNumBuckets(7).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(7, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-02-01T00:00:05.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-01T00:00:06.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-01T00:00:07.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(3); + assertEquals("2017-02-01T00:00:08.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(4); + assertEquals("2017-02-01T00:00:09.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(5); + assertEquals("2017-02-01T00:00:10.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(6); + assertEquals("2017-02-01T00:00:11.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + } + ); + } + + private void testSearchCase(Query query, List dataset, + Consumer configure, + Consumer verify) throws IOException { + executeTestCase(false, query, dataset, configure, verify); + } + + private void testSearchAndReduceCase(Query query, List dataset, + Consumer configure, + Consumer verify) throws IOException { + executeTestCase(true, query, dataset, configure, verify); + } + + private void testBothCases(Query query, List dataset, + Consumer configure, + Consumer verify) throws IOException { + testSearchCase(query, dataset, configure, verify); + testSearchAndReduceCase(query, dataset, configure, verify); + } + + private void executeTestCase(boolean reduced, Query query, List dataset, + Consumer configure, + Consumer verify) throws IOException { + + try (Directory directory = newDirectory()) { + try (RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory)) { + Document document = new Document(); + for (String date : dataset) { + if (frequently()) { + indexWriter.commit(); + } + + long instant = asLong(date); + document.add(new SortedNumericDocValuesField(DATE_FIELD, instant)); + document.add(new LongPoint(INSTANT_FIELD, instant)); + indexWriter.addDocument(document); + document.clear(); + } + } + + try (IndexReader indexReader = DirectoryReader.open(directory)) { + IndexSearcher indexSearcher = newSearcher(indexReader, true, true); + + AutoDateHistogramAggregationBuilder aggregationBuilder = new AutoDateHistogramAggregationBuilder("_name"); + if (configure != null) { + configure.accept(aggregationBuilder); + } + + DateFieldMapper.Builder builder = new DateFieldMapper.Builder("_name"); + DateFieldMapper.DateFieldType fieldType = builder.fieldType(); + fieldType.setHasDocValues(true); + fieldType.setName(aggregationBuilder.field()); + + InternalAutoDateHistogram histogram; + if (reduced) { + histogram = searchAndReduce(indexSearcher, query, aggregationBuilder, fieldType); + } else { + histogram = search(indexSearcher, query, aggregationBuilder, fieldType); + } + verify.accept(histogram); + } + } + } + + private static long asLong(String dateTime) { + return DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.parser().parseDateTime(dateTime).getMillis(); + } +} diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java new file mode 100644 index 0000000000000..46783beafbc6a --- /dev/null +++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java @@ -0,0 +1,153 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.bucket.histogram; + +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.rounding.DateTimeUnit; +import org.elasticsearch.common.rounding.Rounding; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.InternalMultiBucketAggregationTestCase; +import org.elasticsearch.search.aggregations.ParsedMultiBucketAggregation; +import org.elasticsearch.search.aggregations.bucket.histogram.InternalAutoDateHistogram.BucketInfo; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.joda.time.DateTime; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import static org.elasticsearch.common.unit.TimeValue.timeValueHours; +import static org.elasticsearch.common.unit.TimeValue.timeValueMinutes; +import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds; + +public class InternalAutoDateHistogramTests extends InternalMultiBucketAggregationTestCase { + + private DocValueFormat format; + private Rounding[] roundings; + + @Override + public void setUp() throws Exception { + super.setUp(); + format = randomNumericDocValueFormat(); + + roundings = new Rounding[6]; + roundings[0] = Rounding.builder(DateTimeUnit.SECOND_OF_MINUTE).build(); + roundings[1] = Rounding.builder(DateTimeUnit.MINUTES_OF_HOUR).build(); + roundings[2] = Rounding.builder(DateTimeUnit.HOUR_OF_DAY).build(); + roundings[3] = Rounding.builder(DateTimeUnit.DAY_OF_MONTH).build(); + roundings[4] = Rounding.builder(DateTimeUnit.MONTH_OF_YEAR).build(); + roundings[5] = Rounding.builder(DateTimeUnit.YEAR_OF_CENTURY).build(); + } + + @Override + protected InternalAutoDateHistogram createTestInstance(String name, + List pipelineAggregators, + Map metaData, + InternalAggregations aggregations) { + int nbBuckets = randomNumberOfBuckets(); + int targetBuckets = randomIntBetween(1, nbBuckets * 2 + 1); + List buckets = new ArrayList<>(nbBuckets); + long startingDate = System.currentTimeMillis(); + + long interval = randomIntBetween(1, 3); + long intervalMillis = randomFrom(timeValueSeconds(interval), timeValueMinutes(interval), timeValueHours(interval)).getMillis(); + + for (int i = 0; i < nbBuckets; i++) { + long key = startingDate + (intervalMillis * i); + buckets.add(i, new InternalAutoDateHistogram.Bucket(key, randomIntBetween(1, 100), format, aggregations)); + } + InternalAggregations subAggregations = new InternalAggregations(Collections.emptyList()); + BucketInfo bucketInfo = new BucketInfo(roundings, randomIntBetween(0, roundings.length - 1), subAggregations); + + return new InternalAutoDateHistogram(name, buckets, targetBuckets, bucketInfo, format, pipelineAggregators, metaData); + } + + @Override + protected void assertReduced(InternalAutoDateHistogram reduced, List inputs) { + int roundingIdx = 0; + for (InternalAutoDateHistogram histogram : inputs) { + if (histogram.getBucketInfo().roundingIdx > roundingIdx) { + roundingIdx = histogram.getBucketInfo().roundingIdx; + } + } + Map expectedCounts = new TreeMap<>(); + for (Histogram histogram : inputs) { + for (Histogram.Bucket bucket : histogram.getBuckets()) { + expectedCounts.compute(roundings[roundingIdx].round(((DateTime) bucket.getKey()).getMillis()), + (key, oldValue) -> (oldValue == null ? 0 : oldValue) + bucket.getDocCount()); + } + } + Map actualCounts = new TreeMap<>(); + for (Histogram.Bucket bucket : reduced.getBuckets()) { + actualCounts.compute(((DateTime) bucket.getKey()).getMillis(), + (key, oldValue) -> (oldValue == null ? 0 : oldValue) + bucket.getDocCount()); + } + assertEquals(expectedCounts, actualCounts); + } + + @Override + protected Writeable.Reader instanceReader() { + return InternalAutoDateHistogram::new; + } + + @Override + protected Class implementationClass() { + return ParsedAutoDateHistogram.class; + } + + @Override + protected InternalAutoDateHistogram mutateInstance(InternalAutoDateHistogram instance) { + String name = instance.getName(); + List buckets = instance.getBuckets(); + int targetBuckets = instance.getTargetBuckets(); + BucketInfo bucketInfo = instance.getBucketInfo(); + List pipelineAggregators = instance.pipelineAggregators(); + Map metaData = instance.getMetaData(); + switch (between(0, 3)) { + case 0: + name += randomAlphaOfLength(5); + break; + case 1: + buckets = new ArrayList<>(buckets); + buckets.add(new InternalAutoDateHistogram.Bucket(randomNonNegativeLong(), randomIntBetween(1, 100), format, + InternalAggregations.EMPTY)); + break; + case 2: + int roundingIdx = bucketInfo.roundingIdx == bucketInfo.roundings.length - 1 ? 0 : bucketInfo.roundingIdx + 1; + bucketInfo = new BucketInfo(bucketInfo.roundings, roundingIdx, bucketInfo.emptySubAggregations); + break; + case 3: + if (metaData == null) { + metaData = new HashMap<>(1); + } else { + metaData = new HashMap<>(instance.getMetaData()); + } + metaData.put(randomAlphaOfLength(15), randomInt()); + break; + default: + throw new AssertionError("Illegal randomisation branch"); + } + return new InternalAutoDateHistogram(name, buckets, targetBuckets, bucketInfo, format, pipelineAggregators, metaData); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/SearchModule.java b/server/src/main/java/org/elasticsearch/search/SearchModule.java index 323cca7961737..1ea2b18152482 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchModule.java +++ b/server/src/main/java/org/elasticsearch/search/SearchModule.java @@ -110,8 +110,10 @@ import org.elasticsearch.search.aggregations.bucket.geogrid.InternalGeoHashGrid; import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.global.InternalGlobal; +import org.elasticsearch.search.aggregations.bucket.histogram.AutoDateHistogramAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.histogram.InternalAutoDateHistogram; import org.elasticsearch.search.aggregations.bucket.histogram.InternalDateHistogram; import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram; import org.elasticsearch.search.aggregations.bucket.missing.InternalMissing; @@ -395,6 +397,8 @@ private void registerAggregations(List plugins) { HistogramAggregationBuilder::parse).addResultReader(InternalHistogram::new)); registerAggregation(new AggregationSpec(DateHistogramAggregationBuilder.NAME, DateHistogramAggregationBuilder::new, DateHistogramAggregationBuilder::parse).addResultReader(InternalDateHistogram::new)); + registerAggregation(new AggregationSpec(AutoDateHistogramAggregationBuilder.NAME, AutoDateHistogramAggregationBuilder::new, + AutoDateHistogramAggregationBuilder::parse).addResultReader(InternalAutoDateHistogram::new)); registerAggregation(new AggregationSpec(GeoDistanceAggregationBuilder.NAME, GeoDistanceAggregationBuilder::new, GeoDistanceAggregationBuilder::parse).addResultReader(InternalGeoDistance::new)); registerAggregation(new AggregationSpec(GeoGridAggregationBuilder.NAME, GeoGridAggregationBuilder::new, diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java index 02cf3adf88ad7..f94011544e0d5 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java @@ -84,6 +84,19 @@ public final void collectExistingBucket(LeafBucketCollector subCollector, int do subCollector.collect(doc, bucketOrd); } + public final void mergeBuckets(long[] mergeMap, long newNumBuckets) { + try (IntArray oldDocCounts = docCounts) { + docCounts = bigArrays.newIntArray(newNumBuckets, true); + docCounts.fill(0, newNumBuckets, 0); + for (int i = 0; i < oldDocCounts.size(); i++) { + int docCount = oldDocCounts.get(i); + if (docCount != 0) { + docCounts.increment(mergeMap[i], docCount); + } + } + } + } + public IntArray getDocCounts() { return docCounts; } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java index 8d879b88b3dca..e234ac51716a2 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java @@ -28,13 +28,13 @@ import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.BucketOrder; import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalOrder; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; -import org.elasticsearch.search.aggregations.BucketOrder; -import org.elasticsearch.search.aggregations.InternalOrder; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.internal.SearchContext; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java index dfa12db0cd31c..d16aa6ded71fb 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java @@ -424,7 +424,7 @@ private void addEmptyBuckets(List list, ReduceContext reduceContext) { iter.add(new InternalDateHistogram.Bucket(key, 0, keyed, format, reducedEmptySubAggs)); key = nextKey(key).longValue(); } - assert key == nextBucket.key; + assert key == nextBucket.key : "key: " + key + ", nextBucket.key: " + nextBucket.key; } lastBucket = iter.next(); } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/AggregationsTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/AggregationsTests.java index 29c187f59a88a..19e99e831eff8 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/AggregationsTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/AggregationsTests.java @@ -37,6 +37,7 @@ import org.elasticsearch.search.aggregations.bucket.filter.InternalFiltersTests; import org.elasticsearch.search.aggregations.bucket.geogrid.InternalGeoHashGridTests; import org.elasticsearch.search.aggregations.bucket.global.InternalGlobalTests; +import org.elasticsearch.search.aggregations.bucket.histogram.InternalAutoDateHistogramTests; import org.elasticsearch.search.aggregations.bucket.histogram.InternalDateHistogramTests; import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogramTests; import org.elasticsearch.search.aggregations.bucket.missing.InternalMissingTests; @@ -125,6 +126,7 @@ private static List getAggsTests() { aggsTests.add(new InternalGeoCentroidTests()); aggsTests.add(new InternalHistogramTests()); aggsTests.add(new InternalDateHistogramTests()); + aggsTests.add(new InternalAutoDateHistogramTests()); aggsTests.add(new LongTermsTests()); aggsTests.add(new DoubleTermsTests()); aggsTests.add(new StringTermsTests()); diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalAggregationTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/InternalAggregationTestCase.java index 8f5fe5d5622e7..cddb98a44d8d9 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalAggregationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalAggregationTestCase.java @@ -54,8 +54,10 @@ import org.elasticsearch.search.aggregations.bucket.geogrid.ParsedGeoHashGrid; import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.global.ParsedGlobal; +import org.elasticsearch.search.aggregations.bucket.histogram.AutoDateHistogramAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.histogram.ParsedAutoDateHistogram; import org.elasticsearch.search.aggregations.bucket.histogram.ParsedDateHistogram; import org.elasticsearch.search.aggregations.bucket.histogram.ParsedHistogram; import org.elasticsearch.search.aggregations.bucket.missing.MissingAggregationBuilder; @@ -182,6 +184,7 @@ public abstract class InternalAggregationTestCase map.put(GeoCentroidAggregationBuilder.NAME, (p, c) -> ParsedGeoCentroid.fromXContent(p, (String) c)); map.put(HistogramAggregationBuilder.NAME, (p, c) -> ParsedHistogram.fromXContent(p, (String) c)); map.put(DateHistogramAggregationBuilder.NAME, (p, c) -> ParsedDateHistogram.fromXContent(p, (String) c)); + map.put(AutoDateHistogramAggregationBuilder.NAME, (p, c) -> ParsedAutoDateHistogram.fromXContent(p, (String) c)); map.put(StringTerms.NAME, (p, c) -> ParsedStringTerms.fromXContent(p, (String) c)); map.put(LongTerms.NAME, (p, c) -> ParsedLongTerms.fromXContent(p, (String) c)); map.put(DoubleTerms.NAME, (p, c) -> ParsedDoubleTerms.fromXContent(p, (String) c)); diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalMultiBucketAggregationTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/InternalMultiBucketAggregationTestCase.java index 952b6c027945b..6f0aebe23966b 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalMultiBucketAggregationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalMultiBucketAggregationTestCase.java @@ -149,7 +149,8 @@ private void assertMultiBucketsAggregations(Aggregation expected, Aggregation ac protected void assertMultiBucketsAggregation(MultiBucketsAggregation expected, MultiBucketsAggregation actual, boolean checkOrder) { Class parsedClass = implementationClass(); assertNotNull("Parsed aggregation class must not be null", parsedClass); - assertTrue(parsedClass.isInstance(actual)); + assertTrue("Unexpected parsed class, expected instance of: " + actual + ", but was: " + parsedClass, + parsedClass.isInstance(actual)); assertTrue(expected instanceof InternalAggregation); assertEquals(expected.getName(), actual.getName()); From 29e76f38fc64f16bbfa58f5b0986f27bd65fd415 Mon Sep 17 00:00:00 2001 From: Colin Goodheart-Smithe Date: Mon, 18 Sep 2017 14:13:02 +0100 Subject: [PATCH 02/24] Adds documentation --- docs/reference/aggregations/bucket.asciidoc | 2 + .../autodatehistogram-aggregation.asciidoc | 242 ++++++++++++++++++ 2 files changed, 244 insertions(+) create mode 100644 docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc diff --git a/docs/reference/aggregations/bucket.asciidoc b/docs/reference/aggregations/bucket.asciidoc index 1233e0d9b7398..ddb55e8d34c8e 100644 --- a/docs/reference/aggregations/bucket.asciidoc +++ b/docs/reference/aggregations/bucket.asciidoc @@ -19,6 +19,8 @@ the limit will fail with an exception. include::bucket/adjacency-matrix-aggregation.asciidoc[] +include::bucket/autodatehistogram-aggregation.asciidoc[] + include::bucket/children-aggregation.asciidoc[] include::bucket/composite-aggregation.asciidoc[] diff --git a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc new file mode 100644 index 0000000000000..b4c5be4b33f61 --- /dev/null +++ b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc @@ -0,0 +1,242 @@ +[[search-aggregations-bucket-autodatehistogram-aggregation]] +=== Auto-interval Date Histogram Aggregation + +A multi-bucket aggregation similar to the <> except +instead of providing an interval to use as the width of each bucket, a target number of buckets is provided +indicating the number of buckets needed and the interval of the buckets is automatically chosen to best achieve +that target. + +Requesting a target of 10 buckets. + +[source,js] +-------------------------------------------------- +POST /sales/_search?size=0 +{ + "aggs" : { + "sales_over_time" : { + "auto_date_histogram" : { + "field" : "date", + "target_buckets" : 10 + } + } + } +} +-------------------------------------------------- +// CONSOLE +// TEST[setup:sales] + +==== Keys + +Internally, a date is represented as a 64 bit number representing a timestamp +in milliseconds-since-the-epoch. These timestamps are returned as the bucket +++key++s. The `key_as_string` is the same timestamp converted to a formatted +date string using the format specified with the `format` parameter: + +TIP: If no `format` is specified, then it will use the first date +<> specified in the field mapping. + +[source,js] +-------------------------------------------------- +POST /sales/_search?size=0 +{ + "aggs" : { + "sales_over_time" : { + "date_histogram" : { + "field" : "date", + "target_buckets" : 10, + "format" : "yyyy-MM-dd" <1> + } + } + } +} +-------------------------------------------------- +// CONSOLE +// TEST[setup:sales] + +<1> Supports expressive date <> + +Response: + +[source,js] +-------------------------------------------------- +{ + ... + "aggregations": { + "sales_over_time": { + "buckets": [ + { + "key_as_string": "2015-01-01", + "key": 1420070400000, + "doc_count": 3 + }, + { + "key_as_string": "2015-02-01", + "key": 1422748800000, + "doc_count": 2 + }, + { + "key_as_string": "2015-03-01", + "key": 1425168000000, + "doc_count": 2 + } + ] + } + } +} +-------------------------------------------------- +// TESTRESPONSE[s/\.\.\./"took": $body.took,"timed_out": false,"_shards": $body._shards,"hits": $body.hits,/] + +==== Time Zone + +Date-times are stored in Elasticsearch in UTC. By default, all bucketing and +rounding is also done in UTC. The `time_zone` parameter can be used to indicate +that bucketing should use a different time zone. + +Time zones may either be specified as an ISO 8601 UTC offset (e.g. `+01:00` or +`-08:00`) or as a timezone id, an identifier used in the TZ database like +`America/Los_Angeles`. + +Consider the following example: + +[source,js] +--------------------------------- +PUT my_index/log/1?refresh +{ + "date": "2015-10-01T00:30:00Z" +} + +PUT my_index/log/2?refresh +{ + "date": "2015-10-01T01:30:00Z" +} + +GET my_index/_search?size=0 +{ + "aggs": { + "by_day": { + "date_histogram": { + "field": "date", + "target_buckets" : 10 + } + } + } +} +--------------------------------- +// CONSOLE + +UTC is used if no time zone is specified, which would result in both of these +documents being placed into the same day bucket, which starts at midnight UTC +on 1 October 2015: + +[source,js] +--------------------------------- +{ + ... + "aggregations": { + "by_day": { + "buckets": [ + { + "key_as_string": "2015-10-01T00:00:00.000Z", + "key": 1443657600000, + "doc_count": 2 + } + ] + } + } +} +--------------------------------- +// TESTRESPONSE[s/\.\.\./"took": $body.took,"timed_out": false,"_shards": $body._shards,"hits": $body.hits,/] + +If a `time_zone` of `-01:00` is specified, then midnight starts at one hour before +midnight UTC: + +[source,js] +--------------------------------- +GET my_index/_search?size=0 +{ + "aggs": { + "by_day": { + "date_histogram": { + "field": "date", + "target_buckets" : 10, + "time_zone": "-01:00" + } + } + } +} +--------------------------------- +// CONSOLE +// TEST[continued] + +Now the first document falls into the bucket for 30 September 2015, while the +second document falls into the bucket for 1 October 2015: + +[source,js] +--------------------------------- +{ + ... + "aggregations": { + "by_day": { + "buckets": [ + { + "key_as_string": "2015-09-30T00:00:00.000-01:00", <1> + "key": 1443574800000, + "doc_count": 1 + }, + { + "key_as_string": "2015-10-01T00:00:00.000-01:00", <1> + "key": 1443661200000, + "doc_count": 1 + } + ] + } + } +} +--------------------------------- +// TESTRESPONSE[s/\.\.\./"took": $body.took,"timed_out": false,"_shards": $body._shards,"hits": $body.hits,/] + +<1> The `key_as_string` value represents midnight on each day + in the specified time zone. + +WARNING: When using time zones that follow DST (daylight savings time) changes, +buckets close to the moment when those changes happen can have slightly different +sizes than neighbouring buckets. +For example, consider a DST start in the `CET` time zone: on 27 March 2016 at 2am, +clocks were turned forward 1 hour to 3am local time. If the result of the aggregation +was daily buckets, the bucket covering that day will only hold data for 23 hours +instead of the usual 24 hours for other buckets. The same is true for shorter intervals +like e.g. 12h. Here, we will have only a 11h bucket on the morning of 27 March when the +DST shift happens. + +==== Scripts + +Like with the normal <>, both document level +scripts and value level scripts are supported. This aggregation does not however, support the `min_doc_count`, +`extended_bounds` and `order` parameters. + +==== Missing value + +The `missing` parameter defines how documents that are missing a value should be treated. +By default they will be ignored but it is also possible to treat them as if they +had a value. + +[source,js] +-------------------------------------------------- +POST /sales/_search?size=0 +{ + "aggs" : { + "sale_date" : { + "date_histogram" : { + "field" : "date", + "target_buckets": 10, + "missing": "2000/01/01" <1> + } + } + } +} +-------------------------------------------------- +// CONSOLE +// TEST[setup:sales] + +<1> Documents without a value in the `publish_date` field will fall into the same bucket as documents that have the value `2000-01-01`. + From 1480aac176743ce194bcab9bd2a4f585b6d8054b Mon Sep 17 00:00:00 2001 From: Colin Goodheart-Smithe Date: Wed, 20 Sep 2017 10:07:17 +0100 Subject: [PATCH 03/24] Added sub aggregator test --- .../MergingBucketsDeferringCollector.java | 15 +- .../AutoDateHistogramAggregator.java | 2 +- .../AutoDateHistogramAggregatorTests.java | 133 ++++++++++++++++++ .../autodatehistogram-aggregation.asciidoc | 18 +-- 4 files changed, 157 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java index 72653440c800c..1d4c9f08f6d11 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java @@ -102,6 +102,7 @@ public void collect(int doc, long bucket) throws IOException { } public void mergeBuckets(long[] mergeMap) { + List newEntries = new ArrayList<>(); for (Entry sourceEntry : entries) { PackedLongValues.Builder newBuckets = PackedLongValues.packedBuilder(PackedInts.DEFAULT); @@ -112,6 +113,18 @@ public void mergeBuckets(long[] mergeMap) { newEntries.add(new Entry(sourceEntry.context, sourceEntry.docDeltas, newBuckets.build())); } entries = newEntries; + + // if there are buckets that have been collected in the current segment + // we need to update the bucket ordinals there too + if (buckets.size() > 0) { + PackedLongValues currentBuckets = buckets.build(); + PackedLongValues.Builder newBuckets = PackedLongValues.packedBuilder(PackedInts.DEFAULT); + for (PackedLongValues.Iterator itr = currentBuckets.iterator(); itr.hasNext();) { + long bucket = itr.next(); + newBuckets.add(mergeMap[(int) bucket]); + } + buckets = newBuckets; + } } @Override @@ -194,7 +207,7 @@ public InternalAggregation buildAggregation(long bucket) throws IOException { } final long rebasedBucket = selectedBuckets.find(bucket); if (rebasedBucket == -1) { - throw new IllegalStateException("Cannot build for a bucket which has not been collected"); + throw new IllegalStateException("Cannot build for a bucket which has not been collected [" + bucket + "]"); } return in.buildAggregation(rebasedBucket); } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java index 293ee2fcbee74..cf2531fca6394 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java @@ -160,7 +160,7 @@ public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOE long[] bucketOrdArray = new long[(int) bucketOrds.size()]; for (int i = 0; i < bucketOrds.size(); i++) { - bucketOrdArray[i] = bucketOrds.get(i); + bucketOrdArray[i] = i; } runDeferredCollections(bucketOrdArray); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java index 16cf4e392bc01..4399c5d8237fb 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java @@ -31,7 +31,9 @@ import org.apache.lucene.search.Query; import org.apache.lucene.store.Directory; import org.elasticsearch.index.mapper.DateFieldMapper; +import org.elasticsearch.search.aggregations.AggregationBuilders; import org.elasticsearch.search.aggregations.AggregatorTestCase; +import org.elasticsearch.search.aggregations.metrics.stats.Stats; import java.io.IOException; import java.util.Arrays; @@ -76,6 +78,137 @@ public void testMatchAllDocs() throws IOException { ); } + public void testSubAggregations() throws IOException { + Query query = new MatchAllDocsQuery(); + + testSearchCase(query, dataset, + aggregation -> aggregation.setNumBuckets(6).field(DATE_FIELD) + .subAggregation(AggregationBuilders.stats("stats").field(DATE_FIELD)), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(6, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2010-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + Stats stats = bucket.getAggregations().get("stats"); + assertEquals("2010-03-12T01:07:45.000Z", stats.getMinAsString()); + assertEquals("2010-04-27T03:43:34.000Z", stats.getMaxAsString()); + assertEquals(2L, stats.getCount()); + + bucket = buckets.get(1); + assertEquals("2012-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertEquals("2012-05-18T04:11:00.000Z", stats.getMinAsString()); + assertEquals("2012-05-18T04:11:00.000Z", stats.getMaxAsString()); + assertEquals(1L, stats.getCount()); + + bucket = buckets.get(2); + assertEquals("2013-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertEquals("2013-05-29T05:11:31.000Z", stats.getMinAsString()); + assertEquals("2013-10-31T08:24:05.000Z", stats.getMaxAsString()); + assertEquals(2L, stats.getCount()); + + bucket = buckets.get(3); + assertEquals("2015-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertEquals("2015-02-13T13:09:32.000Z", stats.getMinAsString()); + assertEquals("2015-11-13T16:14:34.000Z", stats.getMaxAsString()); + assertEquals(3L, stats.getCount()); + + bucket = buckets.get(4); + assertEquals("2016-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertEquals("2016-03-04T17:09:50.000Z", stats.getMinAsString()); + assertEquals("2016-03-04T17:09:50.000Z", stats.getMaxAsString()); + assertEquals(1L, stats.getCount()); + + bucket = buckets.get(5); + assertEquals("2017-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertEquals("2017-12-12T22:55:46.000Z", stats.getMinAsString()); + assertEquals("2017-12-12T22:55:46.000Z", stats.getMaxAsString()); + assertEquals(1L, stats.getCount()); + }); + testSearchAndReduceCase(query, dataset, + aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD) + .subAggregation(AggregationBuilders.stats("stats").field(DATE_FIELD)), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(8, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2010-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + Stats stats = bucket.getAggregations().get("stats"); + assertEquals("2010-03-12T01:07:45.000Z", stats.getMinAsString()); + assertEquals("2010-04-27T03:43:34.000Z", stats.getMaxAsString()); + assertEquals(2L, stats.getCount()); + + bucket = buckets.get(1); + assertEquals("2011-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertTrue(Double.isInfinite(stats.getMin())); + assertTrue(Double.isInfinite(stats.getMax())); + assertEquals(0L, stats.getCount()); + + bucket = buckets.get(2); + assertEquals("2012-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertEquals("2012-05-18T04:11:00.000Z", stats.getMinAsString()); + assertEquals("2012-05-18T04:11:00.000Z", stats.getMaxAsString()); + assertEquals(1L, stats.getCount()); + + bucket = buckets.get(3); + assertEquals("2013-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertEquals("2013-05-29T05:11:31.000Z", stats.getMinAsString()); + assertEquals("2013-10-31T08:24:05.000Z", stats.getMaxAsString()); + assertEquals(2L, stats.getCount()); + + bucket = buckets.get(4); + assertEquals("2014-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertTrue(Double.isInfinite(stats.getMin())); + assertTrue(Double.isInfinite(stats.getMax())); + assertEquals(0L, stats.getCount()); + + bucket = buckets.get(5); + assertEquals("2015-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertEquals("2015-02-13T13:09:32.000Z", stats.getMinAsString()); + assertEquals("2015-11-13T16:14:34.000Z", stats.getMaxAsString()); + assertEquals(3L, stats.getCount()); + + bucket = buckets.get(6); + assertEquals("2016-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertEquals("2016-03-04T17:09:50.000Z", stats.getMinAsString()); + assertEquals("2016-03-04T17:09:50.000Z", stats.getMaxAsString()); + assertEquals(1L, stats.getCount()); + + bucket = buckets.get(7); + assertEquals("2017-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertEquals("2017-12-12T22:55:46.000Z", stats.getMinAsString()); + assertEquals("2017-12-12T22:55:46.000Z", stats.getMaxAsString()); + assertEquals(1L, stats.getCount()); + }); + } + public void testNoDocs() throws IOException { Query query = new MatchNoDocsQuery(); List dates = Collections.emptyList(); diff --git a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc index b4c5be4b33f61..1470a88efdff4 100644 --- a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc @@ -16,7 +16,7 @@ POST /sales/_search?size=0 "sales_over_time" : { "auto_date_histogram" : { "field" : "date", - "target_buckets" : 10 + "buckets" : 10 } } } @@ -41,9 +41,9 @@ POST /sales/_search?size=0 { "aggs" : { "sales_over_time" : { - "date_histogram" : { + "auto_date_histogram" : { "field" : "date", - "target_buckets" : 10, + "buckets" : 10, "format" : "yyyy-MM-dd" <1> } } @@ -114,9 +114,9 @@ GET my_index/_search?size=0 { "aggs": { "by_day": { - "date_histogram": { + "auto_date_histogram": { "field": "date", - "target_buckets" : 10 + "buckets" : 10 } } } @@ -156,9 +156,9 @@ GET my_index/_search?size=0 { "aggs": { "by_day": { - "date_histogram": { + "auto_date_histogram": { "field": "date", - "target_buckets" : 10, + "buckets" : 10, "time_zone": "-01:00" } } @@ -226,9 +226,9 @@ POST /sales/_search?size=0 { "aggs" : { "sale_date" : { - "date_histogram" : { + "auto_date_histogram" : { "field" : "date", - "target_buckets": 10, + "buckets": 10, "missing": "2000/01/01" <1> } } From fbe907d61391e9a7b38d9ae667e030a195522bae Mon Sep 17 00:00:00 2001 From: Colin Goodheart-Smithe Date: Tue, 26 Sep 2017 10:53:48 +0100 Subject: [PATCH 04/24] Fixes failing docs test --- .../autodatehistogram-aggregation.asciidoc | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc index 1470a88efdff4..f655a1616f014 100644 --- a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc @@ -110,13 +110,18 @@ PUT my_index/log/2?refresh "date": "2015-10-01T01:30:00Z" } +PUT my_index/log/3?refresh +{ + "date": "2015-10-01T02:30:00Z" +} + GET my_index/_search?size=0 { "aggs": { "by_day": { "auto_date_histogram": { "field": "date", - "buckets" : 10 + "buckets" : 2 } } } @@ -124,7 +129,7 @@ GET my_index/_search?size=0 --------------------------------- // CONSOLE -UTC is used if no time zone is specified, which would result in both of these +UTC is used if no time zone is specified, which would result in all of these documents being placed into the same day bucket, which starts at midnight UTC on 1 October 2015: @@ -138,7 +143,7 @@ on 1 October 2015: { "key_as_string": "2015-10-01T00:00:00.000Z", "key": 1443657600000, - "doc_count": 2 + "doc_count": 3 } ] } @@ -158,7 +163,7 @@ GET my_index/_search?size=0 "by_day": { "auto_date_histogram": { "field": "date", - "buckets" : 10, + "buckets" : 2, "time_zone": "-01:00" } } @@ -169,7 +174,7 @@ GET my_index/_search?size=0 // TEST[continued] Now the first document falls into the bucket for 30 September 2015, while the -second document falls into the bucket for 1 October 2015: +other two documents fall into the bucket for 1 October 2015: [source,js] --------------------------------- @@ -186,7 +191,7 @@ second document falls into the bucket for 1 October 2015: { "key_as_string": "2015-10-01T00:00:00.000-01:00", <1> "key": 1443661200000, - "doc_count": 1 + "doc_count": 2 } ] } From 2aec5bc93aa29535d726d5bde49294f640224714 Mon Sep 17 00:00:00 2001 From: Colin Goodheart-Smithe Date: Fri, 9 Mar 2018 11:23:38 +0000 Subject: [PATCH 05/24] Brings branch up to date with master changes --- .../MergingBucketsDeferringCollector.java | 230 ++++++ .../AutoDateHistogramAggregationBuilder.java | 150 ++++ .../AutoDateHistogramAggregator.java | 196 +++++ .../AutoDateHistogramAggregatorFactory.java | 72 ++ .../histogram/InternalAutoDateHistogram.java | 527 ++++++++++++++ .../histogram/ParsedAutoDateHistogram.java | 91 +++ .../bucket/AutoDateHistogramTests.java | 44 ++ .../AutoDateHistogramAggregatorTests.java | 669 ++++++++++++++++++ .../InternalAutoDateHistogramTests.java | 153 ++++ 9 files changed, 2132 insertions(+) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/ParsedAutoDateHistogram.java create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/bucket/AutoDateHistogramTests.java create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java new file mode 100644 index 0000000000000..1d4c9f08f6d11 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java @@ -0,0 +1,230 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.bucket; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.Weight; +import org.apache.lucene.util.packed.PackedInts; +import org.apache.lucene.util.packed.PackedLongValues; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.LongHash; +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.BucketCollector; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +public class MergingBucketsDeferringCollector extends DeferringBucketCollector { + + List entries = new ArrayList<>(); + BucketCollector collector; + final SearchContext searchContext; + LeafReaderContext context; + PackedLongValues.Builder docDeltas; + PackedLongValues.Builder buckets; + long maxBucket = -1; + boolean finished = false; + LongHash selectedBuckets; + + public MergingBucketsDeferringCollector(SearchContext context) { + this.searchContext = context; + } + + @Override + public void setDeferredCollector(Iterable deferredCollectors) { + this.collector = BucketCollector.wrap(deferredCollectors); + } + + @Override + public boolean needsScores() { + if (collector == null) { + throw new IllegalStateException(); + } + return collector.needsScores(); + } + + @Override + public void preCollection() throws IOException { + collector.preCollection(); + } + + private void finishLeaf() { + if (context != null) { + entries.add(new Entry(context, docDeltas.build(), buckets.build())); + } + context = null; + docDeltas = null; + buckets = null; + } + + @Override + public LeafBucketCollector getLeafCollector(LeafReaderContext ctx) throws IOException { + finishLeaf(); + + context = ctx; + docDeltas = PackedLongValues.packedBuilder(PackedInts.DEFAULT); + buckets = PackedLongValues.packedBuilder(PackedInts.DEFAULT); + + return new LeafBucketCollector() { + int lastDoc = 0; + + @Override + public void collect(int doc, long bucket) throws IOException { + docDeltas.add(doc - lastDoc); + buckets.add(bucket); + lastDoc = doc; + maxBucket = Math.max(maxBucket, bucket); + } + }; + } + + public void mergeBuckets(long[] mergeMap) { + + List newEntries = new ArrayList<>(); + for (Entry sourceEntry : entries) { + PackedLongValues.Builder newBuckets = PackedLongValues.packedBuilder(PackedInts.DEFAULT); + for (PackedLongValues.Iterator itr = sourceEntry.buckets.iterator(); itr.hasNext();) { + long bucket = itr.next(); + newBuckets.add(mergeMap[(int) bucket]); + } + newEntries.add(new Entry(sourceEntry.context, sourceEntry.docDeltas, newBuckets.build())); + } + entries = newEntries; + + // if there are buckets that have been collected in the current segment + // we need to update the bucket ordinals there too + if (buckets.size() > 0) { + PackedLongValues currentBuckets = buckets.build(); + PackedLongValues.Builder newBuckets = PackedLongValues.packedBuilder(PackedInts.DEFAULT); + for (PackedLongValues.Iterator itr = currentBuckets.iterator(); itr.hasNext();) { + long bucket = itr.next(); + newBuckets.add(mergeMap[(int) bucket]); + } + buckets = newBuckets; + } + } + + @Override + public void postCollection() throws IOException { + finishLeaf(); + finished = true; + } + + /** + * Replay the wrapped collector, but only on a selection of buckets. + */ + @Override + public void prepareSelectedBuckets(long... selectedBuckets) throws IOException { + if (!finished) { + throw new IllegalStateException("Cannot replay yet, collection is not finished: postCollect() has not been called"); + } + if (this.selectedBuckets != null) { + throw new IllegalStateException("Already been replayed"); + } + + final LongHash hash = new LongHash(selectedBuckets.length, BigArrays.NON_RECYCLING_INSTANCE); + for (long bucket : selectedBuckets) { + hash.add(bucket); + } + this.selectedBuckets = hash; + + boolean needsScores = collector.needsScores(); + Weight weight = null; + if (needsScores) { + weight = searchContext.searcher().createNormalizedWeight(searchContext.query(), true); + } + for (Entry entry : entries) { + final LeafBucketCollector leafCollector = collector.getLeafCollector(entry.context); + DocIdSetIterator docIt = null; + if (needsScores && entry.docDeltas.size() > 0) { + Scorer scorer = weight.scorer(entry.context); + // We don't need to check if the scorer is null + // since we are sure that there are documents to replay + // (entry.docDeltas it not empty). + docIt = scorer.iterator(); + leafCollector.setScorer(scorer); + } + final PackedLongValues.Iterator docDeltaIterator = entry.docDeltas.iterator(); + final PackedLongValues.Iterator buckets = entry.buckets.iterator(); + int doc = 0; + for (long i = 0, end = entry.docDeltas.size(); i < end; ++i) { + doc += docDeltaIterator.next(); + final long bucket = buckets.next(); + final long rebasedBucket = hash.find(bucket); + if (rebasedBucket != -1) { + if (needsScores) { + if (docIt.docID() < doc) { + docIt.advance(doc); + } + // aggregations should only be replayed on matching + // documents + assert docIt.docID() == doc; + } + leafCollector.collect(doc, rebasedBucket); + } + } + } + + collector.postCollection(); + } + + /** + * Wrap the provided aggregator so that it behaves (almost) as if it had + * been collected directly. + */ + @Override + public Aggregator wrap(final Aggregator in) { + + return new WrappedAggregator(in) { + + @Override + public InternalAggregation buildAggregation(long bucket) throws IOException { + if (selectedBuckets == null) { + throw new IllegalStateException("Collection has not been replayed yet."); + } + final long rebasedBucket = selectedBuckets.find(bucket); + if (rebasedBucket == -1) { + throw new IllegalStateException("Cannot build for a bucket which has not been collected [" + bucket + "]"); + } + return in.buildAggregation(rebasedBucket); + } + + }; + } + + private static class Entry { + final LeafReaderContext context; + final PackedLongValues docDeltas; + final PackedLongValues buckets; + + Entry(LeafReaderContext context, PackedLongValues docDeltas, PackedLongValues buckets) { + this.context = context; + this.docDeltas = docDeltas; + this.buckets = buckets; + } + } + +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java new file mode 100644 index 0000000000000..600de3cd328ad --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java @@ -0,0 +1,150 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.bucket.histogram; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.rounding.DateTimeUnit; +import org.elasticsearch.common.rounding.Rounding; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.search.aggregations.AggregationBuilder; +import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; +import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.support.ValueType; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; +import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; +import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper; +import org.elasticsearch.search.aggregations.support.ValuesSourceType; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.Map; +import java.util.Objects; + +public class AutoDateHistogramAggregationBuilder + extends ValuesSourceAggregationBuilder { + + public static final String NAME = "auto_date_histogram"; + + public static final ParseField NUM_BUCKETS_FIELD = new ParseField("buckets"); + + private static final ObjectParser PARSER; + static { + PARSER = new ObjectParser<>(AutoDateHistogramAggregationBuilder.NAME); + ValuesSourceParserHelper.declareNumericFields(PARSER, true, true, true); + + PARSER.declareInt(AutoDateHistogramAggregationBuilder::setNumBuckets, NUM_BUCKETS_FIELD); + } + + public static AutoDateHistogramAggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException { + return PARSER.parse(parser, new AutoDateHistogramAggregationBuilder(aggregationName), null); + } + + private int numBuckets = 10; + + /** Create a new builder with the given name. */ + public AutoDateHistogramAggregationBuilder(String name) { + super(name, ValuesSourceType.NUMERIC, ValueType.DATE); + } + + /** Read from a stream, for internal use only. */ + public AutoDateHistogramAggregationBuilder(StreamInput in) throws IOException { + super(in, ValuesSourceType.NUMERIC, ValueType.DATE); + numBuckets = in.readVInt(); + } + + protected AutoDateHistogramAggregationBuilder(AutoDateHistogramAggregationBuilder clone, Builder factoriesBuilder, + Map metaData) { + super(clone, factoriesBuilder, metaData); + this.numBuckets = clone.numBuckets; + } + + @Override + protected AggregationBuilder shallowCopy(Builder factoriesBuilder, Map metaData) { + return new AutoDateHistogramAggregationBuilder(this, factoriesBuilder, metaData); + } + + @Override + protected void innerWriteTo(StreamOutput out) throws IOException { + out.writeVInt(numBuckets); + } + + @Override + public String getType() { + return NAME; + } + + public AutoDateHistogramAggregationBuilder setNumBuckets(int numBuckets) { + if (numBuckets <= 0) { + throw new IllegalArgumentException(NUM_BUCKETS_FIELD.getPreferredName() + " must be greater than 0 for [" + name + "]"); + } + this.numBuckets = numBuckets; + return this; + } + + public int getNumBuckets() { + return numBuckets; + } + + @Override + protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + Rounding[] roundings = new Rounding[6]; + roundings[0] = createRounding(DateTimeUnit.SECOND_OF_MINUTE); + roundings[1] = createRounding(DateTimeUnit.MINUTES_OF_HOUR); + roundings[2] = createRounding(DateTimeUnit.HOUR_OF_DAY); + roundings[3] = createRounding(DateTimeUnit.DAY_OF_MONTH); + roundings[4] = createRounding(DateTimeUnit.MONTH_OF_YEAR); + roundings[5] = createRounding(DateTimeUnit.YEAR_OF_CENTURY); + return new AutoDateHistogramAggregatorFactory(name, config, numBuckets, roundings, context, parent, subFactoriesBuilder, metaData); + } + + private Rounding createRounding(DateTimeUnit interval) { + Rounding.Builder tzRoundingBuilder = Rounding.builder(interval); + if (timeZone() != null) { + tzRoundingBuilder.timeZone(timeZone()); + } + Rounding rounding = tzRoundingBuilder.build(); + return rounding; + } + + @Override + protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + builder.field(NUM_BUCKETS_FIELD.getPreferredName(), numBuckets); + return builder; + } + + @Override + protected int innerHashCode() { + return Objects.hash(numBuckets); + } + + @Override + protected boolean innerEquals(Object obj) { + AutoDateHistogramAggregationBuilder other = (AutoDateHistogramAggregationBuilder) obj; + return Objects.equals(numBuckets, other.numBuckets); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java new file mode 100644 index 0000000000000..cf2531fca6394 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java @@ -0,0 +1,196 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.search.aggregations.bucket.histogram; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.util.CollectionUtil; +import org.elasticsearch.common.inject.internal.Nullable; +import org.elasticsearch.common.lease.Releasables; +import org.elasticsearch.common.rounding.Rounding; +import org.elasticsearch.common.util.LongHash; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.BucketOrder; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; +import org.elasticsearch.search.aggregations.bucket.DeferableBucketAggregator; +import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector; +import org.elasticsearch.search.aggregations.bucket.MergingBucketsDeferringCollector; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * An aggregator for date values. Every date is rounded down using a configured + * {@link Rounding}. + * + * @see Rounding + */ +class AutoDateHistogramAggregator extends DeferableBucketAggregator { + + private final ValuesSource.Numeric valuesSource; + private final DocValueFormat formatter; + private final Rounding[] roundings; + private int roundingIdx = 0; + + private LongHash bucketOrds; + private int targetBuckets; + private MergingBucketsDeferringCollector deferringCollector; + + AutoDateHistogramAggregator(String name, AggregatorFactories factories, int numBuckets, Rounding[] roundings, + @Nullable ValuesSource.Numeric valuesSource, DocValueFormat formatter, SearchContext aggregationContext, Aggregator parent, + List pipelineAggregators, Map metaData) throws IOException { + + super(name, factories, aggregationContext, parent, pipelineAggregators, metaData); + this.targetBuckets = numBuckets; + this.valuesSource = valuesSource; + this.formatter = formatter; + this.roundings = roundings; + + bucketOrds = new LongHash(1, aggregationContext.bigArrays()); + + } + + @Override + public boolean needsScores() { + return (valuesSource != null && valuesSource.needsScores()) || super.needsScores(); + } + + @Override + protected boolean shouldDefer(Aggregator aggregator) { + return true; + } + + @Override + public DeferringBucketCollector getDeferringCollector() { + deferringCollector = new MergingBucketsDeferringCollector(context); + return deferringCollector; + } + + @Override + public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, + final LeafBucketCollector sub) throws IOException { + if (valuesSource == null) { + return LeafBucketCollector.NO_OP_COLLECTOR; + } + final SortedNumericDocValues values = valuesSource.longValues(ctx); + return new LeafBucketCollectorBase(sub, values) { + @Override + public void collect(int doc, long bucket) throws IOException { + assert bucket == 0; + if (values.advanceExact(doc)) { + final int valuesCount = values.docValueCount(); + + long previousRounded = Long.MIN_VALUE; + for (int i = 0; i < valuesCount; ++i) { + long value = values.nextValue(); + long rounded = roundings[roundingIdx].round(value); + assert rounded >= previousRounded; + if (rounded == previousRounded) { + continue; + } + long bucketOrd = bucketOrds.add(rounded); + if (bucketOrd < 0) { // already seen + bucketOrd = -1 - bucketOrd; + collectExistingBucket(sub, doc, bucketOrd); + } else { + collectBucket(sub, doc, bucketOrd); + while (bucketOrds.size() > targetBuckets) { + increaseRounding(); + } + } + previousRounded = rounded; + } + } + } + + private void increaseRounding() { + try (LongHash oldBucketOrds = bucketOrds) { + LongHash newBucketOrds = new LongHash(1, context.bigArrays()); + long[] mergeMap = new long[(int) oldBucketOrds.size()]; + Rounding newRounding = roundings[++roundingIdx]; + for (int i = 0; i < oldBucketOrds.size(); i++) { + long oldKey = oldBucketOrds.get(i); + long newKey = newRounding.round(oldKey); + long newBucketOrd = newBucketOrds.add(newKey); + if (newBucketOrd >= 0) { + mergeMap[i] = newBucketOrd; + } else { + mergeMap[i] = -1 - newBucketOrd; + } + } + mergeBuckets(mergeMap, newBucketOrds.size()); + if (deferringCollector != null) { + deferringCollector.mergeBuckets(mergeMap); + } + bucketOrds = newBucketOrds; + } + } + }; + } + + @Override + public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { + assert owningBucketOrdinal == 0; + + long[] bucketOrdArray = new long[(int) bucketOrds.size()]; + for (int i = 0; i < bucketOrds.size(); i++) { + bucketOrdArray[i] = i; + } + + runDeferredCollections(bucketOrdArray); + + List buckets = new ArrayList<>((int) bucketOrds.size()); + for (long i = 0; i < bucketOrds.size(); i++) { + buckets.add(new InternalAutoDateHistogram.Bucket(bucketOrds.get(i), bucketDocCount(i), formatter, bucketAggregations(i))); + } + + // the contract of the histogram aggregation is that shards must return + // buckets ordered by key in ascending order + CollectionUtil.introSort(buckets, BucketOrder.key(true).comparator(this)); + + // value source will be null for unmapped fields + InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundings, roundingIdx, + buildEmptySubAggregations()); + + return new InternalAutoDateHistogram(name, buckets, targetBuckets, emptyBucketInfo, formatter, pipelineAggregators(), metaData()); + } + + @Override + public InternalAggregation buildEmptyAggregation() { + InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundings, roundingIdx, + buildEmptySubAggregations()); + return new InternalAutoDateHistogram(name, Collections.emptyList(), targetBuckets, emptyBucketInfo, formatter, + pipelineAggregators(), metaData()); + } + + @Override + public void doClose() { + Releasables.close(bucketOrds); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java new file mode 100644 index 0000000000000..18ce727204151 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java @@ -0,0 +1,72 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.bucket.histogram; + +import org.elasticsearch.common.rounding.Rounding; +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; +import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +public final class AutoDateHistogramAggregatorFactory + extends ValuesSourceAggregatorFactory { + + private final int numBuckets; + private Rounding[] roundings; + + public AutoDateHistogramAggregatorFactory(String name, ValuesSourceConfig config, int numBuckets, Rounding[] roundings, + SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, + Map metaData) throws IOException { + super(name, config, context, parent, subFactoriesBuilder, metaData); + this.numBuckets = numBuckets; + this.roundings = roundings; + } + + @Override + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, Aggregator parent, boolean collectsFromSingleBucket, + List pipelineAggregators, Map metaData) throws IOException { + if (collectsFromSingleBucket == false) { + return asMultiBucketAggregator(this, context, parent); + } + return createAggregator(valuesSource, parent, pipelineAggregators, metaData); + } + + private Aggregator createAggregator(ValuesSource.Numeric valuesSource, Aggregator parent, List pipelineAggregators, + Map metaData) throws IOException { + return new AutoDateHistogramAggregator(name, factories, numBuckets, roundings, valuesSource, config.format(), context, parent, + pipelineAggregators, + metaData); + } + + @Override + protected Aggregator createUnmapped(Aggregator parent, List pipelineAggregators, Map metaData) + throws IOException { + return createAggregator(null, parent, pipelineAggregators, metaData); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java new file mode 100644 index 0000000000000..4b17da4283d16 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java @@ -0,0 +1,527 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.search.aggregations.bucket.histogram; + +import org.apache.lucene.util.PriorityQueue; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.rounding.Rounding; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.Aggregations; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; +import org.elasticsearch.search.aggregations.KeyComparable; +import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.ListIterator; +import java.util.Map; +import java.util.Objects; + +/** + * Implementation of {@link Histogram}. + */ +public final class InternalAutoDateHistogram extends + InternalMultiBucketAggregation implements Histogram, HistogramFactory { + + public static class Bucket extends InternalMultiBucketAggregation.InternalBucket implements Histogram.Bucket, KeyComparable { + + final long key; + final long docCount; + final InternalAggregations aggregations; + protected final transient DocValueFormat format; + + public Bucket(long key, long docCount, DocValueFormat format, + InternalAggregations aggregations) { + this.format = format; + this.key = key; + this.docCount = docCount; + this.aggregations = aggregations; + } + + /** + * Read from a stream. + */ + public Bucket(StreamInput in, DocValueFormat format) throws IOException { + this.format = format; + key = in.readLong(); + docCount = in.readVLong(); + aggregations = InternalAggregations.readAggregations(in); + } + + @Override + public boolean equals(Object obj) { + if (obj == null || obj.getClass() != InternalAutoDateHistogram.Bucket.class) { + return false; + } + InternalAutoDateHistogram.Bucket that = (InternalAutoDateHistogram.Bucket) obj; + // No need to take the keyed and format parameters into account, + // they are already stored and tested on the InternalDateHistogram object + return key == that.key + && docCount == that.docCount + && Objects.equals(aggregations, that.aggregations); + } + + @Override + public int hashCode() { + return Objects.hash(getClass(), key, docCount, aggregations); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeLong(key); + out.writeVLong(docCount); + aggregations.writeTo(out); + } + + @Override + public String getKeyAsString() { + return format.format(key); + } + + @Override + public Object getKey() { + return new DateTime(key, DateTimeZone.UTC); + } + + @Override + public long getDocCount() { + return docCount; + } + + @Override + public Aggregations getAggregations() { + return aggregations; + } + + Bucket reduce(List buckets, Rounding rounding, ReduceContext context) { + List aggregations = new ArrayList<>(buckets.size()); + long docCount = 0; + for (Bucket bucket : buckets) { + docCount += bucket.docCount; + aggregations.add((InternalAggregations) bucket.getAggregations()); + } + InternalAggregations aggs = InternalAggregations.reduce(aggregations, context); + return new InternalAutoDateHistogram.Bucket(rounding.round(key), docCount, format, aggs); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + String keyAsString = format.format(key); + builder.startObject(); + if (format != DocValueFormat.RAW) { + builder.field(CommonFields.KEY_AS_STRING.getPreferredName(), keyAsString); + } + builder.field(CommonFields.KEY.getPreferredName(), key); + builder.field(CommonFields.DOC_COUNT.getPreferredName(), docCount); + aggregations.toXContentInternal(builder, params); + builder.endObject(); + return builder; + } + + @Override + public int compareKey(Bucket other) { + return Long.compare(key, other.key); + } + + public DocValueFormat getFormatter() { + return format; + } + } + + static class BucketInfo { + + final Rounding[] roundings; + final int roundingIdx; + final InternalAggregations emptySubAggregations; + + BucketInfo(Rounding[] roundings, int roundingIdx, InternalAggregations subAggregations) { + this.roundings = roundings; + this.roundingIdx = roundingIdx; + this.emptySubAggregations = subAggregations; + } + + BucketInfo(StreamInput in) throws IOException { + int size = in.readVInt(); + roundings = new Rounding[size]; + for (int i = 0; i < size; i++) { + roundings[i] = Rounding.Streams.read(in); + } + roundingIdx = in.readVInt(); + emptySubAggregations = InternalAggregations.readAggregations(in); + } + + void writeTo(StreamOutput out) throws IOException { + out.writeVInt(roundings.length); + for (Rounding rounding : roundings) { + Rounding.Streams.write(rounding, out); + } + out.writeVInt(roundingIdx); + emptySubAggregations.writeTo(out); + } + + @Override + public boolean equals(Object obj) { + if (obj == null || getClass() != obj.getClass()) { + return false; + } + BucketInfo that = (BucketInfo) obj; + return Objects.deepEquals(roundings, that.roundings) + && Objects.equals(roundingIdx, that.roundingIdx) + && Objects.equals(emptySubAggregations, that.emptySubAggregations); + } + + @Override + public int hashCode() { + return Objects.hash(getClass(), Arrays.hashCode(roundings), roundingIdx, emptySubAggregations); + } + } + + private final List buckets; + private final DocValueFormat format; + private final BucketInfo bucketInfo; + private final int targetBuckets; + + + InternalAutoDateHistogram(String name, List buckets, int targetBuckets, BucketInfo emptyBucketInfo, DocValueFormat formatter, + List pipelineAggregators, Map metaData) { + super(name, pipelineAggregators, metaData); + this.buckets = buckets; + this.bucketInfo = emptyBucketInfo; + this.format = formatter; + this.targetBuckets = targetBuckets; + } + + /** + * Stream from a stream. + */ + public InternalAutoDateHistogram(StreamInput in) throws IOException { + super(in); + bucketInfo = new BucketInfo(in); + format = in.readNamedWriteable(DocValueFormat.class); + buckets = in.readList(stream -> new Bucket(stream, format)); + this.targetBuckets = in.readVInt(); + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + bucketInfo.writeTo(out); + out.writeNamedWriteable(format); + out.writeList(buckets); + out.writeVInt(targetBuckets); + } + + @Override + public String getWriteableName() { + return AutoDateHistogramAggregationBuilder.NAME; + } + + @Override + public List getBuckets() { + return Collections.unmodifiableList(buckets); + } + + DocValueFormat getFormatter() { + return format; + } + + public int getTargetBuckets() { + return targetBuckets; + } + + public BucketInfo getBucketInfo() { + return bucketInfo; + } + + @Override + public InternalAutoDateHistogram create(List buckets) { + return new InternalAutoDateHistogram(name, buckets, targetBuckets, bucketInfo, format, pipelineAggregators(), metaData); + } + + @Override + public Bucket createBucket(InternalAggregations aggregations, Bucket prototype) { + return new Bucket(prototype.key, prototype.docCount, prototype.format, aggregations); + } + + private static class IteratorAndCurrent { + + private final Iterator iterator; + private Bucket current; + + IteratorAndCurrent(Iterator iterator) { + this.iterator = iterator; + current = iterator.next(); + } + + } + + /** + * This method works almost exactly the same as + * InternalDateHistogram#reduceBuckets(List, ReduceContext), the different + * here is that we need to round all the keys we see using the highest level + * rounding returned across all the shards so the resolution of the buckets + * is the same and they can be reduced together. + */ + private BucketReduceResult reduceBuckets(List aggregations, ReduceContext reduceContext) { + + // First we need to find the highest level rounding used across all the + // shards + int reduceRoundingIdx = 0; + for (InternalAggregation aggregation : aggregations) { + int aggRoundingIdx = ((InternalAutoDateHistogram) aggregation).bucketInfo.roundingIdx; + if (aggRoundingIdx > reduceRoundingIdx) { + reduceRoundingIdx = aggRoundingIdx; + } + } + // This rounding will be used to reduce all the buckets + Rounding reduceRounding = bucketInfo.roundings[reduceRoundingIdx]; + + final PriorityQueue pq = new PriorityQueue(aggregations.size()) { + @Override + protected boolean lessThan(IteratorAndCurrent a, IteratorAndCurrent b) { + return a.current.key < b.current.key; + } + }; + for (InternalAggregation aggregation : aggregations) { + InternalAutoDateHistogram histogram = (InternalAutoDateHistogram) aggregation; + if (histogram.buckets.isEmpty() == false) { + pq.add(new IteratorAndCurrent(histogram.buckets.iterator())); + } + } + + List reducedBuckets = new ArrayList<>(); + if (pq.size() > 0) { + // list of buckets coming from different shards that have the same key + List currentBuckets = new ArrayList<>(); + double key = reduceRounding.round(pq.top().current.key); + + do { + final IteratorAndCurrent top = pq.top(); + + if (reduceRounding.round(top.current.key) != key) { + // the key changes, reduce what we already buffered and reset the buffer for current buckets + final Bucket reduced = currentBuckets.get(0).reduce(currentBuckets, reduceRounding, reduceContext); + reducedBuckets.add(reduced); + currentBuckets.clear(); + key = reduceRounding.round(top.current.key); + } + + currentBuckets.add(top.current); + + if (top.iterator.hasNext()) { + final Bucket next = top.iterator.next(); + assert next.key > top.current.key : "shards must return data sorted by key"; + top.current = next; + pq.updateTop(); + } else { + pq.pop(); + } + } while (pq.size() > 0); + + if (currentBuckets.isEmpty() == false) { + final Bucket reduced = currentBuckets.get(0).reduce(currentBuckets, reduceRounding, reduceContext); + reducedBuckets.add(reduced); + } + } + + return mergeBucketsIfNeeded(reducedBuckets, reduceRoundingIdx, reduceRounding, reduceContext); + } + + private BucketReduceResult mergeBucketsIfNeeded(List reducedBuckets, int reduceRoundingIdx, Rounding reduceRounding, + ReduceContext reduceContext) { + while (reducedBuckets.size() > targetBuckets && reduceRoundingIdx < bucketInfo.roundings.length - 1) { + reduceRoundingIdx++; + reduceRounding = bucketInfo.roundings[reduceRoundingIdx]; + reducedBuckets = mergeBuckets(reducedBuckets, reduceRounding, reduceContext); + } + return new BucketReduceResult(reducedBuckets, reduceRounding, reduceRoundingIdx); + } + + private List mergeBuckets(List reducedBuckets, Rounding reduceRounding, ReduceContext reduceContext) { + List mergedBuckets = new ArrayList<>(); + + List sameKeyedBuckets = new ArrayList<>(); + double key = Double.NaN; + for (Bucket bucket : reducedBuckets) { + long roundedBucketKey = reduceRounding.round(bucket.key); + if (Double.isNaN(key)) { + key = roundedBucketKey; + sameKeyedBuckets.add(createBucket(key, bucket.docCount, bucket.aggregations)); + } else if (roundedBucketKey == key) { + sameKeyedBuckets.add(createBucket(key, bucket.docCount, bucket.aggregations)); + } else { + mergedBuckets.add(sameKeyedBuckets.get(0).reduce(sameKeyedBuckets, reduceRounding, reduceContext)); + sameKeyedBuckets.clear(); + key = roundedBucketKey; + sameKeyedBuckets.add(createBucket(key, bucket.docCount, bucket.aggregations)); + } + } + if (sameKeyedBuckets.isEmpty() == false) { + mergedBuckets.add(sameKeyedBuckets.get(0).reduce(sameKeyedBuckets, reduceRounding, reduceContext)); + } + reducedBuckets = mergedBuckets; + return reducedBuckets; + } + + private static class BucketReduceResult { + List buckets; + Rounding rounding; + int roundingIdx; + + BucketReduceResult(List buckets, Rounding rounding, int roundingIdx) { + this.buckets = buckets; + this.rounding = rounding; + this.roundingIdx = roundingIdx; + + } + } + + private BucketReduceResult addEmptyBuckets(BucketReduceResult currentResult, ReduceContext reduceContext) { + List list = currentResult.buckets; + if (list.isEmpty()) { + return currentResult; + } + int roundingIdx = getAppropriateRounding(list.get(0).key, list.get(list.size() - 1).key, currentResult.roundingIdx, + bucketInfo.roundings); + Rounding rounding = bucketInfo.roundings[roundingIdx]; + // merge buckets using the new rounding + list = mergeBuckets(list, rounding, reduceContext); + + Bucket lastBucket = null; + ListIterator iter = list.listIterator(); + InternalAggregations reducedEmptySubAggs = InternalAggregations.reduce(Collections.singletonList(bucketInfo.emptySubAggregations), + reduceContext); + + // Add the empty buckets within the data, + // e.g. if the data series is [1,2,3,7] there're 3 empty buckets that will be created for 4,5,6 + while (iter.hasNext()) { + Bucket nextBucket = list.get(iter.nextIndex()); + if (lastBucket != null) { + long key = rounding.nextRoundingValue(lastBucket.key); + while (key < nextBucket.key) { + iter.add(new InternalAutoDateHistogram.Bucket(key, 0, format, reducedEmptySubAggs)); + key = rounding.nextRoundingValue(key); + } + assert key == nextBucket.key : "key: " + key + ", nextBucket.key: " + nextBucket.key; + } + lastBucket = iter.next(); + } + return new BucketReduceResult(list, rounding, roundingIdx); + } + + private int getAppropriateRounding(long minKey, long maxKey, int roundingIdx, Rounding[] roundings) { + if (roundingIdx == roundings.length - 1) { + return roundingIdx; + } + int currentRoundingIdx = roundingIdx; + int requiredBuckets = 0; + do { + Rounding currentRounding = roundings[currentRoundingIdx]; + long currentKey = minKey; + requiredBuckets = 0; + while (currentKey < maxKey) { + requiredBuckets++; + currentKey = currentRounding.nextRoundingValue(currentKey); + } + currentRoundingIdx++; + } while (requiredBuckets > targetBuckets && currentRoundingIdx < roundings.length); + // The loop will increase past the correct rounding index here so we + // need to subtract one to get the rounding index we need + return currentRoundingIdx - 1; + } + + @Override + public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { + BucketReduceResult reducedBucketsResult = reduceBuckets(aggregations, reduceContext); + + // adding empty buckets if needed + reducedBucketsResult = addEmptyBuckets(reducedBucketsResult, reduceContext); + + // Adding empty buckets may have tipped us over the target so merge the buckets again if needed + reducedBucketsResult = mergeBucketsIfNeeded(reducedBucketsResult.buckets, reducedBucketsResult.roundingIdx, + reducedBucketsResult.rounding, reduceContext); + + BucketInfo bucketInfo = new BucketInfo(this.bucketInfo.roundings, reducedBucketsResult.roundingIdx, + this.bucketInfo.emptySubAggregations); + + return new InternalAutoDateHistogram(getName(), reducedBucketsResult.buckets, targetBuckets, bucketInfo, format, + pipelineAggregators(), getMetaData()); + } + + @Override + public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + builder.startArray(CommonFields.BUCKETS.getPreferredName()); + for (Bucket bucket : buckets) { + bucket.toXContent(builder, params); + } + builder.endArray(); + return builder; + } + + // HistogramFactory method impls + + @Override + public Number getKey(MultiBucketsAggregation.Bucket bucket) { + return ((Bucket) bucket).key; + } + + @Override + public Number nextKey(Number key) { + return bucketInfo.roundings[bucketInfo.roundingIdx].nextRoundingValue(key.longValue()); + } + + @Override + public InternalAggregation createAggregation(List buckets) { + // convert buckets to the right type + List buckets2 = new ArrayList<>(buckets.size()); + for (Object b : buckets) { + buckets2.add((Bucket) b); + } + buckets2 = Collections.unmodifiableList(buckets2); + return new InternalAutoDateHistogram(name, buckets2, targetBuckets, bucketInfo, format, pipelineAggregators(), getMetaData()); + } + + @Override + public Bucket createBucket(Number key, long docCount, InternalAggregations aggregations) { + return new Bucket(key.longValue(), docCount, format, aggregations); + } + + @Override + protected boolean doEquals(Object obj) { + InternalAutoDateHistogram that = (InternalAutoDateHistogram) obj; + return Objects.equals(buckets, that.buckets) + && Objects.equals(format, that.format) + && Objects.equals(bucketInfo, that.bucketInfo); + } + + @Override + protected int doHashCode() { + return Objects.hash(buckets, format, bucketInfo); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/ParsedAutoDateHistogram.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/ParsedAutoDateHistogram.java new file mode 100644 index 0000000000000..caca44f9f2ea7 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/ParsedAutoDateHistogram.java @@ -0,0 +1,91 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.bucket.histogram; + +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.search.aggregations.ParsedMultiBucketAggregation; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; + +import java.io.IOException; +import java.util.List; + +public class ParsedAutoDateHistogram extends ParsedMultiBucketAggregation implements Histogram { + + @Override + public String getType() { + return AutoDateHistogramAggregationBuilder.NAME; + } + + @Override + public List getBuckets() { + return buckets; + } + + private static ObjectParser PARSER = + new ObjectParser<>(ParsedAutoDateHistogram.class.getSimpleName(), true, ParsedAutoDateHistogram::new); + static { + declareMultiBucketAggregationFields(PARSER, + parser -> ParsedBucket.fromXContent(parser, false), + parser -> ParsedBucket.fromXContent(parser, true)); + } + + public static ParsedAutoDateHistogram fromXContent(XContentParser parser, String name) throws IOException { + ParsedAutoDateHistogram aggregation = PARSER.parse(parser, null); + aggregation.setName(name); + return aggregation; + } + + public static class ParsedBucket extends ParsedMultiBucketAggregation.ParsedBucket implements Histogram.Bucket { + + private Long key; + + @Override + public Object getKey() { + if (key != null) { + return new DateTime(key, DateTimeZone.UTC); + } + return null; + } + + @Override + public String getKeyAsString() { + String keyAsString = super.getKeyAsString(); + if (keyAsString != null) { + return keyAsString; + } + if (key != null) { + return Long.toString(key); + } + return null; + } + + @Override + protected XContentBuilder keyToXContent(XContentBuilder builder) throws IOException { + return builder.field(CommonFields.KEY.getPreferredName(), key); + } + + static ParsedBucket fromXContent(XContentParser parser, boolean keyed) throws IOException { + return parseXContent(parser, keyed, ParsedBucket::new, (p, bucket) -> bucket.key = p.longValue()); + } + } +} diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/AutoDateHistogramTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/AutoDateHistogramTests.java new file mode 100644 index 0000000000000..3a10edf183376 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/AutoDateHistogramTests.java @@ -0,0 +1,44 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.bucket; + +import org.elasticsearch.search.aggregations.BaseAggregationTestCase; +import org.elasticsearch.search.aggregations.bucket.histogram.AutoDateHistogramAggregationBuilder; + +public class AutoDateHistogramTests extends BaseAggregationTestCase { + + @Override + protected AutoDateHistogramAggregationBuilder createTestAggregatorBuilder() { + AutoDateHistogramAggregationBuilder builder = new AutoDateHistogramAggregationBuilder(randomAlphaOfLengthBetween(1, 10)); + builder.field(INT_FIELD_NAME); + builder.setNumBuckets(randomIntBetween(1, 100000)); + if (randomBoolean()) { + builder.format("###.##"); + } + if (randomBoolean()) { + builder.missing(randomIntBetween(0, 10)); + } + if (randomBoolean()) { + builder.timeZone(randomDateTimeZone()); + } + return builder; + } + +} diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java new file mode 100644 index 0000000000000..4399c5d8237fb --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java @@ -0,0 +1,669 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.bucket.histogram; + +import org.apache.lucene.document.Document; +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.MatchNoDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.store.Directory; +import org.elasticsearch.index.mapper.DateFieldMapper; +import org.elasticsearch.search.aggregations.AggregationBuilders; +import org.elasticsearch.search.aggregations.AggregatorTestCase; +import org.elasticsearch.search.aggregations.metrics.stats.Stats; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.function.Consumer; + +public class AutoDateHistogramAggregatorTests extends AggregatorTestCase { + + private static final String DATE_FIELD = "date"; + private static final String INSTANT_FIELD = "instant"; + + private static final List dataset = Arrays.asList( + "2010-03-12T01:07:45", + "2010-04-27T03:43:34", + "2012-05-18T04:11:00", + "2013-05-29T05:11:31", + "2013-10-31T08:24:05", + "2015-02-13T13:09:32", + "2015-06-24T13:47:43", + "2015-11-13T16:14:34", + "2016-03-04T17:09:50", + "2017-12-12T22:55:46"); + + public void testMatchNoDocs() throws IOException { + testBothCases(new MatchNoDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(10).field(DATE_FIELD), + histogram -> assertEquals(0, histogram.getBuckets().size()) + ); + } + + public void testMatchAllDocs() throws IOException { + Query query = new MatchAllDocsQuery(); + + testSearchCase(query, dataset, + aggregation -> aggregation.setNumBuckets(6).field(DATE_FIELD), + histogram -> assertEquals(6, histogram.getBuckets().size()) + ); + testSearchAndReduceCase(query, dataset, + aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD), + histogram -> assertEquals(8, histogram.getBuckets().size()) + ); + } + + public void testSubAggregations() throws IOException { + Query query = new MatchAllDocsQuery(); + + testSearchCase(query, dataset, + aggregation -> aggregation.setNumBuckets(6).field(DATE_FIELD) + .subAggregation(AggregationBuilders.stats("stats").field(DATE_FIELD)), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(6, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2010-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + Stats stats = bucket.getAggregations().get("stats"); + assertEquals("2010-03-12T01:07:45.000Z", stats.getMinAsString()); + assertEquals("2010-04-27T03:43:34.000Z", stats.getMaxAsString()); + assertEquals(2L, stats.getCount()); + + bucket = buckets.get(1); + assertEquals("2012-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertEquals("2012-05-18T04:11:00.000Z", stats.getMinAsString()); + assertEquals("2012-05-18T04:11:00.000Z", stats.getMaxAsString()); + assertEquals(1L, stats.getCount()); + + bucket = buckets.get(2); + assertEquals("2013-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertEquals("2013-05-29T05:11:31.000Z", stats.getMinAsString()); + assertEquals("2013-10-31T08:24:05.000Z", stats.getMaxAsString()); + assertEquals(2L, stats.getCount()); + + bucket = buckets.get(3); + assertEquals("2015-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertEquals("2015-02-13T13:09:32.000Z", stats.getMinAsString()); + assertEquals("2015-11-13T16:14:34.000Z", stats.getMaxAsString()); + assertEquals(3L, stats.getCount()); + + bucket = buckets.get(4); + assertEquals("2016-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertEquals("2016-03-04T17:09:50.000Z", stats.getMinAsString()); + assertEquals("2016-03-04T17:09:50.000Z", stats.getMaxAsString()); + assertEquals(1L, stats.getCount()); + + bucket = buckets.get(5); + assertEquals("2017-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertEquals("2017-12-12T22:55:46.000Z", stats.getMinAsString()); + assertEquals("2017-12-12T22:55:46.000Z", stats.getMaxAsString()); + assertEquals(1L, stats.getCount()); + }); + testSearchAndReduceCase(query, dataset, + aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD) + .subAggregation(AggregationBuilders.stats("stats").field(DATE_FIELD)), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(8, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2010-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + Stats stats = bucket.getAggregations().get("stats"); + assertEquals("2010-03-12T01:07:45.000Z", stats.getMinAsString()); + assertEquals("2010-04-27T03:43:34.000Z", stats.getMaxAsString()); + assertEquals(2L, stats.getCount()); + + bucket = buckets.get(1); + assertEquals("2011-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertTrue(Double.isInfinite(stats.getMin())); + assertTrue(Double.isInfinite(stats.getMax())); + assertEquals(0L, stats.getCount()); + + bucket = buckets.get(2); + assertEquals("2012-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertEquals("2012-05-18T04:11:00.000Z", stats.getMinAsString()); + assertEquals("2012-05-18T04:11:00.000Z", stats.getMaxAsString()); + assertEquals(1L, stats.getCount()); + + bucket = buckets.get(3); + assertEquals("2013-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertEquals("2013-05-29T05:11:31.000Z", stats.getMinAsString()); + assertEquals("2013-10-31T08:24:05.000Z", stats.getMaxAsString()); + assertEquals(2L, stats.getCount()); + + bucket = buckets.get(4); + assertEquals("2014-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertTrue(Double.isInfinite(stats.getMin())); + assertTrue(Double.isInfinite(stats.getMax())); + assertEquals(0L, stats.getCount()); + + bucket = buckets.get(5); + assertEquals("2015-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertEquals("2015-02-13T13:09:32.000Z", stats.getMinAsString()); + assertEquals("2015-11-13T16:14:34.000Z", stats.getMaxAsString()); + assertEquals(3L, stats.getCount()); + + bucket = buckets.get(6); + assertEquals("2016-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertEquals("2016-03-04T17:09:50.000Z", stats.getMinAsString()); + assertEquals("2016-03-04T17:09:50.000Z", stats.getMaxAsString()); + assertEquals(1L, stats.getCount()); + + bucket = buckets.get(7); + assertEquals("2017-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + stats = bucket.getAggregations().get("stats"); + assertEquals("2017-12-12T22:55:46.000Z", stats.getMinAsString()); + assertEquals("2017-12-12T22:55:46.000Z", stats.getMaxAsString()); + assertEquals(1L, stats.getCount()); + }); + } + + public void testNoDocs() throws IOException { + Query query = new MatchNoDocsQuery(); + List dates = Collections.emptyList(); + Consumer aggregation = agg -> agg.setNumBuckets(10).field(DATE_FIELD); + + testSearchCase(query, dates, aggregation, + histogram -> assertEquals(0, histogram.getBuckets().size()) + ); + testSearchAndReduceCase(query, dates, aggregation, + histogram -> assertNull(histogram) + ); + } + + public void testAggregateWrongField() throws IOException { + testBothCases(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(10).field("wrong_field"), + histogram -> assertEquals(0, histogram.getBuckets().size()) + ); + } + + public void testIntervalYear() throws IOException { + testBothCases(LongPoint.newRangeQuery(INSTANT_FIELD, asLong("2015-01-01"), asLong("2017-12-31")), dataset, + aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(3, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2015-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2016-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + } + ); + } + + public void testIntervalMonth() throws IOException { + testBothCases(new MatchAllDocsQuery(), + Arrays.asList("2017-01-01", "2017-02-02", "2017-02-03", "2017-03-04", "2017-03-05", "2017-03-06"), + aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(3, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-03-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + } + ); + } + + public void testIntervalDay() throws IOException { + testSearchCase(new MatchAllDocsQuery(), + Arrays.asList("2017-02-01", "2017-02-02", "2017-02-02", "2017-02-03", "2017-02-03", "2017-02-03", "2017-02-05"), + aggregation -> aggregation.setNumBuckets(5).field(DATE_FIELD), histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(4, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-02-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-02T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-03T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + + bucket = buckets.get(3); + assertEquals("2017-02-05T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), + Arrays.asList( + "2017-02-01", + "2017-02-02", + "2017-02-02", + "2017-02-03", + "2017-02-03", + "2017-02-03", + "2017-02-05" + ), + aggregation -> aggregation.setNumBuckets(5).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(5, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-02-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-02T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-03T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + + bucket = buckets.get(3); + assertEquals("2017-02-04T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(4); + assertEquals("2017-02-05T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + } + ); + } + + public void testIntervalHour() throws IOException { + testSearchCase(new MatchAllDocsQuery(), + Arrays.asList( + "2017-02-01T09:02:00.000Z", + "2017-02-01T09:35:00.000Z", + "2017-02-01T10:15:00.000Z", + "2017-02-01T13:06:00.000Z", + "2017-02-01T14:04:00.000Z", + "2017-02-01T14:05:00.000Z", + "2017-02-01T15:59:00.000Z", + "2017-02-01T16:06:00.000Z", + "2017-02-01T16:48:00.000Z", + "2017-02-01T16:59:00.000Z" + ), + aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(6, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-02-01T09:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-01T10:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-01T13:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(3); + assertEquals("2017-02-01T14:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(4); + assertEquals("2017-02-01T15:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(5); + assertEquals("2017-02-01T16:00:00.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + } + ); + testSearchAndReduceCase(new MatchAllDocsQuery(), + Arrays.asList( + "2017-02-01T09:02:00.000Z", + "2017-02-01T09:35:00.000Z", + "2017-02-01T10:15:00.000Z", + "2017-02-01T13:06:00.000Z", + "2017-02-01T14:04:00.000Z", + "2017-02-01T14:05:00.000Z", + "2017-02-01T15:59:00.000Z", + "2017-02-01T16:06:00.000Z", + "2017-02-01T16:48:00.000Z", + "2017-02-01T16:59:00.000Z" + ), + aggregation -> aggregation.setNumBuckets(10).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(8, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-02-01T09:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-01T10:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-01T11:00:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(3); + assertEquals("2017-02-01T12:00:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(4); + assertEquals("2017-02-01T13:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(5); + assertEquals("2017-02-01T14:00:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(6); + assertEquals("2017-02-01T15:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(7); + assertEquals("2017-02-01T16:00:00.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + } + ); + } + + public void testIntervalMinute() throws IOException { + testSearchCase(new MatchAllDocsQuery(), + Arrays.asList( + "2017-02-01T09:02:35.000Z", + "2017-02-01T09:02:59.000Z", + "2017-02-01T09:15:37.000Z", + "2017-02-01T09:16:04.000Z", + "2017-02-01T09:16:42.000Z" + ), + aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(3, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-02-01T09:02:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-01T09:15:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-01T09:16:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + } + ); + testSearchAndReduceCase(new MatchAllDocsQuery(), + Arrays.asList( + "2017-02-01T09:02:35.000Z", + "2017-02-01T09:02:59.000Z", + "2017-02-01T09:15:37.000Z", + "2017-02-01T09:16:04.000Z", + "2017-02-01T09:16:42.000Z" + ), + aggregation -> aggregation.setNumBuckets(15).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(15, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-02-01T09:02:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-01T09:03:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-01T09:04:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(3); + assertEquals("2017-02-01T09:05:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(4); + assertEquals("2017-02-01T09:06:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(5); + assertEquals("2017-02-01T09:07:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(6); + assertEquals("2017-02-01T09:08:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(7); + assertEquals("2017-02-01T09:09:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(8); + assertEquals("2017-02-01T09:10:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(9); + assertEquals("2017-02-01T09:11:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(10); + assertEquals("2017-02-01T09:12:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(11); + assertEquals("2017-02-01T09:13:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(12); + assertEquals("2017-02-01T09:14:00.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(13); + assertEquals("2017-02-01T09:15:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(14); + assertEquals("2017-02-01T09:16:00.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + } + ); + } + + public void testIntervalSecond() throws IOException { + testSearchCase(new MatchAllDocsQuery(), + Arrays.asList("2017-02-01T00:00:05.015Z", "2017-02-01T00:00:07.299Z", "2017-02-01T00:00:07.074Z", + "2017-02-01T00:00:11.688Z", "2017-02-01T00:00:11.210Z", "2017-02-01T00:00:11.380Z"), + aggregation -> aggregation.setNumBuckets(7).field(DATE_FIELD), histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(3, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-02-01T00:00:05.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-01T00:00:07.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-01T00:00:11.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), + Arrays.asList( + "2017-02-01T00:00:05.015Z", + "2017-02-01T00:00:07.299Z", + "2017-02-01T00:00:07.074Z", + "2017-02-01T00:00:11.688Z", + "2017-02-01T00:00:11.210Z", + "2017-02-01T00:00:11.380Z" + ), + aggregation -> aggregation.setNumBuckets(7).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(7, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-02-01T00:00:05.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-01T00:00:06.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-01T00:00:07.000Z", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(3); + assertEquals("2017-02-01T00:00:08.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(4); + assertEquals("2017-02-01T00:00:09.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(5); + assertEquals("2017-02-01T00:00:10.000Z", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(6); + assertEquals("2017-02-01T00:00:11.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + } + ); + } + + private void testSearchCase(Query query, List dataset, + Consumer configure, + Consumer verify) throws IOException { + executeTestCase(false, query, dataset, configure, verify); + } + + private void testSearchAndReduceCase(Query query, List dataset, + Consumer configure, + Consumer verify) throws IOException { + executeTestCase(true, query, dataset, configure, verify); + } + + private void testBothCases(Query query, List dataset, + Consumer configure, + Consumer verify) throws IOException { + testSearchCase(query, dataset, configure, verify); + testSearchAndReduceCase(query, dataset, configure, verify); + } + + private void executeTestCase(boolean reduced, Query query, List dataset, + Consumer configure, + Consumer verify) throws IOException { + + try (Directory directory = newDirectory()) { + try (RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory)) { + Document document = new Document(); + for (String date : dataset) { + if (frequently()) { + indexWriter.commit(); + } + + long instant = asLong(date); + document.add(new SortedNumericDocValuesField(DATE_FIELD, instant)); + document.add(new LongPoint(INSTANT_FIELD, instant)); + indexWriter.addDocument(document); + document.clear(); + } + } + + try (IndexReader indexReader = DirectoryReader.open(directory)) { + IndexSearcher indexSearcher = newSearcher(indexReader, true, true); + + AutoDateHistogramAggregationBuilder aggregationBuilder = new AutoDateHistogramAggregationBuilder("_name"); + if (configure != null) { + configure.accept(aggregationBuilder); + } + + DateFieldMapper.Builder builder = new DateFieldMapper.Builder("_name"); + DateFieldMapper.DateFieldType fieldType = builder.fieldType(); + fieldType.setHasDocValues(true); + fieldType.setName(aggregationBuilder.field()); + + InternalAutoDateHistogram histogram; + if (reduced) { + histogram = searchAndReduce(indexSearcher, query, aggregationBuilder, fieldType); + } else { + histogram = search(indexSearcher, query, aggregationBuilder, fieldType); + } + verify.accept(histogram); + } + } + } + + private static long asLong(String dateTime) { + return DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.parser().parseDateTime(dateTime).getMillis(); + } +} diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java new file mode 100644 index 0000000000000..5c7eff6bec061 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java @@ -0,0 +1,153 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.bucket.histogram; + +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.rounding.DateTimeUnit; +import org.elasticsearch.common.rounding.Rounding; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.ParsedMultiBucketAggregation; +import org.elasticsearch.search.aggregations.bucket.histogram.InternalAutoDateHistogram.BucketInfo; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.elasticsearch.test.InternalMultiBucketAggregationTestCase; +import org.joda.time.DateTime; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import static org.elasticsearch.common.unit.TimeValue.timeValueHours; +import static org.elasticsearch.common.unit.TimeValue.timeValueMinutes; +import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds; + +public class InternalAutoDateHistogramTests extends InternalMultiBucketAggregationTestCase { + + private DocValueFormat format; + private Rounding[] roundings; + + @Override + public void setUp() throws Exception { + super.setUp(); + format = randomNumericDocValueFormat(); + + roundings = new Rounding[6]; + roundings[0] = Rounding.builder(DateTimeUnit.SECOND_OF_MINUTE).build(); + roundings[1] = Rounding.builder(DateTimeUnit.MINUTES_OF_HOUR).build(); + roundings[2] = Rounding.builder(DateTimeUnit.HOUR_OF_DAY).build(); + roundings[3] = Rounding.builder(DateTimeUnit.DAY_OF_MONTH).build(); + roundings[4] = Rounding.builder(DateTimeUnit.MONTH_OF_YEAR).build(); + roundings[5] = Rounding.builder(DateTimeUnit.YEAR_OF_CENTURY).build(); + } + + @Override + protected InternalAutoDateHistogram createTestInstance(String name, + List pipelineAggregators, + Map metaData, + InternalAggregations aggregations) { + int nbBuckets = randomNumberOfBuckets(); + int targetBuckets = randomIntBetween(1, nbBuckets * 2 + 1); + List buckets = new ArrayList<>(nbBuckets); + long startingDate = System.currentTimeMillis(); + + long interval = randomIntBetween(1, 3); + long intervalMillis = randomFrom(timeValueSeconds(interval), timeValueMinutes(interval), timeValueHours(interval)).getMillis(); + + for (int i = 0; i < nbBuckets; i++) { + long key = startingDate + (intervalMillis * i); + buckets.add(i, new InternalAutoDateHistogram.Bucket(key, randomIntBetween(1, 100), format, aggregations)); + } + InternalAggregations subAggregations = new InternalAggregations(Collections.emptyList()); + BucketInfo bucketInfo = new BucketInfo(roundings, randomIntBetween(0, roundings.length - 1), subAggregations); + + return new InternalAutoDateHistogram(name, buckets, targetBuckets, bucketInfo, format, pipelineAggregators, metaData); + } + + @Override + protected void assertReduced(InternalAutoDateHistogram reduced, List inputs) { + int roundingIdx = 0; + for (InternalAutoDateHistogram histogram : inputs) { + if (histogram.getBucketInfo().roundingIdx > roundingIdx) { + roundingIdx = histogram.getBucketInfo().roundingIdx; + } + } + Map expectedCounts = new TreeMap<>(); + for (Histogram histogram : inputs) { + for (Histogram.Bucket bucket : histogram.getBuckets()) { + expectedCounts.compute(roundings[roundingIdx].round(((DateTime) bucket.getKey()).getMillis()), + (key, oldValue) -> (oldValue == null ? 0 : oldValue) + bucket.getDocCount()); + } + } + Map actualCounts = new TreeMap<>(); + for (Histogram.Bucket bucket : reduced.getBuckets()) { + actualCounts.compute(((DateTime) bucket.getKey()).getMillis(), + (key, oldValue) -> (oldValue == null ? 0 : oldValue) + bucket.getDocCount()); + } + assertEquals(expectedCounts, actualCounts); + } + + @Override + protected Writeable.Reader instanceReader() { + return InternalAutoDateHistogram::new; + } + + @Override + protected Class implementationClass() { + return ParsedAutoDateHistogram.class; + } + + @Override + protected InternalAutoDateHistogram mutateInstance(InternalAutoDateHistogram instance) { + String name = instance.getName(); + List buckets = instance.getBuckets(); + int targetBuckets = instance.getTargetBuckets(); + BucketInfo bucketInfo = instance.getBucketInfo(); + List pipelineAggregators = instance.pipelineAggregators(); + Map metaData = instance.getMetaData(); + switch (between(0, 3)) { + case 0: + name += randomAlphaOfLength(5); + break; + case 1: + buckets = new ArrayList<>(buckets); + buckets.add(new InternalAutoDateHistogram.Bucket(randomNonNegativeLong(), randomIntBetween(1, 100), format, + InternalAggregations.EMPTY)); + break; + case 2: + int roundingIdx = bucketInfo.roundingIdx == bucketInfo.roundings.length - 1 ? 0 : bucketInfo.roundingIdx + 1; + bucketInfo = new BucketInfo(bucketInfo.roundings, roundingIdx, bucketInfo.emptySubAggregations); + break; + case 3: + if (metaData == null) { + metaData = new HashMap<>(1); + } else { + metaData = new HashMap<>(instance.getMetaData()); + } + metaData.put(randomAlphaOfLength(15), randomInt()); + break; + default: + throw new AssertionError("Illegal randomisation branch"); + } + return new InternalAutoDateHistogram(name, buckets, targetBuckets, bucketInfo, format, pipelineAggregators, metaData); + } +} From 33458f61ab7d6581248cb87223c27a92fb17370c Mon Sep 17 00:00:00 2001 From: Colin Goodheart-Smithe Date: Fri, 9 Mar 2018 15:09:39 +0000 Subject: [PATCH 06/24] trying to get tests to pass again --- .../histogram/AutoDateHistogramAggregator.java | 1 + .../histogram/InternalAutoDateHistogram.java | 15 +++++++++++++-- 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java index cf2531fca6394..268b87250c9a0 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java @@ -157,6 +157,7 @@ private void increaseRounding() { @Override public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { assert owningBucketOrdinal == 0; + consumeBucketsAndMaybeBreak((int) bucketOrds.size()); long[] bucketOrdArray = new long[(int) bucketOrds.size()]; for (int i = 0; i < bucketOrds.size(); i++) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java index 4b17da4283d16..78e87f236ad9b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java @@ -327,7 +327,12 @@ protected boolean lessThan(IteratorAndCurrent a, IteratorAndCurrent b) { if (reduceRounding.round(top.current.key) != key) { // the key changes, reduce what we already buffered and reset the buffer for current buckets final Bucket reduced = currentBuckets.get(0).reduce(currentBuckets, reduceRounding, reduceContext); - reducedBuckets.add(reduced); + if (reduceContext.isFinalReduce() == false) { + reduceContext.consumeBucketsAndMaybeBreak(1); + reducedBuckets.add(reduced); + } else { + reduceContext.consumeBucketsAndMaybeBreak(-countInnerBucket(reduced)); + } currentBuckets.clear(); key = reduceRounding.round(top.current.key); } @@ -346,7 +351,12 @@ protected boolean lessThan(IteratorAndCurrent a, IteratorAndCurrent b) { if (currentBuckets.isEmpty() == false) { final Bucket reduced = currentBuckets.get(0).reduce(currentBuckets, reduceRounding, reduceContext); - reducedBuckets.add(reduced); + if (reduceContext.isFinalReduce() == false) { + reduceContext.consumeBucketsAndMaybeBreak(1); + reducedBuckets.add(reduced); + } else { + reduceContext.consumeBucketsAndMaybeBreak(-countInnerBucket(reduced)); + } } } @@ -425,6 +435,7 @@ private BucketReduceResult addEmptyBuckets(BucketReduceResult currentResult, Red if (lastBucket != null) { long key = rounding.nextRoundingValue(lastBucket.key); while (key < nextBucket.key) { + reduceContext.consumeBucketsAndMaybeBreak(1); iter.add(new InternalAutoDateHistogram.Bucket(key, 0, format, reducedEmptySubAggs)); key = rounding.nextRoundingValue(key); } From 678802a75b0998a0a8baa27b9e7227597ad5534b Mon Sep 17 00:00:00 2001 From: Colin Goodheart-Smithe Date: Mon, 12 Mar 2018 10:22:26 +0000 Subject: [PATCH 07/24] Fixes multiBucketConsumer accounting --- .../histogram/InternalAutoDateHistogram.java | 21 ++++++++----------- 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java index 78e87f236ad9b..fcfbd2c66d12b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java @@ -327,12 +327,8 @@ protected boolean lessThan(IteratorAndCurrent a, IteratorAndCurrent b) { if (reduceRounding.round(top.current.key) != key) { // the key changes, reduce what we already buffered and reset the buffer for current buckets final Bucket reduced = currentBuckets.get(0).reduce(currentBuckets, reduceRounding, reduceContext); - if (reduceContext.isFinalReduce() == false) { - reduceContext.consumeBucketsAndMaybeBreak(1); - reducedBuckets.add(reduced); - } else { - reduceContext.consumeBucketsAndMaybeBreak(-countInnerBucket(reduced)); - } + reduceContext.consumeBucketsAndMaybeBreak(1); + reducedBuckets.add(reduced); currentBuckets.clear(); key = reduceRounding.round(top.current.key); } @@ -351,12 +347,8 @@ protected boolean lessThan(IteratorAndCurrent a, IteratorAndCurrent b) { if (currentBuckets.isEmpty() == false) { final Bucket reduced = currentBuckets.get(0).reduce(currentBuckets, reduceRounding, reduceContext); - if (reduceContext.isFinalReduce() == false) { - reduceContext.consumeBucketsAndMaybeBreak(1); - reducedBuckets.add(reduced); - } else { - reduceContext.consumeBucketsAndMaybeBreak(-countInnerBucket(reduced)); - } + reduceContext.consumeBucketsAndMaybeBreak(1); + reducedBuckets.add(reduced); } } @@ -382,17 +374,22 @@ private List mergeBuckets(List reducedBuckets, Rounding reduceRo long roundedBucketKey = reduceRounding.round(bucket.key); if (Double.isNaN(key)) { key = roundedBucketKey; + reduceContext.consumeBucketsAndMaybeBreak(-countInnerBucket(bucket) - 1); sameKeyedBuckets.add(createBucket(key, bucket.docCount, bucket.aggregations)); } else if (roundedBucketKey == key) { + reduceContext.consumeBucketsAndMaybeBreak(-countInnerBucket(bucket) - 1); sameKeyedBuckets.add(createBucket(key, bucket.docCount, bucket.aggregations)); } else { + reduceContext.consumeBucketsAndMaybeBreak(1); mergedBuckets.add(sameKeyedBuckets.get(0).reduce(sameKeyedBuckets, reduceRounding, reduceContext)); sameKeyedBuckets.clear(); key = roundedBucketKey; + reduceContext.consumeBucketsAndMaybeBreak(-countInnerBucket(bucket) - 1); sameKeyedBuckets.add(createBucket(key, bucket.docCount, bucket.aggregations)); } } if (sameKeyedBuckets.isEmpty() == false) { + reduceContext.consumeBucketsAndMaybeBreak(1); mergedBuckets.add(sameKeyedBuckets.get(0).reduce(sameKeyedBuckets, reduceRounding, reduceContext)); } reducedBuckets = mergedBuckets; From 993c782d117892af9a3c86a51921cdee630a3ac5 Mon Sep 17 00:00:00 2001 From: Colin Goodheart-Smithe Date: Tue, 13 Mar 2018 14:58:05 +0000 Subject: [PATCH 08/24] Collects more buckets than needed on shards This gives us more options at reduce time in terms of how we do the final merge of the buckeets to produce the final result --- .../AutoDateHistogramAggregator.java | 10 +++-- .../histogram/InternalAutoDateHistogram.java | 40 ++++++++++++++----- .../AutoDateHistogramAggregatorTests.java | 12 +++--- .../InternalAutoDateHistogramTests.java | 17 ++++++-- 4 files changed, 55 insertions(+), 24 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java index 268b87250c9a0..bba468f44f12c 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java @@ -61,6 +61,7 @@ class AutoDateHistogramAggregator extends DeferableBucketAggregator { private LongHash bucketOrds; private int targetBuckets; private MergingBucketsDeferringCollector deferringCollector; + private int numCollectedValues = 0; AutoDateHistogramAggregator(String name, AggregatorFactories factories, int numBuckets, Rounding[] roundings, @Nullable ValuesSource.Numeric valuesSource, DocValueFormat formatter, SearchContext aggregationContext, Aggregator parent, @@ -109,6 +110,7 @@ public void collect(int doc, long bucket) throws IOException { long previousRounded = Long.MIN_VALUE; for (int i = 0; i < valuesCount; ++i) { long value = values.nextValue(); + numCollectedValues++; long rounded = roundings[roundingIdx].round(value); assert rounded >= previousRounded; if (rounded == previousRounded) { @@ -120,7 +122,8 @@ public void collect(int doc, long bucket) throws IOException { collectExistingBucket(sub, doc, bucketOrd); } else { collectBucket(sub, doc, bucketOrd); - while (bucketOrds.size() > targetBuckets) { + double maxBuckets = Math.max(targetBuckets, targetBuckets * Math.log(numCollectedValues)); + while (bucketOrds.size() > maxBuckets) { increaseRounding(); } } @@ -179,14 +182,15 @@ public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOE InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundings, roundingIdx, buildEmptySubAggregations()); - return new InternalAutoDateHistogram(name, buckets, targetBuckets, emptyBucketInfo, formatter, pipelineAggregators(), metaData()); + return new InternalAutoDateHistogram(name, buckets, targetBuckets, numCollectedValues, emptyBucketInfo, formatter, + pipelineAggregators(), metaData()); } @Override public InternalAggregation buildEmptyAggregation() { InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundings, roundingIdx, buildEmptySubAggregations()); - return new InternalAutoDateHistogram(name, Collections.emptyList(), targetBuckets, emptyBucketInfo, formatter, + return new InternalAutoDateHistogram(name, Collections.emptyList(), targetBuckets, numCollectedValues, emptyBucketInfo, formatter, pipelineAggregators(), metaData()); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java index fcfbd2c66d12b..30b23d2166008 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java @@ -43,6 +43,7 @@ import java.util.ListIterator; import java.util.Map; import java.util.Objects; +import java.util.stream.Collectors; /** * Implementation of {@link Histogram}. @@ -207,15 +208,17 @@ public int hashCode() { private final DocValueFormat format; private final BucketInfo bucketInfo; private final int targetBuckets; + private final long numValuesCollected; - InternalAutoDateHistogram(String name, List buckets, int targetBuckets, BucketInfo emptyBucketInfo, DocValueFormat formatter, - List pipelineAggregators, Map metaData) { + InternalAutoDateHistogram(String name, List buckets, int targetBuckets, long numValuesCollected, BucketInfo emptyBucketInfo, + DocValueFormat formatter, List pipelineAggregators, Map metaData) { super(name, pipelineAggregators, metaData); this.buckets = buckets; this.bucketInfo = emptyBucketInfo; this.format = formatter; this.targetBuckets = targetBuckets; + this.numValuesCollected = numValuesCollected; } /** @@ -227,6 +230,7 @@ public InternalAutoDateHistogram(StreamInput in) throws IOException { format = in.readNamedWriteable(DocValueFormat.class); buckets = in.readList(stream -> new Bucket(stream, format)); this.targetBuckets = in.readVInt(); + this.numValuesCollected = in.readVLong(); } @Override @@ -235,6 +239,7 @@ protected void doWriteTo(StreamOutput out) throws IOException { out.writeNamedWriteable(format); out.writeList(buckets); out.writeVInt(targetBuckets); + out.writeVLong(numValuesCollected); } @Override @@ -255,13 +260,18 @@ public int getTargetBuckets() { return targetBuckets; } + public long getNumValuesCollected() { + return numValuesCollected; + } + public BucketInfo getBucketInfo() { return bucketInfo; } @Override public InternalAutoDateHistogram create(List buckets) { - return new InternalAutoDateHistogram(name, buckets, targetBuckets, bucketInfo, format, pipelineAggregators(), metaData); + return new InternalAutoDateHistogram(name, buckets, targetBuckets, numValuesCollected, bucketInfo, format, pipelineAggregators(), + metaData); } @Override @@ -365,7 +375,8 @@ private BucketReduceResult mergeBucketsIfNeeded(List reducedBuckets, int return new BucketReduceResult(reducedBuckets, reduceRounding, reduceRoundingIdx); } - private List mergeBuckets(List reducedBuckets, Rounding reduceRounding, ReduceContext reduceContext) { + private List mergeBuckets(List reducedBuckets, Rounding reduceRounding, + ReduceContext reduceContext) { List mergedBuckets = new ArrayList<>(); List sameKeyedBuckets = new ArrayList<>(); @@ -409,12 +420,13 @@ private static class BucketReduceResult { } } - private BucketReduceResult addEmptyBuckets(BucketReduceResult currentResult, ReduceContext reduceContext) { + private BucketReduceResult addEmptyBuckets(BucketReduceResult currentResult, long numValuesCollected, ReduceContext reduceContext) { List list = currentResult.buckets; if (list.isEmpty()) { return currentResult; } - int roundingIdx = getAppropriateRounding(list.get(0).key, list.get(list.size() - 1).key, currentResult.roundingIdx, + double maxBuckets = Math.max(targetBuckets, targetBuckets * Math.log(numValuesCollected)); + int roundingIdx = getAppropriateRounding(list.get(0).key, list.get(list.size() - 1).key, maxBuckets, currentResult.roundingIdx, bucketInfo.roundings); Rounding rounding = bucketInfo.roundings[roundingIdx]; // merge buckets using the new rounding @@ -443,7 +455,7 @@ private BucketReduceResult addEmptyBuckets(BucketReduceResult currentResult, Red return new BucketReduceResult(list, rounding, roundingIdx); } - private int getAppropriateRounding(long minKey, long maxKey, int roundingIdx, Rounding[] roundings) { + private int getAppropriateRounding(long minKey, long maxKey, double maxBuckets, int roundingIdx, Rounding[] roundings) { if (roundingIdx == roundings.length - 1) { return roundingIdx; } @@ -458,7 +470,7 @@ private int getAppropriateRounding(long minKey, long maxKey, int roundingIdx, Ro currentKey = currentRounding.nextRoundingValue(currentKey); } currentRoundingIdx++; - } while (requiredBuckets > targetBuckets && currentRoundingIdx < roundings.length); + } while (requiredBuckets > maxBuckets && currentRoundingIdx < roundings.length); // The loop will increase past the correct rounding index here so we // need to subtract one to get the rounding index we need return currentRoundingIdx - 1; @@ -466,10 +478,13 @@ private int getAppropriateRounding(long minKey, long maxKey, int roundingIdx, Ro @Override public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { + long numValuesCollected = aggregations.stream() + .collect(Collectors.summingLong(agg -> ((InternalAutoDateHistogram) agg).getNumValuesCollected())); + BucketReduceResult reducedBucketsResult = reduceBuckets(aggregations, reduceContext); // adding empty buckets if needed - reducedBucketsResult = addEmptyBuckets(reducedBucketsResult, reduceContext); + reducedBucketsResult = addEmptyBuckets(reducedBucketsResult, numValuesCollected, reduceContext); // Adding empty buckets may have tipped us over the target so merge the buckets again if needed reducedBucketsResult = mergeBucketsIfNeeded(reducedBucketsResult.buckets, reducedBucketsResult.roundingIdx, @@ -478,7 +493,7 @@ public InternalAggregation doReduce(List aggregations, Redu BucketInfo bucketInfo = new BucketInfo(this.bucketInfo.roundings, reducedBucketsResult.roundingIdx, this.bucketInfo.emptySubAggregations); - return new InternalAutoDateHistogram(getName(), reducedBucketsResult.buckets, targetBuckets, bucketInfo, format, + return new InternalAutoDateHistogram(getName(), reducedBucketsResult.buckets, targetBuckets, numValuesCollected, bucketInfo, format, pipelineAggregators(), getMetaData()); } @@ -512,7 +527,8 @@ public InternalAggregation createAggregation(List aggregation.setNumBuckets(6).field(DATE_FIELD), + aggregation -> aggregation.setNumBuckets(3).field(DATE_FIELD), histogram -> assertEquals(6, histogram.getBuckets().size()) ); testSearchAndReduceCase(query, dataset, @@ -82,7 +82,7 @@ public void testSubAggregations() throws IOException { Query query = new MatchAllDocsQuery(); testSearchCase(query, dataset, - aggregation -> aggregation.setNumBuckets(6).field(DATE_FIELD) + aggregation -> aggregation.setNumBuckets(3).field(DATE_FIELD) .subAggregation(AggregationBuilders.stats("stats").field(DATE_FIELD)), histogram -> { List buckets = histogram.getBuckets(); @@ -231,7 +231,7 @@ public void testAggregateWrongField() throws IOException { public void testIntervalYear() throws IOException { testBothCases(LongPoint.newRangeQuery(INSTANT_FIELD, asLong("2015-01-01"), asLong("2017-12-31")), dataset, - aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), + aggregation -> aggregation.setNumBuckets(3).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); assertEquals(3, buckets.size()); @@ -254,7 +254,7 @@ public void testIntervalYear() throws IOException { public void testIntervalMonth() throws IOException { testBothCases(new MatchAllDocsQuery(), Arrays.asList("2017-01-01", "2017-02-02", "2017-02-03", "2017-03-04", "2017-03-05", "2017-03-06"), - aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), + aggregation -> aggregation.setNumBuckets(3).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); assertEquals(3, buckets.size()); @@ -349,7 +349,7 @@ public void testIntervalHour() throws IOException { "2017-02-01T16:48:00.000Z", "2017-02-01T16:59:00.000Z" ), - aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD), + aggregation -> aggregation.setNumBuckets(3).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); assertEquals(6, buckets.size()); @@ -441,7 +441,7 @@ public void testIntervalMinute() throws IOException { "2017-02-01T09:16:04.000Z", "2017-02-01T09:16:42.000Z" ), - aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), + aggregation -> aggregation.setNumBuckets(3).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); assertEquals(3, buckets.size()); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java index 5c7eff6bec061..5aa7a7cb1b1d6 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java @@ -79,8 +79,9 @@ protected InternalAutoDateHistogram createTestInstance(String name, } InternalAggregations subAggregations = new InternalAggregations(Collections.emptyList()); BucketInfo bucketInfo = new BucketInfo(roundings, randomIntBetween(0, roundings.length - 1), subAggregations); - - return new InternalAutoDateHistogram(name, buckets, targetBuckets, bucketInfo, format, pipelineAggregators, metaData); + long numValuesCollected = randomNonNegativeLong(); + return new InternalAutoDateHistogram(name, buckets, targetBuckets, numValuesCollected, bucketInfo, format, pipelineAggregators, + metaData); } @Override @@ -121,10 +122,11 @@ protected InternalAutoDateHistogram mutateInstance(InternalAutoDateHistogram ins String name = instance.getName(); List buckets = instance.getBuckets(); int targetBuckets = instance.getTargetBuckets(); + long numValuesCollected = instance.getNumValuesCollected(); BucketInfo bucketInfo = instance.getBucketInfo(); List pipelineAggregators = instance.pipelineAggregators(); Map metaData = instance.getMetaData(); - switch (between(0, 3)) { + switch (between(0, 5)) { case 0: name += randomAlphaOfLength(5); break; @@ -145,9 +147,16 @@ protected InternalAutoDateHistogram mutateInstance(InternalAutoDateHistogram ins } metaData.put(randomAlphaOfLength(15), randomInt()); break; + case 4: + targetBuckets += between(1, 100); + break; + case 5: + numValuesCollected += between(1, 100); + break; default: throw new AssertionError("Illegal randomisation branch"); } - return new InternalAutoDateHistogram(name, buckets, targetBuckets, bucketInfo, format, pipelineAggregators, metaData); + return new InternalAutoDateHistogram(name, buckets, targetBuckets, numValuesCollected, bucketInfo, format, pipelineAggregators, + metaData); } } From ee5e721b9eede9949c2f90b328fe684b3c886783 Mon Sep 17 00:00:00 2001 From: Colin Goodheart-Smithe Date: Thu, 15 Mar 2018 11:25:19 +0000 Subject: [PATCH 09/24] Revert "Collects more buckets than needed on shards" This reverts commit 993c782d117892af9a3c86a51921cdee630a3ac5. --- .../AutoDateHistogramAggregator.java | 10 ++--- .../histogram/InternalAutoDateHistogram.java | 40 +++++-------------- .../AutoDateHistogramAggregatorTests.java | 12 +++--- .../InternalAutoDateHistogramTests.java | 17 ++------ 4 files changed, 24 insertions(+), 55 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java index bba468f44f12c..268b87250c9a0 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java @@ -61,7 +61,6 @@ class AutoDateHistogramAggregator extends DeferableBucketAggregator { private LongHash bucketOrds; private int targetBuckets; private MergingBucketsDeferringCollector deferringCollector; - private int numCollectedValues = 0; AutoDateHistogramAggregator(String name, AggregatorFactories factories, int numBuckets, Rounding[] roundings, @Nullable ValuesSource.Numeric valuesSource, DocValueFormat formatter, SearchContext aggregationContext, Aggregator parent, @@ -110,7 +109,6 @@ public void collect(int doc, long bucket) throws IOException { long previousRounded = Long.MIN_VALUE; for (int i = 0; i < valuesCount; ++i) { long value = values.nextValue(); - numCollectedValues++; long rounded = roundings[roundingIdx].round(value); assert rounded >= previousRounded; if (rounded == previousRounded) { @@ -122,8 +120,7 @@ public void collect(int doc, long bucket) throws IOException { collectExistingBucket(sub, doc, bucketOrd); } else { collectBucket(sub, doc, bucketOrd); - double maxBuckets = Math.max(targetBuckets, targetBuckets * Math.log(numCollectedValues)); - while (bucketOrds.size() > maxBuckets) { + while (bucketOrds.size() > targetBuckets) { increaseRounding(); } } @@ -182,15 +179,14 @@ public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOE InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundings, roundingIdx, buildEmptySubAggregations()); - return new InternalAutoDateHistogram(name, buckets, targetBuckets, numCollectedValues, emptyBucketInfo, formatter, - pipelineAggregators(), metaData()); + return new InternalAutoDateHistogram(name, buckets, targetBuckets, emptyBucketInfo, formatter, pipelineAggregators(), metaData()); } @Override public InternalAggregation buildEmptyAggregation() { InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundings, roundingIdx, buildEmptySubAggregations()); - return new InternalAutoDateHistogram(name, Collections.emptyList(), targetBuckets, numCollectedValues, emptyBucketInfo, formatter, + return new InternalAutoDateHistogram(name, Collections.emptyList(), targetBuckets, emptyBucketInfo, formatter, pipelineAggregators(), metaData()); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java index 30b23d2166008..fcfbd2c66d12b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java @@ -43,7 +43,6 @@ import java.util.ListIterator; import java.util.Map; import java.util.Objects; -import java.util.stream.Collectors; /** * Implementation of {@link Histogram}. @@ -208,17 +207,15 @@ public int hashCode() { private final DocValueFormat format; private final BucketInfo bucketInfo; private final int targetBuckets; - private final long numValuesCollected; - InternalAutoDateHistogram(String name, List buckets, int targetBuckets, long numValuesCollected, BucketInfo emptyBucketInfo, - DocValueFormat formatter, List pipelineAggregators, Map metaData) { + InternalAutoDateHistogram(String name, List buckets, int targetBuckets, BucketInfo emptyBucketInfo, DocValueFormat formatter, + List pipelineAggregators, Map metaData) { super(name, pipelineAggregators, metaData); this.buckets = buckets; this.bucketInfo = emptyBucketInfo; this.format = formatter; this.targetBuckets = targetBuckets; - this.numValuesCollected = numValuesCollected; } /** @@ -230,7 +227,6 @@ public InternalAutoDateHistogram(StreamInput in) throws IOException { format = in.readNamedWriteable(DocValueFormat.class); buckets = in.readList(stream -> new Bucket(stream, format)); this.targetBuckets = in.readVInt(); - this.numValuesCollected = in.readVLong(); } @Override @@ -239,7 +235,6 @@ protected void doWriteTo(StreamOutput out) throws IOException { out.writeNamedWriteable(format); out.writeList(buckets); out.writeVInt(targetBuckets); - out.writeVLong(numValuesCollected); } @Override @@ -260,18 +255,13 @@ public int getTargetBuckets() { return targetBuckets; } - public long getNumValuesCollected() { - return numValuesCollected; - } - public BucketInfo getBucketInfo() { return bucketInfo; } @Override public InternalAutoDateHistogram create(List buckets) { - return new InternalAutoDateHistogram(name, buckets, targetBuckets, numValuesCollected, bucketInfo, format, pipelineAggregators(), - metaData); + return new InternalAutoDateHistogram(name, buckets, targetBuckets, bucketInfo, format, pipelineAggregators(), metaData); } @Override @@ -375,8 +365,7 @@ private BucketReduceResult mergeBucketsIfNeeded(List reducedBuckets, int return new BucketReduceResult(reducedBuckets, reduceRounding, reduceRoundingIdx); } - private List mergeBuckets(List reducedBuckets, Rounding reduceRounding, - ReduceContext reduceContext) { + private List mergeBuckets(List reducedBuckets, Rounding reduceRounding, ReduceContext reduceContext) { List mergedBuckets = new ArrayList<>(); List sameKeyedBuckets = new ArrayList<>(); @@ -420,13 +409,12 @@ private static class BucketReduceResult { } } - private BucketReduceResult addEmptyBuckets(BucketReduceResult currentResult, long numValuesCollected, ReduceContext reduceContext) { + private BucketReduceResult addEmptyBuckets(BucketReduceResult currentResult, ReduceContext reduceContext) { List list = currentResult.buckets; if (list.isEmpty()) { return currentResult; } - double maxBuckets = Math.max(targetBuckets, targetBuckets * Math.log(numValuesCollected)); - int roundingIdx = getAppropriateRounding(list.get(0).key, list.get(list.size() - 1).key, maxBuckets, currentResult.roundingIdx, + int roundingIdx = getAppropriateRounding(list.get(0).key, list.get(list.size() - 1).key, currentResult.roundingIdx, bucketInfo.roundings); Rounding rounding = bucketInfo.roundings[roundingIdx]; // merge buckets using the new rounding @@ -455,7 +443,7 @@ private BucketReduceResult addEmptyBuckets(BucketReduceResult currentResult, lon return new BucketReduceResult(list, rounding, roundingIdx); } - private int getAppropriateRounding(long minKey, long maxKey, double maxBuckets, int roundingIdx, Rounding[] roundings) { + private int getAppropriateRounding(long minKey, long maxKey, int roundingIdx, Rounding[] roundings) { if (roundingIdx == roundings.length - 1) { return roundingIdx; } @@ -470,7 +458,7 @@ private int getAppropriateRounding(long minKey, long maxKey, double maxBuckets, currentKey = currentRounding.nextRoundingValue(currentKey); } currentRoundingIdx++; - } while (requiredBuckets > maxBuckets && currentRoundingIdx < roundings.length); + } while (requiredBuckets > targetBuckets && currentRoundingIdx < roundings.length); // The loop will increase past the correct rounding index here so we // need to subtract one to get the rounding index we need return currentRoundingIdx - 1; @@ -478,13 +466,10 @@ private int getAppropriateRounding(long minKey, long maxKey, double maxBuckets, @Override public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { - long numValuesCollected = aggregations.stream() - .collect(Collectors.summingLong(agg -> ((InternalAutoDateHistogram) agg).getNumValuesCollected())); - BucketReduceResult reducedBucketsResult = reduceBuckets(aggregations, reduceContext); // adding empty buckets if needed - reducedBucketsResult = addEmptyBuckets(reducedBucketsResult, numValuesCollected, reduceContext); + reducedBucketsResult = addEmptyBuckets(reducedBucketsResult, reduceContext); // Adding empty buckets may have tipped us over the target so merge the buckets again if needed reducedBucketsResult = mergeBucketsIfNeeded(reducedBucketsResult.buckets, reducedBucketsResult.roundingIdx, @@ -493,7 +478,7 @@ public InternalAggregation doReduce(List aggregations, Redu BucketInfo bucketInfo = new BucketInfo(this.bucketInfo.roundings, reducedBucketsResult.roundingIdx, this.bucketInfo.emptySubAggregations); - return new InternalAutoDateHistogram(getName(), reducedBucketsResult.buckets, targetBuckets, numValuesCollected, bucketInfo, format, + return new InternalAutoDateHistogram(getName(), reducedBucketsResult.buckets, targetBuckets, bucketInfo, format, pipelineAggregators(), getMetaData()); } @@ -527,8 +512,7 @@ public InternalAggregation createAggregation(List aggregation.setNumBuckets(3).field(DATE_FIELD), + aggregation -> aggregation.setNumBuckets(6).field(DATE_FIELD), histogram -> assertEquals(6, histogram.getBuckets().size()) ); testSearchAndReduceCase(query, dataset, @@ -82,7 +82,7 @@ public void testSubAggregations() throws IOException { Query query = new MatchAllDocsQuery(); testSearchCase(query, dataset, - aggregation -> aggregation.setNumBuckets(3).field(DATE_FIELD) + aggregation -> aggregation.setNumBuckets(6).field(DATE_FIELD) .subAggregation(AggregationBuilders.stats("stats").field(DATE_FIELD)), histogram -> { List buckets = histogram.getBuckets(); @@ -231,7 +231,7 @@ public void testAggregateWrongField() throws IOException { public void testIntervalYear() throws IOException { testBothCases(LongPoint.newRangeQuery(INSTANT_FIELD, asLong("2015-01-01"), asLong("2017-12-31")), dataset, - aggregation -> aggregation.setNumBuckets(3).field(DATE_FIELD), + aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); assertEquals(3, buckets.size()); @@ -254,7 +254,7 @@ public void testIntervalYear() throws IOException { public void testIntervalMonth() throws IOException { testBothCases(new MatchAllDocsQuery(), Arrays.asList("2017-01-01", "2017-02-02", "2017-02-03", "2017-03-04", "2017-03-05", "2017-03-06"), - aggregation -> aggregation.setNumBuckets(3).field(DATE_FIELD), + aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); assertEquals(3, buckets.size()); @@ -349,7 +349,7 @@ public void testIntervalHour() throws IOException { "2017-02-01T16:48:00.000Z", "2017-02-01T16:59:00.000Z" ), - aggregation -> aggregation.setNumBuckets(3).field(DATE_FIELD), + aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); assertEquals(6, buckets.size()); @@ -441,7 +441,7 @@ public void testIntervalMinute() throws IOException { "2017-02-01T09:16:04.000Z", "2017-02-01T09:16:42.000Z" ), - aggregation -> aggregation.setNumBuckets(3).field(DATE_FIELD), + aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); assertEquals(3, buckets.size()); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java index 5aa7a7cb1b1d6..5c7eff6bec061 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java @@ -79,9 +79,8 @@ protected InternalAutoDateHistogram createTestInstance(String name, } InternalAggregations subAggregations = new InternalAggregations(Collections.emptyList()); BucketInfo bucketInfo = new BucketInfo(roundings, randomIntBetween(0, roundings.length - 1), subAggregations); - long numValuesCollected = randomNonNegativeLong(); - return new InternalAutoDateHistogram(name, buckets, targetBuckets, numValuesCollected, bucketInfo, format, pipelineAggregators, - metaData); + + return new InternalAutoDateHistogram(name, buckets, targetBuckets, bucketInfo, format, pipelineAggregators, metaData); } @Override @@ -122,11 +121,10 @@ protected InternalAutoDateHistogram mutateInstance(InternalAutoDateHistogram ins String name = instance.getName(); List buckets = instance.getBuckets(); int targetBuckets = instance.getTargetBuckets(); - long numValuesCollected = instance.getNumValuesCollected(); BucketInfo bucketInfo = instance.getBucketInfo(); List pipelineAggregators = instance.pipelineAggregators(); Map metaData = instance.getMetaData(); - switch (between(0, 5)) { + switch (between(0, 3)) { case 0: name += randomAlphaOfLength(5); break; @@ -147,16 +145,9 @@ protected InternalAutoDateHistogram mutateInstance(InternalAutoDateHistogram ins } metaData.put(randomAlphaOfLength(15), randomInt()); break; - case 4: - targetBuckets += between(1, 100); - break; - case 5: - numValuesCollected += between(1, 100); - break; default: throw new AssertionError("Illegal randomisation branch"); } - return new InternalAutoDateHistogram(name, buckets, targetBuckets, numValuesCollected, bucketInfo, format, pipelineAggregators, - metaData); + return new InternalAutoDateHistogram(name, buckets, targetBuckets, bucketInfo, format, pipelineAggregators, metaData); } } From 5bedd6f956c8026acffe53e9e8245844250f4f62 Mon Sep 17 00:00:00 2001 From: Colin Goodheart-Smithe Date: Thu, 15 Mar 2018 13:19:28 +0000 Subject: [PATCH 10/24] Adds ability to merge within a rounding --- .../AutoDateHistogramAggregationBuilder.java | 59 +++++- .../AutoDateHistogramAggregator.java | 18 +- .../AutoDateHistogramAggregatorFactory.java | 10 +- .../histogram/InternalAutoDateHistogram.java | 105 ++++++++--- .../AutoDateHistogramAggregatorTests.java | 175 ++++++++++-------- .../InternalAutoDateHistogramTests.java | 25 +-- 6 files changed, 256 insertions(+), 136 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java index 600de3cd328ad..0a83c5cd350e0 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java @@ -22,6 +22,7 @@ import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.rounding.DateTimeUnit; import org.elasticsearch.common.rounding.Rounding; import org.elasticsearch.common.xcontent.ObjectParser; @@ -41,6 +42,7 @@ import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; +import java.util.Arrays; import java.util.Map; import java.util.Objects; @@ -112,13 +114,13 @@ public int getNumBuckets() { @Override protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { - Rounding[] roundings = new Rounding[6]; - roundings[0] = createRounding(DateTimeUnit.SECOND_OF_MINUTE); - roundings[1] = createRounding(DateTimeUnit.MINUTES_OF_HOUR); - roundings[2] = createRounding(DateTimeUnit.HOUR_OF_DAY); - roundings[3] = createRounding(DateTimeUnit.DAY_OF_MONTH); - roundings[4] = createRounding(DateTimeUnit.MONTH_OF_YEAR); - roundings[5] = createRounding(DateTimeUnit.YEAR_OF_CENTURY); + RoundingInfo[] roundings = new RoundingInfo[6]; + roundings[0] = new RoundingInfo(createRounding(DateTimeUnit.SECOND_OF_MINUTE), 1, 5, 10, 30); + roundings[1] = new RoundingInfo(createRounding(DateTimeUnit.MINUTES_OF_HOUR), 1, 5, 10, 30); + roundings[2] = new RoundingInfo(createRounding(DateTimeUnit.HOUR_OF_DAY), 1, 3, 12); + roundings[3] = new RoundingInfo(createRounding(DateTimeUnit.DAY_OF_MONTH), 1, 7); + roundings[4] = new RoundingInfo(createRounding(DateTimeUnit.MONTH_OF_YEAR), 1, 3); + roundings[5] = new RoundingInfo(createRounding(DateTimeUnit.YEAR_OF_CENTURY), 1, 10, 20, 50, 100); return new AutoDateHistogramAggregatorFactory(name, config, numBuckets, roundings, context, parent, subFactoriesBuilder, metaData); } @@ -147,4 +149,47 @@ protected boolean innerEquals(Object obj) { AutoDateHistogramAggregationBuilder other = (AutoDateHistogramAggregationBuilder) obj; return Objects.equals(numBuckets, other.numBuckets); } + + public static class RoundingInfo implements Writeable { + final Rounding rounding; + final int[] innerIntervals; + + public RoundingInfo(Rounding rounding, int... innerIntervals) { + this.rounding = rounding; + this.innerIntervals = innerIntervals; + } + + public RoundingInfo(StreamInput in) throws IOException { + rounding = Rounding.Streams.read(in); + innerIntervals = in.readIntArray(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + Rounding.Streams.write(rounding, out); + out.writeIntArray(innerIntervals); + } + + public int getMaximumInnerInterval() { + return innerIntervals[innerIntervals.length - 1]; + } + + @Override + public int hashCode() { + return Objects.hash(rounding, Arrays.hashCode(innerIntervals)); + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + if (obj.getClass() != getClass()) { + return false; + } + RoundingInfo other = (RoundingInfo) obj; + return Objects.equals(rounding, other.rounding) && + Objects.deepEquals(innerIntervals, other.innerIntervals); + } + } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java index 268b87250c9a0..4d15719b5f9e7 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java @@ -35,6 +35,7 @@ import org.elasticsearch.search.aggregations.bucket.DeferableBucketAggregator; import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector; import org.elasticsearch.search.aggregations.bucket.MergingBucketsDeferringCollector; +import org.elasticsearch.search.aggregations.bucket.histogram.AutoDateHistogramAggregationBuilder.RoundingInfo; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.internal.SearchContext; @@ -55,14 +56,14 @@ class AutoDateHistogramAggregator extends DeferableBucketAggregator { private final ValuesSource.Numeric valuesSource; private final DocValueFormat formatter; - private final Rounding[] roundings; + private final RoundingInfo[] roundingInfos; private int roundingIdx = 0; private LongHash bucketOrds; private int targetBuckets; private MergingBucketsDeferringCollector deferringCollector; - AutoDateHistogramAggregator(String name, AggregatorFactories factories, int numBuckets, Rounding[] roundings, + AutoDateHistogramAggregator(String name, AggregatorFactories factories, int numBuckets, RoundingInfo[] roundingInfos, @Nullable ValuesSource.Numeric valuesSource, DocValueFormat formatter, SearchContext aggregationContext, Aggregator parent, List pipelineAggregators, Map metaData) throws IOException { @@ -70,7 +71,7 @@ class AutoDateHistogramAggregator extends DeferableBucketAggregator { this.targetBuckets = numBuckets; this.valuesSource = valuesSource; this.formatter = formatter; - this.roundings = roundings; + this.roundingInfos = roundingInfos; bucketOrds = new LongHash(1, aggregationContext.bigArrays()); @@ -109,7 +110,7 @@ public void collect(int doc, long bucket) throws IOException { long previousRounded = Long.MIN_VALUE; for (int i = 0; i < valuesCount; ++i) { long value = values.nextValue(); - long rounded = roundings[roundingIdx].round(value); + long rounded = roundingInfos[roundingIdx].rounding.round(value); assert rounded >= previousRounded; if (rounded == previousRounded) { continue; @@ -120,7 +121,8 @@ public void collect(int doc, long bucket) throws IOException { collectExistingBucket(sub, doc, bucketOrd); } else { collectBucket(sub, doc, bucketOrd); - while (bucketOrds.size() > targetBuckets) { + while (roundingIdx < roundingInfos.length - 1 + && bucketOrds.size() > (targetBuckets * roundingInfos[roundingIdx].getMaximumInnerInterval())) { increaseRounding(); } } @@ -133,7 +135,7 @@ private void increaseRounding() { try (LongHash oldBucketOrds = bucketOrds) { LongHash newBucketOrds = new LongHash(1, context.bigArrays()); long[] mergeMap = new long[(int) oldBucketOrds.size()]; - Rounding newRounding = roundings[++roundingIdx]; + Rounding newRounding = roundingInfos[++roundingIdx].rounding; for (int i = 0; i < oldBucketOrds.size(); i++) { long oldKey = oldBucketOrds.get(i); long newKey = newRounding.round(oldKey); @@ -176,7 +178,7 @@ public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOE CollectionUtil.introSort(buckets, BucketOrder.key(true).comparator(this)); // value source will be null for unmapped fields - InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundings, roundingIdx, + InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundingInfos, roundingIdx, buildEmptySubAggregations()); return new InternalAutoDateHistogram(name, buckets, targetBuckets, emptyBucketInfo, formatter, pipelineAggregators(), metaData()); @@ -184,7 +186,7 @@ public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOE @Override public InternalAggregation buildEmptyAggregation() { - InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundings, roundingIdx, + InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundingInfos, roundingIdx, buildEmptySubAggregations()); return new InternalAutoDateHistogram(name, Collections.emptyList(), targetBuckets, emptyBucketInfo, formatter, pipelineAggregators(), metaData()); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java index 18ce727204151..051f2f9f6e7c7 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java @@ -19,10 +19,10 @@ package org.elasticsearch.search.aggregations.bucket.histogram; -import org.elasticsearch.common.rounding.Rounding; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.bucket.histogram.AutoDateHistogramAggregationBuilder.RoundingInfo; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; @@ -38,14 +38,14 @@ public final class AutoDateHistogramAggregatorFactory extends ValuesSourceAggregatorFactory { private final int numBuckets; - private Rounding[] roundings; + private RoundingInfo[] roundingInfos; - public AutoDateHistogramAggregatorFactory(String name, ValuesSourceConfig config, int numBuckets, Rounding[] roundings, + public AutoDateHistogramAggregatorFactory(String name, ValuesSourceConfig config, int numBuckets, RoundingInfo[] roundingInfos, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); this.numBuckets = numBuckets; - this.roundings = roundings; + this.roundingInfos = roundingInfos; } @Override @@ -59,7 +59,7 @@ protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, Aggrega private Aggregator createAggregator(ValuesSource.Numeric valuesSource, Aggregator parent, List pipelineAggregators, Map metaData) throws IOException { - return new AutoDateHistogramAggregator(name, factories, numBuckets, roundings, valuesSource, config.format(), context, parent, + return new AutoDateHistogramAggregator(name, factories, numBuckets, roundingInfos, valuesSource, config.format(), context, parent, pipelineAggregators, metaData); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java index fcfbd2c66d12b..4f1addb341bb6 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java @@ -30,6 +30,7 @@ import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; import org.elasticsearch.search.aggregations.KeyComparable; import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation; +import org.elasticsearch.search.aggregations.bucket.histogram.AutoDateHistogramAggregationBuilder.RoundingInfo; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; @@ -157,30 +158,30 @@ public DocValueFormat getFormatter() { static class BucketInfo { - final Rounding[] roundings; + final RoundingInfo[] roundingInfos; final int roundingIdx; final InternalAggregations emptySubAggregations; - BucketInfo(Rounding[] roundings, int roundingIdx, InternalAggregations subAggregations) { - this.roundings = roundings; + BucketInfo(RoundingInfo[] roundings, int roundingIdx, InternalAggregations subAggregations) { + this.roundingInfos = roundings; this.roundingIdx = roundingIdx; this.emptySubAggregations = subAggregations; } BucketInfo(StreamInput in) throws IOException { int size = in.readVInt(); - roundings = new Rounding[size]; + roundingInfos = new RoundingInfo[size]; for (int i = 0; i < size; i++) { - roundings[i] = Rounding.Streams.read(in); + roundingInfos[i] = new RoundingInfo(in); } roundingIdx = in.readVInt(); emptySubAggregations = InternalAggregations.readAggregations(in); } void writeTo(StreamOutput out) throws IOException { - out.writeVInt(roundings.length); - for (Rounding rounding : roundings) { - Rounding.Streams.write(rounding, out); + out.writeVInt(roundingInfos.length); + for (RoundingInfo roundingInfo : roundingInfos) { + roundingInfo.writeTo(out); } out.writeVInt(roundingIdx); emptySubAggregations.writeTo(out); @@ -192,14 +193,14 @@ public boolean equals(Object obj) { return false; } BucketInfo that = (BucketInfo) obj; - return Objects.deepEquals(roundings, that.roundings) + return Objects.deepEquals(roundingInfos, that.roundingInfos) && Objects.equals(roundingIdx, that.roundingIdx) && Objects.equals(emptySubAggregations, that.emptySubAggregations); } @Override public int hashCode() { - return Objects.hash(getClass(), Arrays.hashCode(roundings), roundingIdx, emptySubAggregations); + return Objects.hash(getClass(), Arrays.hashCode(roundingInfos), roundingIdx, emptySubAggregations); } } @@ -300,7 +301,8 @@ private BucketReduceResult reduceBuckets(List aggregations, } } // This rounding will be used to reduce all the buckets - Rounding reduceRounding = bucketInfo.roundings[reduceRoundingIdx]; + RoundingInfo reduceRoundingInfo = bucketInfo.roundingInfos[reduceRoundingIdx]; + Rounding reduceRounding = reduceRoundingInfo.rounding; final PriorityQueue pq = new PriorityQueue(aggregations.size()) { @Override @@ -352,17 +354,18 @@ protected boolean lessThan(IteratorAndCurrent a, IteratorAndCurrent b) { } } - return mergeBucketsIfNeeded(reducedBuckets, reduceRoundingIdx, reduceRounding, reduceContext); + return mergeBucketsIfNeeded(reducedBuckets, reduceRoundingIdx, reduceRoundingInfo, reduceContext); } - private BucketReduceResult mergeBucketsIfNeeded(List reducedBuckets, int reduceRoundingIdx, Rounding reduceRounding, + private BucketReduceResult mergeBucketsIfNeeded(List reducedBuckets, int reduceRoundingIdx, RoundingInfo reduceRoundingInfo, ReduceContext reduceContext) { - while (reducedBuckets.size() > targetBuckets && reduceRoundingIdx < bucketInfo.roundings.length - 1) { + while (reducedBuckets.size() > (targetBuckets * reduceRoundingInfo.getMaximumInnerInterval()) + && reduceRoundingIdx < bucketInfo.roundingInfos.length - 1) { reduceRoundingIdx++; - reduceRounding = bucketInfo.roundings[reduceRoundingIdx]; - reducedBuckets = mergeBuckets(reducedBuckets, reduceRounding, reduceContext); + reduceRoundingInfo = bucketInfo.roundingInfos[reduceRoundingIdx]; + reducedBuckets = mergeBuckets(reducedBuckets, reduceRoundingInfo.rounding, reduceContext); } - return new BucketReduceResult(reducedBuckets, reduceRounding, reduceRoundingIdx); + return new BucketReduceResult(reducedBuckets, reduceRoundingInfo, reduceRoundingIdx); } private List mergeBuckets(List reducedBuckets, Rounding reduceRounding, ReduceContext reduceContext) { @@ -398,12 +401,12 @@ private List mergeBuckets(List reducedBuckets, Rounding reduceRo private static class BucketReduceResult { List buckets; - Rounding rounding; + RoundingInfo roundingInfo; int roundingIdx; - BucketReduceResult(List buckets, Rounding rounding, int roundingIdx) { + BucketReduceResult(List buckets, RoundingInfo roundingInfo, int roundingIdx) { this.buckets = buckets; - this.rounding = rounding; + this.roundingInfo = roundingInfo; this.roundingIdx = roundingIdx; } @@ -415,8 +418,9 @@ private BucketReduceResult addEmptyBuckets(BucketReduceResult currentResult, Red return currentResult; } int roundingIdx = getAppropriateRounding(list.get(0).key, list.get(list.size() - 1).key, currentResult.roundingIdx, - bucketInfo.roundings); - Rounding rounding = bucketInfo.roundings[roundingIdx]; + bucketInfo.roundingInfos); + RoundingInfo roundingInfo = bucketInfo.roundingInfos[roundingIdx]; + Rounding rounding = roundingInfo.rounding; // merge buckets using the new rounding list = mergeBuckets(list, rounding, reduceContext); @@ -440,17 +444,17 @@ private BucketReduceResult addEmptyBuckets(BucketReduceResult currentResult, Red } lastBucket = iter.next(); } - return new BucketReduceResult(list, rounding, roundingIdx); + return new BucketReduceResult(list, roundingInfo, roundingIdx); } - private int getAppropriateRounding(long minKey, long maxKey, int roundingIdx, Rounding[] roundings) { + private int getAppropriateRounding(long minKey, long maxKey, int roundingIdx, RoundingInfo[] roundings) { if (roundingIdx == roundings.length - 1) { return roundingIdx; } int currentRoundingIdx = roundingIdx; int requiredBuckets = 0; do { - Rounding currentRounding = roundings[currentRoundingIdx]; + Rounding currentRounding = roundings[currentRoundingIdx].rounding; long currentKey = minKey; requiredBuckets = 0; while (currentKey < maxKey) { @@ -458,7 +462,8 @@ private int getAppropriateRounding(long minKey, long maxKey, int roundingIdx, Ro currentKey = currentRounding.nextRoundingValue(currentKey); } currentRoundingIdx++; - } while (requiredBuckets > targetBuckets && currentRoundingIdx < roundings.length); + } while (requiredBuckets > (targetBuckets * roundings[roundingIdx].getMaximumInnerInterval()) + && currentRoundingIdx < roundings.length); // The loop will increase past the correct rounding index here so we // need to subtract one to get the rounding index we need return currentRoundingIdx - 1; @@ -473,15 +478,57 @@ public InternalAggregation doReduce(List aggregations, Redu // Adding empty buckets may have tipped us over the target so merge the buckets again if needed reducedBucketsResult = mergeBucketsIfNeeded(reducedBucketsResult.buckets, reducedBucketsResult.roundingIdx, - reducedBucketsResult.rounding, reduceContext); + reducedBucketsResult.roundingInfo, reduceContext); + + // Now finally see if we need to merge consecutive buckets together to make a coarser interval at the same rounding + reducedBucketsResult = maybeMergeConsecutiveBuckets(reducedBucketsResult, reduceContext); - BucketInfo bucketInfo = new BucketInfo(this.bucketInfo.roundings, reducedBucketsResult.roundingIdx, + BucketInfo bucketInfo = new BucketInfo(this.bucketInfo.roundingInfos, reducedBucketsResult.roundingIdx, this.bucketInfo.emptySubAggregations); return new InternalAutoDateHistogram(getName(), reducedBucketsResult.buckets, targetBuckets, bucketInfo, format, pipelineAggregators(), getMetaData()); } + private BucketReduceResult maybeMergeConsecutiveBuckets(BucketReduceResult reducedBucketsResult, ReduceContext reduceContext) { + List buckets = reducedBucketsResult.buckets; + RoundingInfo roundingInfo = reducedBucketsResult.roundingInfo; + int roundingIdx = reducedBucketsResult.roundingIdx; + if (buckets.size() > targetBuckets) { + for (int interval : roundingInfo.innerIntervals) { + int resultingBuckets = buckets.size() / interval; + if (resultingBuckets <= targetBuckets) { + return mergeConsecutiveBuckets(buckets, interval, roundingIdx, roundingInfo, reduceContext); + } + } + } + return reducedBucketsResult; + } + + private BucketReduceResult mergeConsecutiveBuckets(List reducedBuckets, int mergeInterval, int roundingIdx, + RoundingInfo roundingInfo, ReduceContext reduceContext) { + List mergedBuckets = new ArrayList<>(); + List sameKeyedBuckets = new ArrayList<>(); + + double key = roundingInfo.rounding.round(reducedBuckets.get(0).key); + for (int i = 0; i < reducedBuckets.size(); i++) { + Bucket bucket = reducedBuckets.get(i); + if (i % mergeInterval == 0 && sameKeyedBuckets.isEmpty() == false) { + reduceContext.consumeBucketsAndMaybeBreak(1); + mergedBuckets.add(sameKeyedBuckets.get(0).reduce(sameKeyedBuckets, roundingInfo.rounding, reduceContext)); + sameKeyedBuckets.clear(); + key = roundingInfo.rounding.round(bucket.key); + } + reduceContext.consumeBucketsAndMaybeBreak(-countInnerBucket(bucket) - 1); + sameKeyedBuckets.add(createBucket(key, bucket.docCount, bucket.aggregations)); + } + if (sameKeyedBuckets.isEmpty() == false) { + reduceContext.consumeBucketsAndMaybeBreak(1); + mergedBuckets.add(sameKeyedBuckets.get(0).reduce(sameKeyedBuckets, roundingInfo.rounding, reduceContext)); + } + return new BucketReduceResult(mergedBuckets, roundingInfo, roundingIdx); + } + @Override public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { builder.startArray(CommonFields.BUCKETS.getPreferredName()); @@ -501,7 +548,7 @@ public Number getKey(MultiBucketsAggregation.Bucket bucket) { @Override public Number nextKey(Number key) { - return bucketInfo.roundings[bucketInfo.roundingIdx].nextRoundingValue(key.longValue()); + return bucketInfo.roundingInfos[bucketInfo.roundingIdx].rounding.nextRoundingValue(key.longValue()); } @Override diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java index 4399c5d8237fb..6ac0a4485aed8 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java @@ -70,7 +70,7 @@ public void testMatchAllDocs() throws IOException { testSearchCase(query, dataset, aggregation -> aggregation.setNumBuckets(6).field(DATE_FIELD), - histogram -> assertEquals(6, histogram.getBuckets().size()) + histogram -> assertEquals(10, histogram.getBuckets().size()) ); testSearchAndReduceCase(query, dataset, aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD), @@ -80,62 +80,6 @@ public void testMatchAllDocs() throws IOException { public void testSubAggregations() throws IOException { Query query = new MatchAllDocsQuery(); - - testSearchCase(query, dataset, - aggregation -> aggregation.setNumBuckets(6).field(DATE_FIELD) - .subAggregation(AggregationBuilders.stats("stats").field(DATE_FIELD)), - histogram -> { - List buckets = histogram.getBuckets(); - assertEquals(6, buckets.size()); - - Histogram.Bucket bucket = buckets.get(0); - assertEquals("2010-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); - Stats stats = bucket.getAggregations().get("stats"); - assertEquals("2010-03-12T01:07:45.000Z", stats.getMinAsString()); - assertEquals("2010-04-27T03:43:34.000Z", stats.getMaxAsString()); - assertEquals(2L, stats.getCount()); - - bucket = buckets.get(1); - assertEquals("2012-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - stats = bucket.getAggregations().get("stats"); - assertEquals("2012-05-18T04:11:00.000Z", stats.getMinAsString()); - assertEquals("2012-05-18T04:11:00.000Z", stats.getMaxAsString()); - assertEquals(1L, stats.getCount()); - - bucket = buckets.get(2); - assertEquals("2013-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); - stats = bucket.getAggregations().get("stats"); - assertEquals("2013-05-29T05:11:31.000Z", stats.getMinAsString()); - assertEquals("2013-10-31T08:24:05.000Z", stats.getMaxAsString()); - assertEquals(2L, stats.getCount()); - - bucket = buckets.get(3); - assertEquals("2015-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(3, bucket.getDocCount()); - stats = bucket.getAggregations().get("stats"); - assertEquals("2015-02-13T13:09:32.000Z", stats.getMinAsString()); - assertEquals("2015-11-13T16:14:34.000Z", stats.getMaxAsString()); - assertEquals(3L, stats.getCount()); - - bucket = buckets.get(4); - assertEquals("2016-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - stats = bucket.getAggregations().get("stats"); - assertEquals("2016-03-04T17:09:50.000Z", stats.getMinAsString()); - assertEquals("2016-03-04T17:09:50.000Z", stats.getMaxAsString()); - assertEquals(1L, stats.getCount()); - - bucket = buckets.get(5); - assertEquals("2017-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - stats = bucket.getAggregations().get("stats"); - assertEquals("2017-12-12T22:55:46.000Z", stats.getMinAsString()); - assertEquals("2017-12-12T22:55:46.000Z", stats.getMaxAsString()); - assertEquals(1L, stats.getCount()); - }); testSearchAndReduceCase(query, dataset, aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD) .subAggregation(AggregationBuilders.stats("stats").field(DATE_FIELD)), @@ -230,7 +174,34 @@ public void testAggregateWrongField() throws IOException { } public void testIntervalYear() throws IOException { - testBothCases(LongPoint.newRangeQuery(INSTANT_FIELD, asLong("2015-01-01"), asLong("2017-12-31")), dataset, + testSearchCase(LongPoint.newRangeQuery(INSTANT_FIELD, asLong("2015-01-01"), asLong("2017-12-31")), dataset, + aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(5, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2015-02-13T13:09:32.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2015-06-24T13:47:43.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2015-11-13T16:14:34.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(3); + assertEquals("2016-03-04T17:09:50.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(4); + assertEquals("2017-12-12T22:55:46.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + } + ); + testSearchAndReduceCase(LongPoint.newRangeQuery(INSTANT_FIELD, asLong("2015-01-01"), asLong("2017-12-31")), dataset, aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); @@ -252,7 +223,37 @@ public void testIntervalYear() throws IOException { } public void testIntervalMonth() throws IOException { - testBothCases(new MatchAllDocsQuery(), + testSearchCase(new MatchAllDocsQuery(), + Arrays.asList("2017-01-01", "2017-02-02", "2017-02-03", "2017-03-04", "2017-03-05", "2017-03-06"), + aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(6, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-01-01T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-02T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-03T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(3); + assertEquals("2017-03-04T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(4); + assertEquals("2017-03-05T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(5); + assertEquals("2017-03-06T00:00:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), Arrays.asList("2017-01-01", "2017-02-02", "2017-02-03", "2017-03-04", "2017-03-05", "2017-03-06"), aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), histogram -> { @@ -352,31 +353,47 @@ public void testIntervalHour() throws IOException { aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); - assertEquals(6, buckets.size()); + assertEquals(10, buckets.size()); Histogram.Bucket bucket = buckets.get(0); - assertEquals("2017-02-01T09:00:00.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); + assertEquals("2017-02-01T09:02:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); bucket = buckets.get(1); - assertEquals("2017-02-01T10:00:00.000Z", bucket.getKeyAsString()); + assertEquals("2017-02-01T09:35:00.000Z", bucket.getKeyAsString()); assertEquals(1, bucket.getDocCount()); bucket = buckets.get(2); - assertEquals("2017-02-01T13:00:00.000Z", bucket.getKeyAsString()); + assertEquals("2017-02-01T10:15:00.000Z", bucket.getKeyAsString()); assertEquals(1, bucket.getDocCount()); bucket = buckets.get(3); - assertEquals("2017-02-01T14:00:00.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); + assertEquals("2017-02-01T13:06:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); bucket = buckets.get(4); - assertEquals("2017-02-01T15:00:00.000Z", bucket.getKeyAsString()); + assertEquals("2017-02-01T14:04:00.000Z", bucket.getKeyAsString()); assertEquals(1, bucket.getDocCount()); bucket = buckets.get(5); - assertEquals("2017-02-01T16:00:00.000Z", bucket.getKeyAsString()); - assertEquals(3, bucket.getDocCount()); + assertEquals("2017-02-01T14:05:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(6); + assertEquals("2017-02-01T15:59:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(7); + assertEquals("2017-02-01T16:06:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(8); + assertEquals("2017-02-01T16:48:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(9); + assertEquals("2017-02-01T16:59:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); } ); testSearchAndReduceCase(new MatchAllDocsQuery(), @@ -444,19 +461,27 @@ public void testIntervalMinute() throws IOException { aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); - assertEquals(3, buckets.size()); + assertEquals(5, buckets.size()); Histogram.Bucket bucket = buckets.get(0); - assertEquals("2017-02-01T09:02:00.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); + assertEquals("2017-02-01T09:02:35.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); bucket = buckets.get(1); - assertEquals("2017-02-01T09:15:00.000Z", bucket.getKeyAsString()); + assertEquals("2017-02-01T09:02:59.000Z", bucket.getKeyAsString()); assertEquals(1, bucket.getDocCount()); bucket = buckets.get(2); - assertEquals("2017-02-01T09:16:00.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); + assertEquals("2017-02-01T09:15:37.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(3); + assertEquals("2017-02-01T09:16:04.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(4); + assertEquals("2017-02-01T09:16:42.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); } ); testSearchAndReduceCase(new MatchAllDocsQuery(), diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java index 5c7eff6bec061..389371efd79aa 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java @@ -25,6 +25,7 @@ import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.ParsedMultiBucketAggregation; +import org.elasticsearch.search.aggregations.bucket.histogram.AutoDateHistogramAggregationBuilder.RoundingInfo; import org.elasticsearch.search.aggregations.bucket.histogram.InternalAutoDateHistogram.BucketInfo; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.test.InternalMultiBucketAggregationTestCase; @@ -44,20 +45,20 @@ public class InternalAutoDateHistogramTests extends InternalMultiBucketAggregationTestCase { private DocValueFormat format; - private Rounding[] roundings; + private RoundingInfo[] roundingInfos; @Override public void setUp() throws Exception { super.setUp(); format = randomNumericDocValueFormat(); - roundings = new Rounding[6]; - roundings[0] = Rounding.builder(DateTimeUnit.SECOND_OF_MINUTE).build(); - roundings[1] = Rounding.builder(DateTimeUnit.MINUTES_OF_HOUR).build(); - roundings[2] = Rounding.builder(DateTimeUnit.HOUR_OF_DAY).build(); - roundings[3] = Rounding.builder(DateTimeUnit.DAY_OF_MONTH).build(); - roundings[4] = Rounding.builder(DateTimeUnit.MONTH_OF_YEAR).build(); - roundings[5] = Rounding.builder(DateTimeUnit.YEAR_OF_CENTURY).build(); + roundingInfos = new RoundingInfo[6]; + roundingInfos[0] = new RoundingInfo(Rounding.builder(DateTimeUnit.SECOND_OF_MINUTE).build(), 1, 5, 10, 30); + roundingInfos[1] = new RoundingInfo(Rounding.builder(DateTimeUnit.MINUTES_OF_HOUR).build(), 1, 5, 10, 30); + roundingInfos[2] = new RoundingInfo(Rounding.builder(DateTimeUnit.HOUR_OF_DAY).build(), 1, 3, 12); + roundingInfos[3] = new RoundingInfo(Rounding.builder(DateTimeUnit.DAY_OF_MONTH).build(), 1, 7); + roundingInfos[4] = new RoundingInfo(Rounding.builder(DateTimeUnit.MONTH_OF_YEAR).build(), 1, 3); + roundingInfos[5] = new RoundingInfo(Rounding.builder(DateTimeUnit.YEAR_OF_CENTURY).build(), 1, 10, 20, 50, 100); } @Override @@ -78,7 +79,7 @@ protected InternalAutoDateHistogram createTestInstance(String name, buckets.add(i, new InternalAutoDateHistogram.Bucket(key, randomIntBetween(1, 100), format, aggregations)); } InternalAggregations subAggregations = new InternalAggregations(Collections.emptyList()); - BucketInfo bucketInfo = new BucketInfo(roundings, randomIntBetween(0, roundings.length - 1), subAggregations); + BucketInfo bucketInfo = new BucketInfo(roundingInfos, randomIntBetween(0, roundingInfos.length - 1), subAggregations); return new InternalAutoDateHistogram(name, buckets, targetBuckets, bucketInfo, format, pipelineAggregators, metaData); } @@ -94,7 +95,7 @@ protected void assertReduced(InternalAutoDateHistogram reduced, List expectedCounts = new TreeMap<>(); for (Histogram histogram : inputs) { for (Histogram.Bucket bucket : histogram.getBuckets()) { - expectedCounts.compute(roundings[roundingIdx].round(((DateTime) bucket.getKey()).getMillis()), + expectedCounts.compute(roundingInfos[roundingIdx].rounding.round(((DateTime) bucket.getKey()).getMillis()), (key, oldValue) -> (oldValue == null ? 0 : oldValue) + bucket.getDocCount()); } } @@ -134,8 +135,8 @@ protected InternalAutoDateHistogram mutateInstance(InternalAutoDateHistogram ins InternalAggregations.EMPTY)); break; case 2: - int roundingIdx = bucketInfo.roundingIdx == bucketInfo.roundings.length - 1 ? 0 : bucketInfo.roundingIdx + 1; - bucketInfo = new BucketInfo(bucketInfo.roundings, roundingIdx, bucketInfo.emptySubAggregations); + int roundingIdx = bucketInfo.roundingIdx == bucketInfo.roundingInfos.length - 1 ? 0 : bucketInfo.roundingIdx + 1; + bucketInfo = new BucketInfo(bucketInfo.roundingInfos, roundingIdx, bucketInfo.emptySubAggregations); break; case 3: if (metaData == null) { From fa3b90158bf2681f41073c9379856623928e49a4 Mon Sep 17 00:00:00 2001 From: Colin Goodheart-Smithe Date: Mon, 19 Mar 2018 12:01:42 +0000 Subject: [PATCH 11/24] Fixes nonn-timezone doc test failure --- .../aggregations/bucket/autodatehistogram-aggregation.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc index f655a1616f014..e8b887cbb8765 100644 --- a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc @@ -43,7 +43,7 @@ POST /sales/_search?size=0 "sales_over_time" : { "auto_date_histogram" : { "field" : "date", - "buckets" : 10, + "buckets" : 5, "format" : "yyyy-MM-dd" <1> } } From 7c4d01914e88cc51cc92c19be1c1cc79a1afd3ee Mon Sep 17 00:00:00 2001 From: Colin Goodheart-Smithe Date: Mon, 19 Mar 2018 14:45:00 +0000 Subject: [PATCH 12/24] Fix time zone tests --- .../autodatehistogram-aggregation.asciidoc | 38 ++++++++++---- .../AutoDateHistogramAggregatorTests.java | 52 +++++++++++++++++++ 2 files changed, 79 insertions(+), 11 deletions(-) diff --git a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc index e8b887cbb8765..0b0f0cd4c3b42 100644 --- a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc @@ -129,9 +129,8 @@ GET my_index/_search?size=0 --------------------------------- // CONSOLE -UTC is used if no time zone is specified, which would result in all of these -documents being placed into the same day bucket, which starts at midnight UTC -on 1 October 2015: +UTC is used if no time zone is specified, three 1-hour buckets are returned +starting at midnight UTC on 1 October 2015: [source,js] --------------------------------- @@ -142,8 +141,18 @@ on 1 October 2015: "buckets": [ { "key_as_string": "2015-10-01T00:00:00.000Z", - "key": 1443657600000, - "doc_count": 3 + "key": 1443657600000, + "doc_count": 1 + }, + { + "key_as_string": "2015-10-01T01:00:00.000Z", + "key": 1443661200000, + "doc_count": 1 + }, + { + "key_as_string": "2015-10-01T02:00:00.000Z", + "key": 1443664800000, + "doc_count": 1 } ] } @@ -173,8 +182,10 @@ GET my_index/_search?size=0 // CONSOLE // TEST[continued] -Now the first document falls into the bucket for 30 September 2015, while the -other two documents fall into the bucket for 1 October 2015: + +Now three 1-hour buckets are still returned but the first bucket starts at +11:00pm on 30 September 2015 since that is the local time for the bucket in +the specified time zone. [source,js] --------------------------------- @@ -184,14 +195,19 @@ other two documents fall into the bucket for 1 October 2015: "by_day": { "buckets": [ { - "key_as_string": "2015-09-30T00:00:00.000-01:00", <1> - "key": 1443574800000, + "key_as_string": "2015-09-30T23:00:00.000-01:00", + "key": 1443657600000, "doc_count": 1 }, { - "key_as_string": "2015-10-01T00:00:00.000-01:00", <1> + "key_as_string": "2015-10-01T00:00:00.000-01:00", "key": 1443661200000, - "doc_count": 2 + "doc_count": 1 + }, + { + "key_as_string": "2015-10-01T01:00:00.000-01:00", + "key": 1443664800000, + "doc_count": 1 } ] } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java index 6ac0a4485aed8..4e23394b87361 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java @@ -34,6 +34,7 @@ import org.elasticsearch.search.aggregations.AggregationBuilders; import org.elasticsearch.search.aggregations.AggregatorTestCase; import org.elasticsearch.search.aggregations.metrics.stats.Stats; +import org.joda.time.DateTimeZone; import java.io.IOException; import java.util.Arrays; @@ -336,6 +337,57 @@ public void testIntervalDay() throws IOException { ); } + public void testIntervalDayWithTZ() throws IOException { + testSearchCase(new MatchAllDocsQuery(), + Arrays.asList("2017-02-01", "2017-02-02", "2017-02-02", "2017-02-03", "2017-02-03", "2017-02-03", "2017-02-05"), + aggregation -> aggregation.setNumBuckets(5).field(DATE_FIELD).timeZone(DateTimeZone.forOffsetHours(-1)), histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(4, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-01-31T23:00:00.000-01:00", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-01T23:00:00.000-01:00", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-02T23:00:00.000-01:00", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + + bucket = buckets.get(3); + assertEquals("2017-02-04T23:00:00.000-01:00", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), + Arrays.asList("2017-02-01", "2017-02-02", "2017-02-02", "2017-02-03", "2017-02-03", "2017-02-03", "2017-02-05"), + aggregation -> aggregation.setNumBuckets(5).field(DATE_FIELD).timeZone(DateTimeZone.forOffsetHours(-1)), histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(5, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-01-31T00:00:00.000-01:00", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-01T00:00:00.000-01:00", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-02T00:00:00.000-01:00", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + + bucket = buckets.get(3); + assertEquals("2017-02-03T00:00:00.000-01:00", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(4); + assertEquals("2017-02-04T00:00:00.000-01:00", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + }); + } + public void testIntervalHour() throws IOException { testSearchCase(new MatchAllDocsQuery(), Arrays.asList( From 65c056eca29516cbc21d47a7c986dd0bac47e40b Mon Sep 17 00:00:00 2001 From: Colin Goodheart-Smithe Date: Mon, 19 Mar 2018 15:50:24 +0000 Subject: [PATCH 13/24] iterates on tests --- .../autodatehistogram-aggregation.asciidoc | 4 +- .../AutoDateHistogramAggregatorTests.java | 113 ++++++++++++++++++ 2 files changed, 115 insertions(+), 2 deletions(-) diff --git a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc index 0b0f0cd4c3b42..f5b5c6006c78d 100644 --- a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc @@ -121,7 +121,7 @@ GET my_index/_search?size=0 "by_day": { "auto_date_histogram": { "field": "date", - "buckets" : 2 + "buckets" : 3 } } } @@ -172,7 +172,7 @@ GET my_index/_search?size=0 "by_day": { "auto_date_histogram": { "field": "date", - "buckets" : 2, + "buckets" : 3, "time_zone": "-01:00" } } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java index 4e23394b87361..d131e4f169d36 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java @@ -501,6 +501,119 @@ public void testIntervalHour() throws IOException { ); } + public void testIntervalHourWithTZ() throws IOException { + testSearchCase(new MatchAllDocsQuery(), + Arrays.asList( + "2017-02-01T09:02:00.000Z", + "2017-02-01T09:35:00.000Z", + "2017-02-01T10:15:00.000Z", + "2017-02-01T13:06:00.000Z", + "2017-02-01T14:04:00.000Z", + "2017-02-01T14:05:00.000Z", + "2017-02-01T15:59:00.000Z", + "2017-02-01T16:06:00.000Z", + "2017-02-01T16:48:00.000Z", + "2017-02-01T16:59:00.000Z" + ), + aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD).timeZone(DateTimeZone.forOffsetHours(-1)), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(10, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-02-01T08:02:00.000-01:00", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-01T08:35:00.000-01:00", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-01T09:15:00.000-01:00", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(3); + assertEquals("2017-02-01T12:06:00.000-01:00", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(4); + assertEquals("2017-02-01T13:04:00.000-01:00", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(5); + assertEquals("2017-02-01T13:05:00.000-01:00", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(6); + assertEquals("2017-02-01T14:59:00.000-01:00", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(7); + assertEquals("2017-02-01T15:06:00.000-01:00", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(8); + assertEquals("2017-02-01T15:48:00.000-01:00", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(9); + assertEquals("2017-02-01T15:59:00.000-01:00", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + } + ); + testSearchAndReduceCase(new MatchAllDocsQuery(), + Arrays.asList( + "2017-02-01T09:02:00.000Z", + "2017-02-01T09:35:00.000Z", + "2017-02-01T10:15:00.000Z", + "2017-02-01T13:06:00.000Z", + "2017-02-01T14:04:00.000Z", + "2017-02-01T14:05:00.000Z", + "2017-02-01T15:59:00.000Z", + "2017-02-01T16:06:00.000Z", + "2017-02-01T16:48:00.000Z", + "2017-02-01T16:59:00.000Z" + ), + aggregation -> aggregation.setNumBuckets(10).field(DATE_FIELD).timeZone(DateTimeZone.forOffsetHours(-1)), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(8, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-02-01T08:00:00.000-01:00", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-01T09:00:00.000-01:00", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-01T10:00:00.000-01:00", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(3); + assertEquals("2017-02-01T11:00:00.000-01:00", bucket.getKeyAsString()); + assertEquals(0, bucket.getDocCount()); + + bucket = buckets.get(4); + assertEquals("2017-02-01T12:00:00.000-01:00", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(5); + assertEquals("2017-02-01T13:00:00.000-01:00", bucket.getKeyAsString()); + assertEquals(2, bucket.getDocCount()); + + bucket = buckets.get(6); + assertEquals("2017-02-01T14:00:00.000-01:00", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(7); + assertEquals("2017-02-01T15:00:00.000-01:00", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + } + ); + } + public void testIntervalMinute() throws IOException { testSearchCase(new MatchAllDocsQuery(), Arrays.asList( From 4fac5bfa306a47dd656317250470d4aa0057b598 Mon Sep 17 00:00:00 2001 From: Colin Goodheart-Smithe Date: Thu, 22 Mar 2018 11:44:56 +0000 Subject: [PATCH 14/24] Adds test case and documentation changes Added some notes in the documentation about the intervals that can bbe returned. Also added a test case that utilises the merging of conseecutive buckets --- .../autodatehistogram-aggregation.asciidoc | 20 +++- .../AutoDateHistogramAggregationBuilder.java | 2 +- .../AutoDateHistogramAggregatorTests.java | 93 +++++++++++++++++++ 3 files changed, 113 insertions(+), 2 deletions(-) diff --git a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc index f5b5c6006c78d..9d1d0424ce4a0 100644 --- a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc @@ -4,7 +4,7 @@ A multi-bucket aggregation similar to the <> except instead of providing an interval to use as the width of each bucket, a target number of buckets is provided indicating the number of buckets needed and the interval of the buckets is automatically chosen to best achieve -that target. +that target. The number of buckets returned will always be less than or equal to this target number. Requesting a target of 10 buckets. @@ -86,6 +86,24 @@ Response: -------------------------------------------------- // TESTRESPONSE[s/\.\.\./"took": $body.took,"timed_out": false,"_shards": $body._shards,"hits": $body.hits,/] +=== Intervals + +The interval of the returned buckets is selected based on the data collected by the +aggregation so that the number of buckets returned is less than or equal to the number +requested. The possible intervals returned are: + +[horizontal] +seconds:: In multiples of 1, 5, 10 and 30 +minutes:: In multiples of 1, 5, 10 and 30 +hours:: In multiples of 1, 3 and 12 +days:: In multiples of 1, and 7 +months:: In multiples of 1, and 3 +years:: In multiples of 1, 5, 10, 20, 50 and 100 + +In the worst case, where the number of daily buckets are too many for the requested +number of buckets, the number of buckets returned will be 1/7th of the number of +buckets requested. + ==== Time Zone Date-times are stored in Elasticsearch in UTC. By default, all bucketing and diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java index 0a83c5cd350e0..5d61c068b56a2 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java @@ -120,7 +120,7 @@ public int getNumBuckets() { roundings[2] = new RoundingInfo(createRounding(DateTimeUnit.HOUR_OF_DAY), 1, 3, 12); roundings[3] = new RoundingInfo(createRounding(DateTimeUnit.DAY_OF_MONTH), 1, 7); roundings[4] = new RoundingInfo(createRounding(DateTimeUnit.MONTH_OF_YEAR), 1, 3); - roundings[5] = new RoundingInfo(createRounding(DateTimeUnit.YEAR_OF_CENTURY), 1, 10, 20, 50, 100); + roundings[5] = new RoundingInfo(createRounding(DateTimeUnit.YEAR_OF_CENTURY), 1, 5, 10, 20, 50, 100); return new AutoDateHistogramAggregatorFactory(name, config, numBuckets, roundings, context, parent, subFactoriesBuilder, metaData); } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java index d131e4f169d36..dcdbd8b3e695c 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java @@ -614,6 +614,99 @@ public void testIntervalHourWithTZ() throws IOException { ); } + public void testInterval3Hour() throws IOException { + testSearchCase(new MatchAllDocsQuery(), + Arrays.asList( + "2017-02-01T09:02:00.000Z", + "2017-02-01T09:35:00.000Z", + "2017-02-01T10:15:00.000Z", + "2017-02-01T13:06:00.000Z", + "2017-02-01T14:04:00.000Z", + "2017-02-01T14:05:00.000Z", + "2017-02-01T15:59:00.000Z", + "2017-02-01T16:06:00.000Z", + "2017-02-01T16:48:00.000Z", + "2017-02-01T16:59:00.000Z" + ), + aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(10, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-02-01T09:02:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-01T09:35:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-01T10:15:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(3); + assertEquals("2017-02-01T13:06:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(4); + assertEquals("2017-02-01T14:04:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(5); + assertEquals("2017-02-01T14:05:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(6); + assertEquals("2017-02-01T15:59:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(7); + assertEquals("2017-02-01T16:06:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(8); + assertEquals("2017-02-01T16:48:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + + bucket = buckets.get(9); + assertEquals("2017-02-01T16:59:00.000Z", bucket.getKeyAsString()); + assertEquals(1, bucket.getDocCount()); + } + ); + testSearchAndReduceCase(new MatchAllDocsQuery(), + Arrays.asList( + "2017-02-01T09:02:00.000Z", + "2017-02-01T09:35:00.000Z", + "2017-02-01T10:15:00.000Z", + "2017-02-01T13:06:00.000Z", + "2017-02-01T14:04:00.000Z", + "2017-02-01T14:05:00.000Z", + "2017-02-01T15:59:00.000Z", + "2017-02-01T16:06:00.000Z", + "2017-02-01T16:48:00.000Z", + "2017-02-01T16:59:00.000Z" + ), + aggregation -> aggregation.setNumBuckets(6).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(3, buckets.size()); + + Histogram.Bucket bucket = buckets.get(0); + assertEquals("2017-02-01T09:00:00.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + + bucket = buckets.get(1); + assertEquals("2017-02-01T12:00:00.000Z", bucket.getKeyAsString()); + assertEquals(3, bucket.getDocCount()); + + bucket = buckets.get(2); + assertEquals("2017-02-01T15:00:00.000Z", bucket.getKeyAsString()); + assertEquals(4, bucket.getDocCount()); + } + ); + } + public void testIntervalMinute() throws IOException { testSearchCase(new MatchAllDocsQuery(), Arrays.asList( From fa6c841ab752a6c87415cd72557cd15b8a74d328 Mon Sep 17 00:00:00 2001 From: Colin Goodheart-Smithe Date: Mon, 26 Mar 2018 10:39:42 +0100 Subject: [PATCH 15/24] Fixes performance bug The bug meant that getAppropriate rounding look a huge amount of time if the range of the data was large but also sparsely populated. In these situations the rounding would be very low so iterating through the rounding values from the min key to the max keey look a long time (~120 seconds in one test). The solution is to add a rough estimate first which chooses the rounding based just on the long values of the min and max keeys alone but selects the rounding one lower than the one it thinks is appropriate so the accurate method can choose the final rounding taking into account the fact that intervals are not always fixed length. Thee commit also adds more tests --- .../AutoDateHistogramAggregationBuilder.java | 22 +- .../histogram/InternalAutoDateHistogram.java | 17 + .../AutoDateHistogramAggregatorTests.java | 346 ++++++++++++++++++ 3 files changed, 378 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java index 5d61c068b56a2..a5f1fe8e4f0a3 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java @@ -115,12 +115,12 @@ public int getNumBuckets() { protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { RoundingInfo[] roundings = new RoundingInfo[6]; - roundings[0] = new RoundingInfo(createRounding(DateTimeUnit.SECOND_OF_MINUTE), 1, 5, 10, 30); - roundings[1] = new RoundingInfo(createRounding(DateTimeUnit.MINUTES_OF_HOUR), 1, 5, 10, 30); - roundings[2] = new RoundingInfo(createRounding(DateTimeUnit.HOUR_OF_DAY), 1, 3, 12); - roundings[3] = new RoundingInfo(createRounding(DateTimeUnit.DAY_OF_MONTH), 1, 7); - roundings[4] = new RoundingInfo(createRounding(DateTimeUnit.MONTH_OF_YEAR), 1, 3); - roundings[5] = new RoundingInfo(createRounding(DateTimeUnit.YEAR_OF_CENTURY), 1, 5, 10, 20, 50, 100); + roundings[0] = new RoundingInfo(createRounding(DateTimeUnit.SECOND_OF_MINUTE), 1000L, 1, 5, 10, 30); + roundings[1] = new RoundingInfo(createRounding(DateTimeUnit.MINUTES_OF_HOUR), 60 * 1000L, 1, 5, 10, 30); + roundings[2] = new RoundingInfo(createRounding(DateTimeUnit.HOUR_OF_DAY), 60 * 60 * 1000L, 1, 3, 12); + roundings[3] = new RoundingInfo(createRounding(DateTimeUnit.DAY_OF_MONTH), 24 * 60 * 60 * 1000L, 1, 7); + roundings[4] = new RoundingInfo(createRounding(DateTimeUnit.MONTH_OF_YEAR), 30 * 24 * 60 * 60 * 1000L, 1, 3); + roundings[5] = new RoundingInfo(createRounding(DateTimeUnit.YEAR_OF_CENTURY), 365 * 24 * 60 * 60 * 1000L, 1, 5, 10, 20, 50, 100); return new AutoDateHistogramAggregatorFactory(name, config, numBuckets, roundings, context, parent, subFactoriesBuilder, metaData); } @@ -153,20 +153,24 @@ protected boolean innerEquals(Object obj) { public static class RoundingInfo implements Writeable { final Rounding rounding; final int[] innerIntervals; + final long roughEstimateDurationMillis; - public RoundingInfo(Rounding rounding, int... innerIntervals) { + public RoundingInfo(Rounding rounding, long roughEstimateDurationMillis, int... innerIntervals) { this.rounding = rounding; + this.roughEstimateDurationMillis = roughEstimateDurationMillis; this.innerIntervals = innerIntervals; } public RoundingInfo(StreamInput in) throws IOException { rounding = Rounding.Streams.read(in); + roughEstimateDurationMillis = in.readVLong(); innerIntervals = in.readIntArray(); } @Override public void writeTo(StreamOutput out) throws IOException { Rounding.Streams.write(rounding, out); + out.writeVLong(roughEstimateDurationMillis); out.writeIntArray(innerIntervals); } @@ -174,6 +178,10 @@ public int getMaximumInnerInterval() { return innerIntervals[innerIntervals.length - 1]; } + public long getRoughEstimateDurationMillis() { + return roughEstimateDurationMillis; + } + @Override public int hashCode() { return Objects.hash(rounding, Arrays.hashCode(innerIntervals)); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java index 4f1addb341bb6..c305e9784216e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java @@ -452,6 +452,23 @@ private int getAppropriateRounding(long minKey, long maxKey, int roundingIdx, Ro return roundingIdx; } int currentRoundingIdx = roundingIdx; + + // Getting the accurate number of required buckets can be slow for large + // ranges at low roundings so get a rough estimate of the rounding first + // so we are at most 1 away from the correct rounding and then get the + // accurate rounding value + for (int i = currentRoundingIdx + 1; i < roundings.length; i++) { + long dataDuration = maxKey - minKey; + long roughEstimateRequiredBuckets = dataDuration / roundings[i].getRoughEstimateDurationMillis(); + if (roughEstimateRequiredBuckets < targetBuckets * roundings[i].getMaximumInnerInterval()) { + currentRoundingIdx = i - 1; + break; + } else if (i == roundingIdx - 1) { + currentRoundingIdx = i; + break; + } + } + int requiredBuckets = 0; do { Rounding currentRounding = roundings[currentRoundingIdx].rounding; diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java index dcdbd8b3e695c..a1e863dd6bf25 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java @@ -34,9 +34,15 @@ import org.elasticsearch.search.aggregations.AggregationBuilders; import org.elasticsearch.search.aggregations.AggregatorTestCase; import org.elasticsearch.search.aggregations.metrics.stats.Stats; +import org.hamcrest.Matchers; +import org.joda.time.DateTime; import org.joda.time.DateTimeZone; +import org.joda.time.chrono.ISOChronology; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -614,6 +620,346 @@ public void testIntervalHourWithTZ() throws IOException { ); } + public void testAllSecondIntervals() throws IOException { + DateTimeFormatter format = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss.SSSZ"); + List dataset = new ArrayList<>(); + DateTime startDate = new DateTime(2017, 01, 01, 00, 00, 00, ISOChronology.getInstanceUTC()); + for (int i = 0; i < 600; i++) { + DateTime date = startDate.plusSeconds(i); + dataset.add(format.print(date)); + } + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(600).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(600, buckets.size()); + for (int i = 0; i < 600; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusSeconds(i), bucket.getKey()); + assertEquals(1, bucket.getDocCount()); + } + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(300).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(120, buckets.size()); + for (int i = 0; i < 120; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusSeconds(i * 5), bucket.getKey()); + assertEquals(5, bucket.getDocCount()); + } + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(100).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(60, buckets.size()); + for (int i = 0; i < 60; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusSeconds(i * 10), bucket.getKey()); + assertEquals(10, bucket.getDocCount()); + } + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(50).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(20, buckets.size()); + for (int i = 0; i < 20; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusSeconds(i * 30), bucket.getKey()); + assertEquals(30, bucket.getDocCount()); + } + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(15).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(10, buckets.size()); + for (int i = 0; i < 10; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusMinutes(i), bucket.getKey()); + assertEquals(60, bucket.getDocCount()); + } + }); + } + + public void testAllMinuteIntervals() throws IOException { + DateTimeFormatter format = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss.SSSZ"); + List dataset = new ArrayList<>(); + DateTime startDate = new DateTime(2017, 01, 01, 00, 00, 00, ISOChronology.getInstanceUTC()); + for (int i = 0; i < 600; i++) { + DateTime date = startDate.plusMinutes(i); + dataset.add(format.print(date)); + } + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(600).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(600, buckets.size()); + for (int i = 0; i < 600; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusMinutes(i), bucket.getKey()); + assertEquals(1, bucket.getDocCount()); + } + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(300).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(120, buckets.size()); + for (int i = 0; i < 120; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusMinutes(i * 5), bucket.getKey()); + assertEquals(5, bucket.getDocCount()); + } + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(100).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(60, buckets.size()); + for (int i = 0; i < 60; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusMinutes(i * 10), bucket.getKey()); + assertEquals(10, bucket.getDocCount()); + } + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(50).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(20, buckets.size()); + for (int i = 0; i < 20; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusMinutes(i * 30), bucket.getKey()); + assertEquals(30, bucket.getDocCount()); + } + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(15).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(10, buckets.size()); + for (int i = 0; i < 10; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusHours(i), bucket.getKey()); + assertEquals(60, bucket.getDocCount()); + } + }); + } + + public void testAllHourIntervals() throws IOException { + DateTimeFormatter format = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss.SSSZ"); + List dataset = new ArrayList<>(); + DateTime startDate = new DateTime(2017, 01, 01, 00, 00, 00, ISOChronology.getInstanceUTC()); + for (int i = 0; i < 600; i++) { + DateTime date = startDate.plusHours(i); + dataset.add(format.print(date)); + } + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(600).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(600, buckets.size()); + for (int i = 0; i < 600; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusHours(i), bucket.getKey()); + assertEquals(1, bucket.getDocCount()); + } + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(300).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(200, buckets.size()); + for (int i = 0; i < 200; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusHours(i * 3), bucket.getKey()); + assertEquals(3, bucket.getDocCount()); + } + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(100).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(50, buckets.size()); + for (int i = 0; i < 50; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusHours(i * 12), bucket.getKey()); + assertEquals(12, bucket.getDocCount()); + } + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(30).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(25, buckets.size()); + for (int i = 0; i < 25; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusDays(i), bucket.getKey()); + assertEquals(24, bucket.getDocCount()); + } + }); + } + + public void testAllDayIntervals() throws IOException { + DateTimeFormatter format = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss.SSSZ"); + List dataset = new ArrayList<>(); + DateTime startDate = new DateTime(2017, 01, 01, 00, 00, 00, ISOChronology.getInstanceUTC()); + for (int i = 0; i < 700; i++) { + DateTime date = startDate.plusDays(i); + dataset.add(format.print(date)); + } + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(700).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(700, buckets.size()); + for (int i = 0; i < 700; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusDays(i), bucket.getKey()); + assertEquals(1, bucket.getDocCount()); + } + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(300).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(100, buckets.size()); + for (int i = 0; i < 100; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusDays(i * 7), bucket.getKey()); + assertEquals(7, bucket.getDocCount()); + } + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(30).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(24, buckets.size()); + for (int i = 0; i < 24; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusMonths(i), bucket.getKey()); + assertThat(bucket.getDocCount(), Matchers.lessThanOrEqualTo(31L)); + } + }); + } + + public void testAllMonthIntervals() throws IOException { + DateTimeFormatter format = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss.SSSZ"); + List dataset = new ArrayList<>(); + DateTime startDate = new DateTime(2017, 01, 01, 00, 00, 00, ISOChronology.getInstanceUTC()); + for (int i = 0; i < 600; i++) { + DateTime date = startDate.plusMonths(i); + dataset.add(format.print(date)); + } + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(600).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(600, buckets.size()); + for (int i = 0; i < 600; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusMonths(i), bucket.getKey()); + assertEquals(1, bucket.getDocCount()); + } + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(300).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(200, buckets.size()); + for (int i = 0; i < 200; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusMonths(i * 3), bucket.getKey()); + assertEquals(3, bucket.getDocCount()); + } + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + aggregation -> aggregation.setNumBuckets(60).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(50, buckets.size()); + for (int i = 0; i < 50; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusYears(i), bucket.getKey()); + assertEquals(12, bucket.getDocCount()); + } + }); + } + + public void testAllYearIntervals() throws IOException { + DateTimeFormatter format = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss.SSSZ"); + List dataset = new ArrayList<>(); + DateTime startDate = new DateTime(2017, 01, 01, 00, 00, 00, ISOChronology.getInstanceUTC()); + for (int i = 0; i < 600; i++) { + DateTime date = startDate.plusYears(i); + dataset.add(format.print(date)); + } + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(600).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(600, buckets.size()); + for (int i = 0; i < 600; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusYears(i), bucket.getKey()); + assertEquals(1, bucket.getDocCount()); + } + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(300).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(120, buckets.size()); + for (int i = 0; i < 120; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusYears(i * 5), bucket.getKey()); + assertEquals(5, bucket.getDocCount()); + } + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(100).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(60, buckets.size()); + for (int i = 0; i < 60; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusYears(i * 10), bucket.getKey()); + assertEquals(10, bucket.getDocCount()); + } + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(50).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(30, buckets.size()); + for (int i = 0; i < 30; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusYears(i * 20), bucket.getKey()); + assertEquals(20, bucket.getDocCount()); + } + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(20).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(12, buckets.size()); + for (int i = 0; i < 12; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusYears(i * 50), bucket.getKey()); + assertEquals(50, bucket.getDocCount()); + } + }); + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(10).field(DATE_FIELD), + histogram -> { + List buckets = histogram.getBuckets(); + assertEquals(6, buckets.size()); + for (int i = 0; i < 6; i++) { + Histogram.Bucket bucket = buckets.get(i); + assertEquals(startDate.plusYears(i * 100), bucket.getKey()); + assertEquals(100, bucket.getDocCount()); + } + }); + } + public void testInterval3Hour() throws IOException { testSearchCase(new MatchAllDocsQuery(), Arrays.asList( From c3a713421feb85aebf05bbe344707061b8eb681d Mon Sep 17 00:00:00 2001 From: Colin Goodheart-Smithe Date: Mon, 26 Mar 2018 11:48:39 +0100 Subject: [PATCH 16/24] Changes to only do complex reduction on final reduce --- .../histogram/InternalAutoDateHistogram.java | 20 ++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java index c305e9784216e..3f658204a86a5 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java @@ -490,15 +490,17 @@ private int getAppropriateRounding(long minKey, long maxKey, int roundingIdx, Ro public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { BucketReduceResult reducedBucketsResult = reduceBuckets(aggregations, reduceContext); - // adding empty buckets if needed - reducedBucketsResult = addEmptyBuckets(reducedBucketsResult, reduceContext); - - // Adding empty buckets may have tipped us over the target so merge the buckets again if needed - reducedBucketsResult = mergeBucketsIfNeeded(reducedBucketsResult.buckets, reducedBucketsResult.roundingIdx, - reducedBucketsResult.roundingInfo, reduceContext); - - // Now finally see if we need to merge consecutive buckets together to make a coarser interval at the same rounding - reducedBucketsResult = maybeMergeConsecutiveBuckets(reducedBucketsResult, reduceContext); + if (reduceContext.isFinalReduce()) { + // adding empty buckets if needed + reducedBucketsResult = addEmptyBuckets(reducedBucketsResult, reduceContext); + + // Adding empty buckets may have tipped us over the target so merge the buckets again if needed + reducedBucketsResult = mergeBucketsIfNeeded(reducedBucketsResult.buckets, reducedBucketsResult.roundingIdx, + reducedBucketsResult.roundingInfo, reduceContext); + + // Now finally see if we need to merge consecutive buckets together to make a coarser interval at the same rounding + reducedBucketsResult = maybeMergeConsecutiveBuckets(reducedBucketsResult, reduceContext); + } BucketInfo bucketInfo = new BucketInfo(this.bucketInfo.roundingInfos, reducedBucketsResult.roundingIdx, this.bucketInfo.emptySubAggregations); From 42fc64b0a75f9ba9844dbe52977020ea78a4f678 Mon Sep 17 00:00:00 2001 From: Paul Sanwald Date: Mon, 18 Jun 2018 16:50:43 -0500 Subject: [PATCH 17/24] merge latest with master --- .../MergingBucketsDeferringCollector.java | 230 ------ .../AutoDateHistogramAggregationBuilder.java | 137 ---- .../AutoDateHistogramAggregator.java | 196 ----- .../AutoDateHistogramAggregatorFactory.java | 72 -- .../histogram/InternalAutoDateHistogram.java | 527 -------------- .../histogram/ParsedAutoDateHistogram.java | 91 --- .../bucket/AutoDateHistogramTests.java | 44 -- .../AutoDateHistogramAggregatorTests.java | 669 ------------------ .../InternalAutoDateHistogramTests.java | 153 ---- .../AutoDateHistogramAggregator.java | 2 +- .../histogram/InternalAutoDateHistogram.java | 8 +- 11 files changed, 5 insertions(+), 2124 deletions(-) delete mode 100644 core/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java delete mode 100644 core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java delete mode 100644 core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java delete mode 100644 core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java delete mode 100644 core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java delete mode 100644 core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/ParsedAutoDateHistogram.java delete mode 100644 core/src/test/java/org/elasticsearch/search/aggregations/bucket/AutoDateHistogramTests.java delete mode 100644 core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java delete mode 100644 core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java deleted file mode 100644 index 1d4c9f08f6d11..0000000000000 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java +++ /dev/null @@ -1,230 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.search.aggregations.bucket; - -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.search.Scorer; -import org.apache.lucene.search.Weight; -import org.apache.lucene.util.packed.PackedInts; -import org.apache.lucene.util.packed.PackedLongValues; -import org.elasticsearch.common.util.BigArrays; -import org.elasticsearch.common.util.LongHash; -import org.elasticsearch.search.aggregations.Aggregator; -import org.elasticsearch.search.aggregations.BucketCollector; -import org.elasticsearch.search.aggregations.InternalAggregation; -import org.elasticsearch.search.aggregations.LeafBucketCollector; -import org.elasticsearch.search.internal.SearchContext; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -public class MergingBucketsDeferringCollector extends DeferringBucketCollector { - - List entries = new ArrayList<>(); - BucketCollector collector; - final SearchContext searchContext; - LeafReaderContext context; - PackedLongValues.Builder docDeltas; - PackedLongValues.Builder buckets; - long maxBucket = -1; - boolean finished = false; - LongHash selectedBuckets; - - public MergingBucketsDeferringCollector(SearchContext context) { - this.searchContext = context; - } - - @Override - public void setDeferredCollector(Iterable deferredCollectors) { - this.collector = BucketCollector.wrap(deferredCollectors); - } - - @Override - public boolean needsScores() { - if (collector == null) { - throw new IllegalStateException(); - } - return collector.needsScores(); - } - - @Override - public void preCollection() throws IOException { - collector.preCollection(); - } - - private void finishLeaf() { - if (context != null) { - entries.add(new Entry(context, docDeltas.build(), buckets.build())); - } - context = null; - docDeltas = null; - buckets = null; - } - - @Override - public LeafBucketCollector getLeafCollector(LeafReaderContext ctx) throws IOException { - finishLeaf(); - - context = ctx; - docDeltas = PackedLongValues.packedBuilder(PackedInts.DEFAULT); - buckets = PackedLongValues.packedBuilder(PackedInts.DEFAULT); - - return new LeafBucketCollector() { - int lastDoc = 0; - - @Override - public void collect(int doc, long bucket) throws IOException { - docDeltas.add(doc - lastDoc); - buckets.add(bucket); - lastDoc = doc; - maxBucket = Math.max(maxBucket, bucket); - } - }; - } - - public void mergeBuckets(long[] mergeMap) { - - List newEntries = new ArrayList<>(); - for (Entry sourceEntry : entries) { - PackedLongValues.Builder newBuckets = PackedLongValues.packedBuilder(PackedInts.DEFAULT); - for (PackedLongValues.Iterator itr = sourceEntry.buckets.iterator(); itr.hasNext();) { - long bucket = itr.next(); - newBuckets.add(mergeMap[(int) bucket]); - } - newEntries.add(new Entry(sourceEntry.context, sourceEntry.docDeltas, newBuckets.build())); - } - entries = newEntries; - - // if there are buckets that have been collected in the current segment - // we need to update the bucket ordinals there too - if (buckets.size() > 0) { - PackedLongValues currentBuckets = buckets.build(); - PackedLongValues.Builder newBuckets = PackedLongValues.packedBuilder(PackedInts.DEFAULT); - for (PackedLongValues.Iterator itr = currentBuckets.iterator(); itr.hasNext();) { - long bucket = itr.next(); - newBuckets.add(mergeMap[(int) bucket]); - } - buckets = newBuckets; - } - } - - @Override - public void postCollection() throws IOException { - finishLeaf(); - finished = true; - } - - /** - * Replay the wrapped collector, but only on a selection of buckets. - */ - @Override - public void prepareSelectedBuckets(long... selectedBuckets) throws IOException { - if (!finished) { - throw new IllegalStateException("Cannot replay yet, collection is not finished: postCollect() has not been called"); - } - if (this.selectedBuckets != null) { - throw new IllegalStateException("Already been replayed"); - } - - final LongHash hash = new LongHash(selectedBuckets.length, BigArrays.NON_RECYCLING_INSTANCE); - for (long bucket : selectedBuckets) { - hash.add(bucket); - } - this.selectedBuckets = hash; - - boolean needsScores = collector.needsScores(); - Weight weight = null; - if (needsScores) { - weight = searchContext.searcher().createNormalizedWeight(searchContext.query(), true); - } - for (Entry entry : entries) { - final LeafBucketCollector leafCollector = collector.getLeafCollector(entry.context); - DocIdSetIterator docIt = null; - if (needsScores && entry.docDeltas.size() > 0) { - Scorer scorer = weight.scorer(entry.context); - // We don't need to check if the scorer is null - // since we are sure that there are documents to replay - // (entry.docDeltas it not empty). - docIt = scorer.iterator(); - leafCollector.setScorer(scorer); - } - final PackedLongValues.Iterator docDeltaIterator = entry.docDeltas.iterator(); - final PackedLongValues.Iterator buckets = entry.buckets.iterator(); - int doc = 0; - for (long i = 0, end = entry.docDeltas.size(); i < end; ++i) { - doc += docDeltaIterator.next(); - final long bucket = buckets.next(); - final long rebasedBucket = hash.find(bucket); - if (rebasedBucket != -1) { - if (needsScores) { - if (docIt.docID() < doc) { - docIt.advance(doc); - } - // aggregations should only be replayed on matching - // documents - assert docIt.docID() == doc; - } - leafCollector.collect(doc, rebasedBucket); - } - } - } - - collector.postCollection(); - } - - /** - * Wrap the provided aggregator so that it behaves (almost) as if it had - * been collected directly. - */ - @Override - public Aggregator wrap(final Aggregator in) { - - return new WrappedAggregator(in) { - - @Override - public InternalAggregation buildAggregation(long bucket) throws IOException { - if (selectedBuckets == null) { - throw new IllegalStateException("Collection has not been replayed yet."); - } - final long rebasedBucket = selectedBuckets.find(bucket); - if (rebasedBucket == -1) { - throw new IllegalStateException("Cannot build for a bucket which has not been collected [" + bucket + "]"); - } - return in.buildAggregation(rebasedBucket); - } - - }; - } - - private static class Entry { - final LeafReaderContext context; - final PackedLongValues docDeltas; - final PackedLongValues buckets; - - Entry(LeafReaderContext context, PackedLongValues docDeltas, PackedLongValues buckets) { - this.context = context; - this.docDeltas = docDeltas; - this.buckets = buckets; - } - } - -} diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java deleted file mode 100644 index 0194ca6789f45..0000000000000 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java +++ /dev/null @@ -1,137 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.search.aggregations.bucket.histogram; - -import org.elasticsearch.common.ParseField; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.rounding.DateTimeUnit; -import org.elasticsearch.common.rounding.Rounding; -import org.elasticsearch.common.xcontent.ObjectParser; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; -import org.elasticsearch.search.aggregations.AggregatorFactory; -import org.elasticsearch.search.aggregations.support.ValueType; -import org.elasticsearch.search.aggregations.support.ValuesSource; -import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; -import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder; -import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; -import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; -import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper; -import org.elasticsearch.search.aggregations.support.ValuesSourceType; -import org.elasticsearch.search.internal.SearchContext; - -import java.io.IOException; -import java.util.Objects; - -public class AutoDateHistogramAggregationBuilder - extends ValuesSourceAggregationBuilder { - - public static final String NAME = "auto_date_histogram"; - - public static final ParseField NUM_BUCKETS_FIELD = new ParseField("buckets"); - - private static final ObjectParser PARSER; - static { - PARSER = new ObjectParser<>(AutoDateHistogramAggregationBuilder.NAME); - ValuesSourceParserHelper.declareNumericFields(PARSER, true, true, true); - - PARSER.declareInt(AutoDateHistogramAggregationBuilder::setNumBuckets, NUM_BUCKETS_FIELD); - } - - public static AutoDateHistogramAggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException { - return PARSER.parse(parser, new AutoDateHistogramAggregationBuilder(aggregationName), null); - } - - private int numBuckets = 10; - - /** Create a new builder with the given name. */ - public AutoDateHistogramAggregationBuilder(String name) { - super(name, ValuesSourceType.NUMERIC, ValueType.DATE); - } - - /** Read from a stream, for internal use only. */ - public AutoDateHistogramAggregationBuilder(StreamInput in) throws IOException { - super(in, ValuesSourceType.NUMERIC, ValueType.DATE); - numBuckets = in.readVInt(); - } - - @Override - protected void innerWriteTo(StreamOutput out) throws IOException { - out.writeVInt(numBuckets); - } - - @Override - public String getType() { - return NAME; - } - - public AutoDateHistogramAggregationBuilder setNumBuckets(int numBuckets) { - if (numBuckets <= 0) { - throw new IllegalArgumentException(NUM_BUCKETS_FIELD.getPreferredName() + " must be greater than 0 for [" + name + "]"); - } - this.numBuckets = numBuckets; - return this; - } - - public int getNumBuckets() { - return numBuckets; - } - - @Override - protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { - Rounding[] roundings = new Rounding[6]; - roundings[0] = createRounding(DateTimeUnit.SECOND_OF_MINUTE); - roundings[1] = createRounding(DateTimeUnit.MINUTES_OF_HOUR); - roundings[2] = createRounding(DateTimeUnit.HOUR_OF_DAY); - roundings[3] = createRounding(DateTimeUnit.DAY_OF_MONTH); - roundings[4] = createRounding(DateTimeUnit.MONTH_OF_YEAR); - roundings[5] = createRounding(DateTimeUnit.YEAR_OF_CENTURY); - return new AutoDateHistogramAggregatorFactory(name, config, numBuckets, roundings, context, parent, subFactoriesBuilder, metaData); - } - - private Rounding createRounding(DateTimeUnit interval) { - Rounding.Builder tzRoundingBuilder = Rounding.builder(interval); - if (timeZone() != null) { - tzRoundingBuilder.timeZone(timeZone()); - } - Rounding rounding = tzRoundingBuilder.build(); - return rounding; - } - - @Override - protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { - builder.field(NUM_BUCKETS_FIELD.getPreferredName(), numBuckets); - return builder; - } - - @Override - protected int innerHashCode() { - return Objects.hash(numBuckets); - } - - @Override - protected boolean innerEquals(Object obj) { - AutoDateHistogramAggregationBuilder other = (AutoDateHistogramAggregationBuilder) obj; - return Objects.equals(numBuckets, other.numBuckets); - } -} diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java deleted file mode 100644 index cf2531fca6394..0000000000000 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java +++ /dev/null @@ -1,196 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.elasticsearch.search.aggregations.bucket.histogram; - -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.SortedNumericDocValues; -import org.apache.lucene.util.CollectionUtil; -import org.elasticsearch.common.inject.internal.Nullable; -import org.elasticsearch.common.lease.Releasables; -import org.elasticsearch.common.rounding.Rounding; -import org.elasticsearch.common.util.LongHash; -import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.aggregations.Aggregator; -import org.elasticsearch.search.aggregations.AggregatorFactories; -import org.elasticsearch.search.aggregations.BucketOrder; -import org.elasticsearch.search.aggregations.InternalAggregation; -import org.elasticsearch.search.aggregations.LeafBucketCollector; -import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; -import org.elasticsearch.search.aggregations.bucket.DeferableBucketAggregator; -import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector; -import org.elasticsearch.search.aggregations.bucket.MergingBucketsDeferringCollector; -import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; -import org.elasticsearch.search.aggregations.support.ValuesSource; -import org.elasticsearch.search.internal.SearchContext; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -/** - * An aggregator for date values. Every date is rounded down using a configured - * {@link Rounding}. - * - * @see Rounding - */ -class AutoDateHistogramAggregator extends DeferableBucketAggregator { - - private final ValuesSource.Numeric valuesSource; - private final DocValueFormat formatter; - private final Rounding[] roundings; - private int roundingIdx = 0; - - private LongHash bucketOrds; - private int targetBuckets; - private MergingBucketsDeferringCollector deferringCollector; - - AutoDateHistogramAggregator(String name, AggregatorFactories factories, int numBuckets, Rounding[] roundings, - @Nullable ValuesSource.Numeric valuesSource, DocValueFormat formatter, SearchContext aggregationContext, Aggregator parent, - List pipelineAggregators, Map metaData) throws IOException { - - super(name, factories, aggregationContext, parent, pipelineAggregators, metaData); - this.targetBuckets = numBuckets; - this.valuesSource = valuesSource; - this.formatter = formatter; - this.roundings = roundings; - - bucketOrds = new LongHash(1, aggregationContext.bigArrays()); - - } - - @Override - public boolean needsScores() { - return (valuesSource != null && valuesSource.needsScores()) || super.needsScores(); - } - - @Override - protected boolean shouldDefer(Aggregator aggregator) { - return true; - } - - @Override - public DeferringBucketCollector getDeferringCollector() { - deferringCollector = new MergingBucketsDeferringCollector(context); - return deferringCollector; - } - - @Override - public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, - final LeafBucketCollector sub) throws IOException { - if (valuesSource == null) { - return LeafBucketCollector.NO_OP_COLLECTOR; - } - final SortedNumericDocValues values = valuesSource.longValues(ctx); - return new LeafBucketCollectorBase(sub, values) { - @Override - public void collect(int doc, long bucket) throws IOException { - assert bucket == 0; - if (values.advanceExact(doc)) { - final int valuesCount = values.docValueCount(); - - long previousRounded = Long.MIN_VALUE; - for (int i = 0; i < valuesCount; ++i) { - long value = values.nextValue(); - long rounded = roundings[roundingIdx].round(value); - assert rounded >= previousRounded; - if (rounded == previousRounded) { - continue; - } - long bucketOrd = bucketOrds.add(rounded); - if (bucketOrd < 0) { // already seen - bucketOrd = -1 - bucketOrd; - collectExistingBucket(sub, doc, bucketOrd); - } else { - collectBucket(sub, doc, bucketOrd); - while (bucketOrds.size() > targetBuckets) { - increaseRounding(); - } - } - previousRounded = rounded; - } - } - } - - private void increaseRounding() { - try (LongHash oldBucketOrds = bucketOrds) { - LongHash newBucketOrds = new LongHash(1, context.bigArrays()); - long[] mergeMap = new long[(int) oldBucketOrds.size()]; - Rounding newRounding = roundings[++roundingIdx]; - for (int i = 0; i < oldBucketOrds.size(); i++) { - long oldKey = oldBucketOrds.get(i); - long newKey = newRounding.round(oldKey); - long newBucketOrd = newBucketOrds.add(newKey); - if (newBucketOrd >= 0) { - mergeMap[i] = newBucketOrd; - } else { - mergeMap[i] = -1 - newBucketOrd; - } - } - mergeBuckets(mergeMap, newBucketOrds.size()); - if (deferringCollector != null) { - deferringCollector.mergeBuckets(mergeMap); - } - bucketOrds = newBucketOrds; - } - } - }; - } - - @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - assert owningBucketOrdinal == 0; - - long[] bucketOrdArray = new long[(int) bucketOrds.size()]; - for (int i = 0; i < bucketOrds.size(); i++) { - bucketOrdArray[i] = i; - } - - runDeferredCollections(bucketOrdArray); - - List buckets = new ArrayList<>((int) bucketOrds.size()); - for (long i = 0; i < bucketOrds.size(); i++) { - buckets.add(new InternalAutoDateHistogram.Bucket(bucketOrds.get(i), bucketDocCount(i), formatter, bucketAggregations(i))); - } - - // the contract of the histogram aggregation is that shards must return - // buckets ordered by key in ascending order - CollectionUtil.introSort(buckets, BucketOrder.key(true).comparator(this)); - - // value source will be null for unmapped fields - InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundings, roundingIdx, - buildEmptySubAggregations()); - - return new InternalAutoDateHistogram(name, buckets, targetBuckets, emptyBucketInfo, formatter, pipelineAggregators(), metaData()); - } - - @Override - public InternalAggregation buildEmptyAggregation() { - InternalAutoDateHistogram.BucketInfo emptyBucketInfo = new InternalAutoDateHistogram.BucketInfo(roundings, roundingIdx, - buildEmptySubAggregations()); - return new InternalAutoDateHistogram(name, Collections.emptyList(), targetBuckets, emptyBucketInfo, formatter, - pipelineAggregators(), metaData()); - } - - @Override - public void doClose() { - Releasables.close(bucketOrds); - } -} diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java deleted file mode 100644 index 18ce727204151..0000000000000 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorFactory.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.search.aggregations.bucket.histogram; - -import org.elasticsearch.common.rounding.Rounding; -import org.elasticsearch.search.aggregations.Aggregator; -import org.elasticsearch.search.aggregations.AggregatorFactories; -import org.elasticsearch.search.aggregations.AggregatorFactory; -import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; -import org.elasticsearch.search.aggregations.support.ValuesSource; -import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; -import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; -import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; -import org.elasticsearch.search.internal.SearchContext; - -import java.io.IOException; -import java.util.List; -import java.util.Map; - -public final class AutoDateHistogramAggregatorFactory - extends ValuesSourceAggregatorFactory { - - private final int numBuckets; - private Rounding[] roundings; - - public AutoDateHistogramAggregatorFactory(String name, ValuesSourceConfig config, int numBuckets, Rounding[] roundings, - SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, - Map metaData) throws IOException { - super(name, config, context, parent, subFactoriesBuilder, metaData); - this.numBuckets = numBuckets; - this.roundings = roundings; - } - - @Override - protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, Aggregator parent, boolean collectsFromSingleBucket, - List pipelineAggregators, Map metaData) throws IOException { - if (collectsFromSingleBucket == false) { - return asMultiBucketAggregator(this, context, parent); - } - return createAggregator(valuesSource, parent, pipelineAggregators, metaData); - } - - private Aggregator createAggregator(ValuesSource.Numeric valuesSource, Aggregator parent, List pipelineAggregators, - Map metaData) throws IOException { - return new AutoDateHistogramAggregator(name, factories, numBuckets, roundings, valuesSource, config.format(), context, parent, - pipelineAggregators, - metaData); - } - - @Override - protected Aggregator createUnmapped(Aggregator parent, List pipelineAggregators, Map metaData) - throws IOException { - return createAggregator(null, parent, pipelineAggregators, metaData); - } -} diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java deleted file mode 100644 index 4b17da4283d16..0000000000000 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java +++ /dev/null @@ -1,527 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.elasticsearch.search.aggregations.bucket.histogram; - -import org.apache.lucene.util.PriorityQueue; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.rounding.Rounding; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.aggregations.Aggregations; -import org.elasticsearch.search.aggregations.InternalAggregation; -import org.elasticsearch.search.aggregations.InternalAggregations; -import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; -import org.elasticsearch.search.aggregations.KeyComparable; -import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation; -import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.ListIterator; -import java.util.Map; -import java.util.Objects; - -/** - * Implementation of {@link Histogram}. - */ -public final class InternalAutoDateHistogram extends - InternalMultiBucketAggregation implements Histogram, HistogramFactory { - - public static class Bucket extends InternalMultiBucketAggregation.InternalBucket implements Histogram.Bucket, KeyComparable { - - final long key; - final long docCount; - final InternalAggregations aggregations; - protected final transient DocValueFormat format; - - public Bucket(long key, long docCount, DocValueFormat format, - InternalAggregations aggregations) { - this.format = format; - this.key = key; - this.docCount = docCount; - this.aggregations = aggregations; - } - - /** - * Read from a stream. - */ - public Bucket(StreamInput in, DocValueFormat format) throws IOException { - this.format = format; - key = in.readLong(); - docCount = in.readVLong(); - aggregations = InternalAggregations.readAggregations(in); - } - - @Override - public boolean equals(Object obj) { - if (obj == null || obj.getClass() != InternalAutoDateHistogram.Bucket.class) { - return false; - } - InternalAutoDateHistogram.Bucket that = (InternalAutoDateHistogram.Bucket) obj; - // No need to take the keyed and format parameters into account, - // they are already stored and tested on the InternalDateHistogram object - return key == that.key - && docCount == that.docCount - && Objects.equals(aggregations, that.aggregations); - } - - @Override - public int hashCode() { - return Objects.hash(getClass(), key, docCount, aggregations); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeLong(key); - out.writeVLong(docCount); - aggregations.writeTo(out); - } - - @Override - public String getKeyAsString() { - return format.format(key); - } - - @Override - public Object getKey() { - return new DateTime(key, DateTimeZone.UTC); - } - - @Override - public long getDocCount() { - return docCount; - } - - @Override - public Aggregations getAggregations() { - return aggregations; - } - - Bucket reduce(List buckets, Rounding rounding, ReduceContext context) { - List aggregations = new ArrayList<>(buckets.size()); - long docCount = 0; - for (Bucket bucket : buckets) { - docCount += bucket.docCount; - aggregations.add((InternalAggregations) bucket.getAggregations()); - } - InternalAggregations aggs = InternalAggregations.reduce(aggregations, context); - return new InternalAutoDateHistogram.Bucket(rounding.round(key), docCount, format, aggs); - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - String keyAsString = format.format(key); - builder.startObject(); - if (format != DocValueFormat.RAW) { - builder.field(CommonFields.KEY_AS_STRING.getPreferredName(), keyAsString); - } - builder.field(CommonFields.KEY.getPreferredName(), key); - builder.field(CommonFields.DOC_COUNT.getPreferredName(), docCount); - aggregations.toXContentInternal(builder, params); - builder.endObject(); - return builder; - } - - @Override - public int compareKey(Bucket other) { - return Long.compare(key, other.key); - } - - public DocValueFormat getFormatter() { - return format; - } - } - - static class BucketInfo { - - final Rounding[] roundings; - final int roundingIdx; - final InternalAggregations emptySubAggregations; - - BucketInfo(Rounding[] roundings, int roundingIdx, InternalAggregations subAggregations) { - this.roundings = roundings; - this.roundingIdx = roundingIdx; - this.emptySubAggregations = subAggregations; - } - - BucketInfo(StreamInput in) throws IOException { - int size = in.readVInt(); - roundings = new Rounding[size]; - for (int i = 0; i < size; i++) { - roundings[i] = Rounding.Streams.read(in); - } - roundingIdx = in.readVInt(); - emptySubAggregations = InternalAggregations.readAggregations(in); - } - - void writeTo(StreamOutput out) throws IOException { - out.writeVInt(roundings.length); - for (Rounding rounding : roundings) { - Rounding.Streams.write(rounding, out); - } - out.writeVInt(roundingIdx); - emptySubAggregations.writeTo(out); - } - - @Override - public boolean equals(Object obj) { - if (obj == null || getClass() != obj.getClass()) { - return false; - } - BucketInfo that = (BucketInfo) obj; - return Objects.deepEquals(roundings, that.roundings) - && Objects.equals(roundingIdx, that.roundingIdx) - && Objects.equals(emptySubAggregations, that.emptySubAggregations); - } - - @Override - public int hashCode() { - return Objects.hash(getClass(), Arrays.hashCode(roundings), roundingIdx, emptySubAggregations); - } - } - - private final List buckets; - private final DocValueFormat format; - private final BucketInfo bucketInfo; - private final int targetBuckets; - - - InternalAutoDateHistogram(String name, List buckets, int targetBuckets, BucketInfo emptyBucketInfo, DocValueFormat formatter, - List pipelineAggregators, Map metaData) { - super(name, pipelineAggregators, metaData); - this.buckets = buckets; - this.bucketInfo = emptyBucketInfo; - this.format = formatter; - this.targetBuckets = targetBuckets; - } - - /** - * Stream from a stream. - */ - public InternalAutoDateHistogram(StreamInput in) throws IOException { - super(in); - bucketInfo = new BucketInfo(in); - format = in.readNamedWriteable(DocValueFormat.class); - buckets = in.readList(stream -> new Bucket(stream, format)); - this.targetBuckets = in.readVInt(); - } - - @Override - protected void doWriteTo(StreamOutput out) throws IOException { - bucketInfo.writeTo(out); - out.writeNamedWriteable(format); - out.writeList(buckets); - out.writeVInt(targetBuckets); - } - - @Override - public String getWriteableName() { - return AutoDateHistogramAggregationBuilder.NAME; - } - - @Override - public List getBuckets() { - return Collections.unmodifiableList(buckets); - } - - DocValueFormat getFormatter() { - return format; - } - - public int getTargetBuckets() { - return targetBuckets; - } - - public BucketInfo getBucketInfo() { - return bucketInfo; - } - - @Override - public InternalAutoDateHistogram create(List buckets) { - return new InternalAutoDateHistogram(name, buckets, targetBuckets, bucketInfo, format, pipelineAggregators(), metaData); - } - - @Override - public Bucket createBucket(InternalAggregations aggregations, Bucket prototype) { - return new Bucket(prototype.key, prototype.docCount, prototype.format, aggregations); - } - - private static class IteratorAndCurrent { - - private final Iterator iterator; - private Bucket current; - - IteratorAndCurrent(Iterator iterator) { - this.iterator = iterator; - current = iterator.next(); - } - - } - - /** - * This method works almost exactly the same as - * InternalDateHistogram#reduceBuckets(List, ReduceContext), the different - * here is that we need to round all the keys we see using the highest level - * rounding returned across all the shards so the resolution of the buckets - * is the same and they can be reduced together. - */ - private BucketReduceResult reduceBuckets(List aggregations, ReduceContext reduceContext) { - - // First we need to find the highest level rounding used across all the - // shards - int reduceRoundingIdx = 0; - for (InternalAggregation aggregation : aggregations) { - int aggRoundingIdx = ((InternalAutoDateHistogram) aggregation).bucketInfo.roundingIdx; - if (aggRoundingIdx > reduceRoundingIdx) { - reduceRoundingIdx = aggRoundingIdx; - } - } - // This rounding will be used to reduce all the buckets - Rounding reduceRounding = bucketInfo.roundings[reduceRoundingIdx]; - - final PriorityQueue pq = new PriorityQueue(aggregations.size()) { - @Override - protected boolean lessThan(IteratorAndCurrent a, IteratorAndCurrent b) { - return a.current.key < b.current.key; - } - }; - for (InternalAggregation aggregation : aggregations) { - InternalAutoDateHistogram histogram = (InternalAutoDateHistogram) aggregation; - if (histogram.buckets.isEmpty() == false) { - pq.add(new IteratorAndCurrent(histogram.buckets.iterator())); - } - } - - List reducedBuckets = new ArrayList<>(); - if (pq.size() > 0) { - // list of buckets coming from different shards that have the same key - List currentBuckets = new ArrayList<>(); - double key = reduceRounding.round(pq.top().current.key); - - do { - final IteratorAndCurrent top = pq.top(); - - if (reduceRounding.round(top.current.key) != key) { - // the key changes, reduce what we already buffered and reset the buffer for current buckets - final Bucket reduced = currentBuckets.get(0).reduce(currentBuckets, reduceRounding, reduceContext); - reducedBuckets.add(reduced); - currentBuckets.clear(); - key = reduceRounding.round(top.current.key); - } - - currentBuckets.add(top.current); - - if (top.iterator.hasNext()) { - final Bucket next = top.iterator.next(); - assert next.key > top.current.key : "shards must return data sorted by key"; - top.current = next; - pq.updateTop(); - } else { - pq.pop(); - } - } while (pq.size() > 0); - - if (currentBuckets.isEmpty() == false) { - final Bucket reduced = currentBuckets.get(0).reduce(currentBuckets, reduceRounding, reduceContext); - reducedBuckets.add(reduced); - } - } - - return mergeBucketsIfNeeded(reducedBuckets, reduceRoundingIdx, reduceRounding, reduceContext); - } - - private BucketReduceResult mergeBucketsIfNeeded(List reducedBuckets, int reduceRoundingIdx, Rounding reduceRounding, - ReduceContext reduceContext) { - while (reducedBuckets.size() > targetBuckets && reduceRoundingIdx < bucketInfo.roundings.length - 1) { - reduceRoundingIdx++; - reduceRounding = bucketInfo.roundings[reduceRoundingIdx]; - reducedBuckets = mergeBuckets(reducedBuckets, reduceRounding, reduceContext); - } - return new BucketReduceResult(reducedBuckets, reduceRounding, reduceRoundingIdx); - } - - private List mergeBuckets(List reducedBuckets, Rounding reduceRounding, ReduceContext reduceContext) { - List mergedBuckets = new ArrayList<>(); - - List sameKeyedBuckets = new ArrayList<>(); - double key = Double.NaN; - for (Bucket bucket : reducedBuckets) { - long roundedBucketKey = reduceRounding.round(bucket.key); - if (Double.isNaN(key)) { - key = roundedBucketKey; - sameKeyedBuckets.add(createBucket(key, bucket.docCount, bucket.aggregations)); - } else if (roundedBucketKey == key) { - sameKeyedBuckets.add(createBucket(key, bucket.docCount, bucket.aggregations)); - } else { - mergedBuckets.add(sameKeyedBuckets.get(0).reduce(sameKeyedBuckets, reduceRounding, reduceContext)); - sameKeyedBuckets.clear(); - key = roundedBucketKey; - sameKeyedBuckets.add(createBucket(key, bucket.docCount, bucket.aggregations)); - } - } - if (sameKeyedBuckets.isEmpty() == false) { - mergedBuckets.add(sameKeyedBuckets.get(0).reduce(sameKeyedBuckets, reduceRounding, reduceContext)); - } - reducedBuckets = mergedBuckets; - return reducedBuckets; - } - - private static class BucketReduceResult { - List buckets; - Rounding rounding; - int roundingIdx; - - BucketReduceResult(List buckets, Rounding rounding, int roundingIdx) { - this.buckets = buckets; - this.rounding = rounding; - this.roundingIdx = roundingIdx; - - } - } - - private BucketReduceResult addEmptyBuckets(BucketReduceResult currentResult, ReduceContext reduceContext) { - List list = currentResult.buckets; - if (list.isEmpty()) { - return currentResult; - } - int roundingIdx = getAppropriateRounding(list.get(0).key, list.get(list.size() - 1).key, currentResult.roundingIdx, - bucketInfo.roundings); - Rounding rounding = bucketInfo.roundings[roundingIdx]; - // merge buckets using the new rounding - list = mergeBuckets(list, rounding, reduceContext); - - Bucket lastBucket = null; - ListIterator iter = list.listIterator(); - InternalAggregations reducedEmptySubAggs = InternalAggregations.reduce(Collections.singletonList(bucketInfo.emptySubAggregations), - reduceContext); - - // Add the empty buckets within the data, - // e.g. if the data series is [1,2,3,7] there're 3 empty buckets that will be created for 4,5,6 - while (iter.hasNext()) { - Bucket nextBucket = list.get(iter.nextIndex()); - if (lastBucket != null) { - long key = rounding.nextRoundingValue(lastBucket.key); - while (key < nextBucket.key) { - iter.add(new InternalAutoDateHistogram.Bucket(key, 0, format, reducedEmptySubAggs)); - key = rounding.nextRoundingValue(key); - } - assert key == nextBucket.key : "key: " + key + ", nextBucket.key: " + nextBucket.key; - } - lastBucket = iter.next(); - } - return new BucketReduceResult(list, rounding, roundingIdx); - } - - private int getAppropriateRounding(long minKey, long maxKey, int roundingIdx, Rounding[] roundings) { - if (roundingIdx == roundings.length - 1) { - return roundingIdx; - } - int currentRoundingIdx = roundingIdx; - int requiredBuckets = 0; - do { - Rounding currentRounding = roundings[currentRoundingIdx]; - long currentKey = minKey; - requiredBuckets = 0; - while (currentKey < maxKey) { - requiredBuckets++; - currentKey = currentRounding.nextRoundingValue(currentKey); - } - currentRoundingIdx++; - } while (requiredBuckets > targetBuckets && currentRoundingIdx < roundings.length); - // The loop will increase past the correct rounding index here so we - // need to subtract one to get the rounding index we need - return currentRoundingIdx - 1; - } - - @Override - public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { - BucketReduceResult reducedBucketsResult = reduceBuckets(aggregations, reduceContext); - - // adding empty buckets if needed - reducedBucketsResult = addEmptyBuckets(reducedBucketsResult, reduceContext); - - // Adding empty buckets may have tipped us over the target so merge the buckets again if needed - reducedBucketsResult = mergeBucketsIfNeeded(reducedBucketsResult.buckets, reducedBucketsResult.roundingIdx, - reducedBucketsResult.rounding, reduceContext); - - BucketInfo bucketInfo = new BucketInfo(this.bucketInfo.roundings, reducedBucketsResult.roundingIdx, - this.bucketInfo.emptySubAggregations); - - return new InternalAutoDateHistogram(getName(), reducedBucketsResult.buckets, targetBuckets, bucketInfo, format, - pipelineAggregators(), getMetaData()); - } - - @Override - public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { - builder.startArray(CommonFields.BUCKETS.getPreferredName()); - for (Bucket bucket : buckets) { - bucket.toXContent(builder, params); - } - builder.endArray(); - return builder; - } - - // HistogramFactory method impls - - @Override - public Number getKey(MultiBucketsAggregation.Bucket bucket) { - return ((Bucket) bucket).key; - } - - @Override - public Number nextKey(Number key) { - return bucketInfo.roundings[bucketInfo.roundingIdx].nextRoundingValue(key.longValue()); - } - - @Override - public InternalAggregation createAggregation(List buckets) { - // convert buckets to the right type - List buckets2 = new ArrayList<>(buckets.size()); - for (Object b : buckets) { - buckets2.add((Bucket) b); - } - buckets2 = Collections.unmodifiableList(buckets2); - return new InternalAutoDateHistogram(name, buckets2, targetBuckets, bucketInfo, format, pipelineAggregators(), getMetaData()); - } - - @Override - public Bucket createBucket(Number key, long docCount, InternalAggregations aggregations) { - return new Bucket(key.longValue(), docCount, format, aggregations); - } - - @Override - protected boolean doEquals(Object obj) { - InternalAutoDateHistogram that = (InternalAutoDateHistogram) obj; - return Objects.equals(buckets, that.buckets) - && Objects.equals(format, that.format) - && Objects.equals(bucketInfo, that.bucketInfo); - } - - @Override - protected int doHashCode() { - return Objects.hash(buckets, format, bucketInfo); - } -} diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/ParsedAutoDateHistogram.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/ParsedAutoDateHistogram.java deleted file mode 100644 index caca44f9f2ea7..0000000000000 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/ParsedAutoDateHistogram.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.search.aggregations.bucket.histogram; - -import org.elasticsearch.common.xcontent.ObjectParser; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.search.aggregations.ParsedMultiBucketAggregation; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; - -import java.io.IOException; -import java.util.List; - -public class ParsedAutoDateHistogram extends ParsedMultiBucketAggregation implements Histogram { - - @Override - public String getType() { - return AutoDateHistogramAggregationBuilder.NAME; - } - - @Override - public List getBuckets() { - return buckets; - } - - private static ObjectParser PARSER = - new ObjectParser<>(ParsedAutoDateHistogram.class.getSimpleName(), true, ParsedAutoDateHistogram::new); - static { - declareMultiBucketAggregationFields(PARSER, - parser -> ParsedBucket.fromXContent(parser, false), - parser -> ParsedBucket.fromXContent(parser, true)); - } - - public static ParsedAutoDateHistogram fromXContent(XContentParser parser, String name) throws IOException { - ParsedAutoDateHistogram aggregation = PARSER.parse(parser, null); - aggregation.setName(name); - return aggregation; - } - - public static class ParsedBucket extends ParsedMultiBucketAggregation.ParsedBucket implements Histogram.Bucket { - - private Long key; - - @Override - public Object getKey() { - if (key != null) { - return new DateTime(key, DateTimeZone.UTC); - } - return null; - } - - @Override - public String getKeyAsString() { - String keyAsString = super.getKeyAsString(); - if (keyAsString != null) { - return keyAsString; - } - if (key != null) { - return Long.toString(key); - } - return null; - } - - @Override - protected XContentBuilder keyToXContent(XContentBuilder builder) throws IOException { - return builder.field(CommonFields.KEY.getPreferredName(), key); - } - - static ParsedBucket fromXContent(XContentParser parser, boolean keyed) throws IOException { - return parseXContent(parser, keyed, ParsedBucket::new, (p, bucket) -> bucket.key = p.longValue()); - } - } -} diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/AutoDateHistogramTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/AutoDateHistogramTests.java deleted file mode 100644 index 3a10edf183376..0000000000000 --- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/AutoDateHistogramTests.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.search.aggregations.bucket; - -import org.elasticsearch.search.aggregations.BaseAggregationTestCase; -import org.elasticsearch.search.aggregations.bucket.histogram.AutoDateHistogramAggregationBuilder; - -public class AutoDateHistogramTests extends BaseAggregationTestCase { - - @Override - protected AutoDateHistogramAggregationBuilder createTestAggregatorBuilder() { - AutoDateHistogramAggregationBuilder builder = new AutoDateHistogramAggregationBuilder(randomAlphaOfLengthBetween(1, 10)); - builder.field(INT_FIELD_NAME); - builder.setNumBuckets(randomIntBetween(1, 100000)); - if (randomBoolean()) { - builder.format("###.##"); - } - if (randomBoolean()) { - builder.missing(randomIntBetween(0, 10)); - } - if (randomBoolean()) { - builder.timeZone(randomDateTimeZone()); - } - return builder; - } - -} diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java deleted file mode 100644 index 4399c5d8237fb..0000000000000 --- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java +++ /dev/null @@ -1,669 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.search.aggregations.bucket.histogram; - -import org.apache.lucene.document.Document; -import org.apache.lucene.document.LongPoint; -import org.apache.lucene.document.SortedNumericDocValuesField; -import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.RandomIndexWriter; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.MatchAllDocsQuery; -import org.apache.lucene.search.MatchNoDocsQuery; -import org.apache.lucene.search.Query; -import org.apache.lucene.store.Directory; -import org.elasticsearch.index.mapper.DateFieldMapper; -import org.elasticsearch.search.aggregations.AggregationBuilders; -import org.elasticsearch.search.aggregations.AggregatorTestCase; -import org.elasticsearch.search.aggregations.metrics.stats.Stats; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.function.Consumer; - -public class AutoDateHistogramAggregatorTests extends AggregatorTestCase { - - private static final String DATE_FIELD = "date"; - private static final String INSTANT_FIELD = "instant"; - - private static final List dataset = Arrays.asList( - "2010-03-12T01:07:45", - "2010-04-27T03:43:34", - "2012-05-18T04:11:00", - "2013-05-29T05:11:31", - "2013-10-31T08:24:05", - "2015-02-13T13:09:32", - "2015-06-24T13:47:43", - "2015-11-13T16:14:34", - "2016-03-04T17:09:50", - "2017-12-12T22:55:46"); - - public void testMatchNoDocs() throws IOException { - testBothCases(new MatchNoDocsQuery(), dataset, - aggregation -> aggregation.setNumBuckets(10).field(DATE_FIELD), - histogram -> assertEquals(0, histogram.getBuckets().size()) - ); - } - - public void testMatchAllDocs() throws IOException { - Query query = new MatchAllDocsQuery(); - - testSearchCase(query, dataset, - aggregation -> aggregation.setNumBuckets(6).field(DATE_FIELD), - histogram -> assertEquals(6, histogram.getBuckets().size()) - ); - testSearchAndReduceCase(query, dataset, - aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD), - histogram -> assertEquals(8, histogram.getBuckets().size()) - ); - } - - public void testSubAggregations() throws IOException { - Query query = new MatchAllDocsQuery(); - - testSearchCase(query, dataset, - aggregation -> aggregation.setNumBuckets(6).field(DATE_FIELD) - .subAggregation(AggregationBuilders.stats("stats").field(DATE_FIELD)), - histogram -> { - List buckets = histogram.getBuckets(); - assertEquals(6, buckets.size()); - - Histogram.Bucket bucket = buckets.get(0); - assertEquals("2010-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); - Stats stats = bucket.getAggregations().get("stats"); - assertEquals("2010-03-12T01:07:45.000Z", stats.getMinAsString()); - assertEquals("2010-04-27T03:43:34.000Z", stats.getMaxAsString()); - assertEquals(2L, stats.getCount()); - - bucket = buckets.get(1); - assertEquals("2012-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - stats = bucket.getAggregations().get("stats"); - assertEquals("2012-05-18T04:11:00.000Z", stats.getMinAsString()); - assertEquals("2012-05-18T04:11:00.000Z", stats.getMaxAsString()); - assertEquals(1L, stats.getCount()); - - bucket = buckets.get(2); - assertEquals("2013-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); - stats = bucket.getAggregations().get("stats"); - assertEquals("2013-05-29T05:11:31.000Z", stats.getMinAsString()); - assertEquals("2013-10-31T08:24:05.000Z", stats.getMaxAsString()); - assertEquals(2L, stats.getCount()); - - bucket = buckets.get(3); - assertEquals("2015-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(3, bucket.getDocCount()); - stats = bucket.getAggregations().get("stats"); - assertEquals("2015-02-13T13:09:32.000Z", stats.getMinAsString()); - assertEquals("2015-11-13T16:14:34.000Z", stats.getMaxAsString()); - assertEquals(3L, stats.getCount()); - - bucket = buckets.get(4); - assertEquals("2016-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - stats = bucket.getAggregations().get("stats"); - assertEquals("2016-03-04T17:09:50.000Z", stats.getMinAsString()); - assertEquals("2016-03-04T17:09:50.000Z", stats.getMaxAsString()); - assertEquals(1L, stats.getCount()); - - bucket = buckets.get(5); - assertEquals("2017-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - stats = bucket.getAggregations().get("stats"); - assertEquals("2017-12-12T22:55:46.000Z", stats.getMinAsString()); - assertEquals("2017-12-12T22:55:46.000Z", stats.getMaxAsString()); - assertEquals(1L, stats.getCount()); - }); - testSearchAndReduceCase(query, dataset, - aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD) - .subAggregation(AggregationBuilders.stats("stats").field(DATE_FIELD)), - histogram -> { - List buckets = histogram.getBuckets(); - assertEquals(8, buckets.size()); - - Histogram.Bucket bucket = buckets.get(0); - assertEquals("2010-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); - Stats stats = bucket.getAggregations().get("stats"); - assertEquals("2010-03-12T01:07:45.000Z", stats.getMinAsString()); - assertEquals("2010-04-27T03:43:34.000Z", stats.getMaxAsString()); - assertEquals(2L, stats.getCount()); - - bucket = buckets.get(1); - assertEquals("2011-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(0, bucket.getDocCount()); - stats = bucket.getAggregations().get("stats"); - assertTrue(Double.isInfinite(stats.getMin())); - assertTrue(Double.isInfinite(stats.getMax())); - assertEquals(0L, stats.getCount()); - - bucket = buckets.get(2); - assertEquals("2012-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - stats = bucket.getAggregations().get("stats"); - assertEquals("2012-05-18T04:11:00.000Z", stats.getMinAsString()); - assertEquals("2012-05-18T04:11:00.000Z", stats.getMaxAsString()); - assertEquals(1L, stats.getCount()); - - bucket = buckets.get(3); - assertEquals("2013-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); - stats = bucket.getAggregations().get("stats"); - assertEquals("2013-05-29T05:11:31.000Z", stats.getMinAsString()); - assertEquals("2013-10-31T08:24:05.000Z", stats.getMaxAsString()); - assertEquals(2L, stats.getCount()); - - bucket = buckets.get(4); - assertEquals("2014-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(0, bucket.getDocCount()); - stats = bucket.getAggregations().get("stats"); - assertTrue(Double.isInfinite(stats.getMin())); - assertTrue(Double.isInfinite(stats.getMax())); - assertEquals(0L, stats.getCount()); - - bucket = buckets.get(5); - assertEquals("2015-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(3, bucket.getDocCount()); - stats = bucket.getAggregations().get("stats"); - assertEquals("2015-02-13T13:09:32.000Z", stats.getMinAsString()); - assertEquals("2015-11-13T16:14:34.000Z", stats.getMaxAsString()); - assertEquals(3L, stats.getCount()); - - bucket = buckets.get(6); - assertEquals("2016-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - stats = bucket.getAggregations().get("stats"); - assertEquals("2016-03-04T17:09:50.000Z", stats.getMinAsString()); - assertEquals("2016-03-04T17:09:50.000Z", stats.getMaxAsString()); - assertEquals(1L, stats.getCount()); - - bucket = buckets.get(7); - assertEquals("2017-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - stats = bucket.getAggregations().get("stats"); - assertEquals("2017-12-12T22:55:46.000Z", stats.getMinAsString()); - assertEquals("2017-12-12T22:55:46.000Z", stats.getMaxAsString()); - assertEquals(1L, stats.getCount()); - }); - } - - public void testNoDocs() throws IOException { - Query query = new MatchNoDocsQuery(); - List dates = Collections.emptyList(); - Consumer aggregation = agg -> agg.setNumBuckets(10).field(DATE_FIELD); - - testSearchCase(query, dates, aggregation, - histogram -> assertEquals(0, histogram.getBuckets().size()) - ); - testSearchAndReduceCase(query, dates, aggregation, - histogram -> assertNull(histogram) - ); - } - - public void testAggregateWrongField() throws IOException { - testBothCases(new MatchAllDocsQuery(), dataset, - aggregation -> aggregation.setNumBuckets(10).field("wrong_field"), - histogram -> assertEquals(0, histogram.getBuckets().size()) - ); - } - - public void testIntervalYear() throws IOException { - testBothCases(LongPoint.newRangeQuery(INSTANT_FIELD, asLong("2015-01-01"), asLong("2017-12-31")), dataset, - aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), - histogram -> { - List buckets = histogram.getBuckets(); - assertEquals(3, buckets.size()); - - Histogram.Bucket bucket = buckets.get(0); - assertEquals("2015-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(3, bucket.getDocCount()); - - bucket = buckets.get(1); - assertEquals("2016-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - - bucket = buckets.get(2); - assertEquals("2017-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - } - ); - } - - public void testIntervalMonth() throws IOException { - testBothCases(new MatchAllDocsQuery(), - Arrays.asList("2017-01-01", "2017-02-02", "2017-02-03", "2017-03-04", "2017-03-05", "2017-03-06"), - aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), - histogram -> { - List buckets = histogram.getBuckets(); - assertEquals(3, buckets.size()); - - Histogram.Bucket bucket = buckets.get(0); - assertEquals("2017-01-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - - bucket = buckets.get(1); - assertEquals("2017-02-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); - - bucket = buckets.get(2); - assertEquals("2017-03-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(3, bucket.getDocCount()); - } - ); - } - - public void testIntervalDay() throws IOException { - testSearchCase(new MatchAllDocsQuery(), - Arrays.asList("2017-02-01", "2017-02-02", "2017-02-02", "2017-02-03", "2017-02-03", "2017-02-03", "2017-02-05"), - aggregation -> aggregation.setNumBuckets(5).field(DATE_FIELD), histogram -> { - List buckets = histogram.getBuckets(); - assertEquals(4, buckets.size()); - - Histogram.Bucket bucket = buckets.get(0); - assertEquals("2017-02-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - - bucket = buckets.get(1); - assertEquals("2017-02-02T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); - - bucket = buckets.get(2); - assertEquals("2017-02-03T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(3, bucket.getDocCount()); - - bucket = buckets.get(3); - assertEquals("2017-02-05T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - }); - testSearchAndReduceCase(new MatchAllDocsQuery(), - Arrays.asList( - "2017-02-01", - "2017-02-02", - "2017-02-02", - "2017-02-03", - "2017-02-03", - "2017-02-03", - "2017-02-05" - ), - aggregation -> aggregation.setNumBuckets(5).field(DATE_FIELD), - histogram -> { - List buckets = histogram.getBuckets(); - assertEquals(5, buckets.size()); - - Histogram.Bucket bucket = buckets.get(0); - assertEquals("2017-02-01T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - - bucket = buckets.get(1); - assertEquals("2017-02-02T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); - - bucket = buckets.get(2); - assertEquals("2017-02-03T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(3, bucket.getDocCount()); - - bucket = buckets.get(3); - assertEquals("2017-02-04T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(0, bucket.getDocCount()); - - bucket = buckets.get(4); - assertEquals("2017-02-05T00:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - } - ); - } - - public void testIntervalHour() throws IOException { - testSearchCase(new MatchAllDocsQuery(), - Arrays.asList( - "2017-02-01T09:02:00.000Z", - "2017-02-01T09:35:00.000Z", - "2017-02-01T10:15:00.000Z", - "2017-02-01T13:06:00.000Z", - "2017-02-01T14:04:00.000Z", - "2017-02-01T14:05:00.000Z", - "2017-02-01T15:59:00.000Z", - "2017-02-01T16:06:00.000Z", - "2017-02-01T16:48:00.000Z", - "2017-02-01T16:59:00.000Z" - ), - aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD), - histogram -> { - List buckets = histogram.getBuckets(); - assertEquals(6, buckets.size()); - - Histogram.Bucket bucket = buckets.get(0); - assertEquals("2017-02-01T09:00:00.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); - - bucket = buckets.get(1); - assertEquals("2017-02-01T10:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - - bucket = buckets.get(2); - assertEquals("2017-02-01T13:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - - bucket = buckets.get(3); - assertEquals("2017-02-01T14:00:00.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); - - bucket = buckets.get(4); - assertEquals("2017-02-01T15:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - - bucket = buckets.get(5); - assertEquals("2017-02-01T16:00:00.000Z", bucket.getKeyAsString()); - assertEquals(3, bucket.getDocCount()); - } - ); - testSearchAndReduceCase(new MatchAllDocsQuery(), - Arrays.asList( - "2017-02-01T09:02:00.000Z", - "2017-02-01T09:35:00.000Z", - "2017-02-01T10:15:00.000Z", - "2017-02-01T13:06:00.000Z", - "2017-02-01T14:04:00.000Z", - "2017-02-01T14:05:00.000Z", - "2017-02-01T15:59:00.000Z", - "2017-02-01T16:06:00.000Z", - "2017-02-01T16:48:00.000Z", - "2017-02-01T16:59:00.000Z" - ), - aggregation -> aggregation.setNumBuckets(10).field(DATE_FIELD), - histogram -> { - List buckets = histogram.getBuckets(); - assertEquals(8, buckets.size()); - - Histogram.Bucket bucket = buckets.get(0); - assertEquals("2017-02-01T09:00:00.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); - - bucket = buckets.get(1); - assertEquals("2017-02-01T10:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - - bucket = buckets.get(2); - assertEquals("2017-02-01T11:00:00.000Z", bucket.getKeyAsString()); - assertEquals(0, bucket.getDocCount()); - - bucket = buckets.get(3); - assertEquals("2017-02-01T12:00:00.000Z", bucket.getKeyAsString()); - assertEquals(0, bucket.getDocCount()); - - bucket = buckets.get(4); - assertEquals("2017-02-01T13:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - - bucket = buckets.get(5); - assertEquals("2017-02-01T14:00:00.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); - - bucket = buckets.get(6); - assertEquals("2017-02-01T15:00:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - - bucket = buckets.get(7); - assertEquals("2017-02-01T16:00:00.000Z", bucket.getKeyAsString()); - assertEquals(3, bucket.getDocCount()); - } - ); - } - - public void testIntervalMinute() throws IOException { - testSearchCase(new MatchAllDocsQuery(), - Arrays.asList( - "2017-02-01T09:02:35.000Z", - "2017-02-01T09:02:59.000Z", - "2017-02-01T09:15:37.000Z", - "2017-02-01T09:16:04.000Z", - "2017-02-01T09:16:42.000Z" - ), - aggregation -> aggregation.setNumBuckets(4).field(DATE_FIELD), - histogram -> { - List buckets = histogram.getBuckets(); - assertEquals(3, buckets.size()); - - Histogram.Bucket bucket = buckets.get(0); - assertEquals("2017-02-01T09:02:00.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); - - bucket = buckets.get(1); - assertEquals("2017-02-01T09:15:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - - bucket = buckets.get(2); - assertEquals("2017-02-01T09:16:00.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); - } - ); - testSearchAndReduceCase(new MatchAllDocsQuery(), - Arrays.asList( - "2017-02-01T09:02:35.000Z", - "2017-02-01T09:02:59.000Z", - "2017-02-01T09:15:37.000Z", - "2017-02-01T09:16:04.000Z", - "2017-02-01T09:16:42.000Z" - ), - aggregation -> aggregation.setNumBuckets(15).field(DATE_FIELD), - histogram -> { - List buckets = histogram.getBuckets(); - assertEquals(15, buckets.size()); - - Histogram.Bucket bucket = buckets.get(0); - assertEquals("2017-02-01T09:02:00.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); - - bucket = buckets.get(1); - assertEquals("2017-02-01T09:03:00.000Z", bucket.getKeyAsString()); - assertEquals(0, bucket.getDocCount()); - - bucket = buckets.get(2); - assertEquals("2017-02-01T09:04:00.000Z", bucket.getKeyAsString()); - assertEquals(0, bucket.getDocCount()); - - bucket = buckets.get(3); - assertEquals("2017-02-01T09:05:00.000Z", bucket.getKeyAsString()); - assertEquals(0, bucket.getDocCount()); - - bucket = buckets.get(4); - assertEquals("2017-02-01T09:06:00.000Z", bucket.getKeyAsString()); - assertEquals(0, bucket.getDocCount()); - - bucket = buckets.get(5); - assertEquals("2017-02-01T09:07:00.000Z", bucket.getKeyAsString()); - assertEquals(0, bucket.getDocCount()); - - bucket = buckets.get(6); - assertEquals("2017-02-01T09:08:00.000Z", bucket.getKeyAsString()); - assertEquals(0, bucket.getDocCount()); - - bucket = buckets.get(7); - assertEquals("2017-02-01T09:09:00.000Z", bucket.getKeyAsString()); - assertEquals(0, bucket.getDocCount()); - - bucket = buckets.get(8); - assertEquals("2017-02-01T09:10:00.000Z", bucket.getKeyAsString()); - assertEquals(0, bucket.getDocCount()); - - bucket = buckets.get(9); - assertEquals("2017-02-01T09:11:00.000Z", bucket.getKeyAsString()); - assertEquals(0, bucket.getDocCount()); - - bucket = buckets.get(10); - assertEquals("2017-02-01T09:12:00.000Z", bucket.getKeyAsString()); - assertEquals(0, bucket.getDocCount()); - - bucket = buckets.get(11); - assertEquals("2017-02-01T09:13:00.000Z", bucket.getKeyAsString()); - assertEquals(0, bucket.getDocCount()); - - bucket = buckets.get(12); - assertEquals("2017-02-01T09:14:00.000Z", bucket.getKeyAsString()); - assertEquals(0, bucket.getDocCount()); - - bucket = buckets.get(13); - assertEquals("2017-02-01T09:15:00.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - - bucket = buckets.get(14); - assertEquals("2017-02-01T09:16:00.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); - } - ); - } - - public void testIntervalSecond() throws IOException { - testSearchCase(new MatchAllDocsQuery(), - Arrays.asList("2017-02-01T00:00:05.015Z", "2017-02-01T00:00:07.299Z", "2017-02-01T00:00:07.074Z", - "2017-02-01T00:00:11.688Z", "2017-02-01T00:00:11.210Z", "2017-02-01T00:00:11.380Z"), - aggregation -> aggregation.setNumBuckets(7).field(DATE_FIELD), histogram -> { - List buckets = histogram.getBuckets(); - assertEquals(3, buckets.size()); - - Histogram.Bucket bucket = buckets.get(0); - assertEquals("2017-02-01T00:00:05.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - - bucket = buckets.get(1); - assertEquals("2017-02-01T00:00:07.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); - - bucket = buckets.get(2); - assertEquals("2017-02-01T00:00:11.000Z", bucket.getKeyAsString()); - assertEquals(3, bucket.getDocCount()); - }); - testSearchAndReduceCase(new MatchAllDocsQuery(), - Arrays.asList( - "2017-02-01T00:00:05.015Z", - "2017-02-01T00:00:07.299Z", - "2017-02-01T00:00:07.074Z", - "2017-02-01T00:00:11.688Z", - "2017-02-01T00:00:11.210Z", - "2017-02-01T00:00:11.380Z" - ), - aggregation -> aggregation.setNumBuckets(7).field(DATE_FIELD), - histogram -> { - List buckets = histogram.getBuckets(); - assertEquals(7, buckets.size()); - - Histogram.Bucket bucket = buckets.get(0); - assertEquals("2017-02-01T00:00:05.000Z", bucket.getKeyAsString()); - assertEquals(1, bucket.getDocCount()); - - bucket = buckets.get(1); - assertEquals("2017-02-01T00:00:06.000Z", bucket.getKeyAsString()); - assertEquals(0, bucket.getDocCount()); - - bucket = buckets.get(2); - assertEquals("2017-02-01T00:00:07.000Z", bucket.getKeyAsString()); - assertEquals(2, bucket.getDocCount()); - - bucket = buckets.get(3); - assertEquals("2017-02-01T00:00:08.000Z", bucket.getKeyAsString()); - assertEquals(0, bucket.getDocCount()); - - bucket = buckets.get(4); - assertEquals("2017-02-01T00:00:09.000Z", bucket.getKeyAsString()); - assertEquals(0, bucket.getDocCount()); - - bucket = buckets.get(5); - assertEquals("2017-02-01T00:00:10.000Z", bucket.getKeyAsString()); - assertEquals(0, bucket.getDocCount()); - - bucket = buckets.get(6); - assertEquals("2017-02-01T00:00:11.000Z", bucket.getKeyAsString()); - assertEquals(3, bucket.getDocCount()); - } - ); - } - - private void testSearchCase(Query query, List dataset, - Consumer configure, - Consumer verify) throws IOException { - executeTestCase(false, query, dataset, configure, verify); - } - - private void testSearchAndReduceCase(Query query, List dataset, - Consumer configure, - Consumer verify) throws IOException { - executeTestCase(true, query, dataset, configure, verify); - } - - private void testBothCases(Query query, List dataset, - Consumer configure, - Consumer verify) throws IOException { - testSearchCase(query, dataset, configure, verify); - testSearchAndReduceCase(query, dataset, configure, verify); - } - - private void executeTestCase(boolean reduced, Query query, List dataset, - Consumer configure, - Consumer verify) throws IOException { - - try (Directory directory = newDirectory()) { - try (RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory)) { - Document document = new Document(); - for (String date : dataset) { - if (frequently()) { - indexWriter.commit(); - } - - long instant = asLong(date); - document.add(new SortedNumericDocValuesField(DATE_FIELD, instant)); - document.add(new LongPoint(INSTANT_FIELD, instant)); - indexWriter.addDocument(document); - document.clear(); - } - } - - try (IndexReader indexReader = DirectoryReader.open(directory)) { - IndexSearcher indexSearcher = newSearcher(indexReader, true, true); - - AutoDateHistogramAggregationBuilder aggregationBuilder = new AutoDateHistogramAggregationBuilder("_name"); - if (configure != null) { - configure.accept(aggregationBuilder); - } - - DateFieldMapper.Builder builder = new DateFieldMapper.Builder("_name"); - DateFieldMapper.DateFieldType fieldType = builder.fieldType(); - fieldType.setHasDocValues(true); - fieldType.setName(aggregationBuilder.field()); - - InternalAutoDateHistogram histogram; - if (reduced) { - histogram = searchAndReduce(indexSearcher, query, aggregationBuilder, fieldType); - } else { - histogram = search(indexSearcher, query, aggregationBuilder, fieldType); - } - verify.accept(histogram); - } - } - } - - private static long asLong(String dateTime) { - return DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.parser().parseDateTime(dateTime).getMillis(); - } -} diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java deleted file mode 100644 index 46783beafbc6a..0000000000000 --- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogramTests.java +++ /dev/null @@ -1,153 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.search.aggregations.bucket.histogram; - -import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.rounding.DateTimeUnit; -import org.elasticsearch.common.rounding.Rounding; -import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.aggregations.InternalAggregations; -import org.elasticsearch.search.aggregations.InternalMultiBucketAggregationTestCase; -import org.elasticsearch.search.aggregations.ParsedMultiBucketAggregation; -import org.elasticsearch.search.aggregations.bucket.histogram.InternalAutoDateHistogram.BucketInfo; -import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; -import org.joda.time.DateTime; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; - -import static org.elasticsearch.common.unit.TimeValue.timeValueHours; -import static org.elasticsearch.common.unit.TimeValue.timeValueMinutes; -import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds; - -public class InternalAutoDateHistogramTests extends InternalMultiBucketAggregationTestCase { - - private DocValueFormat format; - private Rounding[] roundings; - - @Override - public void setUp() throws Exception { - super.setUp(); - format = randomNumericDocValueFormat(); - - roundings = new Rounding[6]; - roundings[0] = Rounding.builder(DateTimeUnit.SECOND_OF_MINUTE).build(); - roundings[1] = Rounding.builder(DateTimeUnit.MINUTES_OF_HOUR).build(); - roundings[2] = Rounding.builder(DateTimeUnit.HOUR_OF_DAY).build(); - roundings[3] = Rounding.builder(DateTimeUnit.DAY_OF_MONTH).build(); - roundings[4] = Rounding.builder(DateTimeUnit.MONTH_OF_YEAR).build(); - roundings[5] = Rounding.builder(DateTimeUnit.YEAR_OF_CENTURY).build(); - } - - @Override - protected InternalAutoDateHistogram createTestInstance(String name, - List pipelineAggregators, - Map metaData, - InternalAggregations aggregations) { - int nbBuckets = randomNumberOfBuckets(); - int targetBuckets = randomIntBetween(1, nbBuckets * 2 + 1); - List buckets = new ArrayList<>(nbBuckets); - long startingDate = System.currentTimeMillis(); - - long interval = randomIntBetween(1, 3); - long intervalMillis = randomFrom(timeValueSeconds(interval), timeValueMinutes(interval), timeValueHours(interval)).getMillis(); - - for (int i = 0; i < nbBuckets; i++) { - long key = startingDate + (intervalMillis * i); - buckets.add(i, new InternalAutoDateHistogram.Bucket(key, randomIntBetween(1, 100), format, aggregations)); - } - InternalAggregations subAggregations = new InternalAggregations(Collections.emptyList()); - BucketInfo bucketInfo = new BucketInfo(roundings, randomIntBetween(0, roundings.length - 1), subAggregations); - - return new InternalAutoDateHistogram(name, buckets, targetBuckets, bucketInfo, format, pipelineAggregators, metaData); - } - - @Override - protected void assertReduced(InternalAutoDateHistogram reduced, List inputs) { - int roundingIdx = 0; - for (InternalAutoDateHistogram histogram : inputs) { - if (histogram.getBucketInfo().roundingIdx > roundingIdx) { - roundingIdx = histogram.getBucketInfo().roundingIdx; - } - } - Map expectedCounts = new TreeMap<>(); - for (Histogram histogram : inputs) { - for (Histogram.Bucket bucket : histogram.getBuckets()) { - expectedCounts.compute(roundings[roundingIdx].round(((DateTime) bucket.getKey()).getMillis()), - (key, oldValue) -> (oldValue == null ? 0 : oldValue) + bucket.getDocCount()); - } - } - Map actualCounts = new TreeMap<>(); - for (Histogram.Bucket bucket : reduced.getBuckets()) { - actualCounts.compute(((DateTime) bucket.getKey()).getMillis(), - (key, oldValue) -> (oldValue == null ? 0 : oldValue) + bucket.getDocCount()); - } - assertEquals(expectedCounts, actualCounts); - } - - @Override - protected Writeable.Reader instanceReader() { - return InternalAutoDateHistogram::new; - } - - @Override - protected Class implementationClass() { - return ParsedAutoDateHistogram.class; - } - - @Override - protected InternalAutoDateHistogram mutateInstance(InternalAutoDateHistogram instance) { - String name = instance.getName(); - List buckets = instance.getBuckets(); - int targetBuckets = instance.getTargetBuckets(); - BucketInfo bucketInfo = instance.getBucketInfo(); - List pipelineAggregators = instance.pipelineAggregators(); - Map metaData = instance.getMetaData(); - switch (between(0, 3)) { - case 0: - name += randomAlphaOfLength(5); - break; - case 1: - buckets = new ArrayList<>(buckets); - buckets.add(new InternalAutoDateHistogram.Bucket(randomNonNegativeLong(), randomIntBetween(1, 100), format, - InternalAggregations.EMPTY)); - break; - case 2: - int roundingIdx = bucketInfo.roundingIdx == bucketInfo.roundings.length - 1 ? 0 : bucketInfo.roundingIdx + 1; - bucketInfo = new BucketInfo(bucketInfo.roundings, roundingIdx, bucketInfo.emptySubAggregations); - break; - case 3: - if (metaData == null) { - metaData = new HashMap<>(1); - } else { - metaData = new HashMap<>(instance.getMetaData()); - } - metaData.put(randomAlphaOfLength(15), randomInt()); - break; - default: - throw new AssertionError("Illegal randomisation branch"); - } - return new InternalAutoDateHistogram(name, buckets, targetBuckets, bucketInfo, format, pipelineAggregators, metaData); - } -} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java index 4d15719b5f9e7..f86145386f1df 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java @@ -21,7 +21,7 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.util.CollectionUtil; -import org.elasticsearch.common.inject.internal.Nullable; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.rounding.Rounding; import org.elasticsearch.common.util.LongHash; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java index 3f658204a86a5..fc53cf7a9e8d3 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java @@ -103,7 +103,7 @@ public void writeTo(StreamOutput out) throws IOException { @Override public String getKeyAsString() { - return format.format(key); + return (String) format.format(key); } @Override @@ -134,7 +134,7 @@ Bucket reduce(List buckets, Rounding rounding, ReduceContext context) { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - String keyAsString = format.format(key); + String keyAsString = (String) format.format(key); builder.startObject(); if (format != DocValueFormat.RAW) { builder.field(CommonFields.KEY_AS_STRING.getPreferredName(), keyAsString); @@ -493,11 +493,11 @@ public InternalAggregation doReduce(List aggregations, Redu if (reduceContext.isFinalReduce()) { // adding empty buckets if needed reducedBucketsResult = addEmptyBuckets(reducedBucketsResult, reduceContext); - + // Adding empty buckets may have tipped us over the target so merge the buckets again if needed reducedBucketsResult = mergeBucketsIfNeeded(reducedBucketsResult.buckets, reducedBucketsResult.roundingIdx, reducedBucketsResult.roundingInfo, reduceContext); - + // Now finally see if we need to merge consecutive buckets together to make a coarser interval at the same rounding reducedBucketsResult = maybeMergeConsecutiveBuckets(reducedBucketsResult, reduceContext); } From 3b54eca3e0ac196df88fd68f78abf8064b35a2d0 Mon Sep 17 00:00:00 2001 From: Paul Sanwald Date: Thu, 21 Jun 2018 14:15:03 -0500 Subject: [PATCH 18/24] correct tests and add a new test case for 10k buckets --- .../AutoDateHistogramAggregationBuilder.java | 12 +++-- .../histogram/InternalAutoDateHistogram.java | 4 +- .../AutoDateHistogramAggregatorTests.java | 54 ++++++++++++------- 3 files changed, 45 insertions(+), 25 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java index a5f1fe8e4f0a3..465ccd8014bb2 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java @@ -61,6 +61,8 @@ public class AutoDateHistogramAggregationBuilder PARSER.declareInt(AutoDateHistogramAggregationBuilder::setNumBuckets, NUM_BUCKETS_FIELD); } + public static final int BUCKET_LIMIT = 10000; + public static AutoDateHistogramAggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException { return PARSER.parse(parser, new AutoDateHistogramAggregationBuilder(aggregationName), null); } @@ -103,6 +105,10 @@ public AutoDateHistogramAggregationBuilder setNumBuckets(int numBuckets) { if (numBuckets <= 0) { throw new IllegalArgumentException(NUM_BUCKETS_FIELD.getPreferredName() + " must be greater than 0 for [" + name + "]"); } + if (numBuckets > BUCKET_LIMIT) { + throw new IllegalArgumentException( + String.format("%s should be less than %d for %s", NUM_BUCKETS_FIELD.getPreferredName(), BUCKET_LIMIT, name)); + } this.numBuckets = numBuckets; return this; } @@ -160,7 +166,7 @@ public RoundingInfo(Rounding rounding, long roughEstimateDurationMillis, int... this.roughEstimateDurationMillis = roughEstimateDurationMillis; this.innerIntervals = innerIntervals; } - + public RoundingInfo(StreamInput in) throws IOException { rounding = Rounding.Streams.read(in); roughEstimateDurationMillis = in.readVLong(); @@ -173,7 +179,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVLong(roughEstimateDurationMillis); out.writeIntArray(innerIntervals); } - + public int getMaximumInnerInterval() { return innerIntervals[innerIntervals.length - 1]; } @@ -186,7 +192,7 @@ public long getRoughEstimateDurationMillis() { public int hashCode() { return Objects.hash(rounding, Arrays.hashCode(innerIntervals)); } - + @Override public boolean equals(Object obj) { if (obj == null) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java index fc53cf7a9e8d3..27c195cbdae75 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalAutoDateHistogram.java @@ -103,7 +103,7 @@ public void writeTo(StreamOutput out) throws IOException { @Override public String getKeyAsString() { - return (String) format.format(key); + return format.format(key).toString(); } @Override @@ -134,7 +134,7 @@ Bucket reduce(List buckets, Rounding rounding, ReduceContext context) { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - String keyAsString = (String) format.format(key); + String keyAsString = format.format(key).toString(); builder.startObject(); if (format != DocValueFormat.RAW) { builder.field(CommonFields.KEY_AS_STRING.getPreferredName(), keyAsString); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java index a1e863dd6bf25..9ee9664cb32b2 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java @@ -30,6 +30,7 @@ import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.store.Directory; +import org.elasticsearch.common.network.NetworkUtils; import org.elasticsearch.index.mapper.DateFieldMapper; import org.elasticsearch.search.aggregations.AggregationBuilders; import org.elasticsearch.search.aggregations.AggregatorTestCase; @@ -48,6 +49,8 @@ import java.util.List; import java.util.function.Consumer; +import static org.hamcrest.Matchers.containsString; + public class AutoDateHistogramAggregatorTests extends AggregatorTestCase { private static final String DATE_FIELD = "date"; @@ -282,6 +285,17 @@ public void testIntervalMonth() throws IOException { ); } + public void testWithLargeNumberOfBuckets() { + Query query = new MatchAllDocsQuery(); + IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, + () -> testSearchCase(query, dataset, + aggregation -> aggregation.setNumBuckets(AutoDateHistogramAggregationBuilder.BUCKET_LIMIT+1).field(DATE_FIELD), + // since an exception is thrown, this assertion won't be invoked. + histogram -> assertTrue(false) + )); + assertThat(exception.getMessage(), containsString("should be less than")); + } + public void testIntervalDay() throws IOException { testSearchCase(new MatchAllDocsQuery(), Arrays.asList("2017-02-01", "2017-02-02", "2017-02-02", "2017-02-03", "2017-02-03", "2017-02-03", "2017-02-05"), @@ -628,7 +642,7 @@ public void testAllSecondIntervals() throws IOException { DateTime date = startDate.plusSeconds(i); dataset.add(format.print(date)); } - testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(600).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); @@ -639,7 +653,7 @@ public void testAllSecondIntervals() throws IOException { assertEquals(1, bucket.getDocCount()); } }); - testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(300).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); @@ -650,7 +664,7 @@ public void testAllSecondIntervals() throws IOException { assertEquals(5, bucket.getDocCount()); } }); - testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(100).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); @@ -661,7 +675,7 @@ public void testAllSecondIntervals() throws IOException { assertEquals(10, bucket.getDocCount()); } }); - testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(50).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); @@ -672,7 +686,7 @@ public void testAllSecondIntervals() throws IOException { assertEquals(30, bucket.getDocCount()); } }); - testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(15).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); @@ -693,7 +707,7 @@ public void testAllMinuteIntervals() throws IOException { DateTime date = startDate.plusMinutes(i); dataset.add(format.print(date)); } - testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(600).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); @@ -704,7 +718,7 @@ public void testAllMinuteIntervals() throws IOException { assertEquals(1, bucket.getDocCount()); } }); - testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(300).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); @@ -715,7 +729,7 @@ public void testAllMinuteIntervals() throws IOException { assertEquals(5, bucket.getDocCount()); } }); - testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(100).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); @@ -726,7 +740,7 @@ public void testAllMinuteIntervals() throws IOException { assertEquals(10, bucket.getDocCount()); } }); - testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(50).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); @@ -737,7 +751,7 @@ public void testAllMinuteIntervals() throws IOException { assertEquals(30, bucket.getDocCount()); } }); - testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(15).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); @@ -758,7 +772,7 @@ public void testAllHourIntervals() throws IOException { DateTime date = startDate.plusHours(i); dataset.add(format.print(date)); } - testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(600).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); @@ -769,7 +783,7 @@ public void testAllHourIntervals() throws IOException { assertEquals(1, bucket.getDocCount()); } }); - testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(300).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); @@ -780,7 +794,7 @@ public void testAllHourIntervals() throws IOException { assertEquals(3, bucket.getDocCount()); } }); - testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(100).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); @@ -791,7 +805,7 @@ public void testAllHourIntervals() throws IOException { assertEquals(12, bucket.getDocCount()); } }); - testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(30).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); @@ -812,7 +826,7 @@ public void testAllDayIntervals() throws IOException { DateTime date = startDate.plusDays(i); dataset.add(format.print(date)); } - testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(700).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); @@ -823,7 +837,7 @@ public void testAllDayIntervals() throws IOException { assertEquals(1, bucket.getDocCount()); } }); - testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(300).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); @@ -834,7 +848,7 @@ public void testAllDayIntervals() throws IOException { assertEquals(7, bucket.getDocCount()); } }); - testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(30).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); @@ -855,7 +869,7 @@ public void testAllMonthIntervals() throws IOException { DateTime date = startDate.plusMonths(i); dataset.add(format.print(date)); } - testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(600).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); @@ -866,7 +880,7 @@ public void testAllMonthIntervals() throws IOException { assertEquals(1, bucket.getDocCount()); } }); - testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(300).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); @@ -877,7 +891,7 @@ public void testAllMonthIntervals() throws IOException { assertEquals(3, bucket.getDocCount()); } }); - testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(60).field(DATE_FIELD), histogram -> { List buckets = histogram.getBuckets(); From d153836e032684e771e9d24a856138b957ce8514 Mon Sep 17 00:00:00 2001 From: Paul Sanwald Date: Wed, 27 Jun 2018 17:58:55 -0400 Subject: [PATCH 19/24] refactor to perform bucket number check in innerBuild --- .../AutoDateHistogramAggregationBuilder.java | 20 +++++++++++++------ .../AutoDateHistogramAggregatorTests.java | 7 +++++-- 2 files changed, 19 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java index 465ccd8014bb2..dc337fabb1e00 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java @@ -31,6 +31,7 @@ import org.elasticsearch.search.aggregations.AggregationBuilder; import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.MultiBucketConsumerService; import org.elasticsearch.search.aggregations.support.ValueType; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; @@ -61,8 +62,6 @@ public class AutoDateHistogramAggregationBuilder PARSER.declareInt(AutoDateHistogramAggregationBuilder::setNumBuckets, NUM_BUCKETS_FIELD); } - public static final int BUCKET_LIMIT = 10000; - public static AutoDateHistogramAggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException { return PARSER.parse(parser, new AutoDateHistogramAggregationBuilder(aggregationName), null); } @@ -105,10 +104,6 @@ public AutoDateHistogramAggregationBuilder setNumBuckets(int numBuckets) { if (numBuckets <= 0) { throw new IllegalArgumentException(NUM_BUCKETS_FIELD.getPreferredName() + " must be greater than 0 for [" + name + "]"); } - if (numBuckets > BUCKET_LIMIT) { - throw new IllegalArgumentException( - String.format("%s should be less than %d for %s", NUM_BUCKETS_FIELD.getPreferredName(), BUCKET_LIMIT, name)); - } this.numBuckets = numBuckets; return this; } @@ -127,6 +122,19 @@ public int getNumBuckets() { roundings[3] = new RoundingInfo(createRounding(DateTimeUnit.DAY_OF_MONTH), 24 * 60 * 60 * 1000L, 1, 7); roundings[4] = new RoundingInfo(createRounding(DateTimeUnit.MONTH_OF_YEAR), 30 * 24 * 60 * 60 * 1000L, 1, 3); roundings[5] = new RoundingInfo(createRounding(DateTimeUnit.YEAR_OF_CENTURY), 365 * 24 * 60 * 60 * 1000L, 1, 5, 10, 20, 50, 100); + + int maxRoundingInterval = Arrays.stream(roundings,0, roundings.length-1) + .map(rounding -> rounding.innerIntervals) + .flatMapToInt(Arrays::stream) + .boxed() + .reduce(Integer::max).get(); + // TODO[PCS] what's the best way to get a hold of SearchService here, so I'm not just using default? + int bucketCeiling = MultiBucketConsumerService.DEFAULT_MAX_BUCKETS / maxRoundingInterval; + if (numBuckets > bucketCeiling) { + throw new IllegalArgumentException( + String.format("%s must be less than %d", NUM_BUCKETS_FIELD.getPreferredName(), bucketCeiling) + ); + } return new AutoDateHistogramAggregatorFactory(name, config, numBuckets, roundings, context, parent, subFactoriesBuilder, metaData); } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java index 9ee9664cb32b2..6b1632d490dd3 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java @@ -31,9 +31,11 @@ import org.apache.lucene.search.Query; import org.apache.lucene.store.Directory; import org.elasticsearch.common.network.NetworkUtils; +import org.elasticsearch.common.rounding.DateTimeUnit; import org.elasticsearch.index.mapper.DateFieldMapper; import org.elasticsearch.search.aggregations.AggregationBuilders; import org.elasticsearch.search.aggregations.AggregatorTestCase; +import org.elasticsearch.search.aggregations.MultiBucketConsumerService; import org.elasticsearch.search.aggregations.metrics.stats.Stats; import org.hamcrest.Matchers; import org.joda.time.DateTime; @@ -48,6 +50,7 @@ import java.util.Collections; import java.util.List; import java.util.function.Consumer; +import java.util.stream.Collectors; import static org.hamcrest.Matchers.containsString; @@ -289,11 +292,11 @@ public void testWithLargeNumberOfBuckets() { Query query = new MatchAllDocsQuery(); IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> testSearchCase(query, dataset, - aggregation -> aggregation.setNumBuckets(AutoDateHistogramAggregationBuilder.BUCKET_LIMIT+1).field(DATE_FIELD), + aggregation -> aggregation.setNumBuckets(MultiBucketConsumerService.DEFAULT_MAX_BUCKETS+1).field(DATE_FIELD), // since an exception is thrown, this assertion won't be invoked. histogram -> assertTrue(false) )); - assertThat(exception.getMessage(), containsString("should be less than")); + assertThat(exception.getMessage(), containsString("must be less than")); } public void testIntervalDay() throws IOException { From e6cda9a812cf8b458c5721a1083fcf551060d93d Mon Sep 17 00:00:00 2001 From: Paul Sanwald Date: Mon, 2 Jul 2018 15:16:39 -0400 Subject: [PATCH 20/24] correctly derive bucket setting, update tests to increase bucket threshold --- .../AutoDateHistogramAggregationBuilder.java | 6 +++-- .../AutoDateHistogramAggregatorTests.java | 25 ++++++++++++++++--- .../aggregations/AggregatorTestCase.java | 3 +-- 3 files changed, 26 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java index dc337fabb1e00..d63193b691564 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java @@ -25,6 +25,7 @@ import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.rounding.DateTimeUnit; import org.elasticsearch.common.rounding.Rounding; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; @@ -128,8 +129,9 @@ public int getNumBuckets() { .flatMapToInt(Arrays::stream) .boxed() .reduce(Integer::max).get(); - // TODO[PCS] what's the best way to get a hold of SearchService here, so I'm not just using default? - int bucketCeiling = MultiBucketConsumerService.DEFAULT_MAX_BUCKETS / maxRoundingInterval; + Settings settings = context.getQueryShardContext().getIndexSettings().getNodeSettings(); + int maxBuckets = MultiBucketConsumerService.MAX_BUCKET_SETTING.get(settings); + int bucketCeiling = maxBuckets / maxRoundingInterval; if (numBuckets > bucketCeiling) { throw new IllegalArgumentException( String.format("%s must be less than %d", NUM_BUCKETS_FIELD.getPreferredName(), bucketCeiling) diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java index 6b1632d490dd3..7cf29e3aa9cc5 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregatorTests.java @@ -30,8 +30,10 @@ import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.store.Directory; -import org.elasticsearch.common.network.NetworkUtils; -import org.elasticsearch.common.rounding.DateTimeUnit; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.DateFieldMapper; import org.elasticsearch.search.aggregations.AggregationBuilders; import org.elasticsearch.search.aggregations.AggregatorTestCase; @@ -50,7 +52,6 @@ import java.util.Collections; import java.util.List; import java.util.function.Consumer; -import java.util.stream.Collectors; import static org.hamcrest.Matchers.containsString; @@ -645,6 +646,7 @@ public void testAllSecondIntervals() throws IOException { DateTime date = startDate.plusSeconds(i); dataset.add(format.print(date)); } + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(600).field(DATE_FIELD), histogram -> { @@ -656,6 +658,7 @@ public void testAllSecondIntervals() throws IOException { assertEquals(1, bucket.getDocCount()); } }); + testSearchAndReduceCase(new MatchAllDocsQuery(), dataset, aggregation -> aggregation.setNumBuckets(300).field(DATE_FIELD), histogram -> { @@ -1265,8 +1268,22 @@ private void testBothCases(Query query, List dataset, testSearchAndReduceCase(query, dataset, configure, verify); } + @Override + protected IndexSettings createIndexSettings() { + Settings nodeSettings = Settings.builder() + .put("search.max_buckets", 100000).build(); + return new IndexSettings( + IndexMetaData.builder("_index").settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfShards(1) + .numberOfReplicas(0) + .creationDate(System.currentTimeMillis()) + .build(), + nodeSettings + ); + } + private void executeTestCase(boolean reduced, Query query, List dataset, - Consumer configure, + Consumer configure, Consumer verify) throws IOException { try (Directory directory = newDirectory()) { diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index 07f25986655d7..e84f2a99a115d 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -87,7 +87,6 @@ import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import static org.elasticsearch.test.InternalAggregationTestCase.DEFAULT_MAX_BUCKETS; /** * Base class for testing {@link Aggregator} implementations. @@ -229,7 +228,7 @@ public boolean shouldCache(Query query) throws IOException { }); when(searchContext.bitsetFilterCache()).thenReturn(new BitsetFilterCache(indexSettings, mock(Listener.class))); doAnswer(invocation -> { - /* Store the releasables so we can release them at the end of the test case. This is important because aggregations don't + /* Store the release-ables so we can release them at the end of the test case. This is important because aggregations don't * close their sub-aggregations. This is fairly similar to what the production code does. */ releasables.add((Releasable) invocation.getArguments()[0]); return null; From fc3c2ec0301d53c9150f84eda9d944f4abc869be Mon Sep 17 00:00:00 2001 From: Paul Sanwald Date: Mon, 2 Jul 2018 15:39:23 -0400 Subject: [PATCH 21/24] fix checkstyle --- .../histogram/AutoDateHistogramAggregationBuilder.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java index d63193b691564..366060835d891 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregationBuilder.java @@ -133,9 +133,8 @@ public int getNumBuckets() { int maxBuckets = MultiBucketConsumerService.MAX_BUCKET_SETTING.get(settings); int bucketCeiling = maxBuckets / maxRoundingInterval; if (numBuckets > bucketCeiling) { - throw new IllegalArgumentException( - String.format("%s must be less than %d", NUM_BUCKETS_FIELD.getPreferredName(), bucketCeiling) - ); + throw new IllegalArgumentException(NUM_BUCKETS_FIELD.getPreferredName()+ + " must be less than " + bucketCeiling); } return new AutoDateHistogramAggregatorFactory(name, config, numBuckets, roundings, context, parent, subFactoriesBuilder, metaData); } From 963e5e84536e785f160ee74dc7cc6a9a18f94715 Mon Sep 17 00:00:00 2001 From: Paul Sanwald Date: Thu, 12 Jul 2018 15:51:31 -0400 Subject: [PATCH 22/24] address code review comments --- .../MergingBucketsDeferringCollector.java | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java index 1d4c9f08f6d11..f357e9d286f54 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/MergingBucketsDeferringCollector.java @@ -37,6 +37,12 @@ import java.util.ArrayList; import java.util.List; +/** + * A specialization of {@link DeferringBucketCollector} that collects all + * matches and then is able to replay a given subset of buckets. Exposes + * mergeBuckets, which can be invoked by the aggregator when increasing the + * rounding interval. + */ public class MergingBucketsDeferringCollector extends DeferringBucketCollector { List entries = new ArrayList<>(); @@ -92,7 +98,7 @@ public LeafBucketCollector getLeafCollector(LeafReaderContext ctx) throws IOExce int lastDoc = 0; @Override - public void collect(int doc, long bucket) throws IOException { + public void collect(int doc, long bucket) { docDeltas.add(doc - lastDoc); buckets.add(bucket); lastDoc = doc; @@ -103,12 +109,12 @@ public void collect(int doc, long bucket) throws IOException { public void mergeBuckets(long[] mergeMap) { - List newEntries = new ArrayList<>(); + List newEntries = new ArrayList<>(entries.size()); for (Entry sourceEntry : entries) { PackedLongValues.Builder newBuckets = PackedLongValues.packedBuilder(PackedInts.DEFAULT); for (PackedLongValues.Iterator itr = sourceEntry.buckets.iterator(); itr.hasNext();) { long bucket = itr.next(); - newBuckets.add(mergeMap[(int) bucket]); + newBuckets.add(mergeMap[Math.toIntExact(bucket)]); } newEntries.add(new Entry(sourceEntry.context, sourceEntry.docDeltas, newBuckets.build())); } @@ -121,14 +127,14 @@ public void mergeBuckets(long[] mergeMap) { PackedLongValues.Builder newBuckets = PackedLongValues.packedBuilder(PackedInts.DEFAULT); for (PackedLongValues.Iterator itr = currentBuckets.iterator(); itr.hasNext();) { long bucket = itr.next(); - newBuckets.add(mergeMap[(int) bucket]); + newBuckets.add(mergeMap[Math.toIntExact(bucket)]); } buckets = newBuckets; } } @Override - public void postCollection() throws IOException { + public void postCollection() { finishLeaf(); finished = true; } @@ -138,7 +144,7 @@ public void postCollection() throws IOException { */ @Override public void prepareSelectedBuckets(long... selectedBuckets) throws IOException { - if (!finished) { + if (finished == false) { throw new IllegalStateException("Cannot replay yet, collection is not finished: postCollect() has not been called"); } if (this.selectedBuckets != null) { From 6ae1a703a5ff41848df65a41665eac71af5d102a Mon Sep 17 00:00:00 2001 From: Paul Sanwald Date: Thu, 12 Jul 2018 16:48:17 -0400 Subject: [PATCH 23/24] add documentation for default buckets --- .../aggregations/bucket/autodatehistogram-aggregation.asciidoc | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc index 9d1d0424ce4a0..2170b520ac966 100644 --- a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc @@ -6,6 +6,8 @@ instead of providing an interval to use as the width of each bucket, a target nu indicating the number of buckets needed and the interval of the buckets is automatically chosen to best achieve that target. The number of buckets returned will always be less than or equal to this target number. +The buckets field is optional, and will default to 10 buckers if not specified. + Requesting a target of 10 buckets. [source,js] From 4e5d4350409038f1b25771004c5a54a3a014b276 Mon Sep 17 00:00:00 2001 From: Paul Sanwald Date: Fri, 13 Jul 2018 10:09:35 -0400 Subject: [PATCH 24/24] fix typo --- .../aggregations/bucket/autodatehistogram-aggregation.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc index 2170b520ac966..28cb65ce6cc48 100644 --- a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc @@ -6,7 +6,7 @@ instead of providing an interval to use as the width of each bucket, a target nu indicating the number of buckets needed and the interval of the buckets is automatically chosen to best achieve that target. The number of buckets returned will always be less than or equal to this target number. -The buckets field is optional, and will default to 10 buckers if not specified. +The buckets field is optional, and will default to 10 buckets if not specified. Requesting a target of 10 buckets.