From a0818d3b876216c7882b3f6b14c613173409c06c Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Fri, 22 Jul 2016 12:16:45 +0200 Subject: [PATCH] Split regular histograms from date histograms. #19551 Currently both aggregations really share the same implementation. This commit splits the implementations so that regular histograms can support decimal intervals/offsets and compute correct buckets for negative decimal values. However the response API is still the same. So for intance both regular histograms and date histograms will produce an `org.elasticsearch.search.aggregations.bucket.histogram.Histogram` aggregation. The optimization to compute an identifier of the rounded value and the rounded value itself has been removed since it was only used by regular histograms, which now do the rounding themselves instead of relying on the Rounding abstraction. Closes #8082 Closes #4847 --- .../common/rounding/Rounding.java | 46 +- .../common/rounding/TimeZoneRounding.java | 27 +- .../elasticsearch/search/SearchModule.java | 3 +- .../AbstractHistogramAggregatorFactory.java | 113 ----- .../histogram/AbstractHistogramBuilder.java | 203 --------- .../DateHistogramAggregationBuilder.java | 197 +++++++-- .../histogram/DateHistogramAggregator.java | 153 +++++++ .../DateHistogramAggregatorFactory.java | 65 ++- .../bucket/histogram/DateHistogramParser.java | 93 +++- .../bucket/histogram/ExtendedBounds.java | 2 +- .../bucket/histogram/Histogram.java | 48 +- .../HistogramAggregationBuilder.java | 208 ++++++++- .../bucket/histogram/HistogramAggregator.java | 78 ++-- .../histogram/HistogramAggregatorFactory.java | 60 ++- .../bucket/histogram/HistogramFactory.java | 48 ++ .../bucket/histogram/HistogramParser.java | 85 ++-- .../histogram/InternalDateHistogram.java | 414 ++++++++++++++++-- .../bucket/histogram/InternalHistogram.java | 328 ++++++-------- .../bucket/histogram/InternalOrder.java | 12 +- .../aggregations/pipeline/BucketHelpers.java | 5 +- ...mulativeSumPipelineAggregationBuilder.java | 19 +- .../CumulativeSumPipelineAggregator.java | 19 +- .../DerivativePipelineAggregationBuilder.java | 18 +- .../DerivativePipelineAggregator.java | 38 +- .../MovAvgPipelineAggregationBuilder.java | 19 +- .../movavg/MovAvgPipelineAggregator.java | 53 +-- .../SerialDiffPipelineAggregator.java | 21 +- .../common/rounding/RoundingTests.java | 11 - .../rounding/TimeZoneRoundingTests.java | 15 - .../search/aggregations/MissingValueIT.java | 8 +- .../aggregations/bucket/DateHistogramIT.java | 2 +- .../bucket/DateHistogramOffsetIT.java | 2 +- .../bucket/DateHistogramTests.java | 3 +- .../aggregations/bucket/HistogramIT.java | 33 +- .../aggregations/bucket/HistogramTests.java | 8 +- .../bucket/histogram/ExtendedBoundsTests.java | 1 + .../aggregations/pipeline/AvgBucketIT.java | 11 +- .../aggregations/pipeline/BucketScriptIT.java | 36 +- .../pipeline/BucketSelectorIT.java | 35 +- .../pipeline/CumulativeSumIT.java | 4 +- .../pipeline/DateDerivativeIT.java | 19 +- .../aggregations/pipeline/DerivativeIT.java | 40 +- .../pipeline/ExtendedStatsBucketIT.java | 13 +- .../aggregations/pipeline/MaxBucketIT.java | 13 +- .../aggregations/pipeline/MinBucketIT.java | 11 +- .../pipeline/PercentilesBucketIT.java | 15 +- .../aggregations/pipeline/StatsBucketIT.java | 11 +- .../aggregations/pipeline/SumBucketIT.java | 11 +- .../pipeline/moving/avg/MovAvgIT.java | 70 ++- .../pipeline/serialdiff/SerialDiffIT.java | 11 +- .../aggregation/AggregationProfilerIT.java | 10 +- .../bucket/histogram-aggregation.asciidoc | 13 +- .../expression/MoreExpressionTests.java | 24 +- 53 files changed, 1685 insertions(+), 1120 deletions(-) delete mode 100644 core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AbstractHistogramAggregatorFactory.java delete mode 100644 core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AbstractHistogramBuilder.java create mode 100644 core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java create mode 100644 core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramFactory.java diff --git a/core/src/main/java/org/elasticsearch/common/rounding/Rounding.java b/core/src/main/java/org/elasticsearch/common/rounding/Rounding.java index 7905ee29d735f..5633cf9f21304 100644 --- a/core/src/main/java/org/elasticsearch/common/rounding/Rounding.java +++ b/core/src/main/java/org/elasticsearch/common/rounding/Rounding.java @@ -35,24 +35,9 @@ public abstract class Rounding implements Streamable { public abstract byte id(); /** - * Given a value, compute a key that uniquely identifies the rounded value although it is not necessarily equal to the rounding value itself. + * Rounds the given value. */ - public abstract long roundKey(long value); - - /** - * Compute the rounded value given the key that identifies it. - */ - public abstract long valueForKey(long key); - - /** - * Rounds the given value, equivalent to calling roundValue(roundKey(value)). - * - * @param value The value to round. - * @return The rounded value. - */ - public final long round(long value) { - return valueForKey(roundKey(value)); - } + public abstract long round(long value); /** * Given the rounded value (which was potentially generated by {@link #round(long)}, returns the next rounding value. For example, with @@ -112,13 +97,8 @@ public static long roundValue(long key, long interval) { } @Override - public long roundKey(long value) { - return roundKey(value, interval); - } - - @Override - public long valueForKey(long key) { - return key * interval; + public long round(long value) { + return roundKey(value, interval) * interval; } @Override @@ -179,13 +159,8 @@ public byte id() { } @Override - public long roundKey(long utcMillis) { - return rounding.roundKey((long) (factor * utcMillis)); - } - - @Override - public long valueForKey(long key) { - return rounding.valueForKey(key); + public long round(long utcMillis) { + return rounding.round((long) (factor * utcMillis)); } @Override @@ -248,13 +223,8 @@ public byte id() { } @Override - public long roundKey(long value) { - return rounding.roundKey(value - offset); - } - - @Override - public long valueForKey(long key) { - return offset + rounding.valueForKey(key); + public long round(long value) { + return rounding.round(value - offset) + offset; } @Override diff --git a/core/src/main/java/org/elasticsearch/common/rounding/TimeZoneRounding.java b/core/src/main/java/org/elasticsearch/common/rounding/TimeZoneRounding.java index e52769d7f3699..932afa15b5680 100644 --- a/core/src/main/java/org/elasticsearch/common/rounding/TimeZoneRounding.java +++ b/core/src/main/java/org/elasticsearch/common/rounding/TimeZoneRounding.java @@ -31,6 +31,9 @@ import java.util.Objects; /** + * A rounding strategy for dates. It is typically used to group together dates + * that are part of the same hour/day/month, taking into account time zones and + * daylight saving times. */ public abstract class TimeZoneRounding extends Rounding { public static final ParseField INTERVAL_FIELD = new ParseField("interval"); @@ -125,7 +128,7 @@ public byte id() { } @Override - public long roundKey(long utcMillis) { + public long round(long utcMillis) { long rounded = field.roundFloor(utcMillis); if (timeZone.isFixed() == false && timeZone.getOffset(utcMillis) != timeZone.getOffset(rounded)) { // in this case, we crossed a time zone transition. In some edge cases this will @@ -138,20 +141,14 @@ public long roundKey(long utcMillis) { return rounded; } - @Override - public long valueForKey(long time) { - assert roundKey(time) == time; - return time; - } - @Override public long nextRoundingValue(long utcMillis) { - long floor = roundKey(utcMillis); + long floor = round(utcMillis); // add one unit and round to get to next rounded value - long next = roundKey(field.add(floor, 1)); + long next = round(field.add(floor, 1)); if (next == floor) { // in rare case we need to add more than one unit - next = roundKey(field.add(floor, 2)); + next = round(field.add(floor, 2)); } return next; } @@ -216,7 +213,7 @@ public byte id() { } @Override - public long roundKey(long utcMillis) { + public long round(long utcMillis) { long timeLocal = timeZone.convertUTCToLocal(utcMillis); long rounded = Rounding.Interval.roundValue(Rounding.Interval.roundKey(timeLocal, interval), interval); long roundedUTC; @@ -225,7 +222,7 @@ public long roundKey(long utcMillis) { // check if we crossed DST transition, in this case we want the last rounded value before the transition long transition = timeZone.previousTransition(utcMillis); if (transition != utcMillis && transition > roundedUTC) { - roundedUTC = roundKey(transition - 1); + roundedUTC = round(transition - 1); } } else { /* @@ -276,12 +273,6 @@ private boolean isInDSTGap(long instantLocal) { return false; } - @Override - public long valueForKey(long time) { - assert roundKey(time) == time; - return time; - } - @Override public long nextRoundingValue(long time) { long timeLocal = time; diff --git a/core/src/main/java/org/elasticsearch/search/SearchModule.java b/core/src/main/java/org/elasticsearch/search/SearchModule.java index c6ce4a040f396..d2e211db6f246 100644 --- a/core/src/main/java/org/elasticsearch/search/SearchModule.java +++ b/core/src/main/java/org/elasticsearch/search/SearchModule.java @@ -116,6 +116,7 @@ import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramParser; import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.histogram.HistogramParser; +import org.elasticsearch.search.aggregations.bucket.histogram.InternalDateHistogram; import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram; import org.elasticsearch.search.aggregations.bucket.missing.InternalMissing; import org.elasticsearch.search.aggregations.bucket.missing.MissingAggregationBuilder; @@ -546,7 +547,7 @@ private void registerBuiltinAggregations() { registerAggregation(new AggregationSpec(HistogramAggregationBuilder::new, new HistogramParser(), HistogramAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalHistogram::new)); registerAggregation(new AggregationSpec(DateHistogramAggregationBuilder::new, new DateHistogramParser(), - DateHistogramAggregationBuilder.AGGREGATION_NAME_FIELD)); + DateHistogramAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalDateHistogram::new)); registerAggregation(new AggregationSpec(GeoDistanceAggregationBuilder::new, new GeoDistanceParser(), GeoDistanceAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalGeoDistance::new)); registerAggregation(new AggregationSpec(GeoGridAggregationBuilder::new, new GeoHashGridParser(), diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AbstractHistogramAggregatorFactory.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AbstractHistogramAggregatorFactory.java deleted file mode 100644 index 1f3be4512c11e..0000000000000 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AbstractHistogramAggregatorFactory.java +++ /dev/null @@ -1,113 +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.ParsingException; -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.InternalAggregation.Type; -import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; -import org.elasticsearch.search.aggregations.support.AggregationContext; -import org.elasticsearch.search.aggregations.support.ValuesSource; -import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; - -import java.io.IOException; -import java.util.List; -import java.util.Map; - -import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; -import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; - -public abstract class AbstractHistogramAggregatorFactory> - extends ValuesSourceAggregatorFactory { - - protected final long interval; - protected final long offset; - protected final InternalOrder order; - protected final boolean keyed; - protected final long minDocCount; - protected final ExtendedBounds extendedBounds; - private final InternalHistogram.Factory histogramFactory; - - public AbstractHistogramAggregatorFactory(String name, Type type, ValuesSourceConfig config, long interval, long offset, - InternalOrder order, boolean keyed, long minDocCount, ExtendedBounds extendedBounds, - InternalHistogram.Factory histogramFactory, AggregationContext context, AggregatorFactory parent, - AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { - super(name, type, config, context, parent, subFactoriesBuilder, metaData); - this.interval = interval; - this.offset = offset; - this.order = order; - this.keyed = keyed; - this.minDocCount = minDocCount; - this.extendedBounds = extendedBounds; - this.histogramFactory = histogramFactory; - } - - public long minDocCount() { - return minDocCount; - } - - @Override - protected Aggregator createUnmapped(Aggregator parent, List pipelineAggregators, Map metaData) - throws IOException { - return createAggregator(null, parent, pipelineAggregators, metaData); - } - - protected Rounding createRounding() { - if (interval < 1) { - throw new ParsingException(null, "[interval] must be 1 or greater for histogram aggregation [" + name() + "]: " + interval); - } - - Rounding rounding = new Rounding.Interval(interval); - if (offset != 0) { - rounding = new Rounding.OffsetRounding(rounding, offset); - } - return rounding; - } - - @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 { - Rounding rounding = createRounding(); - // we need to round the bounds given by the user and we have to do it - // for every aggregator we create - // as the rounding is not necessarily an idempotent operation. - // todo we need to think of a better structure to the factory/agtor - // code so we won't need to do that - ExtendedBounds roundedBounds = null; - if (extendedBounds != null) { - // parse any string bounds to longs and round them - roundedBounds = extendedBounds.parseAndValidate(name, context.searchContext(), config.format()).round(rounding); - } - return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, roundedBounds, valuesSource, - config.format(), histogramFactory, context, parent, pipelineAggregators, metaData); - } - -} diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AbstractHistogramBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AbstractHistogramBuilder.java deleted file mode 100644 index 703748f14a905..0000000000000 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AbstractHistogramBuilder.java +++ /dev/null @@ -1,203 +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.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.rounding.Rounding; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.search.aggregations.support.ValuesSource; -import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder; -import org.elasticsearch.search.aggregations.support.ValuesSourceType; - -import java.io.IOException; -import java.util.Objects; - -public abstract class AbstractHistogramBuilder> - extends ValuesSourceAggregationBuilder { - - protected long interval; - protected long offset = 0; - protected InternalOrder order = (InternalOrder) Histogram.Order.KEY_ASC; - protected boolean keyed = false; - protected long minDocCount = 0; - protected ExtendedBounds extendedBounds; - - protected AbstractHistogramBuilder(String name, InternalHistogram.Factory histogramFactory) { - super(name, histogramFactory.type(), ValuesSourceType.NUMERIC, histogramFactory.valueType()); - } - - /** - * Read from a stream. - */ - protected AbstractHistogramBuilder(StreamInput in, InternalHistogram.Factory histogramFactory) throws IOException { - super(in, histogramFactory.type(), ValuesSourceType.NUMERIC, histogramFactory.valueType()); - interval = in.readVLong(); - offset = in.readLong(); - if (in.readBoolean()) { - order = InternalOrder.Streams.readOrder(in); - } - keyed = in.readBoolean(); - minDocCount = in.readVLong(); - if (in.readBoolean()) { - extendedBounds = new ExtendedBounds(in); - } - } - - @Override - protected void innerWriteTo(StreamOutput out) throws IOException { - out.writeVLong(interval); - out.writeLong(offset); - boolean hasOrder = order != null; - out.writeBoolean(hasOrder); - if (hasOrder) { - InternalOrder.Streams.writeOrder(order, out); - } - out.writeBoolean(keyed); - out.writeVLong(minDocCount); - boolean hasExtendedBounds = extendedBounds != null; - out.writeBoolean(hasExtendedBounds); - if (hasExtendedBounds) { - extendedBounds.writeTo(out); - } - } - - public long interval() { - return interval; - } - - @SuppressWarnings("unchecked") - public AB interval(long interval) { - if (interval < 1) { - throw new IllegalArgumentException("[interval] must be 1 or greater for histogram aggregation [" + name + "]"); - } - this.interval = interval; - return (AB) this; - } - - public long offset() { - return offset; - } - - @SuppressWarnings("unchecked") - public AB offset(long offset) { - this.offset = offset; - return (AB) this; - } - - public Histogram.Order order() { - return order; - } - - @SuppressWarnings("unchecked") - public AB order(Histogram.Order order) { - if (order == null) { - throw new IllegalArgumentException("[order] must not be null: [" + name + "]"); - } - this.order = (InternalOrder) order; - return (AB) this; - } - - public boolean keyed() { - return keyed; - } - - @SuppressWarnings("unchecked") - public AB keyed(boolean keyed) { - this.keyed = keyed; - return (AB) this; - } - - public long minDocCount() { - return minDocCount; - } - - @SuppressWarnings("unchecked") - public AB minDocCount(long minDocCount) { - if (minDocCount < 0) { - throw new IllegalArgumentException( - "[minDocCount] must be greater than or equal to 0. Found [" + minDocCount + "] in [" + name + "]"); - } - this.minDocCount = minDocCount; - return (AB) this; - } - - public ExtendedBounds extendedBounds() { - return extendedBounds; - } - - @SuppressWarnings("unchecked") - public AB extendedBounds(ExtendedBounds extendedBounds) { - if (extendedBounds == null) { - throw new IllegalArgumentException("[extendedBounds] must not be null: [" + name + "]"); - } - this.extendedBounds = extendedBounds; - return (AB) this; - } - - @Override - protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { - - builder.field(Rounding.Interval.INTERVAL_FIELD.getPreferredName()); - doXContentInterval(builder, params); - builder.field(Rounding.OffsetRounding.OFFSET_FIELD.getPreferredName(), offset); - - if (order != null) { - builder.field(HistogramAggregator.ORDER_FIELD.getPreferredName()); - order.toXContent(builder, params); - } - - builder.field(HistogramAggregator.KEYED_FIELD.getPreferredName(), keyed); - - builder.field(HistogramAggregator.MIN_DOC_COUNT_FIELD.getPreferredName(), minDocCount); - - if (extendedBounds != null) { - extendedBounds.toXContent(builder, params); - } - - return builder; - } - - protected XContentBuilder doXContentInterval(XContentBuilder builder, Params params) throws IOException { - builder.value(interval); - return builder; - } - - @Override - public String getWriteableName() { - return InternalHistogram.TYPE.name(); - } - - @Override - protected int innerHashCode() { - return Objects.hash(interval, offset, order, keyed, minDocCount, extendedBounds); - } - - @Override - protected boolean innerEquals(Object obj) { - AbstractHistogramBuilder other = (AbstractHistogramBuilder) obj; - return Objects.equals(interval, other.interval) - && Objects.equals(offset, other.offset) - && Objects.equals(order, other.order) - && Objects.equals(keyed, other.keyed) - && Objects.equals(minDocCount, other.minDocCount) - && Objects.equals(extendedBounds, other.extendedBounds); - } -} diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java index f07bac1a7d185..8cfbefe1ba3a4 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java @@ -27,40 +27,91 @@ import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.support.AggregationContext; +import org.elasticsearch.search.aggregations.support.ValueType; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder; 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.aggregations.support.ValuesSourceType; import java.io.IOException; import java.util.Objects; -public class DateHistogramAggregationBuilder extends AbstractHistogramBuilder { - +/** + * A builder for histograms on date fields. + */ +public class DateHistogramAggregationBuilder + extends ValuesSourceAggregationBuilder { public static final String NAME = InternalDateHistogram.TYPE.name(); public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); + private long interval; private DateHistogramInterval dateHistogramInterval; + private long offset = 0; + private ExtendedBounds extendedBounds; + private InternalOrder order = (InternalOrder) Histogram.Order.KEY_ASC; + private boolean keyed = false; + private long minDocCount = 0; + /** Create a new builder with the given name. */ public DateHistogramAggregationBuilder(String name) { - super(name, InternalDateHistogram.HISTOGRAM_FACTORY); + super(name, InternalDateHistogram.TYPE, ValuesSourceType.NUMERIC, ValueType.DATE); } - /** - * Read from a stream. - */ + /** Read from a stream, for internal use only. */ public DateHistogramAggregationBuilder(StreamInput in) throws IOException { - super(in, InternalDateHistogram.HISTOGRAM_FACTORY); + super(in, InternalDateHistogram.TYPE, ValuesSourceType.NUMERIC, ValueType.DATE); + if (in.readBoolean()) { + order = InternalOrder.Streams.readOrder(in); + } + keyed = in.readBoolean(); + minDocCount = in.readVLong(); + interval = in.readLong(); dateHistogramInterval = in.readOptionalWriteable(DateHistogramInterval::new); + offset = in.readLong(); + extendedBounds = in.readOptionalWriteable(ExtendedBounds::new); } @Override protected void innerWriteTo(StreamOutput out) throws IOException { - super.innerWriteTo(out); + boolean hasOrder = order != null; + out.writeBoolean(hasOrder); + if (hasOrder) { + InternalOrder.Streams.writeOrder(order, out); + } + out.writeBoolean(keyed); + out.writeVLong(minDocCount); + out.writeLong(interval); out.writeOptionalWriteable(dateHistogramInterval); + out.writeLong(offset); + out.writeOptionalWriteable(extendedBounds); + } + + /** Get the current interval in milliseconds that is set on this builder. */ + public double interval() { + return interval; + } + + /** Set the interval on this builder, and return the builder so that calls can be chained. + * If both {@link #interval()} and {@link #dateHistogramInterval()} are set, then the + * {@link #dateHistogramInterval()} wins. */ + public DateHistogramAggregationBuilder interval(long interval) { + if (interval < 1) { + throw new IllegalArgumentException("[interval] must be 1 or greater for histogram aggregation [" + name + "]"); + } + this.interval = interval; + return this; + } + + /** Get the current date interval that is set on this builder. */ + public DateHistogramInterval dateHistogramInterval() { + return dateHistogramInterval; } - /** - * Set the interval. - */ + /** Set the interval on this builder, and return the builder so that calls can be chained. + * If both {@link #interval()} and {@link #dateHistogramInterval()} are set, then the + * {@link #dateHistogramInterval()} wins. */ public DateHistogramAggregationBuilder dateHistogramInterval(DateHistogramInterval dateHistogramInterval) { if (dateHistogramInterval == null) { throw new IllegalArgumentException("[dateHistogramInterval] must not be null: [" + name + "]"); @@ -69,6 +120,20 @@ public DateHistogramAggregationBuilder dateHistogramInterval(DateHistogramInterv return this; } + /** Get the offset to use when rounding, which is a number of milliseconds. */ + public double offset() { + return offset; + } + + /** Set the offset on this builder, which is a number of milliseconds, and + * return the builder so that calls can be chained. */ + public DateHistogramAggregationBuilder offset(long offset) { + this.offset = offset; + return this; + } + + /** Set the offset on this builder, as a time value, and + * return the builder so that calls can be chained. */ public DateHistogramAggregationBuilder offset(String offset) { if (offset == null) { throw new IllegalArgumentException("[offset] must not be null: [" + name + "]"); @@ -76,7 +141,7 @@ public DateHistogramAggregationBuilder offset(String offset) { return offset(parseStringOffset(offset)); } - protected static long parseStringOffset(String offset) { + static long parseStringOffset(String offset) { if (offset.charAt(0) == '-') { return -TimeValue .parseTimeValue(offset.substring(1), null, DateHistogramAggregationBuilder.class.getSimpleName() + ".parseOffset") @@ -88,40 +153,118 @@ protected static long parseStringOffset(String offset) { .millis(); } - public DateHistogramInterval dateHistogramInterval() { - return dateHistogramInterval; + /** Return extended bounds for this histogram, or {@code null} if none are set. */ + public ExtendedBounds extendedBounds() { + return extendedBounds; } - @Override - protected DateHistogramAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { - return new DateHistogramAggregatorFactory(name, type, config, interval, dateHistogramInterval, offset, order, keyed, minDocCount, - extendedBounds, context, parent, subFactoriesBuilder, metaData); + /** Set extended bounds on this histogram, so that buckets would also be + * generated on intervals that did not match any documents. */ + public DateHistogramAggregationBuilder extendedBounds(ExtendedBounds extendedBounds) { + if (extendedBounds == null) { + throw new IllegalArgumentException("[extendedBounds] must not be null: [" + name + "]"); + } + this.extendedBounds = extendedBounds; + return this; } - @Override - public String getWriteableName() { - return NAME; + /** Return the order to use to sort buckets of this histogram. */ + public Histogram.Order order() { + return order; + } + + /** Set a new order on this builder and return the builder so that calls + * can be chained. */ + public DateHistogramAggregationBuilder order(Histogram.Order order) { + if (order == null) { + throw new IllegalArgumentException("[order] must not be null: [" + name + "]"); + } + this.order = (InternalOrder) order; + return this; + } + + /** Return whether buckets should be returned as a hash. In case + * {@code keyed} is false, buckets will be returned as an array. */ + public boolean keyed() { + return keyed; + } + + /** Set whether to return buckets as a hash or as an array, and return the + * builder so that calls can be chained. */ + public DateHistogramAggregationBuilder keyed(boolean keyed) { + this.keyed = keyed; + return this; + } + + /** Return the minimum count of documents that buckets need to have in order + * to be included in the response. */ + public long minDocCount() { + return minDocCount; + } + + /** Set the minimum count of matching documents that buckets need to have + * and return this builder so that calls can be chained. */ + public DateHistogramAggregationBuilder minDocCount(long minDocCount) { + if (minDocCount < 0) { + throw new IllegalArgumentException( + "[minDocCount] must be greater than or equal to 0. Found [" + minDocCount + "] in [" + name + "]"); + } + this.minDocCount = minDocCount; + return this; } @Override - protected XContentBuilder doXContentInterval(XContentBuilder builder, Params params) throws IOException { + protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + if (dateHistogramInterval == null) { - super.doXContentInterval(builder, params); + builder.field(Histogram.INTERVAL_FIELD.getPreferredName(), interval); } else { - builder.value(dateHistogramInterval.toString()); + builder.field(Histogram.INTERVAL_FIELD.getPreferredName(), dateHistogramInterval.toString()); + } + builder.field(Histogram.OFFSET_FIELD.getPreferredName(), offset); + + if (order != null) { + builder.field(Histogram.ORDER_FIELD.getPreferredName()); + order.toXContent(builder, params); } + + builder.field(Histogram.KEYED_FIELD.getPreferredName(), keyed); + + builder.field(Histogram.MIN_DOC_COUNT_FIELD.getPreferredName(), minDocCount); + + if (extendedBounds != null) { + extendedBounds.toXContent(builder, params); + } + return builder; } + @Override + public String getWriteableName() { + return NAME; + } + + @Override + protected ValuesSourceAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig config, + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + return new DateHistogramAggregatorFactory(name, type, config, interval, dateHistogramInterval, offset, order, keyed, minDocCount, + extendedBounds, context, parent, subFactoriesBuilder, metaData); + } + @Override protected int innerHashCode() { - return Objects.hash(super.innerHashCode(), dateHistogramInterval); + return Objects.hash(order, keyed, minDocCount, interval, dateHistogramInterval, minDocCount, extendedBounds); } @Override protected boolean innerEquals(Object obj) { DateHistogramAggregationBuilder other = (DateHistogramAggregationBuilder) obj; - return super.innerEquals(obj) && Objects.equals(dateHistogramInterval, other.dateHistogramInterval); + return Objects.equals(order, other.order) + && Objects.equals(keyed, other.keyed) + && Objects.equals(minDocCount, other.minDocCount) + && Objects.equals(interval, other.interval) + && Objects.equals(dateHistogramInterval, other.dateHistogramInterval) + && Objects.equals(offset, other.offset) + && Objects.equals(extendedBounds, other.extendedBounds); } } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java new file mode 100644 index 0000000000000..cf8325683e281 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.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.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.rounding.TimeZoneRounding; +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.InternalAggregation; +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.support.AggregationContext; +import org.elasticsearch.search.aggregations.support.ValuesSource; + +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 TimeZoneRounding}. + * @see TimeZoneRounding + */ +class DateHistogramAggregator extends BucketsAggregator { + + private final ValuesSource.Numeric valuesSource; + private final DocValueFormat formatter; + private final Rounding rounding; + private final InternalOrder order; + private final boolean keyed; + + private final long minDocCount; + private final ExtendedBounds extendedBounds; + + private final LongHash bucketOrds; + + public DateHistogramAggregator(String name, AggregatorFactories factories, Rounding rounding, InternalOrder order, boolean keyed, + long minDocCount, @Nullable ExtendedBounds extendedBounds, @Nullable ValuesSource.Numeric valuesSource, + DocValueFormat formatter, AggregationContext aggregationContext, + Aggregator parent, List pipelineAggregators, Map metaData) throws IOException { + + super(name, factories, aggregationContext, parent, pipelineAggregators, metaData); + this.rounding = rounding; + this.order = order; + this.keyed = keyed; + this.minDocCount = minDocCount; + this.extendedBounds = extendedBounds; + this.valuesSource = valuesSource; + this.formatter = formatter; + + bucketOrds = new LongHash(1, aggregationContext.bigArrays()); + } + + @Override + public boolean needsScores() { + return (valuesSource != null && valuesSource.needsScores()) || super.needsScores(); + } + + @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; + values.setDocument(doc); + final int valuesCount = values.count(); + + long previousRounded = Long.MIN_VALUE; + for (int i = 0; i < valuesCount; ++i) { + long value = values.valueAt(i); + long rounded = rounding.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); + } + previousRounded = rounded; + } + } + }; + } + + @Override + public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { + assert owningBucketOrdinal == 0; + List buckets = new ArrayList<>((int) bucketOrds.size()); + for (long i = 0; i < bucketOrds.size(); i++) { + buckets.add(new InternalDateHistogram.Bucket(bucketOrds.get(i), bucketDocCount(i), keyed, formatter, bucketAggregations(i))); + } + + // the contract of the histogram aggregation is that shards must return buckets ordered by key in ascending order + CollectionUtil.introSort(buckets, InternalOrder.KEY_ASC.comparator()); + + // value source will be null for unmapped fields + InternalDateHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 + ? new InternalDateHistogram.EmptyBucketInfo(rounding, buildEmptySubAggregations(), extendedBounds) + : null; + return new InternalDateHistogram(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed, + pipelineAggregators(), metaData()); + } + + @Override + public InternalAggregation buildEmptyAggregation() { + InternalDateHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 + ? new InternalDateHistogram.EmptyBucketInfo(rounding, buildEmptySubAggregations(), extendedBounds) + : null; + return new InternalDateHistogram(name, Collections.emptyList(), order, minDocCount, emptyBucketInfo, formatter, keyed, + pipelineAggregators(), metaData()); + } + + @Override + public void doClose() { + Releasables.close(bucketOrds); + } +} diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorFactory.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorFactory.java index 949542a762539..17c6d82a9c3fc 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorFactory.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregatorFactory.java @@ -23,23 +23,29 @@ import org.elasticsearch.common.rounding.Rounding; import org.elasticsearch.common.rounding.TimeZoneRounding; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.InternalAggregation.Type; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; import java.io.IOException; import java.util.HashMap; +import java.util.List; import java.util.Map; import static java.util.Collections.unmodifiableMap; import org.elasticsearch.search.aggregations.support.AggregationContext; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; -public class DateHistogramAggregatorFactory extends AbstractHistogramAggregatorFactory { +public final class DateHistogramAggregatorFactory + extends ValuesSourceAggregatorFactory { + public static final Map DATE_FIELD_UNITS; - private final DateHistogramInterval dateHistogramInterval; static { Map dateFieldUnits = new HashMap<>(); @@ -62,17 +68,33 @@ public class DateHistogramAggregatorFactory extends AbstractHistogramAggregatorF DATE_FIELD_UNITS = unmodifiableMap(dateFieldUnits); } + private final DateHistogramInterval dateHistogramInterval; + private final long interval; + private final long offset; + private final InternalOrder order; + private final boolean keyed; + private final long minDocCount; + private final ExtendedBounds extendedBounds; + public DateHistogramAggregatorFactory(String name, Type type, ValuesSourceConfig config, long interval, DateHistogramInterval dateHistogramInterval, long offset, InternalOrder order, boolean keyed, long minDocCount, ExtendedBounds extendedBounds, AggregationContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { - super(name, type, config, interval, offset, order, keyed, minDocCount, extendedBounds, InternalDateHistogram.HISTOGRAM_FACTORY, - context, parent, subFactoriesBuilder, metaData); + super(name, type, config, context, parent, subFactoriesBuilder, metaData); + this.interval = interval; this.dateHistogramInterval = dateHistogramInterval; + this.offset = offset; + this.order = order; + this.keyed = keyed; + this.minDocCount = minDocCount; + this.extendedBounds = extendedBounds; } - @Override - protected Rounding createRounding() { + public long minDocCount() { + return minDocCount; + } + + private Rounding createRounding() { TimeZoneRounding.Builder tzRoundingBuilder; if (dateHistogramInterval != null) { DateTimeUnit dateTimeUnit = DATE_FIELD_UNITS.get(dateHistogramInterval.toString()); @@ -94,4 +116,35 @@ protected Rounding createRounding() { return rounding; } + @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 { + Rounding rounding = createRounding(); + // we need to round the bounds given by the user and we have to do it + // for every aggregator we create + // as the rounding is not necessarily an idempotent operation. + // todo we need to think of a better structure to the factory/agtor + // code so we won't need to do that + ExtendedBounds roundedBounds = null; + if (extendedBounds != null) { + // parse any string bounds to longs and round them + roundedBounds = extendedBounds.parseAndValidate(name, context.searchContext(), config.format()).round(rounding); + } + return new DateHistogramAggregator(name, factories, rounding, order, keyed, minDocCount, roundedBounds, 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/DateHistogramParser.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramParser.java index b033de5597e34..f139ad18bb0b9 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramParser.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramParser.java @@ -19,8 +19,12 @@ package org.elasticsearch.search.aggregations.bucket.histogram; import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.rounding.Rounding; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentParser.Token; +import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.NumericValuesSourceParser; import org.elasticsearch.search.aggregations.support.ValueType; import org.elasticsearch.search.aggregations.support.ValuesSourceType; @@ -28,17 +32,13 @@ import java.util.Map; /** - * + * A parser for date histograms. This translates json into a + * {@link DateHistogramAggregationBuilder} instance. */ -public class DateHistogramParser extends HistogramParser { +public class DateHistogramParser extends NumericValuesSourceParser { public DateHistogramParser() { - super(true); - } - - @Override - protected Object parseStringInterval(String text) { - return new DateHistogramInterval(text); + super(true, true, true); } @Override @@ -52,6 +52,8 @@ protected DateHistogramAggregationBuilder createFactory(String aggregationName, factory.interval((Long) interval); } else if (interval instanceof DateHistogramInterval) { factory.dateHistogramInterval((DateHistogramInterval) interval); + } else { + throw new IllegalStateException("Unexpected interval class: " + interval.getClass()); } Long offset = (Long) otherOptions.get(Rounding.OffsetRounding.OFFSET_FIELD); if (offset != null) { @@ -62,21 +64,85 @@ protected DateHistogramAggregationBuilder createFactory(String aggregationName, if (extendedBounds != null) { factory.extendedBounds(extendedBounds); } - Boolean keyed = (Boolean) otherOptions.get(HistogramAggregator.KEYED_FIELD); + Boolean keyed = (Boolean) otherOptions.get(Histogram.KEYED_FIELD); if (keyed != null) { factory.keyed(keyed); } - Long minDocCount = (Long) otherOptions.get(HistogramAggregator.MIN_DOC_COUNT_FIELD); + Long minDocCount = (Long) otherOptions.get(Histogram.MIN_DOC_COUNT_FIELD); if (minDocCount != null) { factory.minDocCount(minDocCount); } - InternalOrder order = (InternalOrder) otherOptions.get(HistogramAggregator.ORDER_FIELD); + InternalOrder order = (InternalOrder) otherOptions.get(Histogram.ORDER_FIELD); if (order != null) { factory.order(order); } return factory; } + @Override + protected boolean token(String aggregationName, String currentFieldName, Token token, XContentParser parser, + ParseFieldMatcher parseFieldMatcher, Map otherOptions) throws IOException { + if (token.isValue()) { + if (parseFieldMatcher.match(currentFieldName, Rounding.Interval.INTERVAL_FIELD)) { + if (token == XContentParser.Token.VALUE_STRING) { + otherOptions.put(Rounding.Interval.INTERVAL_FIELD, new DateHistogramInterval(parser.text())); + return true; + } else { + otherOptions.put(Rounding.Interval.INTERVAL_FIELD, parser.longValue()); + return true; + } + } else if (parseFieldMatcher.match(currentFieldName, Histogram.MIN_DOC_COUNT_FIELD)) { + otherOptions.put(Histogram.MIN_DOC_COUNT_FIELD, parser.longValue()); + return true; + } else if (parseFieldMatcher.match(currentFieldName, Histogram.KEYED_FIELD)) { + otherOptions.put(Histogram.KEYED_FIELD, parser.booleanValue()); + return true; + } else if (parseFieldMatcher.match(currentFieldName, Rounding.OffsetRounding.OFFSET_FIELD)) { + if (token == XContentParser.Token.VALUE_STRING) { + otherOptions.put(Rounding.OffsetRounding.OFFSET_FIELD, + DateHistogramAggregationBuilder.parseStringOffset(parser.text())); + return true; + } else { + otherOptions.put(Rounding.OffsetRounding.OFFSET_FIELD, parser.longValue()); + return true; + } + } else { + return false; + } + } else if (token == XContentParser.Token.START_OBJECT) { + if (parseFieldMatcher.match(currentFieldName, Histogram.ORDER_FIELD)) { + InternalOrder order = null; + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + } else if (token == XContentParser.Token.VALUE_STRING) { + String dir = parser.text(); + boolean asc = "asc".equals(dir); + if (!asc && !"desc".equals(dir)) { + throw new ParsingException(parser.getTokenLocation(), "Unknown order direction in aggregation [" + + aggregationName + "]: [" + dir + + "]. Should be either [asc] or [desc]"); + } + order = resolveOrder(currentFieldName, asc); + } + } + otherOptions.put(Histogram.ORDER_FIELD, order); + return true; + } else if (parseFieldMatcher.match(currentFieldName, ExtendedBounds.EXTENDED_BOUNDS_FIELD)) { + try { + otherOptions.put(ExtendedBounds.EXTENDED_BOUNDS_FIELD, ExtendedBounds.PARSER.apply(parser, () -> parseFieldMatcher)); + } catch (Exception e) { + throw new ParsingException(parser.getTokenLocation(), "Error parsing [{}]", e, aggregationName); + } + return true; + } else { + return false; + } + } else { + return false; + } + } + static InternalOrder resolveOrder(String key, boolean asc) { if ("_key".equals(key) || "_time".equals(key)) { return (InternalOrder) (asc ? InternalOrder.KEY_ASC : InternalOrder.KEY_DESC); @@ -86,9 +152,4 @@ static InternalOrder resolveOrder(String key, boolean asc) { } return new InternalOrder.Aggregation(key, asc); } - - @Override - protected long parseStringOffset(String offset) throws IOException { - return DateHistogramAggregationBuilder.parseStringOffset(offset); - } } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/ExtendedBounds.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/ExtendedBounds.java index ee424bbf59ed8..46fae19e49f9d 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/ExtendedBounds.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/ExtendedBounds.java @@ -41,7 +41,7 @@ import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; public class ExtendedBounds implements ToXContent, Writeable { - static final ParseField EXTENDED_BOUNDS_FIELD = new ParseField("extended_bounds"); + static final ParseField EXTENDED_BOUNDS_FIELD = Histogram.EXTENDED_BOUNDS_FIELD; static final ParseField MIN_FIELD = new ParseField("min"); static final ParseField MAX_FIELD = new ParseField("max"); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/Histogram.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/Histogram.java index a70e54190965e..9453ecef59690 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/Histogram.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/Histogram.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.search.aggregations.bucket.histogram; +import org.elasticsearch.common.ParseField; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation; @@ -29,6 +30,13 @@ */ public interface Histogram extends MultiBucketsAggregation { + ParseField INTERVAL_FIELD = new ParseField("interval"); + ParseField OFFSET_FIELD = new ParseField("offset"); + ParseField ORDER_FIELD = new ParseField("order"); + ParseField KEYED_FIELD = new ParseField("keyed"); + ParseField MIN_DOC_COUNT_FIELD = new ParseField("min_doc_count"); + ParseField EXTENDED_BOUNDS_FIELD = new ParseField("extended_bounds"); + /** * A bucket in the histogram where documents fall in */ @@ -40,7 +48,7 @@ interface Bucket extends MultiBucketsAggregation.Bucket { * @return The buckets of this histogram (each bucket representing an interval in the histogram) */ @Override - List getBuckets(); + List getBuckets(); /** @@ -48,38 +56,48 @@ interface Bucket extends MultiBucketsAggregation.Bucket { */ abstract class Order implements ToXContent { - public static final Order KEY_ASC = new InternalOrder((byte) 1, "_key", true, new Comparator() { + private static int compareKey(Histogram.Bucket b1, Histogram.Bucket b2) { + if (b1 instanceof InternalHistogram.Bucket) { + return Double.compare(((InternalHistogram.Bucket) b1).key, ((InternalHistogram.Bucket) b2).key); + } else if (b1 instanceof InternalDateHistogram.Bucket) { + return Long.compare(((InternalDateHistogram.Bucket) b1).key, ((InternalDateHistogram.Bucket) b2).key); + } else { + throw new IllegalStateException("Unexpected impl: " + b1.getClass()); + } + } + + public static final Order KEY_ASC = new InternalOrder((byte) 1, "_key", true, new Comparator() { @Override - public int compare(InternalHistogram.Bucket b1, InternalHistogram.Bucket b2) { - return Long.compare(b1.key, b2.key); + public int compare(Histogram.Bucket b1, Histogram.Bucket b2) { + return compareKey(b1, b2); } }); - public static final Order KEY_DESC = new InternalOrder((byte) 2, "_key", false, new Comparator() { + public static final Order KEY_DESC = new InternalOrder((byte) 2, "_key", false, new Comparator() { @Override - public int compare(InternalHistogram.Bucket b1, InternalHistogram.Bucket b2) { - return -Long.compare(b1.key, b2.key); + public int compare(Histogram.Bucket b1, Histogram.Bucket b2) { + return compareKey(b2, b1); } }); - public static final Order COUNT_ASC = new InternalOrder((byte) 3, "_count", true, new Comparator() { + public static final Order COUNT_ASC = new InternalOrder((byte) 3, "_count", true, new Comparator() { @Override - public int compare(InternalHistogram.Bucket b1, InternalHistogram.Bucket b2) { + public int compare(Histogram.Bucket b1, Histogram.Bucket b2) { int cmp = Long.compare(b1.getDocCount(), b2.getDocCount()); if (cmp == 0) { - cmp = Long.compare(b1.key, b2.key); + cmp = compareKey(b1, b2); } return cmp; } }); - public static final Order COUNT_DESC = new InternalOrder((byte) 4, "_count", false, new Comparator() { + public static final Order COUNT_DESC = new InternalOrder((byte) 4, "_count", false, new Comparator() { @Override - public int compare(InternalHistogram.Bucket b1, InternalHistogram.Bucket b2) { - int cmp = -Long.compare(b1.getDocCount(), b2.getDocCount()); + public int compare(Histogram.Bucket b1, Histogram.Bucket b2) { + int cmp = Long.compare(b2.getDocCount(), b1.getDocCount()); if (cmp == 0) { - cmp = Long.compare(b1.key, b2.key); + cmp = compareKey(b1, b2); } return cmp; } @@ -109,7 +127,7 @@ public static Order aggregation(String aggregationName, String valueName, boolea /** * @return The bucket comparator by which the order will be applied. */ - abstract Comparator comparator(); + abstract Comparator comparator(); } } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregationBuilder.java index 54d52466bbba4..10f339a37c877 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregationBuilder.java @@ -21,38 +21,224 @@ import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.support.AggregationContext; +import org.elasticsearch.search.aggregations.support.ValueType; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder; 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.aggregations.support.ValuesSourceType; import java.io.IOException; +import java.util.Objects; -public class HistogramAggregationBuilder extends AbstractHistogramBuilder { +/** + * A builder for histograms on numeric fields. + */ +public class HistogramAggregationBuilder + extends ValuesSourceAggregationBuilder { public static final String NAME = InternalHistogram.TYPE.name(); public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); + private double interval; + private double offset = 0; + private double minBound = Double.MAX_VALUE; + private double maxBound = Double.MIN_VALUE; + private InternalOrder order = (InternalOrder) Histogram.Order.KEY_ASC; + private boolean keyed = false; + private long minDocCount = 0; + + /** Create a new builder with the given name. */ public HistogramAggregationBuilder(String name) { - super(name, InternalHistogram.HISTOGRAM_FACTORY); + super(name, InternalHistogram.TYPE, ValuesSourceType.NUMERIC, ValueType.DOUBLE); } - /** - * Read from a stream. - */ + /** Read from a stream, for internal use only. */ public HistogramAggregationBuilder(StreamInput in) throws IOException { - super(in, InternalHistogram.HISTOGRAM_FACTORY); + super(in, InternalHistogram.TYPE, ValuesSourceType.NUMERIC, ValueType.DOUBLE); + if (in.readBoolean()) { + order = InternalOrder.Streams.readOrder(in); + } + keyed = in.readBoolean(); + minDocCount = in.readVLong(); + interval = in.readDouble(); + offset = in.readDouble(); + minBound = in.readDouble(); + maxBound = in.readDouble(); } @Override - protected HistogramAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig config, - AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { - return new HistogramAggregatorFactory(name, type, config, interval, offset, order, keyed, minDocCount, extendedBounds, context, - parent, subFactoriesBuilder, metaData); + protected void innerWriteTo(StreamOutput out) throws IOException { + boolean hasOrder = order != null; + out.writeBoolean(hasOrder); + if (hasOrder) { + InternalOrder.Streams.writeOrder(order, out); + } + out.writeBoolean(keyed); + out.writeVLong(minDocCount); + out.writeDouble(interval); + out.writeDouble(offset); + out.writeDouble(minBound); + out.writeDouble(maxBound); + } + + /** Get the current interval that is set on this builder. */ + public double interval() { + return interval; + } + + /** Set the interval on this builder, and return the builder so that calls can be chained. */ + public HistogramAggregationBuilder interval(double interval) { + if (interval <= 0) { + throw new IllegalArgumentException("[interval] must be >0 for histogram aggregation [" + name + "]"); + } + this.interval = interval; + return this; + } + + /** Get the current offset that is set on this builder. */ + public double offset() { + return offset; + } + + /** Set the offset on this builder, and return the builder so that calls can be chained. */ + public HistogramAggregationBuilder offset(double offset) { + this.offset = offset; + return this; + } + + /** Get the current minimum bound that is set on this builder. */ + public double minBound() { + return minBound; + } + + /** Get the current maximum bound that is set on this builder. */ + public double maxBound() { + return maxBound; + } + + /** Set extended bounds on this builder: buckets between {@code minBound} + * and {@code maxBound} will be created even if no documents fell into + * these buckets. It is possible to create half-open bounds by providing + * {@link Double#POSITIVE_INFINITY} as a {@code minBound} or + * {@link Double#NEGATIVE_INFINITY} as a {@code maxBound}. */ + public HistogramAggregationBuilder extendedBounds(double minBound, double maxBound) { + if (minBound == Double.NEGATIVE_INFINITY) { + throw new IllegalArgumentException("minBound must not be -Infinity, got: " + minBound); + } + if (maxBound == Double.POSITIVE_INFINITY) { + throw new IllegalArgumentException("maxBound must not be +Infinity, got: " + maxBound); + } + this.minBound = minBound; + this.maxBound = maxBound; + return this; + } + + /** Return the order to use to sort buckets of this histogram. */ + public Histogram.Order order() { + return order; + } + + /** Set a new order on this builder and return the builder so that calls + * can be chained. */ + public HistogramAggregationBuilder order(Histogram.Order order) { + if (order == null) { + throw new IllegalArgumentException("[order] must not be null: [" + name + "]"); + } + this.order = (InternalOrder) order; + return this; + } + + /** Return whether buckets should be returned as a hash. In case + * {@code keyed} is false, buckets will be returned as an array. */ + public boolean keyed() { + return keyed; + } + + /** Set whether to return buckets as a hash or as an array, and return the + * builder so that calls can be chained. */ + public HistogramAggregationBuilder keyed(boolean keyed) { + this.keyed = keyed; + return this; + } + + /** Return the minimum count of documents that buckets need to have in order + * to be included in the response. */ + public long minDocCount() { + return minDocCount; + } + + /** Set the minimum count of matching documents that buckets need to have + * and return this builder so that calls can be chained. */ + public HistogramAggregationBuilder minDocCount(long minDocCount) { + if (minDocCount < 0) { + throw new IllegalArgumentException( + "[minDocCount] must be greater than or equal to 0. Found [" + minDocCount + "] in [" + name + "]"); + } + this.minDocCount = minDocCount; + return this; + } + + @Override + protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + + builder.field(Histogram.INTERVAL_FIELD.getPreferredName(), interval); + builder.field(Histogram.OFFSET_FIELD.getPreferredName(), offset); + + if (order != null) { + builder.field(Histogram.ORDER_FIELD.getPreferredName()); + order.toXContent(builder, params); + } + + builder.field(Histogram.KEYED_FIELD.getPreferredName(), keyed); + + builder.field(Histogram.MIN_DOC_COUNT_FIELD.getPreferredName(), minDocCount); + + if (Double.isFinite(minBound) || Double.isFinite(maxBound)) { + builder.startObject(Histogram.EXTENDED_BOUNDS_FIELD.getPreferredName()); + if (Double.isFinite(minBound)) { + builder.field("min", minBound); + } + if (Double.isFinite(maxBound)) { + builder.field("max", maxBound); + } + builder.endObject(); + } + + return builder; } @Override public String getWriteableName() { - return NAME; + return InternalHistogram.TYPE.name(); + } + + @Override + protected ValuesSourceAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig config, + AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + return new HistogramAggregatorFactory(name, type, config, interval, offset, order, keyed, minDocCount, minBound, maxBound, + context, parent, subFactoriesBuilder, metaData); + } + + @Override + protected int innerHashCode() { + return Objects.hash(order, keyed, minDocCount, interval, offset, minBound, maxBound); + } + + @Override + protected boolean innerEquals(Object obj) { + HistogramAggregationBuilder other = (HistogramAggregationBuilder) obj; + return Objects.equals(order, other.order) + && Objects.equals(keyed, other.keyed) + && Objects.equals(minDocCount, other.minDocCount) + && Objects.equals(interval, other.interval) + && Objects.equals(offset, other.offset) + && Objects.equals(minBound, other.minBound) + && Objects.equals(maxBound, other.maxBound); } } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregator.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregator.java index ca797b06049eb..7d102578a720c 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregator.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregator.java @@ -16,16 +16,15 @@ * 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.ParseField; 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.index.fielddata.SortedNumericDoubleValues; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; @@ -33,6 +32,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; +import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram.EmptyBucketInfo; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; @@ -43,38 +43,43 @@ import java.util.List; import java.util.Map; -public class HistogramAggregator extends BucketsAggregator { - - public static final ParseField ORDER_FIELD = new ParseField("order"); - public static final ParseField KEYED_FIELD = new ParseField("keyed"); - public static final ParseField MIN_DOC_COUNT_FIELD = new ParseField("min_doc_count"); +/** + * An aggregator for numeric values. For a given {@code interval}, + * {@code offset} and {@code value}, it returns the highest number that can be + * written as {@code interval * x + offset} and yet is less than or equal to + * {@code value}. + */ +class HistogramAggregator extends BucketsAggregator { private final ValuesSource.Numeric valuesSource; private final DocValueFormat formatter; - private final Rounding rounding; + private final double interval, offset; private final InternalOrder order; private final boolean keyed; - private final long minDocCount; - private final ExtendedBounds extendedBounds; - private final InternalHistogram.Factory histogramFactory; + private final double minBound, maxBound; private final LongHash bucketOrds; - public HistogramAggregator(String name, AggregatorFactories factories, Rounding rounding, InternalOrder order, boolean keyed, - long minDocCount, @Nullable ExtendedBounds extendedBounds, @Nullable ValuesSource.Numeric valuesSource, - DocValueFormat formatter, InternalHistogram.Factory histogramFactory, AggregationContext aggregationContext, - Aggregator parent, List pipelineAggregators, Map metaData) throws IOException { + public HistogramAggregator(String name, AggregatorFactories factories, double interval, double offset, + InternalOrder order, boolean keyed, long minDocCount, double minBound, double maxBound, + @Nullable ValuesSource.Numeric valuesSource, DocValueFormat formatter, + AggregationContext aggregationContext, Aggregator parent, + List pipelineAggregators, Map metaData) throws IOException { super(name, factories, aggregationContext, parent, pipelineAggregators, metaData); - this.rounding = rounding; + if (interval <= 0) { + throw new IllegalArgumentException("interval must be positive, got: " + interval); + } + this.interval = interval; + this.offset = offset; this.order = order; this.keyed = keyed; this.minDocCount = minDocCount; - this.extendedBounds = extendedBounds; + this.minBound = minBound; + this.maxBound = maxBound; this.valuesSource = valuesSource; this.formatter = formatter; - this.histogramFactory = histogramFactory; bucketOrds = new LongHash(1, aggregationContext.bigArrays()); } @@ -90,7 +95,8 @@ public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, if (valuesSource == null) { return LeafBucketCollector.NO_OP_COLLECTOR; } - final SortedNumericDocValues values = valuesSource.longValues(ctx); + + final SortedNumericDoubleValues values = valuesSource.doubleValues(ctx); return new LeafBucketCollectorBase(sub, values) { @Override public void collect(int doc, long bucket) throws IOException { @@ -98,15 +104,15 @@ public void collect(int doc, long bucket) throws IOException { values.setDocument(doc); final int valuesCount = values.count(); - long previousKey = Long.MIN_VALUE; + double previousKey = Double.NEGATIVE_INFINITY; for (int i = 0; i < valuesCount; ++i) { - long value = values.valueAt(i); - long key = rounding.roundKey(value); + double value = values.valueAt(i); + double key = Math.floor((value - offset) / interval); assert key >= previousKey; if (key == previousKey) { continue; } - long bucketOrd = bucketOrds.add(key); + long bucketOrd = bucketOrds.add(Double.doubleToLongBits(key)); if (bucketOrd < 0) { // already seen bucketOrd = -1 - bucketOrd; collectExistingBucket(sub, doc, bucketOrd); @@ -120,26 +126,32 @@ public void collect(int doc, long bucket) throws IOException { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - assert owningBucketOrdinal == 0; + public InternalAggregation buildAggregation(long bucket) throws IOException { + assert bucket == 0; List buckets = new ArrayList<>((int) bucketOrds.size()); for (long i = 0; i < bucketOrds.size(); i++) { - buckets.add(histogramFactory.createBucket(rounding.valueForKey(bucketOrds.get(i)), bucketDocCount(i), bucketAggregations(i), keyed, formatter)); + double roundKey = Double.longBitsToDouble(bucketOrds.get(i)); + double key = roundKey * interval + offset; + buckets.add(new InternalHistogram.Bucket(key, bucketDocCount(i), keyed, formatter, bucketAggregations(i))); } // the contract of the histogram aggregation is that shards must return buckets ordered by key in ascending order CollectionUtil.introSort(buckets, InternalOrder.KEY_ASC.comparator()); - // value source will be null for unmapped fields - InternalHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 ? new InternalHistogram.EmptyBucketInfo(rounding, buildEmptySubAggregations(), extendedBounds) : null; - return histogramFactory.create(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed, pipelineAggregators(), metaData()); + EmptyBucketInfo emptyBucketInfo = null; + if (minDocCount == 0) { + emptyBucketInfo = new EmptyBucketInfo(interval, offset, minBound, maxBound, buildEmptySubAggregations()); + } + return new InternalHistogram(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed, pipelineAggregators(), metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - InternalHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 ? new InternalHistogram.EmptyBucketInfo(rounding, buildEmptySubAggregations(), extendedBounds) : null; - return histogramFactory.create(name, Collections.emptyList(), order, minDocCount, emptyBucketInfo, formatter, keyed, pipelineAggregators(), - metaData()); + EmptyBucketInfo emptyBucketInfo = null; + if (minDocCount == 0) { + emptyBucketInfo = new EmptyBucketInfo(interval, offset, minBound, maxBound, buildEmptySubAggregations()); + } + return new InternalHistogram(name, Collections.emptyList(), order, minDocCount, emptyBucketInfo, formatter, keyed, pipelineAggregators(), metaData()); } @Override diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregatorFactory.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregatorFactory.java index a862aadeb62c1..805aab9ecf5df 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregatorFactory.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregatorFactory.java @@ -19,24 +19,66 @@ package org.elasticsearch.search.aggregations.bucket.histogram; +import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.InternalAggregation.Type; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.elasticsearch.search.aggregations.support.AggregationContext; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; +import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; import java.io.IOException; +import java.util.List; import java.util.Map; -import org.elasticsearch.search.aggregations.support.AggregationContext; -import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; +public final class HistogramAggregatorFactory extends ValuesSourceAggregatorFactory { + + private final double interval, offset; + private final InternalOrder order; + private final boolean keyed; + private final long minDocCount; + private final double minBound, maxBound; + + HistogramAggregatorFactory(String name, Type type, ValuesSourceConfig config, double interval, double offset, + InternalOrder order, boolean keyed, long minDocCount, double minBound, double maxBound, + AggregationContext context, AggregatorFactory parent, + AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { + super(name, type, config, context, parent, subFactoriesBuilder, metaData); + this.interval = interval; + this.offset = offset; + this.order = order; + this.keyed = keyed; + this.minDocCount = minDocCount; + this.minBound = minBound; + this.maxBound = maxBound; + } + + public long minDocCount() { + return minDocCount; + } -public class HistogramAggregatorFactory extends AbstractHistogramAggregatorFactory { + @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 { - public HistogramAggregatorFactory(String name, Type type, ValuesSourceConfig config, long interval, long offset, - InternalOrder order, boolean keyed, long minDocCount, ExtendedBounds extendedBounds, AggregationContext context, - AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, Map metaData) throws IOException { - super(name, type, config, interval, offset, order, keyed, minDocCount, extendedBounds, InternalHistogram.HISTOGRAM_FACTORY, context, - parent, subFactoriesBuilder, metaData); + return new HistogramAggregator(name, factories, interval, offset, order, keyed, minDocCount, minBound, maxBound, 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); + } +} \ No newline at end of file diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramFactory.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramFactory.java new file mode 100644 index 0000000000000..bafa1265fb202 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramFactory.java @@ -0,0 +1,48 @@ +/* + * 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.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation; + +import java.util.List; + +/** Implemented by histogram aggregations and used by pipeline aggregations to insert buckets. */ +// public so that pipeline aggs can use this API: can we fix it? +public interface HistogramFactory { + + /** Get the key for the given bucket. Date histograms must return the + * number of millis since Epoch of the bucket key while numeric histograms + * must return the double value of the key. */ + Number getKey(MultiBucketsAggregation.Bucket bucket); + + /** Given a key returned by {@link #getKey}, compute the lowest key that is + * greater than it. */ + Number nextKey(Number key); + + /** Create an {@link InternalAggregation} object that wraps the given buckets. */ + InternalAggregation createAggregation(List buckets); + + /** Create a {@link MultiBucketsAggregation.Bucket} object that wraps the + * given key, document count and aggregations. */ + MultiBucketsAggregation.Bucket createBucket(Number key, long docCount, InternalAggregations aggregations); + +} diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramParser.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramParser.java index 6897fd2e7f32d..69aed3e733a64 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramParser.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramParser.java @@ -20,8 +20,9 @@ import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseFieldMatcher; +import org.elasticsearch.common.ParseFieldMatcherSupplier; import org.elasticsearch.common.ParsingException; -import org.elasticsearch.common.rounding.Rounding; +import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser.Token; import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.NumericValuesSourceParser; @@ -32,46 +33,51 @@ import java.util.Map; /** - * Parses the histogram request + * A parser for date histograms. This translates json into an + * {@link HistogramAggregationBuilder} instance. */ public class HistogramParser extends NumericValuesSourceParser { - public HistogramParser() { - super(true, true, false); + private static final ObjectParser EXTENDED_BOUNDS_PARSER = new ObjectParser<>( + Histogram.EXTENDED_BOUNDS_FIELD.getPreferredName(), + () -> new double[]{ Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY }); + static { + EXTENDED_BOUNDS_PARSER.declareDouble((bounds, d) -> bounds[0] = d, new ParseField("min")); + EXTENDED_BOUNDS_PARSER.declareDouble((bounds, d) -> bounds[1] = d, new ParseField("max")); } - protected HistogramParser(boolean timezoneAware) { - super(true, true, timezoneAware); + public HistogramParser() { + super(true, true, false); } @Override - protected AbstractHistogramBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType, + protected HistogramAggregationBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType, ValueType targetValueType, Map otherOptions) { HistogramAggregationBuilder factory = new HistogramAggregationBuilder(aggregationName); - Long interval = (Long) otherOptions.get(Rounding.Interval.INTERVAL_FIELD); + Double interval = (Double) otherOptions.get(Histogram.INTERVAL_FIELD); if (interval == null) { throw new ParsingException(null, "Missing required field [interval] for histogram aggregation [" + aggregationName + "]"); } else { factory.interval(interval); } - Long offset = (Long) otherOptions.get(Rounding.OffsetRounding.OFFSET_FIELD); + Double offset = (Double) otherOptions.get(Histogram.OFFSET_FIELD); if (offset != null) { factory.offset(offset); } - ExtendedBounds extendedBounds = (ExtendedBounds) otherOptions.get(ExtendedBounds.EXTENDED_BOUNDS_FIELD); + double[] extendedBounds = (double[]) otherOptions.get(Histogram.EXTENDED_BOUNDS_FIELD); if (extendedBounds != null) { - factory.extendedBounds(extendedBounds); + factory.extendedBounds(extendedBounds[0], extendedBounds[1]); } - Boolean keyed = (Boolean) otherOptions.get(HistogramAggregator.KEYED_FIELD); + Boolean keyed = (Boolean) otherOptions.get(Histogram.KEYED_FIELD); if (keyed != null) { factory.keyed(keyed); } - Long minDocCount = (Long) otherOptions.get(HistogramAggregator.MIN_DOC_COUNT_FIELD); + Long minDocCount = (Long) otherOptions.get(Histogram.MIN_DOC_COUNT_FIELD); if (minDocCount != null) { factory.minDocCount(minDocCount); } - InternalOrder order = (InternalOrder) otherOptions.get(HistogramAggregator.ORDER_FIELD); + InternalOrder order = (InternalOrder) otherOptions.get(Histogram.ORDER_FIELD); if (order != null) { factory.order(order); } @@ -82,33 +88,23 @@ protected AbstractHistogramBuilder createFactory(String aggregationName, Valu protected boolean token(String aggregationName, String currentFieldName, Token token, XContentParser parser, ParseFieldMatcher parseFieldMatcher, Map otherOptions) throws IOException { if (token.isValue()) { - if (parseFieldMatcher.match(currentFieldName, Rounding.Interval.INTERVAL_FIELD)) { - if (token == XContentParser.Token.VALUE_STRING) { - otherOptions.put(Rounding.Interval.INTERVAL_FIELD, parseStringInterval(parser.text())); - return true; - } else { - otherOptions.put(Rounding.Interval.INTERVAL_FIELD, parser.longValue()); - return true; - } - } else if (parseFieldMatcher.match(currentFieldName, HistogramAggregator.MIN_DOC_COUNT_FIELD)) { - otherOptions.put(HistogramAggregator.MIN_DOC_COUNT_FIELD, parser.longValue()); + if (parseFieldMatcher.match(currentFieldName, Histogram.INTERVAL_FIELD)) { + otherOptions.put(Histogram.INTERVAL_FIELD, parser.doubleValue()); return true; - } else if (parseFieldMatcher.match(currentFieldName, HistogramAggregator.KEYED_FIELD)) { - otherOptions.put(HistogramAggregator.KEYED_FIELD, parser.booleanValue()); + } else if (parseFieldMatcher.match(currentFieldName, Histogram.MIN_DOC_COUNT_FIELD)) { + otherOptions.put(Histogram.MIN_DOC_COUNT_FIELD, parser.longValue()); + return true; + } else if (parseFieldMatcher.match(currentFieldName, Histogram.KEYED_FIELD)) { + otherOptions.put(Histogram.KEYED_FIELD, parser.booleanValue()); + return true; + } else if (parseFieldMatcher.match(currentFieldName, Histogram.OFFSET_FIELD)) { + otherOptions.put(Histogram.OFFSET_FIELD, parser.doubleValue()); return true; - } else if (parseFieldMatcher.match(currentFieldName, Rounding.OffsetRounding.OFFSET_FIELD)) { - if (token == XContentParser.Token.VALUE_STRING) { - otherOptions.put(Rounding.OffsetRounding.OFFSET_FIELD, parseStringOffset(parser.text())); - return true; - } else { - otherOptions.put(Rounding.OffsetRounding.OFFSET_FIELD, parser.longValue()); - return true; - } } else { return false; } } else if (token == XContentParser.Token.START_OBJECT) { - if (parseFieldMatcher.match(currentFieldName, HistogramAggregator.ORDER_FIELD)) { + if (parseFieldMatcher.match(currentFieldName, Histogram.ORDER_FIELD)) { InternalOrder order = null; while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { if (token == XContentParser.Token.FIELD_NAME) { @@ -124,14 +120,11 @@ protected boolean token(String aggregationName, String currentFieldName, Token t order = resolveOrder(currentFieldName, asc); } } - otherOptions.put(HistogramAggregator.ORDER_FIELD, order); + otherOptions.put(Histogram.ORDER_FIELD, order); return true; - } else if (parseFieldMatcher.match(currentFieldName, ExtendedBounds.EXTENDED_BOUNDS_FIELD)) { - try { - otherOptions.put(ExtendedBounds.EXTENDED_BOUNDS_FIELD, ExtendedBounds.PARSER.apply(parser, () -> parseFieldMatcher)); - } catch (Exception e) { - throw new ParsingException(parser.getTokenLocation(), "Error parsing [{}]", e, aggregationName); - } + } else if (parseFieldMatcher.match(currentFieldName, Histogram.EXTENDED_BOUNDS_FIELD)) { + double[] bounds = EXTENDED_BOUNDS_PARSER.apply(parser, () -> parseFieldMatcher); + otherOptions.put(Histogram.EXTENDED_BOUNDS_FIELD, bounds); return true; } else { return false; @@ -141,14 +134,6 @@ protected boolean token(String aggregationName, String currentFieldName, Token t } } - protected Object parseStringInterval(String interval) { - return Long.valueOf(interval); - } - - protected long parseStringOffset(String offset) throws IOException { - return Long.valueOf(offset); - } - static InternalOrder resolveOrder(String key, boolean asc) { if ("_key".equals(key)) { return (InternalOrder) (asc ? InternalOrder.KEY_ASC : InternalOrder.KEY_DESC); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java index 0dfc9c8e5326f..4d46c2c185008 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java @@ -18,36 +18,71 @@ */ package org.elasticsearch.search.aggregations.bucket.histogram; +import org.apache.lucene.util.CollectionUtil; +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.AggregationExecutionException; -import org.elasticsearch.search.aggregations.InternalAggregation.Type; +import org.elasticsearch.search.aggregations.Aggregations; +import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregations; -import org.elasticsearch.search.aggregations.support.ValueType; +import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; +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.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.ListIterator; +import java.util.Map; /** - * Results of a date_historgram aggregation. + * Imelementation of {@link Histogram}. */ -public class InternalDateHistogram { +public final class InternalDateHistogram extends InternalMultiBucketAggregation + implements Histogram, HistogramFactory { - public static final Factory HISTOGRAM_FACTORY = new Factory(); static final Type TYPE = new Type("date_histogram"); - static class Bucket extends InternalHistogram.Bucket { - Bucket(long key, long docCount, InternalAggregations aggregations, boolean keyed, DocValueFormat formatter, - InternalHistogram.Factory factory) { - super(key, docCount, keyed, formatter, factory, aggregations); + public static class Bucket extends InternalMultiBucketAggregation.InternalBucket implements Histogram.Bucket { + + final long key; + final long docCount; + final InternalAggregations aggregations; + private final transient boolean keyed; + protected final transient DocValueFormat format; + + public Bucket(long key, long docCount, boolean keyed, DocValueFormat format, + InternalAggregations aggregations) { + this.format = format; + this.keyed = keyed; + this.key = key; + this.docCount = docCount; + this.aggregations = aggregations; } /** * Read from a stream. */ - Bucket(StreamInput in, boolean keyed, DocValueFormat formatter, InternalHistogram.Factory factory) throws IOException { - super(in, keyed, formatter, factory); + public Bucket(StreamInput in, boolean keyed, DocValueFormat format) throws IOException { + this.format = format; + this.keyed = keyed; + key = in.readLong(); + docCount = in.readVLong(); + aggregations = InternalAggregations.readAggregations(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeLong(key); + out.writeVLong(docCount); + aggregations.writeTo(out); } @Override @@ -56,53 +91,356 @@ public String getKeyAsString() { } @Override - public DateTime getKey() { + public Object getKey() { return new DateTime(key, DateTimeZone.UTC); } @Override - public String toString() { - return getKeyAsString(); + public long getDocCount() { + return docCount; } - } - static class Factory extends InternalHistogram.Factory { + @Override + public Aggregations getAggregations() { + return aggregations; + } - Factory() { + Bucket reduce(List buckets, 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 InternalDateHistogram.Bucket(key, docCount, keyed, format, aggs); } @Override - public Type type() { - return TYPE; + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + String keyAsString = format.format(key); + if (keyed) { + builder.startObject(keyAsString); + } else { + builder.startObject(); + } + if (format != DocValueFormat.RAW) { + builder.field(CommonFields.KEY_AS_STRING, keyAsString); + } + builder.field(CommonFields.KEY, key); + builder.field(CommonFields.DOC_COUNT, docCount); + aggregations.toXContentInternal(builder, params); + builder.endObject(); + return builder; } - @Override - public ValueType valueType() { - return ValueType.DATE; + public DocValueFormat getFormatter() { + return format; } - @Override - public InternalDateHistogram.Bucket createBucket(InternalAggregations aggregations, InternalDateHistogram.Bucket prototype) { - return new Bucket(prototype.key, prototype.docCount, aggregations, prototype.getKeyed(), prototype.format, this); + public boolean getKeyed() { + return keyed; } + } - @Override - public InternalDateHistogram.Bucket createBucket(Object key, long docCount, InternalAggregations aggregations, boolean keyed, - DocValueFormat formatter) { - if (key instanceof Number) { - return new Bucket(((Number) key).longValue(), docCount, aggregations, keyed, formatter, this); - } else if (key instanceof DateTime) { - return new Bucket(((DateTime) key).getMillis(), docCount, aggregations, keyed, formatter, this); + static class EmptyBucketInfo { + + final Rounding rounding; + final InternalAggregations subAggregations; + final ExtendedBounds bounds; + + EmptyBucketInfo(Rounding rounding, InternalAggregations subAggregations) { + this(rounding, subAggregations, null); + } + + EmptyBucketInfo(Rounding rounding, InternalAggregations subAggregations, ExtendedBounds bounds) { + this.rounding = rounding; + this.subAggregations = subAggregations; + this.bounds = bounds; + } + + EmptyBucketInfo(StreamInput in) throws IOException { + rounding = Rounding.Streams.read(in); + subAggregations = InternalAggregations.readAggregations(in); + bounds = in.readOptionalWriteable(ExtendedBounds::new); + } + + void writeTo(StreamOutput out) throws IOException { + Rounding.Streams.write(rounding, out); + subAggregations.writeTo(out); + out.writeOptionalWriteable(bounds); + } + + } + + private final List buckets; + private final InternalOrder order; + private final DocValueFormat format; + private final boolean keyed; + private final long minDocCount; + private final EmptyBucketInfo emptyBucketInfo; + + InternalDateHistogram(String name, List buckets, InternalOrder order, long minDocCount, EmptyBucketInfo emptyBucketInfo, + DocValueFormat formatter, boolean keyed, List pipelineAggregators, + Map metaData) { + super(name, pipelineAggregators, metaData); + this.buckets = buckets; + this.order = order; + assert (minDocCount == 0) == (emptyBucketInfo != null); + this.minDocCount = minDocCount; + this.emptyBucketInfo = emptyBucketInfo; + this.format = formatter; + this.keyed = keyed; + } + + /** + * Stream from a stream. + */ + public InternalDateHistogram(StreamInput in) throws IOException { + super(in); + order = InternalOrder.Streams.readOrder(in); + minDocCount = in.readVLong(); + if (minDocCount == 0) { + emptyBucketInfo = new EmptyBucketInfo(in); + } else { + emptyBucketInfo = null; + } + format = in.readNamedWriteable(DocValueFormat.class); + keyed = in.readBoolean(); + buckets = in.readList(stream -> new Bucket(stream, keyed, format)); + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + InternalOrder.Streams.writeOrder(order, out); + out.writeVLong(minDocCount); + if (minDocCount == 0) { + emptyBucketInfo.writeTo(out); + } + out.writeNamedWriteable(format); + out.writeBoolean(keyed); + out.writeList(buckets); + } + + @Override + public String getWriteableName() { + return DateHistogramAggregationBuilder.NAME; + } + + @Override + public List getBuckets() { + return Collections.unmodifiableList(buckets); + } + + @Override + public InternalDateHistogram create(List buckets) { + return new InternalDateHistogram(name, buckets, order, minDocCount, emptyBucketInfo, format, + keyed, pipelineAggregators(), metaData); + } + + @Override + public Bucket createBucket(InternalAggregations aggregations, Bucket prototype) { + return new Bucket(prototype.key, prototype.docCount, prototype.keyed, prototype.format, aggregations); + } + + private static class IteratorAndCurrent { + + private final Iterator iterator; + private Bucket current; + + IteratorAndCurrent(Iterator iterator) { + this.iterator = iterator; + current = iterator.next(); + } + + } + + private List reduceBuckets(List aggregations, ReduceContext reduceContext) { + + 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) { + InternalDateHistogram histogram = (InternalDateHistogram) 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 = pq.top().current.key; + + do { + final IteratorAndCurrent top = pq.top(); + + if (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, reduceContext); + if (reduced.getDocCount() >= minDocCount) { + reducedBuckets.add(reduced); + } + currentBuckets.clear(); + key = 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, reduceContext); + if (reduced.getDocCount() >= minDocCount) { + reducedBuckets.add(reduced); + } + } + } + + return reducedBuckets; + } + + private void addEmptyBuckets(List list, ReduceContext reduceContext) { + Bucket lastBucket = null; + ExtendedBounds bounds = emptyBucketInfo.bounds; + ListIterator iter = list.listIterator(); + + // first adding all the empty buckets *before* the actual data (based on th extended_bounds.min the user requested) + InternalAggregations reducedEmptySubAggs = InternalAggregations.reduce(Collections.singletonList(emptyBucketInfo.subAggregations), + reduceContext); + if (bounds != null) { + Bucket firstBucket = iter.hasNext() ? list.get(iter.nextIndex()) : null; + if (firstBucket == null) { + if (bounds.getMin() != null && bounds.getMax() != null) { + long key = bounds.getMin(); + long max = bounds.getMax(); + while (key <= max) { + iter.add(new InternalDateHistogram.Bucket(key, 0, keyed, format, reducedEmptySubAggs)); + key = emptyBucketInfo.rounding.nextRoundingValue(key); + } + } } else { - throw new AggregationExecutionException("Expected key of type Number or DateTime but got [" + key + "]"); + if (bounds.getMin() != null) { + long key = bounds.getMin(); + if (key < firstBucket.key) { + while (key < firstBucket.key) { + iter.add(new InternalDateHistogram.Bucket(key, 0, keyed, format, reducedEmptySubAggs)); + key = emptyBucketInfo.rounding.nextRoundingValue(key); + } + } + } } } - @Override - protected Bucket readBucket(StreamInput in, boolean keyed, DocValueFormat format) throws IOException { - return new Bucket(in, keyed, format, this); + // now adding the empty buckets within the actual 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 = emptyBucketInfo.rounding.nextRoundingValue(lastBucket.key); + while (key < nextBucket.key) { + iter.add(new InternalDateHistogram.Bucket(key, 0, keyed, format, reducedEmptySubAggs)); + key = emptyBucketInfo.rounding.nextRoundingValue(key); + } + assert key == nextBucket.key; + } + lastBucket = iter.next(); + } + + // finally, adding the empty buckets *after* the actual data (based on the extended_bounds.max requested by the user) + if (bounds != null && lastBucket != null && bounds.getMax() != null && bounds.getMax() > lastBucket.key) { + long key = emptyBucketInfo.rounding.nextRoundingValue(lastBucket.key); + long max = bounds.getMax(); + while (key <= max) { + iter.add(new InternalDateHistogram.Bucket(key, 0, keyed, format, reducedEmptySubAggs)); + key = emptyBucketInfo.rounding.nextRoundingValue(key); + } + } + } + + @Override + public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { + List reducedBuckets = reduceBuckets(aggregations, reduceContext); + + // adding empty buckets if needed + if (minDocCount == 0) { + addEmptyBuckets(reducedBuckets, reduceContext); } + + if (order == InternalOrder.KEY_ASC) { + // nothing to do, data are already sorted since shards return + // sorted buckets and the merge-sort performed by reduceBuckets + // maintains order + } else if (order == InternalOrder.KEY_DESC) { + // we just need to reverse here... + List reverse = new ArrayList<>(reducedBuckets); + Collections.reverse(reverse); + reducedBuckets = reverse; + } else { + // sorted by sub-aggregation, need to fall back to a costly n*log(n) sort + CollectionUtil.introSort(reducedBuckets, order.comparator()); + } + + return new InternalDateHistogram(getName(), reducedBuckets, order, minDocCount, emptyBucketInfo, + format, keyed, pipelineAggregators(), getMetaData()); } - private InternalDateHistogram() {} + @Override + public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + if (keyed) { + builder.startObject(CommonFields.BUCKETS); + } else { + builder.startArray(CommonFields.BUCKETS); + } + for (Bucket bucket : buckets) { + bucket.toXContent(builder, params); + } + if (keyed) { + builder.endObject(); + } else { + 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 emptyBucketInfo.rounding.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 InternalDateHistogram(name, buckets2, order, minDocCount, emptyBucketInfo, format, + keyed, pipelineAggregators(), getMetaData()); + } + + @Override + public Bucket createBucket(Number key, long docCount, InternalAggregations aggregations) { + return new Bucket(key.longValue(), docCount, keyed, format, aggregations); + } } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogram.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogram.java index 29ae9a1016ae9..4dae51533db3b 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogram.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogram.java @@ -22,16 +22,14 @@ 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.AggregationExecutionException; 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.bucket.MultiBucketsAggregation; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; -import org.elasticsearch.search.aggregations.support.ValueType; import java.io.IOException; import java.util.ArrayList; @@ -42,28 +40,25 @@ import java.util.Map; /** - * TODO should be renamed to InternalNumericHistogram (see comment on {@link Histogram})? + * Imelementation of {@link Histogram}. */ -public class InternalHistogram extends InternalMultiBucketAggregation, B> - implements Histogram { +public final class InternalHistogram extends InternalMultiBucketAggregation + implements Histogram, HistogramFactory { - public static final Factory HISTOGRAM_FACTORY = new Factory(); static final Type TYPE = new Type("histogram"); public static class Bucket extends InternalMultiBucketAggregation.InternalBucket implements Histogram.Bucket { - final long key; + final double key; final long docCount; final InternalAggregations aggregations; private final transient boolean keyed; protected final transient DocValueFormat format; - private final Factory factory; - public Bucket(long key, long docCount, boolean keyed, DocValueFormat format, Factory factory, + public Bucket(double key, long docCount, boolean keyed, DocValueFormat format, InternalAggregations aggregations) { this.format = format; this.keyed = keyed; - this.factory = factory; this.key = key; this.docCount = docCount; this.aggregations = aggregations; @@ -72,26 +67,21 @@ public Bucket(long key, long docCount, boolean keyed, DocValueFormat format, Fac /** * Read from a stream. */ - public Bucket(StreamInput in, boolean keyed, DocValueFormat format, Factory factory) throws IOException { + public Bucket(StreamInput in, boolean keyed, DocValueFormat format) throws IOException { this.format = format; this.keyed = keyed; - this.factory = factory; - key = in.readLong(); + key = in.readDouble(); docCount = in.readVLong(); aggregations = InternalAggregations.readAggregations(in); } @Override public void writeTo(StreamOutput out) throws IOException { - out.writeLong(key); + out.writeDouble(key); out.writeVLong(docCount); aggregations.writeTo(out); } - protected Factory getFactory() { - return factory; - } - @Override public String getKeyAsString() { return format.format(key); @@ -112,8 +102,7 @@ public Aggregations getAggregations() { return aggregations; } - @SuppressWarnings("unchecked") - B reduce(List buckets, ReduceContext context) { + Bucket reduce(List buckets, ReduceContext context) { List aggregations = new ArrayList<>(buckets.size()); long docCount = 0; for (Bucket bucket : buckets) { @@ -121,7 +110,7 @@ B reduce(List buckets, ReduceContext context) { aggregations.add((InternalAggregations) bucket.getAggregations()); } InternalAggregations aggs = InternalAggregations.reduce(aggregations, context); - return (B) getFactory().createBucket(key, docCount, aggs, keyed, format); + return new InternalHistogram.Bucket(key, docCount, keyed, format, aggs); } @Override @@ -153,96 +142,40 @@ public boolean getKeyed() { static class EmptyBucketInfo { - final Rounding rounding; + final double interval, offset, minBound, maxBound; final InternalAggregations subAggregations; - final ExtendedBounds bounds; - - EmptyBucketInfo(Rounding rounding, InternalAggregations subAggregations) { - this(rounding, subAggregations, null); - } - EmptyBucketInfo(Rounding rounding, InternalAggregations subAggregations, ExtendedBounds bounds) { - this.rounding = rounding; + EmptyBucketInfo(double interval, double offset, double minBound, double maxBound, InternalAggregations subAggregations) { + this.interval = interval; + this.offset = offset; + this.minBound = minBound; + this.maxBound = maxBound; this.subAggregations = subAggregations; - this.bounds = bounds; - } - - public static EmptyBucketInfo readFrom(StreamInput in) throws IOException { - Rounding rounding = Rounding.Streams.read(in); - InternalAggregations aggs = InternalAggregations.readAggregations(in); - if (in.readBoolean()) { - return new EmptyBucketInfo(rounding, aggs, new ExtendedBounds(in)); - } - return new EmptyBucketInfo(rounding, aggs); - } - - public static void writeTo(EmptyBucketInfo info, StreamOutput out) throws IOException { - Rounding.Streams.write(info.rounding, out); - info.subAggregations.writeTo(out); - out.writeBoolean(info.bounds != null); - if (info.bounds != null) { - info.bounds.writeTo(out); - } - } - - } - - public static class Factory { - - protected Factory() { - } - - public Type type() { - return TYPE; - } - - public ValueType valueType() { - return ValueType.NUMERIC; - } - - public InternalHistogram create(String name, List buckets, InternalOrder order, long minDocCount, - EmptyBucketInfo emptyBucketInfo, DocValueFormat formatter, boolean keyed, - List pipelineAggregators, - Map metaData) { - return new InternalHistogram<>(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed, this, pipelineAggregators, - metaData); - } - - public InternalHistogram create(List buckets, InternalHistogram prototype) { - return new InternalHistogram<>(prototype.name, buckets, prototype.order, prototype.minDocCount, prototype.emptyBucketInfo, - prototype.format, prototype.keyed, this, prototype.pipelineAggregators(), prototype.metaData); } - @SuppressWarnings("unchecked") - public B createBucket(InternalAggregations aggregations, B prototype) { - return (B) new Bucket(prototype.key, prototype.docCount, prototype.getKeyed(), prototype.format, this, aggregations); + EmptyBucketInfo(StreamInput in) throws IOException { + this(in.readDouble(), in.readDouble(), in.readDouble(), in.readDouble(), InternalAggregations.readAggregations(in)); } - @SuppressWarnings("unchecked") - public B createBucket(Object key, long docCount, InternalAggregations aggregations, boolean keyed, DocValueFormat formatter) { - if (key instanceof Number) { - return (B) new Bucket(((Number) key).longValue(), docCount, keyed, formatter, this, aggregations); - } else { - throw new AggregationExecutionException("Expected key of type Number but got [" + key + "]"); - } + public void writeTo(StreamOutput out) throws IOException { + out.writeDouble(interval); + out.writeDouble(offset); + out.writeDouble(minBound); + out.writeDouble(maxBound); + subAggregations.writeTo(out); } - @SuppressWarnings("unchecked") - protected B readBucket(StreamInput in, boolean keyed, DocValueFormat format) throws IOException { - return (B) new Bucket(in, keyed, format, this); - } } - private final List buckets; + private final List buckets; private final InternalOrder order; private final DocValueFormat format; private final boolean keyed; private final long minDocCount; private final EmptyBucketInfo emptyBucketInfo; - private final Factory factory; - InternalHistogram(String name, List buckets, InternalOrder order, long minDocCount, EmptyBucketInfo emptyBucketInfo, - DocValueFormat formatter, boolean keyed, Factory factory, List pipelineAggregators, + InternalHistogram(String name, List buckets, InternalOrder order, long minDocCount, EmptyBucketInfo emptyBucketInfo, + DocValueFormat formatter, boolean keyed, List pipelineAggregators, Map metaData) { super(name, pipelineAggregators, metaData); this.buckets = buckets; @@ -252,7 +185,6 @@ protected B readBucket(StreamInput in, boolean keyed, DocValueFormat format) thr this.emptyBucketInfo = emptyBucketInfo; this.format = formatter; this.keyed = keyed; - this.factory = factory; } /** @@ -260,37 +192,24 @@ protected B readBucket(StreamInput in, boolean keyed, DocValueFormat format) thr */ public InternalHistogram(StreamInput in) throws IOException { super(in); - factory = resolveFactory(in.readString()); order = InternalOrder.Streams.readOrder(in); minDocCount = in.readVLong(); if (minDocCount == 0) { - emptyBucketInfo = EmptyBucketInfo.readFrom(in); + emptyBucketInfo = new EmptyBucketInfo(in); } else { emptyBucketInfo = null; } format = in.readNamedWriteable(DocValueFormat.class); keyed = in.readBoolean(); - buckets = in.readList(stream -> factory.readBucket(stream, keyed, format)); - } - - @SuppressWarnings("unchecked") - protected static Factory resolveFactory(String factoryType) { - if (factoryType.equals(InternalDateHistogram.TYPE.name())) { - return (Factory) new InternalDateHistogram.Factory(); - } else if (factoryType.equals(TYPE.name())) { - return new Factory<>(); - } else { - throw new IllegalStateException("Invalid histogram factory type [" + factoryType + "]"); - } + buckets = in.readList(stream -> new Bucket(stream, keyed, format)); } @Override protected void doWriteTo(StreamOutput out) throws IOException { - out.writeString(factory.type().name()); InternalOrder.Streams.writeOrder(order, out); out.writeVLong(minDocCount); if (minDocCount == 0) { - EmptyBucketInfo.writeTo(emptyBucketInfo, out); + emptyBucketInfo.writeTo(out); } out.writeNamedWriteable(format); out.writeBoolean(keyed); @@ -303,68 +222,59 @@ public String getWriteableName() { } @Override - public List getBuckets() { - return buckets; - } - - public Factory getFactory() { - return factory; - } - - public Rounding getRounding() { - return emptyBucketInfo.rounding; + public List getBuckets() { + return Collections.unmodifiableList(buckets); } @Override - public InternalHistogram create(List buckets) { - return getFactory().create(buckets, this); + public InternalHistogram create(List buckets) { + return new InternalHistogram(name, buckets, order, minDocCount, emptyBucketInfo, format, keyed, pipelineAggregators(), metaData); } @Override - public B createBucket(InternalAggregations aggregations, B prototype) { - return getFactory().createBucket(aggregations, prototype); + public Bucket createBucket(InternalAggregations aggregations, Bucket prototype) { + return new Bucket(prototype.key, prototype.docCount, prototype.keyed, prototype.format, aggregations); } - private static class IteratorAndCurrent { + private static class IteratorAndCurrent { - private final Iterator iterator; - private B current; + private final Iterator iterator; + private Bucket current; - IteratorAndCurrent(Iterator iterator) { + IteratorAndCurrent(Iterator iterator) { this.iterator = iterator; current = iterator.next(); } } - private List reduceBuckets(List aggregations, ReduceContext reduceContext) { + private List reduceBuckets(List aggregations, ReduceContext reduceContext) { - final PriorityQueue> pq = new PriorityQueue>(aggregations.size()) { + final PriorityQueue pq = new PriorityQueue(aggregations.size()) { @Override - protected boolean lessThan(IteratorAndCurrent a, IteratorAndCurrent b) { + protected boolean lessThan(IteratorAndCurrent a, IteratorAndCurrent b) { return a.current.key < b.current.key; } }; for (InternalAggregation aggregation : aggregations) { - @SuppressWarnings("unchecked") - InternalHistogram histogram = (InternalHistogram) aggregation; + InternalHistogram histogram = (InternalHistogram) aggregation; if (histogram.buckets.isEmpty() == false) { - pq.add(new IteratorAndCurrent<>(histogram.buckets.iterator())); + pq.add(new IteratorAndCurrent(histogram.buckets.iterator())); } } - List reducedBuckets = new ArrayList<>(); + List reducedBuckets = new ArrayList<>(); if (pq.size() > 0) { // list of buckets coming from different shards that have the same key - List currentBuckets = new ArrayList<>(); - long key = pq.top().current.key; + List currentBuckets = new ArrayList<>(); + double key = pq.top().current.key; do { - final IteratorAndCurrent top = pq.top(); + final IteratorAndCurrent top = pq.top(); if (top.current.key != key) { // the key changes, reduce what we already buffered and reset the buffer for current buckets - final B reduced = currentBuckets.get(0).reduce(currentBuckets, reduceContext); + final Bucket reduced = currentBuckets.get(0).reduce(currentBuckets, reduceContext); if (reduced.getDocCount() >= minDocCount) { reducedBuckets.add(reduced); } @@ -375,7 +285,7 @@ protected boolean lessThan(IteratorAndCurrent a, IteratorAndCurrent b) { currentBuckets.add(top.current); if (top.iterator.hasNext()) { - final B next = top.iterator.next(); + final Bucket next = top.iterator.next(); assert next.key > top.current.key : "shards must return data sorted by key"; top.current = next; pq.updateTop(); @@ -385,7 +295,7 @@ protected boolean lessThan(IteratorAndCurrent a, IteratorAndCurrent b) { } while (pq.size() > 0); if (currentBuckets.isEmpty() == false) { - final B reduced = currentBuckets.get(0).reduce(currentBuckets, reduceContext); + final Bucket reduced = currentBuckets.get(0).reduce(currentBuckets, reduceContext); if (reduced.getDocCount() >= minDocCount) { reducedBuckets.add(reduced); } @@ -395,75 +305,62 @@ protected boolean lessThan(IteratorAndCurrent a, IteratorAndCurrent b) { return reducedBuckets; } - private void addEmptyBuckets(List list, ReduceContext reduceContext) { - B lastBucket = null; - ExtendedBounds bounds = emptyBucketInfo.bounds; - ListIterator iter = list.listIterator(); + private double nextKey(double key) { + return round(key + emptyBucketInfo.interval + emptyBucketInfo.interval / 2); + } + + private double round(double key) { + return Math.floor((key - emptyBucketInfo.offset) / emptyBucketInfo.interval) * emptyBucketInfo.interval + emptyBucketInfo.offset; + } + + private void addEmptyBuckets(List list, ReduceContext reduceContext) { + ListIterator iter = list.listIterator(); // first adding all the empty buckets *before* the actual data (based on th extended_bounds.min the user requested) - InternalAggregations reducedEmptySubAggs = InternalAggregations.reduce(Collections.singletonList(emptyBucketInfo.subAggregations), + InternalAggregations reducedEmptySubAggs = InternalAggregations.reduce( + Collections.singletonList(emptyBucketInfo.subAggregations), reduceContext); - if (bounds != null) { - B firstBucket = iter.hasNext() ? list.get(iter.nextIndex()) : null; - if (firstBucket == null) { - if (bounds.getMin() != null && bounds.getMax() != null) { - long key = bounds.getMin(); - long max = bounds.getMax(); - while (key <= max) { - iter.add(getFactory().createBucket(key, 0, - reducedEmptySubAggs, - keyed, format)); - key = emptyBucketInfo.rounding.nextRoundingValue(key); - } - } - } else { - if (bounds.getMin() != null) { - long key = bounds.getMin(); - if (key < firstBucket.key) { - while (key < firstBucket.key) { - iter.add(getFactory().createBucket(key, 0, - reducedEmptySubAggs, - keyed, format)); - key = emptyBucketInfo.rounding.nextRoundingValue(key); - } - } + + if (iter.hasNext() == false) { + // fill with empty buckets + for (double key = round(emptyBucketInfo.minBound); key <= emptyBucketInfo.maxBound; key = nextKey(key)) { + iter.add(new Bucket(key, 0, keyed, format, reducedEmptySubAggs)); + } + } else { + Bucket first = list.get(iter.nextIndex()); + if (Double.isFinite(emptyBucketInfo.minBound)) { + // fill with empty buckets until the first key + for (double key = round(emptyBucketInfo.minBound); key < first.key; key = nextKey(key)) { + iter.add(new Bucket(key, 0, keyed, format, reducedEmptySubAggs)); } } - } - // now adding the empty buckets within the actual 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()) { - B nextBucket = list.get(iter.nextIndex()); - if (lastBucket != null) { - long key = emptyBucketInfo.rounding.nextRoundingValue(lastBucket.key); - while (key < nextBucket.key) { - iter.add(getFactory().createBucket(key, 0, - reducedEmptySubAggs, keyed, - format)); - key = emptyBucketInfo.rounding.nextRoundingValue(key); + // now adding the empty buckets within the actual 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 + Bucket lastBucket = null; + do { + Bucket nextBucket = list.get(iter.nextIndex()); + if (lastBucket != null) { + double key = nextKey(lastBucket.key); + while (key < nextBucket.key) { + iter.add(new Bucket(key, 0, keyed, format, reducedEmptySubAggs)); + key = nextKey(key); + } + assert key == nextBucket.key; } - assert key == nextBucket.key; - } - lastBucket = iter.next(); - } + lastBucket = iter.next(); + } while (iter.hasNext()); - // finally, adding the empty buckets *after* the actual data (based on the extended_bounds.max requested by the user) - if (bounds != null && lastBucket != null && bounds.getMax() != null && bounds.getMax() > lastBucket.key) { - long key = emptyBucketInfo.rounding.nextRoundingValue(lastBucket.key); - long max = bounds.getMax(); - while (key <= max) { - iter.add(getFactory().createBucket(key, 0, - reducedEmptySubAggs, keyed, - format)); - key = emptyBucketInfo.rounding.nextRoundingValue(key); + // finally, adding the empty buckets *after* the actual data (based on the extended_bounds.max requested by the user) + for (double key = nextKey(lastBucket.key); key <= emptyBucketInfo.maxBound; key = nextKey(key)) { + iter.add(new Bucket(key, 0, keyed, format, reducedEmptySubAggs)); } } } @Override public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { - List reducedBuckets = reduceBuckets(aggregations, reduceContext); + List reducedBuckets = reduceBuckets(aggregations, reduceContext); // adding empty buckets if needed if (minDocCount == 0) { @@ -476,7 +373,7 @@ public InternalAggregation doReduce(List aggregations, Redu // maintains order } else if (order == InternalOrder.KEY_DESC) { // we just need to reverse here... - List reverse = new ArrayList<>(reducedBuckets); + List reverse = new ArrayList<>(reducedBuckets); Collections.reverse(reverse); reducedBuckets = reverse; } else { @@ -484,7 +381,7 @@ public InternalAggregation doReduce(List aggregations, Redu CollectionUtil.introSort(reducedBuckets, order.comparator()); } - return getFactory().create(getName(), reducedBuckets, order, minDocCount, emptyBucketInfo, format, keyed, pipelineAggregators(), + return new InternalHistogram(getName(), reducedBuckets, order, minDocCount, emptyBucketInfo, format, keyed, pipelineAggregators(), getMetaData()); } @@ -495,7 +392,7 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th } else { builder.startArray(CommonFields.BUCKETS); } - for (B bucket : buckets) { + for (Bucket bucket : buckets) { bucket.toXContent(builder, params); } if (keyed) { @@ -506,4 +403,33 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th return builder; } + // HistogramFactory method impls + + @Override + public Number getKey(MultiBucketsAggregation.Bucket bucket) { + return ((Bucket) bucket).key; + } + + @Override + public Number nextKey(Number key) { + return nextKey(key.doubleValue()); + } + + @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 InternalHistogram(name, buckets2, order, minDocCount, emptyBucketInfo, format, + keyed, pipelineAggregators(), getMetaData()); + } + + @Override + public Bucket createBucket(Number key, long docCount, InternalAggregations aggregations) { + return new Bucket(key.doubleValue(), docCount, keyed, format, aggregations); + } + } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalOrder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalOrder.java index d19a839ad65bf..5cf2f83baa850 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalOrder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalOrder.java @@ -35,9 +35,9 @@ class InternalOrder extends Histogram.Order { final byte id; final String key; final boolean asc; - final Comparator comparator; + final Comparator comparator; - InternalOrder(byte id, String key, boolean asc, Comparator comparator) { + InternalOrder(byte id, String key, boolean asc, Comparator comparator) { this.id = id; this.key = key; this.asc = asc; @@ -57,7 +57,7 @@ boolean asc() { } @Override - Comparator comparator() { + Comparator comparator() { return comparator; } @@ -90,11 +90,7 @@ static class Aggregation extends InternalOrder { static final byte ID = 0; Aggregation(String key, boolean asc) { - super(ID, key, asc, new MultiBucketsAggregation.Bucket.SubAggregationComparator(key, asc)); - } - - private static String key(String aggName, String valueName) { - return (valueName == null) ? aggName : aggName + "." + valueName; + super(ID, key, asc, new MultiBucketsAggregation.Bucket.SubAggregationComparator(key, asc)); } } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpers.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpers.java index 7e46105346d81..98b5b67b7cf5f 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpers.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpers.java @@ -28,6 +28,7 @@ import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; import org.elasticsearch.search.aggregations.InvalidAggregationPathException; +import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation; import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; import org.elasticsearch.search.aggregations.support.AggregationPath; @@ -145,13 +146,13 @@ public String getName() { * @return The value extracted from bucket found at * aggPath */ - public static Double resolveBucketValue(InternalMultiBucketAggregation agg, + public static Double resolveBucketValue(MultiBucketsAggregation agg, InternalMultiBucketAggregation.Bucket bucket, String aggPath, GapPolicy gapPolicy) { List aggPathsList = AggregationPath.parse(aggPath).getPathElementsAsStringList(); return resolveBucketValue(agg, bucket, aggPathsList, gapPolicy); } - public static Double resolveBucketValue(InternalMultiBucketAggregation agg, + public static Double resolveBucketValue(MultiBucketsAggregation agg, InternalMultiBucketAggregation.Bucket bucket, List aggPathAsList, GapPolicy gapPolicy) { try { Object propertyValue = bucket.getProperty(agg.getName(), aggPathAsList); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/cumulativesum/CumulativeSumPipelineAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/cumulativesum/CumulativeSumPipelineAggregationBuilder.java index 898c5711c8533..7bc5483261201 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/cumulativesum/CumulativeSumPipelineAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/cumulativesum/CumulativeSumPipelineAggregationBuilder.java @@ -29,7 +29,8 @@ import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.PipelineAggregationBuilder; -import org.elasticsearch.search.aggregations.bucket.histogram.AbstractHistogramAggregatorFactory; +import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregatorFactory; +import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregatorFactory; import org.elasticsearch.search.aggregations.pipeline.AbstractPipelineAggregationBuilder; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsParser; @@ -104,15 +105,21 @@ public void doValidate(AggregatorFactory parent, AggregatorFactory[] aggFa throw new IllegalStateException(BUCKETS_PATH.getPreferredName() + " must contain a single entry for aggregation [" + name + "]"); } - if (!(parent instanceof AbstractHistogramAggregatorFactory)) { - throw new IllegalStateException("cumulative sum aggregation [" + name - + "] must have a histogram or date_histogram as parent"); - } else { - AbstractHistogramAggregatorFactory histoParent = (AbstractHistogramAggregatorFactory) parent; + if (parent instanceof HistogramAggregatorFactory) { + HistogramAggregatorFactory histoParent = (HistogramAggregatorFactory) parent; + if (histoParent.minDocCount() != 0) { + throw new IllegalStateException("parent histogram of cumulative sum aggregation [" + name + + "] must have min_doc_count of 0"); + } + } else if (parent instanceof DateHistogramAggregatorFactory) { + DateHistogramAggregatorFactory histoParent = (DateHistogramAggregatorFactory) parent; if (histoParent.minDocCount() != 0) { throw new IllegalStateException("parent histogram of cumulative sum aggregation [" + name + "] must have min_doc_count of 0"); } + } else { + throw new IllegalStateException("cumulative sum aggregation [" + name + + "] must have a histogram or date_histogram as parent"); } } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/cumulativesum/CumulativeSumPipelineAggregator.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/cumulativesum/CumulativeSumPipelineAggregator.java index b9a0033272ce6..98c6f7b2fa29e 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/cumulativesum/CumulativeSumPipelineAggregator.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/cumulativesum/CumulativeSumPipelineAggregator.java @@ -25,7 +25,9 @@ import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext; import org.elasticsearch.search.aggregations.InternalAggregations; -import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram; +import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation; +import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket; +import org.elasticsearch.search.aggregations.bucket.histogram.HistogramFactory; import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy; import org.elasticsearch.search.aggregations.pipeline.InternalSimpleValue; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; @@ -68,23 +70,22 @@ public String getWriteableName() { @Override public InternalAggregation reduce(InternalAggregation aggregation, ReduceContext reduceContext) { - InternalHistogram histo = (InternalHistogram) aggregation; - List buckets = histo.getBuckets(); - InternalHistogram.Factory factory = histo.getFactory(); + MultiBucketsAggregation histo = (MultiBucketsAggregation) aggregation; + List buckets = histo.getBuckets(); + HistogramFactory factory = (HistogramFactory) histo; - List newBuckets = new ArrayList<>(); + List newBuckets = new ArrayList<>(); double sum = 0; - for (InternalHistogram.Bucket bucket : buckets) { + for (Bucket bucket : buckets) { Double thisBucketValue = resolveBucketValue(histo, bucket, bucketsPaths()[0], GapPolicy.INSERT_ZEROS); sum += thisBucketValue; List aggs = StreamSupport.stream(bucket.getAggregations().spliterator(), false).map((p) -> { return (InternalAggregation) p; }).collect(Collectors.toList()); aggs.add(new InternalSimpleValue(name(), sum, formatter, new ArrayList(), metaData())); - InternalHistogram.Bucket newBucket = factory.createBucket(bucket.getKey(), bucket.getDocCount(), - new InternalAggregations(aggs), bucket.getKeyed(), bucket.getFormatter()); + Bucket newBucket = factory.createBucket(factory.getKey(bucket), bucket.getDocCount(), new InternalAggregations(aggs)); newBuckets.add(newBucket); } - return factory.create(newBuckets, histo); + return factory.createAggregation(newBuckets); } } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/derivative/DerivativePipelineAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/derivative/DerivativePipelineAggregationBuilder.java index fd4cb6139b00d..e7b6b89d042d5 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/derivative/DerivativePipelineAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/derivative/DerivativePipelineAggregationBuilder.java @@ -31,9 +31,9 @@ import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.PipelineAggregationBuilder; -import org.elasticsearch.search.aggregations.bucket.histogram.AbstractHistogramAggregatorFactory; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregatorFactory; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; +import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregatorFactory; import org.elasticsearch.search.aggregations.pipeline.AbstractPipelineAggregationBuilder; import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; @@ -162,15 +162,21 @@ public void doValidate(AggregatorFactory parent, AggregatorFactory[] aggFa throw new IllegalStateException(PipelineAggregator.Parser.BUCKETS_PATH.getPreferredName() + " must contain a single entry for aggregation [" + name + "]"); } - if (!(parent instanceof AbstractHistogramAggregatorFactory)) { - throw new IllegalStateException("derivative aggregation [" + name - + "] must have a histogram or date_histogram as parent"); - } else { - AbstractHistogramAggregatorFactory histoParent = (AbstractHistogramAggregatorFactory) parent; + if (parent instanceof HistogramAggregatorFactory) { + HistogramAggregatorFactory histoParent = (HistogramAggregatorFactory) parent; + if (histoParent.minDocCount() != 0) { + throw new IllegalStateException("parent histogram of derivative aggregation [" + name + + "] must have min_doc_count of 0"); + } + } else if (parent instanceof DateHistogramAggregatorFactory) { + DateHistogramAggregatorFactory histoParent = (DateHistogramAggregatorFactory) parent; if (histoParent.minDocCount() != 0) { throw new IllegalStateException("parent histogram of derivative aggregation [" + name + "] must have min_doc_count of 0"); } + } else { + throw new IllegalStateException("derivative aggregation [" + name + + "] must have a histogram or date_histogram as parent"); } } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/derivative/DerivativePipelineAggregator.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/derivative/DerivativePipelineAggregator.java index 7dc2ba76704ee..480f04f545a4e 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/derivative/DerivativePipelineAggregator.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/derivative/DerivativePipelineAggregator.java @@ -22,14 +22,14 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext; import org.elasticsearch.search.aggregations.InternalAggregations; -import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram; +import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation; +import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket; +import org.elasticsearch.search.aggregations.bucket.histogram.HistogramFactory; import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; -import org.joda.time.DateTime; import java.io.IOException; import java.util.ArrayList; @@ -77,28 +77,27 @@ public String getWriteableName() { @Override public InternalAggregation reduce(InternalAggregation aggregation, ReduceContext reduceContext) { - InternalHistogram histo = (InternalHistogram) aggregation; - List buckets = histo.getBuckets(); - InternalHistogram.Factory factory = histo.getFactory(); + MultiBucketsAggregation histo = (MultiBucketsAggregation) aggregation; + List buckets = histo.getBuckets(); + HistogramFactory factory = (HistogramFactory) histo; - List newBuckets = new ArrayList<>(); - Long lastBucketKey = null; + List newBuckets = new ArrayList<>(); + Number lastBucketKey = null; Double lastBucketValue = null; - for (InternalHistogram.Bucket bucket : buckets) { - Long thisBucketKey = resolveBucketKeyAsLong(bucket); + for (Bucket bucket : buckets) { + Number thisBucketKey = factory.getKey(bucket); Double thisBucketValue = resolveBucketValue(histo, bucket, bucketsPaths()[0], gapPolicy); if (lastBucketValue != null && thisBucketValue != null) { double gradient = thisBucketValue - lastBucketValue; double xDiff = -1; if (xAxisUnits != null) { - xDiff = (thisBucketKey - lastBucketKey) / xAxisUnits; + xDiff = (thisBucketKey.doubleValue() - lastBucketKey.doubleValue()) / xAxisUnits; } final List aggs = StreamSupport.stream(bucket.getAggregations().spliterator(), false).map((p) -> { return (InternalAggregation) p; }).collect(Collectors.toList()); aggs.add(new InternalDerivative(name(), gradient, xDiff, formatter, new ArrayList(), metaData())); - InternalHistogram.Bucket newBucket = factory.createBucket(bucket.getKey(), bucket.getDocCount(), new InternalAggregations( - aggs), bucket.getKeyed(), bucket.getFormatter()); + Bucket newBucket = factory.createBucket(factory.getKey(bucket), bucket.getDocCount(), new InternalAggregations(aggs)); newBuckets.add(newBucket); } else { newBuckets.add(bucket); @@ -106,18 +105,7 @@ public InternalAggregation reduce(InternalAggregation aggregation, ReduceContext lastBucketKey = thisBucketKey; lastBucketValue = thisBucketValue; } - return factory.create(newBuckets, histo); + return factory.createAggregation(newBuckets); } - private Long resolveBucketKeyAsLong(InternalHistogram.Bucket bucket) { - Object key = bucket.getKey(); - if (key instanceof DateTime) { - return ((DateTime) key).getMillis(); - } else if (key instanceof Number) { - return ((Number) key).longValue(); - } else { - throw new AggregationExecutionException("InternalBucket keys must be either a Number or a DateTime for aggregation " + name() - + ". Found bucket with key " + key); - } - } } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/movavg/MovAvgPipelineAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/movavg/MovAvgPipelineAggregationBuilder.java index f9d21087d6b4b..89bc721e74a3b 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/movavg/MovAvgPipelineAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/movavg/MovAvgPipelineAggregationBuilder.java @@ -30,7 +30,8 @@ import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.PipelineAggregationBuilder; -import org.elasticsearch.search.aggregations.bucket.histogram.AbstractHistogramAggregatorFactory; +import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregatorFactory; +import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregatorFactory; import org.elasticsearch.search.aggregations.pipeline.AbstractPipelineAggregationBuilder; import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; @@ -267,15 +268,21 @@ public void doValidate(AggregatorFactory parent, AggregatorFactory[] aggFa throw new IllegalStateException(PipelineAggregator.Parser.BUCKETS_PATH.getPreferredName() + " must contain a single entry for aggregation [" + name + "]"); } - if (!(parent instanceof AbstractHistogramAggregatorFactory)) { - throw new IllegalStateException("moving average aggregation [" + name - + "] must have a histogram or date_histogram as parent"); - } else { - AbstractHistogramAggregatorFactory histoParent = (AbstractHistogramAggregatorFactory) parent; + if (parent instanceof HistogramAggregatorFactory) { + HistogramAggregatorFactory histoParent = (HistogramAggregatorFactory) parent; + if (histoParent.minDocCount() != 0) { + throw new IllegalStateException("parent histogram of moving average aggregation [" + name + + "] must have min_doc_count of 0"); + } + } else if (parent instanceof DateHistogramAggregatorFactory) { + DateHistogramAggregatorFactory histoParent = (DateHistogramAggregatorFactory) parent; if (histoParent.minDocCount() != 0) { throw new IllegalStateException("parent histogram of moving average aggregation [" + name + "] must have min_doc_count of 0"); } + } else { + throw new IllegalStateException("moving average aggregation [" + name + + "] must have a histogram or date_histogram as parent"); } } diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/movavg/MovAvgPipelineAggregator.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/movavg/MovAvgPipelineAggregator.java index 216890741bbaa..87aa5bfda63e0 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/movavg/MovAvgPipelineAggregator.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/movavg/MovAvgPipelineAggregator.java @@ -23,16 +23,16 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext; import org.elasticsearch.search.aggregations.InternalAggregations; -import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram; +import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation; +import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket; +import org.elasticsearch.search.aggregations.bucket.histogram.HistogramFactory; import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy; import org.elasticsearch.search.aggregations.pipeline.InternalSimpleValue; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.movavg.models.MovAvgModel; -import org.joda.time.DateTime; import java.io.IOException; import java.util.ArrayList; @@ -93,14 +93,14 @@ public String getWriteableName() { @Override public InternalAggregation reduce(InternalAggregation aggregation, ReduceContext reduceContext) { - InternalHistogram histo = (InternalHistogram) aggregation; - List buckets = histo.getBuckets(); - InternalHistogram.Factory factory = histo.getFactory(); + MultiBucketsAggregation histo = (MultiBucketsAggregation) aggregation; + List buckets = histo.getBuckets(); + HistogramFactory factory = (HistogramFactory) histo; - List newBuckets = new ArrayList<>(); + List newBuckets = new ArrayList<>(); EvictingQueue values = new EvictingQueue<>(this.window); - long lastValidKey = 0; + Number lastValidKey = 0; int lastValidPosition = 0; int counter = 0; @@ -110,12 +110,12 @@ public InternalAggregation reduce(InternalAggregation aggregation, ReduceContext model = minimize(buckets, histo, model); } - for (InternalHistogram.Bucket bucket : buckets) { + for (Bucket bucket : buckets) { Double thisBucketValue = resolveBucketValue(histo, bucket, bucketsPaths()[0], gapPolicy); // Default is to reuse existing bucket. Simplifies the rest of the logic, // since we only change newBucket if we can add to it - InternalHistogram.Bucket newBucket = bucket; + Bucket newBucket = bucket; if (!(thisBucketValue == null || thisBucketValue.equals(Double.NaN))) { @@ -127,18 +127,11 @@ public InternalAggregation reduce(InternalAggregation aggregation, ReduceContext return (InternalAggregation) p; }).collect(Collectors.toList()); aggs.add(new InternalSimpleValue(name(), movavg, formatter, new ArrayList(), metaData())); - newBucket = factory.createBucket(bucket.getKey(), bucket.getDocCount(), new InternalAggregations( - aggs), bucket.getKeyed(), bucket.getFormatter()); + newBucket = factory.createBucket(factory.getKey(bucket), bucket.getDocCount(), new InternalAggregations(aggs)); } if (predict > 0) { - if (bucket.getKey() instanceof Number) { - lastValidKey = ((Number) bucket.getKey()).longValue(); - } else if (bucket.getKey() instanceof DateTime) { - lastValidKey = ((DateTime) bucket.getKey()).getMillis(); - } else { - throw new AggregationExecutionException("Expected key of type Number or DateTime but got [" + lastValidKey + "]"); - } + lastValidKey = factory.getKey(bucket); lastValidPosition = counter; } @@ -150,20 +143,14 @@ public InternalAggregation reduce(InternalAggregation aggregation, ReduceContext } if (buckets.size() > 0 && predict > 0) { - - boolean keyed; - DocValueFormat formatter; - keyed = buckets.get(0).getKeyed(); - formatter = buckets.get(0).getFormatter(); - double[] predictions = model.predict(values, predict); for (int i = 0; i < predictions.length; i++) { List aggs; - long newKey = histo.getRounding().nextRoundingValue(lastValidKey); + Number newKey = factory.nextKey(lastValidKey); if (lastValidPosition + i + 1 < newBuckets.size()) { - InternalHistogram.Bucket bucket = (InternalHistogram.Bucket) newBuckets.get(lastValidPosition + i + 1); + Bucket bucket = newBuckets.get(lastValidPosition + i + 1); // Get the existing aggs in the bucket so we don't clobber data aggs = StreamSupport.stream(bucket.getAggregations().spliterator(), false).map((p) -> { @@ -171,8 +158,7 @@ public InternalAggregation reduce(InternalAggregation aggregation, ReduceContext }).collect(Collectors.toList()); aggs.add(new InternalSimpleValue(name(), predictions[i], formatter, new ArrayList(), metaData())); - InternalHistogram.Bucket newBucket = factory.createBucket(newKey, 0, new InternalAggregations( - aggs), keyed, formatter); + Bucket newBucket = factory.createBucket(newKey, 0, new InternalAggregations(aggs)); // Overwrite the existing bucket with the new version newBuckets.set(lastValidPosition + i + 1, newBucket); @@ -182,8 +168,7 @@ public InternalAggregation reduce(InternalAggregation aggregation, ReduceContext aggs = new ArrayList<>(); aggs.add(new InternalSimpleValue(name(), predictions[i], formatter, new ArrayList(), metaData())); - InternalHistogram.Bucket newBucket = factory.createBucket(newKey, 0, new InternalAggregations( - aggs), keyed, formatter); + Bucket newBucket = factory.createBucket(newKey, 0, new InternalAggregations(aggs)); // Since this is a new bucket, simply append it newBuckets.add(newBucket); @@ -192,16 +177,16 @@ public InternalAggregation reduce(InternalAggregation aggregation, ReduceContext } } - return factory.create(newBuckets, histo); + return factory.createAggregation(newBuckets); } - private MovAvgModel minimize(List buckets, InternalHistogram histo, MovAvgModel model) { + private MovAvgModel minimize(List buckets, MultiBucketsAggregation histo, MovAvgModel model) { int counter = 0; EvictingQueue values = new EvictingQueue<>(this.window); double[] test = new double[window]; - ListIterator iter = buckets.listIterator(buckets.size()); + ListIterator iter = buckets.listIterator(buckets.size()); // We have to walk the iterator backwards because we don't know if/how many buckets are empty. while (iter.hasPrevious() && counter < window) { diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/serialdiff/SerialDiffPipelineAggregator.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/serialdiff/SerialDiffPipelineAggregator.java index b50462107f88d..3216d5527dc76 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/serialdiff/SerialDiffPipelineAggregator.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/serialdiff/SerialDiffPipelineAggregator.java @@ -26,8 +26,10 @@ import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext; +import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation; +import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket; +import org.elasticsearch.search.aggregations.bucket.histogram.HistogramFactory; import org.elasticsearch.search.aggregations.InternalAggregations; -import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram; import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy; import org.elasticsearch.search.aggregations.pipeline.InternalSimpleValue; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; @@ -78,17 +80,17 @@ public String getWriteableName() { @Override public InternalAggregation reduce(InternalAggregation aggregation, ReduceContext reduceContext) { - InternalHistogram histo = (InternalHistogram) aggregation; - List buckets = histo.getBuckets(); - InternalHistogram.Factory factory = histo.getFactory(); + MultiBucketsAggregation histo = (MultiBucketsAggregation) aggregation; + List buckets = histo.getBuckets(); + HistogramFactory factory = (HistogramFactory) histo; - List newBuckets = new ArrayList<>(); + List newBuckets = new ArrayList<>(); EvictingQueue lagWindow = new EvictingQueue<>(lag); int counter = 0; - for (InternalHistogram.Bucket bucket : buckets) { + for (Bucket bucket : buckets) { Double thisBucketValue = resolveBucketValue(histo, bucket, bucketsPaths()[0], gapPolicy); - InternalHistogram.Bucket newBucket = bucket; + Bucket newBucket = bucket; counter += 1; @@ -113,8 +115,7 @@ public InternalAggregation reduce(InternalAggregation aggregation, ReduceContext return (InternalAggregation) p; }).collect(Collectors.toList()); aggs.add(new InternalSimpleValue(name(), diff, formatter, new ArrayList(), metaData())); - newBucket = factory.createBucket(bucket.getKey(), bucket.getDocCount(), new InternalAggregations( - aggs), bucket.getKeyed(), bucket.getFormatter()); + newBucket = factory.createBucket(factory.getKey(bucket), bucket.getDocCount(), new InternalAggregations(aggs)); } @@ -122,6 +123,6 @@ public InternalAggregation reduce(InternalAggregation aggregation, ReduceContext lagWindow.add(thisBucketValue); } - return factory.create(newBuckets, histo); + return factory.createAggregation(newBuckets); } } diff --git a/core/src/test/java/org/elasticsearch/common/rounding/RoundingTests.java b/core/src/test/java/org/elasticsearch/common/rounding/RoundingTests.java index 7819e4b60ab3f..a71cc77ffc103 100644 --- a/core/src/test/java/org/elasticsearch/common/rounding/RoundingTests.java +++ b/core/src/test/java/org/elasticsearch/common/rounding/RoundingTests.java @@ -33,10 +33,8 @@ public void testInterval() { int interval = 10; Rounding.Interval rounding = new Rounding.Interval(interval); int value = 24; - final long key = rounding.roundKey(24); final long r = rounding.round(24); String message = "round(" + value + ", interval=" + interval + ") = " + r; - assertEquals(value/interval, key); assertEquals(value/interval * interval, r); assertEquals(message, 0, r % interval); } @@ -46,13 +44,11 @@ public void testIntervalRandom() { Rounding.Interval rounding = new Rounding.Interval(interval); for (int i = 0; i < 1000; ++i) { long l = Math.max(randomLong(), Long.MIN_VALUE + interval); - final long key = rounding.roundKey(l); final long r = rounding.round(l); String message = "round(" + l + ", interval=" + interval + ") = " + r; assertEquals(message, 0, r % interval); assertThat(message, r, lessThanOrEqualTo(l)); assertThat(message, r + interval, greaterThan(l)); - assertEquals(message, r, key*interval); } } @@ -65,15 +61,11 @@ public void testOffsetRounding() { final long interval = 10; final long offset = 7; Rounding.OffsetRounding rounding = new Rounding.OffsetRounding(new Rounding.Interval(interval), offset); - assertEquals(-1, rounding.roundKey(6)); assertEquals(-3, rounding.round(6)); assertEquals(7, rounding.nextRoundingValue(-3)); - assertEquals(0, rounding.roundKey(7)); assertEquals(7, rounding.round(7)); assertEquals(17, rounding.nextRoundingValue(7)); - assertEquals(0, rounding.roundKey(16)); assertEquals(7, rounding.round(16)); - assertEquals(1, rounding.roundKey(17)); assertEquals(17, rounding.round(17)); assertEquals(27, rounding.nextRoundingValue(17)); } @@ -89,13 +81,10 @@ public void testOffsetRoundingRandom() { Rounding.OffsetRounding rounding = new Rounding.OffsetRounding(internalRounding, offset); long safetyMargin = Math.abs(interval) + Math.abs(offset); // to prevent range overflow long value = Math.max(randomLong() - safetyMargin, Long.MIN_VALUE + safetyMargin); - final long key = rounding.roundKey(value); - final long key_next = rounding.roundKey(value + interval); final long r_value = rounding.round(value); final long nextRoundingValue = rounding.nextRoundingValue(r_value); assertThat("Rounding should be idempotent", r_value, equalTo(rounding.round(r_value))); assertThat("Rounded value smaller than unrounded, regardless of offset", r_value - offset, lessThanOrEqualTo(value - offset)); - assertThat("Key and next_key should differ by one", key_next - key, equalTo(1L)); assertThat("Rounded value <= value < next interval start", r_value + interval, greaterThan(value)); assertThat("NextRounding value should be interval from rounded value", r_value + interval, equalTo(nextRoundingValue)); } diff --git a/core/src/test/java/org/elasticsearch/common/rounding/TimeZoneRoundingTests.java b/core/src/test/java/org/elasticsearch/common/rounding/TimeZoneRoundingTests.java index f9e5f6e3fbb58..d4920e9afe8d4 100644 --- a/core/src/test/java/org/elasticsearch/common/rounding/TimeZoneRoundingTests.java +++ b/core/src/test/java/org/elasticsearch/common/rounding/TimeZoneRoundingTests.java @@ -65,9 +65,6 @@ public void testUTCIntervalRounding() { Rounding tzRounding = TimeZoneRounding.builder(TimeValue.timeValueHours(12)).build(); DateTimeZone tz = DateTimeZone.UTC; assertThat(tzRounding.round(time("2009-02-03T01:01:01")), isDate(time("2009-02-03T00:00:00.000Z"), tz)); - long roundKey = tzRounding.roundKey(time("2009-02-03T01:01:01")); - assertThat(roundKey, isDate(tzRounding.roundKey(time("2009-02-03T00:00:00.000Z")), tz)); - assertThat(tzRounding.valueForKey(roundKey), isDate(time("2009-02-03T00:00:00.000Z"), tz)); assertThat(tzRounding.nextRoundingValue(time("2009-02-03T00:00:00.000Z")), isDate(time("2009-02-03T12:00:00.000Z"), tz)); assertThat(tzRounding.round(time("2009-02-03T13:01:01")), isDate(time("2009-02-03T12:00:00.000Z"), tz)); assertThat(tzRounding.nextRoundingValue(time("2009-02-03T12:00:00.000Z")), isDate(time("2009-02-04T00:00:00.000Z"), tz)); @@ -86,9 +83,6 @@ public void testTimeIntervalTimeZoneRounding() { DateTimeZone tz = DateTimeZone.forOffsetHours(-1); Rounding tzRounding = TimeZoneRounding.builder(TimeValue.timeValueHours(6)).timeZone(tz).build(); assertThat(tzRounding.round(time("2009-02-03T00:01:01")), isDate(time("2009-02-02T19:00:00.000Z"), tz)); - long roundKey = tzRounding.roundKey(time("2009-02-03T00:01:01")); - assertThat(roundKey, equalTo(tzRounding.roundKey(time("2009-02-02T19:00:00.000Z")))); - assertThat(tzRounding.valueForKey(roundKey), isDate(time("2009-02-02T19:00:00.000Z"), tz)); assertThat(tzRounding.nextRoundingValue(time("2009-02-02T19:00:00.000Z")), isDate(time("2009-02-03T01:00:00.000Z"), tz)); assertThat(tzRounding.round(time("2009-02-03T13:01:01")), isDate(time("2009-02-03T13:00:00.000Z"), tz)); @@ -102,9 +96,6 @@ public void testDayIntervalTimeZoneRounding() { DateTimeZone tz = DateTimeZone.forOffsetHours(-8); Rounding tzRounding = TimeZoneRounding.builder(TimeValue.timeValueHours(12)).timeZone(tz).build(); assertThat(tzRounding.round(time("2009-02-03T00:01:01")), isDate(time("2009-02-02T20:00:00.000Z"), tz)); - long roundKey = tzRounding.roundKey(time("2009-02-03T00:01:01")); - assertThat(roundKey, isDate(tzRounding.roundKey(time("2009-02-02T20:00:00.000Z")), tz)); - assertThat(tzRounding.valueForKey(roundKey), isDate(time("2009-02-02T20:00:00.000Z"), tz)); assertThat(tzRounding.nextRoundingValue(time("2009-02-02T20:00:00.000Z")), isDate(time("2009-02-03T08:00:00.000Z"), tz)); assertThat(tzRounding.round(time("2009-02-03T13:01:01")), isDate(time("2009-02-03T08:00:00.000Z"), tz)); @@ -130,17 +121,11 @@ public void testDayTimeZoneRounding() { tz = DateTimeZone.forID("-02:00"); tzRounding = TimeZoneRounding.builder(DateTimeUnit.DAY_OF_MONTH).timeZone(tz).build(); assertThat(tzRounding.round(time("2009-02-03T01:01:01")), isDate(time("2009-02-02T02:00:00"), tz)); - long roundKey = tzRounding.roundKey(time("2009-02-03T01:01:01")); - assertThat(roundKey, isDate(tzRounding.roundKey(time("2009-02-02T02:00:00.000Z")), tz)); - assertThat(tzRounding.valueForKey(roundKey), isDate(time("2009-02-02T02:00:00.000Z"), tz)); assertThat(tzRounding.nextRoundingValue(time("2009-02-02T02:00:00")), isDate(time("2009-02-03T02:00:00"), tz)); // date in Feb-3rd, also in -02:00 timezone tzRounding = TimeZoneRounding.builder(DateTimeUnit.DAY_OF_MONTH).timeZone(tz).build(); assertThat(tzRounding.round(time("2009-02-03T02:01:01")), isDate(time("2009-02-03T02:00:00"), tz)); - roundKey = tzRounding.roundKey(time("2009-02-03T02:01:01")); - assertThat(roundKey, isDate(tzRounding.roundKey(time("2009-02-03T02:00:00.000Z")), tz)); - assertThat(tzRounding.valueForKey(roundKey), isDate(time("2009-02-03T02:00:00.000Z"), tz)); assertThat(tzRounding.nextRoundingValue(time("2009-02-03T02:00:00")), isDate(time("2009-02-04T02:00:00"), tz)); } diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/MissingValueIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/MissingValueIT.java index b675c14a488ad..5b0b0378e463a 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/MissingValueIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/MissingValueIT.java @@ -124,7 +124,7 @@ public void testUnmappedHistogram() { assertSearchResponse(response); Histogram histogram = response.getAggregations().get("my_histogram"); assertEquals(1, histogram.getBuckets().size()); - assertEquals(10L, histogram.getBuckets().get(0).getKey()); + assertEquals(10d, histogram.getBuckets().get(0).getKey()); assertEquals(2, histogram.getBuckets().get(0).getDocCount()); } @@ -133,16 +133,16 @@ public void testHistogram() { assertSearchResponse(response); Histogram histogram = response.getAggregations().get("my_histogram"); assertEquals(2, histogram.getBuckets().size()); - assertEquals(0L, histogram.getBuckets().get(0).getKey()); + assertEquals(0d, histogram.getBuckets().get(0).getKey()); assertEquals(1, histogram.getBuckets().get(0).getDocCount()); - assertEquals(5L, histogram.getBuckets().get(1).getKey()); + assertEquals(5d, histogram.getBuckets().get(1).getKey()); assertEquals(1, histogram.getBuckets().get(1).getDocCount()); response = client().prepareSearch("idx").addAggregation(histogram("my_histogram").field("long").interval(5).missing(3)).get(); assertSearchResponse(response); histogram = response.getAggregations().get("my_histogram"); assertEquals(1, histogram.getBuckets().size()); - assertEquals(0L, histogram.getBuckets().get(0).getKey()); + assertEquals(0d, histogram.getBuckets().get(0).getKey()); assertEquals(2, histogram.getBuckets().get(0).getDocCount()); } diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramIT.java index ace55c2c61640..d56a4b7f60428 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramIT.java @@ -841,7 +841,7 @@ public void testEmptyAggregation() throws Exception { Histogram.Bucket bucket = buckets.get(1); assertThat(bucket, Matchers.notNullValue()); - assertThat(bucket.getKeyAsString(), equalTo("1")); + assertThat(bucket.getKeyAsString(), equalTo("1.0")); Histogram dateHisto = bucket.getAggregations().get("date_histo"); assertThat(dateHisto, Matchers.notNullValue()); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramOffsetIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramOffsetIT.java index d7c2bf1c4609c..158cbe9be9d0d 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramOffsetIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramOffsetIT.java @@ -97,7 +97,7 @@ public void testSingleValueWithPositiveOffset() throws Exception { assertThat(response.getHits().getTotalHits(), equalTo(5L)); Histogram histo = response.getAggregations().get("date_histo"); - List buckets = histo.getBuckets(); + List buckets = histo.getBuckets(); assertThat(buckets.size(), equalTo(2)); checkBucketFor(buckets.get(0), new DateTime(2014, 3, 10, 2, 0, DateTimeZone.UTC), 2L); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramTests.java index ab196632a2076..76e58c715bfc8 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramTests.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramTests.java @@ -20,11 +20,10 @@ package org.elasticsearch.search.aggregations.bucket; import org.elasticsearch.search.aggregations.BaseAggregationTestCase; +import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Order; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; -import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBounds; import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBoundsTests; -import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Order; public class DateHistogramTests extends BaseAggregationTestCase { diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/HistogramIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/HistogramIT.java index 8df4a8ca60833..554df5304b713 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/HistogramIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/HistogramIT.java @@ -27,7 +27,6 @@ import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptService.ScriptType; import org.elasticsearch.search.aggregations.bucket.filter.Filter; -import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBounds; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket; import org.elasticsearch.search.aggregations.metrics.max.Max; @@ -84,7 +83,7 @@ protected Map, Object>> pluginScripts() { Map, Object>> scripts = new HashMap<>(); scripts.put("_value + 1", vars -> { - long value = (long) vars.get("_value"); + double value = (double) vars.get("_value"); return value + 1L; }); @@ -377,7 +376,7 @@ public void testSingleValuedFieldWithSubAggregation() throws Exception { } } assertThat(sum.getValue(), equalTo((double) s)); - assertEquals(propertiesKeys[i], (long) i * interval); + assertEquals(propertiesKeys[i], (double) i * interval); assertThat(propertiesDocCounts[i], equalTo(valueCounts[i])); assertThat(propertiesCounts[i], equalTo((double) s)); } @@ -762,7 +761,7 @@ public void testPartiallyUnmappedWithExtendedBounds() throws Exception { histogram("histo") .field(SINGLE_VALUED_FIELD_NAME) .interval(interval) - .extendedBounds(new ExtendedBounds((long) -1 * 2 * interval, (long) valueCounts.length * interval))) + .extendedBounds(-1 * 2 * interval, valueCounts.length * interval)) .get(); assertSearchResponse(response); @@ -853,7 +852,7 @@ public void testSingleValuedFieldWithExtendedBounds() throws Exception { .field(SINGLE_VALUED_FIELD_NAME) .interval(interval) .minDocCount(0) - .extendedBounds(new ExtendedBounds(boundsMin, boundsMax))) + .extendedBounds(boundsMin, boundsMax)) .execute().actionGet(); if (invalidBoundsError) { @@ -930,7 +929,7 @@ public void testEmptyWithExtendedBounds() throws Exception { .field(SINGLE_VALUED_FIELD_NAME) .interval(interval) .minDocCount(0) - .extendedBounds(new ExtendedBounds(boundsMin, boundsMax))) + .extendedBounds(boundsMin, boundsMax)) .execute().actionGet(); if (invalidBoundsError) { @@ -973,7 +972,27 @@ public void testExeptionOnNegativerInterval() { .addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(-1).minDocCount(0)).execute().actionGet(); fail(); } catch (IllegalArgumentException e) { - assertThat(e.toString(), containsString("[interval] must be 1 or greater for histogram aggregation [histo]")); + assertThat(e.toString(), containsString("[interval] must be >0 for histogram aggregation [histo]")); } } + + public void testDecimalIntervalAndOffset() throws Exception { + assertAcked(prepareCreate("decimal_values").addMapping("type", "d", "type=float").get()); + indexRandom(true, + client().prepareIndex("decimal_values", "type", "1").setSource("d", -0.6), + client().prepareIndex("decimal_values", "type", "2").setSource("d", 0.1)); + + SearchResponse r = client().prepareSearch("decimal_values") + .addAggregation(histogram("histo").field("d").interval(0.7).offset(0.05)) + .get(); + assertSearchResponse(r); + + Histogram histogram = r.getAggregations().get("histo"); + List buckets = histogram.getBuckets(); + assertEquals(2, buckets.size()); + assertEquals(-0.65, (double) buckets.get(0).getKey(), 0.01d); + assertEquals(1, buckets.get(0).getDocCount()); + assertEquals(0.05, (double) buckets.get(1).getKey(), 0.01d); + assertEquals(1, buckets.get(1).getDocCount()); + } } diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/HistogramTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/HistogramTests.java index 54d443b387444..3dc2ba22629be 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/HistogramTests.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/HistogramTests.java @@ -20,10 +20,8 @@ package org.elasticsearch.search.aggregations.bucket; import org.elasticsearch.search.aggregations.BaseAggregationTestCase; -import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBounds; -import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBoundsTests; -import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Order; import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Order; public class HistogramTests extends BaseAggregationTestCase { @@ -31,9 +29,9 @@ public class HistogramTests extends BaseAggregationTestCase_count")).execute().actionGet(); assertSearchResponse(response); @@ -133,7 +132,7 @@ public void testDocCountAsSubAgg() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .subAggregation(avgBucket("avg_bucket", "histo>_count"))).execute().actionGet(); assertSearchResponse(response); @@ -215,7 +214,7 @@ public void testMetricAsSubAgg() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue)) + .extendedBounds(minRandomValue, maxRandomValue) .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME))) .subAggregation(avgBucket("avg_bucket", "histo>sum"))).execute().actionGet(); @@ -268,7 +267,7 @@ public void testMetricAsSubAggWithInsertZeros() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue)) + .extendedBounds(minRandomValue, maxRandomValue) .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME))) .subAggregation(avgBucket("avg_bucket", "histo>sum").gapPolicy(GapPolicy.INSERT_ZEROS))) .execute().actionGet(); @@ -341,7 +340,7 @@ public void testNested() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .subAggregation(avgBucket("avg_histo_bucket", "histo>_count"))) .addAggregation(avgBucket("avg_terms_bucket", "terms>avg_histo_bucket")).execute().actionGet(); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketScriptIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketScriptIT.java index 220dfd2981780..fdd50692e8923 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketScriptIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketScriptIT.java @@ -28,8 +28,6 @@ import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptService.ScriptType; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; -import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram; -import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram.Bucket; import org.elasticsearch.search.aggregations.metrics.sum.Sum; import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy; import org.elasticsearch.test.ESIntegTestCase; @@ -166,10 +164,10 @@ public void testInlineScript() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); - List buckets = histo.getBuckets(); + List buckets = histo.getBuckets(); for (int i = 0; i < buckets.size(); ++i) { Histogram.Bucket bucket = buckets.get(i); @@ -212,10 +210,10 @@ public void testInlineScript2() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); - List buckets = histo.getBuckets(); + List buckets = histo.getBuckets(); for (int i = 0; i < buckets.size(); ++i) { Histogram.Bucket bucket = buckets.get(i); @@ -256,10 +254,10 @@ public void testInlineScriptSingleVariable() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); - List buckets = histo.getBuckets(); + List buckets = histo.getBuckets(); for (int i = 0; i < buckets.size(); ++i) { Histogram.Bucket bucket = buckets.get(i); @@ -299,10 +297,10 @@ public void testInlineScriptNamedVars() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); - List buckets = histo.getBuckets(); + List buckets = histo.getBuckets(); for (int i = 0; i < buckets.size(); ++i) { Histogram.Bucket bucket = buckets.get(i); @@ -347,10 +345,10 @@ public void testInlineScriptWithParams() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); - List buckets = histo.getBuckets(); + List buckets = histo.getBuckets(); for (int i = 0; i < buckets.size(); ++i) { Histogram.Bucket bucket = buckets.get(i); @@ -393,10 +391,10 @@ public void testInlineScriptInsertZeros() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); - List buckets = histo.getBuckets(); + List buckets = histo.getBuckets(); for (int i = 0; i < buckets.size(); ++i) { Histogram.Bucket bucket = buckets.get(i); @@ -446,10 +444,10 @@ public void testStoredScript() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); - List buckets = histo.getBuckets(); + List buckets = histo.getBuckets(); for (int i = 0; i < buckets.size(); ++i) { Histogram.Bucket bucket = buckets.get(i); @@ -492,7 +490,7 @@ public void testUnmapped() throws Exception { assertSearchResponse(response); - InternalHistogram deriv = response.getAggregations().get("histo"); + Histogram deriv = response.getAggregations().get("histo"); assertThat(deriv, notNullValue()); assertThat(deriv.getName(), equalTo("histo")); assertThat(deriv.getBuckets().size(), equalTo(0)); @@ -515,10 +513,10 @@ public void testPartiallyUnmapped() throws Exception { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); - List buckets = histo.getBuckets(); + List buckets = histo.getBuckets(); for (int i = 0; i < buckets.size(); ++i) { Histogram.Bucket bucket = buckets.get(i); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketSelectorIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketSelectorIT.java index 64dc7e50bccd9..26c0d3a4bbef2 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketSelectorIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketSelectorIT.java @@ -29,8 +29,7 @@ import org.elasticsearch.script.ScriptService.ScriptType; import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBounds; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; -import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram; -import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram.Bucket; +import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket; import org.elasticsearch.search.aggregations.metrics.sum.Sum; import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy; import org.elasticsearch.test.ESIntegTestCase; @@ -189,7 +188,7 @@ public void testInlineScript() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -222,7 +221,7 @@ public void testInlineScriptNoBucketsPruned() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -255,7 +254,7 @@ public void testInlineScriptNoBucketsLeft() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -278,7 +277,7 @@ public void testInlineScript2() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -310,7 +309,7 @@ public void testInlineScriptSingleVariable() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -344,7 +343,7 @@ public void testInlineScriptNamedVars() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -377,7 +376,7 @@ public void testInlineScriptWithParams() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -410,7 +409,7 @@ public void testInlineScriptInsertZeros() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -449,7 +448,7 @@ public void testStoredScript() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -482,7 +481,7 @@ public void testUnmapped() throws Exception { assertSearchResponse(response); - InternalHistogram deriv = response.getAggregations().get("histo"); + Histogram deriv = response.getAggregations().get("histo"); assertThat(deriv, notNullValue()); assertThat(deriv.getName(), equalTo("histo")); assertThat(deriv.getBuckets().size(), equalTo(0)); @@ -504,7 +503,7 @@ public void testPartiallyUnmapped() throws Exception { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -531,7 +530,7 @@ public void testEmptyBuckets() { histogram("inner_histo") .field(FIELD_1_NAME) .interval(1) - .extendedBounds(new ExtendedBounds(1L, 4L)) + .extendedBounds(1L, 4L) .minDocCount(0) .subAggregation(derivative("derivative", "_count") .gapPolicy(GapPolicy.INSERT_ZEROS)))) @@ -539,7 +538,7 @@ public void testEmptyBuckets() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -547,7 +546,7 @@ public void testEmptyBuckets() { Histogram.Bucket bucket = buckets.get(0); assertThat(bucket, notNullValue()); - assertThat(bucket.getKeyAsString(), equalTo("1")); + assertThat(bucket.getKeyAsString(), equalTo("1.0")); Histogram innerHisto = bucket.getAggregations().get("inner_histo"); assertThat(innerHisto, notNullValue()); List innerBuckets = innerHisto.getBuckets(); @@ -564,7 +563,7 @@ public void testEmptyBuckets() { bucket = buckets.get(1); assertThat(bucket, notNullValue()); - assertThat(bucket.getKeyAsString(), equalTo("2")); + assertThat(bucket.getKeyAsString(), equalTo("2.0")); innerHisto = bucket.getAggregations().get("inner_histo"); assertThat(innerHisto, notNullValue()); innerBuckets = innerHisto.getBuckets(); @@ -580,7 +579,7 @@ public void testEmptyBuckets() { } bucket = buckets.get(2); assertThat(bucket, notNullValue()); - assertThat(bucket.getKeyAsString(), equalTo("3")); + assertThat(bucket.getKeyAsString(), equalTo("3.0")); innerHisto = bucket.getAggregations().get("inner_histo"); assertThat(innerHisto, notNullValue()); innerBuckets = innerHisto.getBuckets(); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/CumulativeSumIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/CumulativeSumIT.java index 906aa3d457893..6a748bd3c8472 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/CumulativeSumIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/CumulativeSumIT.java @@ -89,7 +89,7 @@ public void setupSuiteScopeCluster() throws Exception { public void testDocCount() throws Exception { SearchResponse response = client().prepareSearch("idx") .addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue)) + .extendedBounds(minRandomValue, maxRandomValue) .subAggregation(cumulativeSum("cumulative_sum", "_count"))).execute().actionGet(); assertSearchResponse(response); @@ -120,7 +120,7 @@ public void testMetric() throws Exception { .prepareSearch("idx") .addAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue)) + .extendedBounds(minRandomValue, maxRandomValue) .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME)) .subAggregation(cumulativeSum("cumulative_sum", "sum"))).execute().actionGet(); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/DateDerivativeIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/DateDerivativeIT.java index abda95dd2536f..e156d5586f7e0 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/DateDerivativeIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/DateDerivativeIT.java @@ -25,7 +25,6 @@ import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket; -import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram; import org.elasticsearch.search.aggregations.metrics.sum.Sum; import org.elasticsearch.search.aggregations.pipeline.derivative.Derivative; import org.elasticsearch.search.aggregations.support.AggregationPath; @@ -118,7 +117,7 @@ public void testSingleValuedField() throws Exception { assertSearchResponse(response); - InternalHistogram deriv = response.getAggregations().get("histo"); + Histogram deriv = response.getAggregations().get("histo"); assertThat(deriv, notNullValue()); assertThat(deriv.getName(), equalTo("histo")); List buckets = deriv.getBuckets(); @@ -161,7 +160,7 @@ public void testSingleValuedFieldNormalised() throws Exception { assertSearchResponse(response); - InternalHistogram deriv = response.getAggregations().get("histo"); + Histogram deriv = response.getAggregations().get("histo"); assertThat(deriv, notNullValue()); assertThat(deriv.getName(), equalTo("histo")); List buckets = deriv.getBuckets(); @@ -221,7 +220,7 @@ public void testSingleValuedFieldNormalised_timeZone_CET_DstStart() throws Excep assertSearchResponse(response); - InternalHistogram deriv = response.getAggregations().get("histo"); + Histogram deriv = response.getAggregations().get("histo"); assertThat(deriv, notNullValue()); assertThat(deriv.getName(), equalTo("histo")); List buckets = deriv.getBuckets(); @@ -259,7 +258,7 @@ public void testSingleValuedFieldNormalised_timeZone_CET_DstEnd() throws Excepti assertSearchResponse(response); - InternalHistogram deriv = response.getAggregations().get("histo"); + Histogram deriv = response.getAggregations().get("histo"); assertThat(deriv, notNullValue()); assertThat(deriv.getName(), equalTo("histo")); List buckets = deriv.getBuckets(); @@ -299,7 +298,7 @@ public void testSingleValuedFieldNormalised_timeZone_AsiaKathmandu() throws Exce assertSearchResponse(response); - InternalHistogram deriv = response.getAggregations().get("histo"); + Histogram deriv = response.getAggregations().get("histo"); assertThat(deriv, notNullValue()); assertThat(deriv.getName(), equalTo("histo")); List buckets = deriv.getBuckets(); @@ -345,7 +344,7 @@ public void testSingleValuedFieldWithSubAggregation() throws Exception { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -413,7 +412,7 @@ public void testMultiValuedField() throws Exception { assertSearchResponse(response); - InternalHistogram deriv = response.getAggregations().get("histo"); + Histogram deriv = response.getAggregations().get("histo"); assertThat(deriv, notNullValue()); assertThat(deriv.getName(), equalTo("histo")); List buckets = deriv.getBuckets(); @@ -468,7 +467,7 @@ public void testUnmapped() throws Exception { assertSearchResponse(response); - InternalHistogram deriv = response.getAggregations().get("histo"); + Histogram deriv = response.getAggregations().get("histo"); assertThat(deriv, notNullValue()); assertThat(deriv.getName(), equalTo("histo")); assertThat(deriv.getBuckets().size(), equalTo(0)); @@ -483,7 +482,7 @@ public void testPartiallyUnmapped() throws Exception { assertSearchResponse(response); - InternalHistogram deriv = response.getAggregations().get("histo"); + Histogram deriv = response.getAggregations().get("histo"); assertThat(deriv, notNullValue()); assertThat(deriv.getName(), equalTo("histo")); List buckets = deriv.getBuckets(); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/DerivativeIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/DerivativeIT.java index f1ee16855a7f2..5a780712f0246 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/DerivativeIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/DerivativeIT.java @@ -25,10 +25,8 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.query.QueryBuilders; -import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBounds; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; -import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram; -import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram.Bucket; +import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket; import org.elasticsearch.search.aggregations.metrics.stats.Stats; import org.elasticsearch.search.aggregations.metrics.sum.Sum; import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy; @@ -180,7 +178,7 @@ public void testDocCountDerivative() { assertSearchResponse(response); - InternalHistogram deriv = response.getAggregations().get("histo"); + Histogram deriv = response.getAggregations().get("histo"); assertThat(deriv, notNullValue()); assertThat(deriv.getName(), equalTo("histo")); List buckets = deriv.getBuckets(); @@ -219,7 +217,7 @@ public void testSingleValuedField_normalised() { assertSearchResponse(response); - InternalHistogram deriv = response.getAggregations().get("histo"); + Histogram deriv = response.getAggregations().get("histo"); assertThat(deriv, notNullValue()); assertThat(deriv.getName(), equalTo("histo")); List buckets = deriv.getBuckets(); @@ -257,7 +255,7 @@ public void testSingleValueAggDerivative() throws Exception { assertSearchResponse(response); - InternalHistogram deriv = response.getAggregations().get("histo"); + Histogram deriv = response.getAggregations().get("histo"); assertThat(deriv, notNullValue()); assertThat(deriv.getName(), equalTo("histo")); assertThat(deriv.getBuckets().size(), equalTo(numValueBuckets)); @@ -286,7 +284,7 @@ public void testSingleValueAggDerivative() throws Exception { assertThat(sumDeriv, nullValue()); } expectedSumPreviousBucket = expectedSum; - assertThat((long) propertiesKeys[i], equalTo((long) i * interval)); + assertThat(propertiesKeys[i], equalTo((double) i * interval)); assertThat((long) propertiesDocCounts[i], equalTo(valueCounts[i])); assertThat((double) propertiesSumCounts[i], equalTo((double) expectedSum)); } @@ -302,7 +300,7 @@ public void testMultiValueAggDerivative() throws Exception { assertSearchResponse(response); - InternalHistogram deriv = response.getAggregations().get("histo"); + Histogram deriv = response.getAggregations().get("histo"); assertThat(deriv, notNullValue()); assertThat(deriv.getName(), equalTo("histo")); assertThat(deriv.getBuckets().size(), equalTo(numValueBuckets)); @@ -331,7 +329,7 @@ public void testMultiValueAggDerivative() throws Exception { assertThat(sumDeriv, nullValue()); } expectedSumPreviousBucket = expectedSum; - assertThat((long) propertiesKeys[i], equalTo((long) i * interval)); + assertThat(propertiesKeys[i], equalTo((double) i * interval)); assertThat((long) propertiesDocCounts[i], equalTo(valueCounts[i])); assertThat((double) propertiesSumCounts[i], equalTo((double) expectedSum)); } @@ -346,7 +344,7 @@ public void testUnmapped() throws Exception { assertSearchResponse(response); - InternalHistogram deriv = response.getAggregations().get("histo"); + Histogram deriv = response.getAggregations().get("histo"); assertThat(deriv, notNullValue()); assertThat(deriv.getName(), equalTo("histo")); assertThat(deriv.getBuckets().size(), equalTo(0)); @@ -361,7 +359,7 @@ public void testPartiallyUnmapped() throws Exception { assertSearchResponse(response); - InternalHistogram deriv = response.getAggregations().get("histo"); + Histogram deriv = response.getAggregations().get("histo"); assertThat(deriv, notNullValue()); assertThat(deriv.getName(), equalTo("histo")); List buckets = deriv.getBuckets(); @@ -390,10 +388,10 @@ public void testDocCountDerivativeWithGaps() throws Exception { assertThat(searchResponse.getHits().getTotalHits(), equalTo(numDocsEmptyIdx)); - InternalHistogram deriv = searchResponse.getAggregations().get("histo"); + Histogram deriv = searchResponse.getAggregations().get("histo"); assertThat(deriv, Matchers.notNullValue()); assertThat(deriv.getName(), equalTo("histo")); - List buckets = deriv.getBuckets(); + List buckets = deriv.getBuckets(); assertThat(buckets.size(), equalTo(valueCounts_empty.length)); for (int i = 0; i < valueCounts_empty.length; i++) { @@ -414,16 +412,16 @@ public void testDocCountDerivativeWithGaps_random() throws Exception { .setQuery(matchAllQuery()) .addAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(1) - .extendedBounds(new ExtendedBounds(0L, (long) numBuckets_empty_rnd - 1)) + .extendedBounds(0L, numBuckets_empty_rnd - 1) .subAggregation(derivative("deriv", "_count").gapPolicy(randomFrom(GapPolicy.values())))) .execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(numDocsEmptyIdx_rnd)); - InternalHistogram deriv = searchResponse.getAggregations().get("histo"); + Histogram deriv = searchResponse.getAggregations().get("histo"); assertThat(deriv, Matchers.notNullValue()); assertThat(deriv.getName(), equalTo("histo")); - List buckets = deriv.getBuckets(); + List buckets = deriv.getBuckets(); assertThat(buckets.size(), equalTo(numBuckets_empty_rnd)); for (int i = 0; i < valueCounts_empty_rnd.length; i++) { @@ -449,7 +447,7 @@ public void testDocCountDerivativeWithGaps_insertZeros() throws Exception { assertThat(searchResponse.getHits().getTotalHits(), equalTo(numDocsEmptyIdx)); - InternalHistogram deriv = searchResponse.getAggregations().get("histo"); + Histogram deriv = searchResponse.getAggregations().get("histo"); assertThat(deriv, Matchers.notNullValue()); assertThat(deriv.getName(), equalTo("histo")); List buckets = deriv.getBuckets(); @@ -478,7 +476,7 @@ public void testSingleValueAggDerivativeWithGaps() throws Exception { assertThat(searchResponse.getHits().getTotalHits(), equalTo(numDocsEmptyIdx)); - InternalHistogram deriv = searchResponse.getAggregations().get("histo"); + Histogram deriv = searchResponse.getAggregations().get("histo"); assertThat(deriv, Matchers.notNullValue()); assertThat(deriv.getName(), equalTo("histo")); List buckets = deriv.getBuckets(); @@ -520,7 +518,7 @@ public void testSingleValueAggDerivativeWithGaps_insertZeros() throws Exception assertThat(searchResponse.getHits().getTotalHits(), equalTo(numDocsEmptyIdx)); - InternalHistogram deriv = searchResponse.getAggregations().get("histo"); + Histogram deriv = searchResponse.getAggregations().get("histo"); assertThat(deriv, Matchers.notNullValue()); assertThat(deriv.getName(), equalTo("histo")); List buckets = deriv.getBuckets(); @@ -553,13 +551,13 @@ public void testSingleValueAggDerivativeWithGaps_random() throws Exception { .setQuery(matchAllQuery()) .addAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(1) - .extendedBounds(new ExtendedBounds(0L, (long) numBuckets_empty_rnd - 1)) + .extendedBounds(0L, (long) numBuckets_empty_rnd - 1) .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME)) .subAggregation(derivative("deriv", "sum").gapPolicy(gapPolicy))).execute().actionGet(); assertThat(searchResponse.getHits().getTotalHits(), equalTo(numDocsEmptyIdx_rnd)); - InternalHistogram deriv = searchResponse.getAggregations().get("histo"); + Histogram deriv = searchResponse.getAggregations().get("histo"); assertThat(deriv, Matchers.notNullValue()); assertThat(deriv.getName(), equalTo("histo")); List buckets = deriv.getBuckets(); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/ExtendedStatsBucketIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/ExtendedStatsBucketIT.java index 450e0db093c81..d4310e581c026 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/ExtendedStatsBucketIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/ExtendedStatsBucketIT.java @@ -23,7 +23,6 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBounds; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket; import org.elasticsearch.search.aggregations.bucket.terms.Terms; @@ -157,7 +156,7 @@ public void testGappyIndexWithSigma() { public void testDocCountTopLevel() throws Exception { SearchResponse response = client().prepareSearch("idx") .addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .addAggregation(extendedStatsBucket("extended_stats_bucket", "histo>_count")).execute().actionGet(); assertSearchResponse(response); @@ -204,7 +203,7 @@ public void testDocCountAsSubAgg() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .subAggregation(extendedStatsBucket("extended_stats_bucket", "histo>_count"))).execute().actionGet(); assertSearchResponse(response); @@ -304,7 +303,7 @@ public void testMetricAsSubAgg() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue)) + .extendedBounds(minRandomValue, maxRandomValue) .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME))) .subAggregation(extendedStatsBucket("extended_stats_bucket", "histo>sum"))).execute().actionGet(); @@ -366,7 +365,7 @@ public void testMetricAsSubAggWithInsertZeros() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue)) + .extendedBounds(minRandomValue, maxRandomValue) .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME))) .subAggregation(extendedStatsBucket("extended_stats_bucket", "histo>sum").gapPolicy(GapPolicy.INSERT_ZEROS))) .execute().actionGet(); @@ -449,7 +448,7 @@ public void testBadSigmaAsSubAgg() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue)) + .extendedBounds(minRandomValue, maxRandomValue) .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME))) .subAggregation(extendedStatsBucket("extended_stats_bucket", "histo>sum") .sigma(-1.0))).execute().actionGet(); @@ -479,7 +478,7 @@ public void testNested() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .subAggregation(extendedStatsBucket("avg_histo_bucket", "histo>_count"))) .addAggregation(extendedStatsBucket("avg_terms_bucket", "terms>avg_histo_bucket.avg")).execute().actionGet(); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/MaxBucketIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/MaxBucketIT.java index af60a44fd1b27..aba941ebb4b6a 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/MaxBucketIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/MaxBucketIT.java @@ -22,7 +22,6 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.search.aggregations.bucket.filter.Filter; -import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBounds; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket; import org.elasticsearch.search.aggregations.bucket.terms.Terms; @@ -99,7 +98,7 @@ public void setupSuiteScopeCluster() throws Exception { public void testDocCountTopLevel() throws Exception { SearchResponse response = client().prepareSearch("idx") .addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .addAggregation(maxBucket("max_bucket", "histo>_count")).execute().actionGet(); assertSearchResponse(response); @@ -142,7 +141,7 @@ public void testDocCountAsSubAgg() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .subAggregation(maxBucket("max_bucket", "histo>_count"))).execute().actionGet(); assertSearchResponse(response); @@ -234,7 +233,7 @@ public void testMetricAsSubAgg() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue)) + .extendedBounds(minRandomValue, maxRandomValue) .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME))) .subAggregation(maxBucket("max_bucket", "histo>sum"))).execute().actionGet(); @@ -290,7 +289,7 @@ public void testMetricAsSubAggOfSingleBucketAgg() throws Exception { filter("filter", termQuery("tag", "tag0")) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue)) + .extendedBounds(minRandomValue, maxRandomValue) .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME))) .subAggregation(maxBucket("max_bucket", "histo>sum"))).execute().actionGet(); @@ -339,7 +338,7 @@ public void testMetricAsSubAggWithInsertZeros() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue)) + .extendedBounds(minRandomValue, maxRandomValue) .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME))) .subAggregation(maxBucket("max_bucket", "histo>sum").gapPolicy(GapPolicy.INSERT_ZEROS))) .execute().actionGet(); @@ -417,7 +416,7 @@ public void testNested() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .subAggregation(maxBucket("max_histo_bucket", "histo>_count"))) .addAggregation(maxBucket("max_terms_bucket", "terms>max_histo_bucket")).execute().actionGet(); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/MinBucketIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/MinBucketIT.java index c3b7cd375ed2d..cbd6824b3a4a3 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/MinBucketIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/MinBucketIT.java @@ -21,7 +21,6 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBounds; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket; import org.elasticsearch.search.aggregations.bucket.terms.Terms; @@ -96,7 +95,7 @@ public void setupSuiteScopeCluster() throws Exception { public void testDocCountTopLevel() throws Exception { SearchResponse response = client().prepareSearch("idx") .addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .addAggregation(minBucket("min_bucket", "histo>_count")).execute().actionGet(); assertSearchResponse(response); @@ -139,7 +138,7 @@ public void testDocCountAsSubAgg() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .subAggregation(minBucket("min_bucket", "histo>_count"))).execute().actionGet(); assertSearchResponse(response); @@ -231,7 +230,7 @@ public void testMetricAsSubAgg() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue)) + .extendedBounds(minRandomValue, maxRandomValue) .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME))) .subAggregation(minBucket("min_bucket", "histo>sum"))).execute().actionGet(); @@ -289,7 +288,7 @@ public void testMetricAsSubAggWithInsertZeros() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue)) + .extendedBounds(minRandomValue, maxRandomValue) .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME))) .subAggregation(minBucket("min_bucket", "histo>sum").gapPolicy(GapPolicy.INSERT_ZEROS))) .execute().actionGet(); @@ -367,7 +366,7 @@ public void testNested() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .subAggregation(minBucket("min_histo_bucket", "histo>_count"))) .addAggregation(minBucket("min_terms_bucket", "terms>min_histo_bucket")).execute().actionGet(); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/PercentilesBucketIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/PercentilesBucketIT.java index b27d4a5a4ce63..3ad2367c5c54c 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/PercentilesBucketIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/PercentilesBucketIT.java @@ -23,7 +23,6 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBounds; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; import org.elasticsearch.search.aggregations.bucket.terms.Terms; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; @@ -98,7 +97,7 @@ public void setupSuiteScopeCluster() throws Exception { public void testDocCountopLevel() throws Exception { SearchResponse response = client().prepareSearch("idx") .addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .addAggregation(percentilesBucket("percentiles_bucket", "histo>_count") .percents(PERCENTS)).execute().actionGet(); @@ -140,7 +139,7 @@ public void testDocCountAsSubAgg() throws Exception { .order(Terms.Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .subAggregation(percentilesBucket("percentiles_bucket", "histo>_count") .percents(PERCENTS))).execute().actionGet(); @@ -265,7 +264,7 @@ public void testMetricAsSubAgg() throws Exception { .order(Terms.Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue)) + .extendedBounds(minRandomValue, maxRandomValue) .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME))) .subAggregation(percentilesBucket("percentiles_bucket", "histo>sum") .percents(PERCENTS))).execute().actionGet(); @@ -321,7 +320,7 @@ public void testMetricAsSubAggWithInsertZeros() throws Exception { .order(Terms.Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue)) + .extendedBounds(minRandomValue, maxRandomValue) .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME))) .subAggregation(percentilesBucket("percentiles_bucket", "histo>sum") .gapPolicy(BucketHelpers.GapPolicy.INSERT_ZEROS) @@ -458,7 +457,7 @@ public void testBadPercents_asSubAgg() throws Exception { .order(Terms.Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .subAggregation(percentilesBucket("percentiles_bucket", "histo>_count") .percents(badPercents))).execute().actionGet(); @@ -489,7 +488,7 @@ public void testNested() throws Exception { .order(Terms.Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .subAggregation(percentilesBucket("percentile_histo_bucket", "histo>_count"))) .addAggregation(percentilesBucket("percentile_terms_bucket", "terms>percentile_histo_bucket.50") .percents(PERCENTS)).execute().actionGet(); @@ -554,7 +553,7 @@ public void testNestedWithDecimal() throws Exception { .order(Terms.Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .subAggregation(percentilesBucket("percentile_histo_bucket", "histo>_count") .percents(percent))) .addAggregation(percentilesBucket("percentile_terms_bucket", "terms>percentile_histo_bucket[99.9]") diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/StatsBucketIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/StatsBucketIT.java index 8c6479ef6bfac..c38dc99bdf939 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/StatsBucketIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/StatsBucketIT.java @@ -21,7 +21,6 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBounds; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket; import org.elasticsearch.search.aggregations.bucket.terms.Terms; @@ -96,7 +95,7 @@ public void setupSuiteScopeCluster() throws Exception { public void testDocCountTopLevel() throws Exception { SearchResponse response = client().prepareSearch("idx") .addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .addAggregation(statsBucket("stats_bucket", "histo>_count")).execute().actionGet(); assertSearchResponse(response); @@ -140,7 +139,7 @@ public void testDocCountAsSubAgg() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .subAggregation(statsBucket("stats_bucket", "histo>_count"))).execute().actionGet(); assertSearchResponse(response); @@ -234,7 +233,7 @@ public void testMetricAsSubAgg() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue)) + .extendedBounds(minRandomValue, maxRandomValue) .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME))) .subAggregation(statsBucket("stats_bucket", "histo>sum"))).execute().actionGet(); @@ -293,7 +292,7 @@ public void testMetricAsSubAggWithInsertZeros() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue)) + .extendedBounds(minRandomValue, maxRandomValue) .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME))) .subAggregation(statsBucket("stats_bucket", "histo>sum").gapPolicy(GapPolicy.INSERT_ZEROS))) .execute().actionGet(); @@ -372,7 +371,7 @@ public void testNested() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .subAggregation(statsBucket("avg_histo_bucket", "histo>_count"))) .addAggregation(statsBucket("avg_terms_bucket", "terms>avg_histo_bucket.avg")).execute().actionGet(); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/SumBucketIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/SumBucketIT.java index 2a9be035ca12f..09582430046f1 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/SumBucketIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/SumBucketIT.java @@ -21,7 +21,6 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBounds; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket; import org.elasticsearch.search.aggregations.bucket.terms.Terms; @@ -95,7 +94,7 @@ public void setupSuiteScopeCluster() throws Exception { public void testDocCountTopLevel() throws Exception { SearchResponse response = client().prepareSearch("idx") .addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .addAggregation(sumBucket("sum_bucket", "histo>_count")).execute().actionGet(); assertSearchResponse(response); @@ -130,7 +129,7 @@ public void testDocCountAsSubAgg() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .subAggregation(sumBucket("sum_bucket", "histo>_count"))).execute().actionGet(); assertSearchResponse(response); @@ -206,7 +205,7 @@ public void testMetricAsSubAgg() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue)) + .extendedBounds(minRandomValue, maxRandomValue) .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME))) .subAggregation(sumBucket("sum_bucket", "histo>sum"))).execute().actionGet(); @@ -256,7 +255,7 @@ public void testMetricAsSubAggWithInsertZeros() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue)) + .extendedBounds(minRandomValue, maxRandomValue) .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME))) .subAggregation(sumBucket("sum_bucket", "histo>sum").gapPolicy(GapPolicy.INSERT_ZEROS))) .execute().actionGet(); @@ -326,7 +325,7 @@ public void testNested() throws Exception { .order(Order.term(true)) .subAggregation( histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval) - .extendedBounds(new ExtendedBounds((long) minRandomValue, (long) maxRandomValue))) + .extendedBounds(minRandomValue, maxRandomValue)) .subAggregation(sumBucket("sum_histo_bucket", "histo>_count"))) .addAggregation(sumBucket("sum_terms_bucket", "terms>sum_histo_bucket")).execute().actionGet(); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/moving/avg/MovAvgIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/moving/avg/MovAvgIT.java index c16d8e8062e94..f24dfe4227097 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/moving/avg/MovAvgIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/moving/avg/MovAvgIT.java @@ -23,10 +23,8 @@ import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.collect.EvictingQueue; -import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBounds; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; -import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram; -import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram.Bucket; +import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket; import org.elasticsearch.search.aggregations.metrics.avg.Avg; import org.elasticsearch.search.aggregations.pipeline.BucketHelpers; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregationHelperTests; @@ -410,7 +408,7 @@ public void testSimpleSingleValuedField() { .prepareSearch("idx").setTypes("type") .addAggregation( histogram("histo").field(INTERVAL_FIELD).interval(interval) - .extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1)))) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) .subAggregation(metric) .subAggregation(movingAvg("movavg_counts","_count") .window(windowSize) @@ -424,7 +422,7 @@ public void testSimpleSingleValuedField() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -458,7 +456,7 @@ public void testLinearSingleValuedField() { .prepareSearch("idx").setTypes("type") .addAggregation( histogram("histo").field(INTERVAL_FIELD).interval(interval) - .extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1)))) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) .subAggregation(metric) .subAggregation(movingAvg("movavg_counts", "_count") .window(windowSize) @@ -472,7 +470,7 @@ public void testLinearSingleValuedField() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -506,7 +504,7 @@ public void testEwmaSingleValuedField() { .prepareSearch("idx").setTypes("type") .addAggregation( histogram("histo").field(INTERVAL_FIELD).interval(interval) - .extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1)))) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) .subAggregation(metric) .subAggregation(movingAvg("movavg_counts", "_count") .window(windowSize) @@ -520,7 +518,7 @@ public void testEwmaSingleValuedField() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -554,7 +552,7 @@ public void testHoltSingleValuedField() { .prepareSearch("idx").setTypes("type") .addAggregation( histogram("histo").field(INTERVAL_FIELD).interval(interval) - .extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1)))) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) .subAggregation(metric) .subAggregation(movingAvg("movavg_counts", "_count") .window(windowSize) @@ -568,7 +566,7 @@ public void testHoltSingleValuedField() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -602,7 +600,7 @@ public void testHoltWintersValuedField() { .prepareSearch("idx").setTypes("type") .addAggregation( histogram("histo").field(INTERVAL_FIELD).interval(interval) - .extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1)))) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) .subAggregation(metric) .subAggregation(movingAvg("movavg_counts", "_count") .window(windowSize) @@ -620,7 +618,7 @@ public void testHoltWintersValuedField() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -665,7 +663,7 @@ public void testPredictNegativeKeysAtStart() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -677,7 +675,7 @@ public void testPredictNegativeKeysAtStart() { for (int i = 1; i < 20; i++) { Bucket bucket = buckets.get(i); assertThat(bucket, notNullValue()); - assertThat((long) bucket.getKey(), equalTo((long) i - 10)); + assertThat(bucket.getKey(), equalTo(i - 10d)); assertThat(bucket.getDocCount(), equalTo(1L)); Avg avgAgg = bucket.getAggregations().get("avg"); assertThat(avgAgg, notNullValue()); @@ -690,7 +688,7 @@ public void testPredictNegativeKeysAtStart() { for (int i = 20; i < 25; i++) { Bucket bucket = buckets.get(i); assertThat(bucket, notNullValue()); - assertThat((long) bucket.getKey(), equalTo((long) i - 10)); + assertThat(bucket.getKey(), equalTo(i - 10d)); assertThat(bucket.getDocCount(), equalTo(0L)); Avg avgAgg = bucket.getAggregations().get("avg"); assertThat(avgAgg, nullValue()); @@ -706,7 +704,7 @@ public void testSizeZeroWindow() { .prepareSearch("idx").setTypes("type") .addAggregation( histogram("histo").field(INTERVAL_FIELD).interval(interval) - .extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1)))) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) .subAggregation(randomMetric("the_metric", VALUE_FIELD)) .subAggregation(movingAvg("movavg_counts", "the_metric") .window(0) @@ -744,7 +742,7 @@ public void testNegativeWindow() { .prepareSearch("idx").setTypes("type") .addAggregation( histogram("histo").field(INTERVAL_FIELD).interval(interval) - .extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1)))) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) .subAggregation(randomMetric("the_metric", VALUE_FIELD)) .subAggregation(movingAvg("movavg_counts", "_count") .window(-10) @@ -772,7 +770,7 @@ public void testNoBucketsInHistogram() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -795,7 +793,7 @@ public void testNoBucketsInHistogramWithPredict() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -808,7 +806,7 @@ public void testZeroPrediction() { .prepareSearch("idx").setTypes("type") .addAggregation( histogram("histo").field(INTERVAL_FIELD).interval(interval) - .extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1)))) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) .subAggregation(randomMetric("the_metric", VALUE_FIELD)) .subAggregation(movingAvg("movavg_counts", "the_metric") .window(windowSize) @@ -829,7 +827,7 @@ public void testNegativePrediction() { .prepareSearch("idx").setTypes("type") .addAggregation( histogram("histo").field(INTERVAL_FIELD).interval(interval) - .extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1)))) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) .subAggregation(randomMetric("the_metric", VALUE_FIELD)) .subAggregation(movingAvg("movavg_counts", "the_metric") .window(windowSize) @@ -850,7 +848,7 @@ public void testHoltWintersNotEnoughData() { .prepareSearch("idx").setTypes("type") .addAggregation( histogram("histo").field(INTERVAL_FIELD).interval(interval) - .extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1)))) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) .subAggregation(metric) .subAggregation(movingAvg("movavg_counts", "_count") .window(10) @@ -889,7 +887,7 @@ public void testTwoMovAvgsWithPredictions() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -897,7 +895,7 @@ public void testTwoMovAvgsWithPredictions() { Bucket bucket = buckets.get(0); assertThat(bucket, notNullValue()); - assertThat((long) bucket.getKey(), equalTo((long) 0)); + assertThat(bucket.getKey(), equalTo(0d)); assertThat(bucket.getDocCount(), equalTo(1L)); Avg avgAgg = bucket.getAggregations().get("avg"); @@ -916,7 +914,7 @@ public void testTwoMovAvgsWithPredictions() { // Second bucket bucket = buckets.get(1); assertThat(bucket, notNullValue()); - assertThat((long) bucket.getKey(), equalTo(1L)); + assertThat(bucket.getKey(), equalTo(1d)); assertThat(bucket.getDocCount(), equalTo(1L)); avgAgg = bucket.getAggregations().get("avg"); @@ -937,7 +935,7 @@ public void testTwoMovAvgsWithPredictions() { for (int i = 2; i < 12; i++) { bucket = buckets.get(i); assertThat(bucket, notNullValue()); - assertThat((long) bucket.getKey(), equalTo((long) i)); + assertThat(bucket.getKey(), equalTo((double) i)); assertThat(bucket.getDocCount(), equalTo(1L)); avgAgg = bucket.getAggregations().get("avg"); @@ -961,7 +959,7 @@ public void testTwoMovAvgsWithPredictions() { for (int i = 12; i < 24; i++) { bucket = buckets.get(i); assertThat(bucket, notNullValue()); - assertThat((long) bucket.getKey(), equalTo((long) i)); + assertThat(bucket.getKey(), equalTo((double) i)); assertThat(bucket.getDocCount(), equalTo(0L)); avgAgg = bucket.getAggregations().get("avg"); @@ -986,7 +984,7 @@ public void testBadModelParams() { .prepareSearch("idx").setTypes("type") .addAggregation( histogram("histo").field(INTERVAL_FIELD).interval(interval) - .extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1)))) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) .subAggregation(metric) .subAggregation(movingAvg("movavg_counts", "_count") .window(10) @@ -1004,7 +1002,7 @@ public void testHoltWintersMinimization() { .prepareSearch("idx").setTypes("type") .addAggregation( histogram("histo").field(INTERVAL_FIELD).interval(interval) - .extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1)))) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) .subAggregation(metric) .subAggregation(movingAvg("movavg_counts", "_count") .window(windowSize) @@ -1022,7 +1020,7 @@ public void testHoltWintersMinimization() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -1090,7 +1088,7 @@ public void testMinimizeNotEnoughData() { .prepareSearch("idx").setTypes("type") .addAggregation( histogram("histo").field(INTERVAL_FIELD).interval(interval) - .extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1)))) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) .subAggregation(metric) .subAggregation(movingAvg("movavg_counts", "_count") .window(numBuckets) @@ -1106,7 +1104,7 @@ public void testMinimizeNotEnoughData() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); List buckets = histo.getBuckets(); @@ -1144,7 +1142,7 @@ public void testCheckIfNonTunableCanBeMinimized() { .prepareSearch("idx").setTypes("type") .addAggregation( histogram("histo").field(INTERVAL_FIELD).interval(interval) - .extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1)))) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) .subAggregation(metric) .subAggregation(movingAvg("movavg_counts", "_count") .window(numBuckets) @@ -1162,7 +1160,7 @@ public void testCheckIfNonTunableCanBeMinimized() { .prepareSearch("idx").setTypes("type") .addAggregation( histogram("histo").field(INTERVAL_FIELD).interval(interval) - .extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1)))) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) .subAggregation(metric) .subAggregation(movingAvg("movavg_counts", "_count") .window(numBuckets) @@ -1192,7 +1190,7 @@ public void testCheckIfTunableCanBeMinimized() { .prepareSearch("idx").setTypes("type") .addAggregation( histogram("histo").field(INTERVAL_FIELD).interval(interval) - .extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1)))) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) .subAggregation(metric) .subAggregation(movingAvg("movavg_counts", "_count") .window(numBuckets) diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/serialdiff/SerialDiffIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/serialdiff/SerialDiffIT.java index 1b263d1af09d4..68257045137d1 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/serialdiff/SerialDiffIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/serialdiff/SerialDiffIT.java @@ -22,9 +22,8 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.collect.EvictingQueue; -import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBounds; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; -import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram; +import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket; import org.elasticsearch.search.aggregations.pipeline.BucketHelpers; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregationHelperTests; import org.elasticsearch.search.aggregations.pipeline.SimpleValue; @@ -232,7 +231,7 @@ public void testBasicDiff() { .prepareSearch("idx").setTypes("type") .addAggregation( histogram("histo").field(INTERVAL_FIELD).interval(interval) - .extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1)))) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) .subAggregation(metric) .subAggregation(diff("diff_counts", "_count") .lag(lag) @@ -244,10 +243,10 @@ public void testBasicDiff() { assertSearchResponse(response); - InternalHistogram histo = response.getAggregations().get("histo"); + Histogram histo = response.getAggregations().get("histo"); assertThat(histo, notNullValue()); assertThat(histo.getName(), equalTo("histo")); - List buckets = histo.getBuckets(); + List buckets = histo.getBuckets(); assertThat("Size of buckets array is not correct.", buckets.size(), equalTo(mockHisto.size())); List expectedCounts = testValues.get(MetricTarget.COUNT.toString()); @@ -279,7 +278,7 @@ public void testInvalidLagSize() { .prepareSearch("idx").setTypes("type") .addAggregation( histogram("histo").field(INTERVAL_FIELD).interval(interval) - .extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1)))) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) .subAggregation(metric) .subAggregation(diff("diff_counts", "_count") .lag(-1) diff --git a/core/src/test/java/org/elasticsearch/search/profile/aggregation/AggregationProfilerIT.java b/core/src/test/java/org/elasticsearch/search/profile/aggregation/AggregationProfilerIT.java index 848b230b3fa5c..f245629a283fc 100644 --- a/core/src/test/java/org/elasticsearch/search/profile/aggregation/AggregationProfilerIT.java +++ b/core/src/test/java/org/elasticsearch/search/profile/aggregation/AggregationProfilerIT.java @@ -21,7 +21,6 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregator; import org.elasticsearch.search.aggregations.bucket.terms.GlobalOrdinalsStringTermsAggregator; import org.elasticsearch.search.aggregations.metrics.avg.AvgAggregator; import org.elasticsearch.search.aggregations.metrics.max.MaxAggregator; @@ -99,7 +98,8 @@ public void testSimpleProfile() { assertThat(aggProfileResultsList.size(), equalTo(1)); ProfileResult histoAggResult = aggProfileResultsList.get(0); assertThat(histoAggResult, notNullValue()); - assertThat(histoAggResult.getQueryName(), equalTo(HistogramAggregator.class.getName())); + assertThat(histoAggResult.getQueryName(), + equalTo("org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregator")); assertThat(histoAggResult.getLuceneDescription(), equalTo("histo")); assertThat(histoAggResult.getProfiledChildren().size(), equalTo(0)); assertThat(histoAggResult.getTime(), greaterThan(0L)); @@ -135,7 +135,8 @@ public void testMultiLevelProfile() { assertThat(aggProfileResultsList.size(), equalTo(1)); ProfileResult histoAggResult = aggProfileResultsList.get(0); assertThat(histoAggResult, notNullValue()); - assertThat(histoAggResult.getQueryName(), equalTo(HistogramAggregator.class.getName())); + assertThat(histoAggResult.getQueryName(), + equalTo("org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregator")); assertThat(histoAggResult.getLuceneDescription(), equalTo("histo")); assertThat(histoAggResult.getTime(), greaterThan(0L)); Map histoBreakdown = histoAggResult.getTimeBreakdown(); @@ -212,7 +213,8 @@ public void testComplexProfile() { assertThat(aggProfileResultsList.size(), equalTo(1)); ProfileResult histoAggResult = aggProfileResultsList.get(0); assertThat(histoAggResult, notNullValue()); - assertThat(histoAggResult.getQueryName(), equalTo(HistogramAggregator.class.getName())); + assertThat(histoAggResult.getQueryName(), + equalTo("org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregator")); assertThat(histoAggResult.getLuceneDescription(), equalTo("histo")); assertThat(histoAggResult.getTime(), greaterThan(0L)); Map histoBreakdown = histoAggResult.getTimeBreakdown(); diff --git a/docs/reference/aggregations/bucket/histogram-aggregation.asciidoc b/docs/reference/aggregations/bucket/histogram-aggregation.asciidoc index aac9db84fdce8..b54a288b587e1 100644 --- a/docs/reference/aggregations/bucket/histogram-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/histogram-aggregation.asciidoc @@ -11,19 +11,10 @@ To make this more formal, here is the rounding function that is used: [source,java] -------------------------------------------------- -rem = value % interval -if (rem < 0) { - rem += interval -} -bucket_key = value - rem +bucket_key = Math.floor((value - offset) / interval) * interval + offset -------------------------------------------------- -From the rounding function above it can be seen that the intervals themselves **must** be integers. - -WARNING: Currently, values are cast to integers before being bucketed, which -might cause negative floating-point values to fall into the wrong bucket. For -instance, `-4.5` with an interval of `2` would be cast to `-4`, and so would -end up in the `-4 <= val < -2` bucket instead of the `-6 <= val < -4` bucket. +The `interval` must be a positive decimal, while the `offset` must be a decimal in `[0, interval[`. The following snippet "buckets" the products based on their `price` by interval of `50`: diff --git a/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/MoreExpressionTests.java b/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/MoreExpressionTests.java index 6796c1e048b55..d71d09f2f3775 100644 --- a/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/MoreExpressionTests.java +++ b/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/MoreExpressionTests.java @@ -19,23 +19,14 @@ package org.elasticsearch.script.expression; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - import org.apache.lucene.expressions.Expression; import org.apache.lucene.expressions.js.JavascriptCompiler; -import org.elasticsearch.Version; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.action.update.UpdateRequestBuilder; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.lucene.search.function.CombineFunction; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.index.query.QueryBuilders; @@ -43,22 +34,25 @@ import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.script.CompiledScript; -import org.elasticsearch.script.Script; import org.elasticsearch.script.GeneralScriptException; +import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptService.ScriptType; import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.aggregations.AggregationBuilders; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; -import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram; -import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram.Bucket; import org.elasticsearch.search.aggregations.metrics.stats.Stats; import org.elasticsearch.search.aggregations.pipeline.SimpleValue; import org.elasticsearch.search.sort.SortBuilders; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.VersionUtils; import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram; @@ -600,10 +594,10 @@ public void testPipelineAggregationScript() throws Exception { "twoSum", "threeSum", "fourSum"))) .execute().actionGet(); - InternalHistogram histogram = response.getAggregations().get("histogram"); + Histogram histogram = response.getAggregations().get("histogram"); assertThat(histogram, notNullValue()); assertThat(histogram.getName(), equalTo("histogram")); - List buckets = histogram.getBuckets(); + List buckets = histogram.getBuckets(); for (int bucketCount = 0; bucketCount < buckets.size(); ++bucketCount) { Histogram.Bucket bucket = buckets.get(bucketCount);