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 extends Bucket> 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, ? extends InternalMultiBucketAggregation.Bucket> 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, ? extends InternalMultiBucketAggregation.Bucket> 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 extends InternalHistogram.Bucket> buckets = histo.getBuckets();
- InternalHistogram.Factory extends InternalHistogram.Bucket> factory = histo.getFactory();
+ MultiBucketsAggregation histo = (MultiBucketsAggregation) aggregation;
+ List extends Bucket> 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 extends InternalHistogram.Bucket> buckets = histo.getBuckets();
- InternalHistogram.Factory extends InternalHistogram.Bucket> factory = histo.getFactory();
+ MultiBucketsAggregation histo = (MultiBucketsAggregation) aggregation;
+ List extends Bucket> 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 extends InternalHistogram.Bucket> buckets = histo.getBuckets();
- InternalHistogram.Factory extends InternalHistogram.Bucket> factory = histo.getFactory();
+ MultiBucketsAggregation histo = (MultiBucketsAggregation) aggregation;
+ List extends Bucket> 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 extends InternalHistogram.Bucket> buckets, InternalHistogram histo, MovAvgModel model) {
+ private MovAvgModel minimize(List extends Bucket> buckets, MultiBucketsAggregation histo, MovAvgModel model) {
int counter = 0;
EvictingQueue values = new EvictingQueue<>(this.window);
double[] test = new double[window];
- ListIterator extends InternalHistogram.Bucket> iter = buckets.listIterator(buckets.size());
+ ListIterator extends Bucket> 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 extends InternalHistogram.Bucket> buckets = histo.getBuckets();
- InternalHistogram.Factory extends InternalHistogram.Bucket> factory = histo.getFactory();
+ MultiBucketsAggregation histo = (MultiBucketsAggregation) aggregation;
+ List extends Bucket> 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 extends Histogram.Bucket> 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 extends Bucket> buckets = histo.getBuckets();
+ List extends Histogram.Bucket> 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 extends Bucket> buckets = histo.getBuckets();
+ List extends Histogram.Bucket> 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 extends Bucket> buckets = histo.getBuckets();
+ List extends Histogram.Bucket> 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 extends Bucket> buckets = histo.getBuckets();
+ List extends Histogram.Bucket> 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 extends Bucket> buckets = histo.getBuckets();
+ List extends Histogram.Bucket> 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 extends Bucket> buckets = histo.getBuckets();
+ List extends Histogram.Bucket> 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 extends Bucket> buckets = histo.getBuckets();
+ List extends Histogram.Bucket> 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 extends Bucket> buckets = histo.getBuckets();
+ List extends Histogram.Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Histogram.Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> 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 extends Bucket> buckets = histo.getBuckets();
@@ -795,7 +793,7 @@ public void testNoBucketsInHistogramWithPredict() {
assertSearchResponse(response);
- InternalHistogram