Split regular histograms from date histograms. #19551
Currently both aggregations really share the same implementation. This commit splits the implementations so that regular histograms can support decimal intervals/offsets and compute correct buckets for negative decimal values. However the response API is still the same. So for intance both regular histograms and date histograms will produce an `org.elasticsearch.search.aggregations.bucket.histogram.Histogram` aggregation. The optimization to compute an identifier of the rounded value and the rounded value itself has been removed since it was only used by regular histograms, which now do the rounding themselves instead of relying on the Rounding abstraction. Closes #8082 Closes #4847
This commit is contained in:
parent
f6aeb35ce8
commit
a0818d3b87
|
@ -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 <code>roundValue(roundKey(value))</code>.
|
||||
*
|
||||
* @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 abstract class Rounding implements Streamable {
|
|||
}
|
||||
|
||||
@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 abstract class Rounding implements Streamable {
|
|||
}
|
||||
|
||||
@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 abstract class Rounding implements Streamable {
|
|||
}
|
||||
|
||||
@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
|
||||
|
|
|
@ -31,6 +31,9 @@ import java.io.IOException;
|
|||
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 abstract class TimeZoneRounding extends Rounding {
|
|||
}
|
||||
|
||||
@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 abstract class TimeZoneRounding extends Rounding {
|
|||
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 abstract class TimeZoneRounding extends Rounding {
|
|||
}
|
||||
|
||||
@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 abstract class TimeZoneRounding extends Rounding {
|
|||
// 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 @@ public abstract class TimeZoneRounding extends Rounding {
|
|||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long valueForKey(long time) {
|
||||
assert roundKey(time) == time;
|
||||
return time;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long nextRoundingValue(long time) {
|
||||
long timeLocal = time;
|
||||
|
|
|
@ -116,6 +116,7 @@ import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggre
|
|||
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 @@ public class SearchModule extends AbstractModule {
|
|||
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(),
|
||||
|
|
|
@ -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<AF extends AbstractHistogramAggregatorFactory<AF>>
|
||||
extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, AF> {
|
||||
|
||||
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<Numeric> config, long interval, long offset,
|
||||
InternalOrder order, boolean keyed, long minDocCount, ExtendedBounds extendedBounds,
|
||||
InternalHistogram.Factory<?> histogramFactory, AggregationContext context, AggregatorFactory<?> parent,
|
||||
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> 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<PipelineAggregator> pipelineAggregators, Map<String, Object> 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<PipelineAggregator> pipelineAggregators, Map<String, Object> 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<PipelineAggregator> pipelineAggregators,
|
||||
Map<String, Object> 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);
|
||||
}
|
||||
|
||||
}
|
|
@ -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<AB extends AbstractHistogramBuilder<AB>>
|
||||
extends ValuesSourceAggregationBuilder<ValuesSource.Numeric, AB> {
|
||||
|
||||
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);
|
||||
}
|
||||
}
|
|
@ -27,40 +27,91 @@ 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 DateHistogramAggregationBuilder extends AbstractHistogramBuilder<DateHistogramAggregationBuilder> {
|
||||
|
||||
/**
|
||||
* A builder for histograms on date fields.
|
||||
*/
|
||||
public class DateHistogramAggregationBuilder
|
||||
extends ValuesSourceAggregationBuilder<ValuesSource.Numeric, DateHistogramAggregationBuilder> {
|
||||
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);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the interval.
|
||||
*/
|
||||
/** 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 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 class DateHistogramAggregationBuilder extends AbstractHistogramBuilder<Da
|
|||
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 class DateHistogramAggregationBuilder extends AbstractHistogramBuilder<Da
|
|||
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,15 +153,90 @@ public class DateHistogramAggregationBuilder extends AbstractHistogramBuilder<Da
|
|||
.millis();
|
||||
}
|
||||
|
||||
public DateHistogramInterval dateHistogramInterval() {
|
||||
return dateHistogramInterval;
|
||||
/** Return extended bounds for this histogram, or {@code null} if none are set. */
|
||||
public ExtendedBounds extendedBounds() {
|
||||
return extendedBounds;
|
||||
}
|
||||
|
||||
/** 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;
|
||||
}
|
||||
|
||||
/** 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 DateHistogramAggregatorFactory innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
|
||||
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
|
||||
return new DateHistogramAggregatorFactory(name, type, config, interval, dateHistogramInterval, offset, order, keyed, minDocCount,
|
||||
extendedBounds, context, parent, subFactoriesBuilder, metaData);
|
||||
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
|
||||
|
||||
if (dateHistogramInterval == null) {
|
||||
builder.field(Histogram.INTERVAL_FIELD.getPreferredName(), interval);
|
||||
} else {
|
||||
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
|
||||
|
@ -105,23 +245,26 @@ public class DateHistogramAggregationBuilder extends AbstractHistogramBuilder<Da
|
|||
}
|
||||
|
||||
@Override
|
||||
protected XContentBuilder doXContentInterval(XContentBuilder builder, Params params) throws IOException {
|
||||
if (dateHistogramInterval == null) {
|
||||
super.doXContentInterval(builder, params);
|
||||
} else {
|
||||
builder.value(dateHistogramInterval.toString());
|
||||
}
|
||||
return builder;
|
||||
protected ValuesSourceAggregatorFactory<Numeric, ?> innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> 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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<PipelineAggregator> pipelineAggregators, Map<String, Object> 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<InternalDateHistogram.Bucket> 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);
|
||||
}
|
||||
}
|
|
@ -23,23 +23,29 @@ import org.elasticsearch.common.rounding.DateTimeUnit;
|
|||
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<DateHistogramAggregatorFactory> {
|
||||
public final class DateHistogramAggregatorFactory
|
||||
extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, DateHistogramAggregatorFactory> {
|
||||
|
||||
public static final Map<String, DateTimeUnit> DATE_FIELD_UNITS;
|
||||
private final DateHistogramInterval dateHistogramInterval;
|
||||
|
||||
static {
|
||||
Map<String, DateTimeUnit> 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<Numeric> config, long interval,
|
||||
DateHistogramInterval dateHistogramInterval, long offset, InternalOrder order, boolean keyed, long minDocCount,
|
||||
ExtendedBounds extendedBounds, AggregationContext context, AggregatorFactory<?> parent,
|
||||
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> 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 @@ public class DateHistogramAggregatorFactory extends AbstractHistogramAggregatorF
|
|||
return rounding;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
|
||||
List<PipelineAggregator> pipelineAggregators, Map<String, Object> 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<PipelineAggregator> pipelineAggregators,
|
||||
Map<String, Object> 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<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
|
||||
throws IOException {
|
||||
return createAggregator(null, parent, pipelineAggregators, metaData);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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.io.IOException;
|
|||
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 @@ public class DateHistogramParser extends HistogramParser {
|
|||
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 @@ public class DateHistogramParser extends HistogramParser {
|
|||
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<ParseField, Object> 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 @@ public class DateHistogramParser extends HistogramParser {
|
|||
}
|
||||
return new InternalOrder.Aggregation(key, asc);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected long parseStringOffset(String offset) throws IOException {
|
||||
return DateHistogramAggregationBuilder.parseStringOffset(offset);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -41,7 +41,7 @@ import java.util.Objects;
|
|||
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");
|
||||
|
||||
|
|
|
@ -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 @@ import java.util.List;
|
|||
*/
|
||||
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 @@ public interface Histogram extends MultiBucketsAggregation {
|
|||
* @return The buckets of this histogram (each bucket representing an interval in the histogram)
|
||||
*/
|
||||
@Override
|
||||
List<? extends Bucket> getBuckets();
|
||||
List<Bucket> getBuckets();
|
||||
|
||||
|
||||
/**
|
||||
|
@ -48,38 +56,48 @@ public interface Histogram extends MultiBucketsAggregation {
|
|||
*/
|
||||
abstract class Order implements ToXContent {
|
||||
|
||||
public static final Order KEY_ASC = new InternalOrder((byte) 1, "_key", true, new Comparator<InternalHistogram.Bucket>() {
|
||||
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<Histogram.Bucket>() {
|
||||
@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<InternalHistogram.Bucket>() {
|
||||
public static final Order KEY_DESC = new InternalOrder((byte) 2, "_key", false, new Comparator<Histogram.Bucket>() {
|
||||
@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<InternalHistogram.Bucket>() {
|
||||
public static final Order COUNT_ASC = new InternalOrder((byte) 3, "_count", true, new Comparator<Histogram.Bucket>() {
|
||||
@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<InternalHistogram.Bucket>() {
|
||||
public static final Order COUNT_DESC = new InternalOrder((byte) 4, "_count", false, new Comparator<Histogram.Bucket>() {
|
||||
@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 interface Histogram extends MultiBucketsAggregation {
|
|||
/**
|
||||
* @return The bucket comparator by which the order will be applied.
|
||||
*/
|
||||
abstract Comparator<InternalHistogram.Bucket> comparator();
|
||||
abstract Comparator<Histogram.Bucket> comparator();
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,38 +21,224 @@ package org.elasticsearch.search.aggregations.bucket.histogram;
|
|||
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.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<HistogramAggregationBuilder> {
|
||||
/**
|
||||
* A builder for histograms on numeric fields.
|
||||
*/
|
||||
public class HistogramAggregationBuilder
|
||||
extends ValuesSourceAggregationBuilder<ValuesSource.Numeric, HistogramAggregationBuilder> {
|
||||
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<Numeric> 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<Numeric, ?> innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> 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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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.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.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.Collections;
|
|||
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<PipelineAggregator> pipelineAggregators, Map<String, Object> 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<PipelineAggregator> pipelineAggregators, Map<String, Object> 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 class HistogramAggregator extends BucketsAggregator {
|
|||
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 class HistogramAggregator extends BucketsAggregator {
|
|||
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 class HistogramAggregator extends BucketsAggregator {
|
|||
}
|
||||
|
||||
@Override
|
||||
public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
|
||||
assert owningBucketOrdinal == 0;
|
||||
public InternalAggregation buildAggregation(long bucket) throws IOException {
|
||||
assert bucket == 0;
|
||||
List<InternalHistogram.Bucket> 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
|
||||
|
|
|
@ -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<ValuesSource.Numeric, HistogramAggregatorFactory> {
|
||||
|
||||
public class HistogramAggregatorFactory extends AbstractHistogramAggregatorFactory<HistogramAggregatorFactory> {
|
||||
private final double interval, offset;
|
||||
private final InternalOrder order;
|
||||
private final boolean keyed;
|
||||
private final long minDocCount;
|
||||
private final double minBound, maxBound;
|
||||
|
||||
public HistogramAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, long interval, long offset,
|
||||
InternalOrder order, boolean keyed, long minDocCount, ExtendedBounds extendedBounds, AggregationContext context,
|
||||
AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
|
||||
super(name, type, config, interval, offset, order, keyed, minDocCount, extendedBounds, InternalHistogram.HISTOGRAM_FACTORY, context,
|
||||
parent, subFactoriesBuilder, metaData);
|
||||
HistogramAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, double interval, double offset,
|
||||
InternalOrder order, boolean keyed, long minDocCount, double minBound, double maxBound,
|
||||
AggregationContext context, AggregatorFactory<?> parent,
|
||||
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> 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;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
|
||||
List<PipelineAggregator> pipelineAggregators, Map<String, Object> 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<PipelineAggregator> pipelineAggregators,
|
||||
Map<String, Object> metaData) throws IOException {
|
||||
|
||||
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<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
|
||||
throws IOException {
|
||||
return createAggregator(null, parent, pipelineAggregators, metaData);
|
||||
}
|
||||
}
|
|
@ -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<MultiBucketsAggregation.Bucket> 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);
|
||||
|
||||
}
|
|
@ -20,8 +20,9 @@ package org.elasticsearch.search.aggregations.bucket.histogram;
|
|||
|
||||
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.io.IOException;
|
|||
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 {
|
||||
|
||||
private static final ObjectParser<double[], ParseFieldMatcherSupplier> 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"));
|
||||
}
|
||||
|
||||
public HistogramParser() {
|
||||
super(true, true, false);
|
||||
}
|
||||
|
||||
protected HistogramParser(boolean timezoneAware) {
|
||||
super(true, true, timezoneAware);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AbstractHistogramBuilder<?> createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
protected HistogramAggregationBuilder createFactory(String aggregationName, ValuesSourceType valuesSourceType,
|
||||
ValueType targetValueType, Map<ParseField, Object> 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 @@ public class HistogramParser extends NumericValuesSourceParser {
|
|||
protected boolean token(String aggregationName, String currentFieldName, Token token, XContentParser parser,
|
||||
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> 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 @@ public class HistogramParser extends NumericValuesSourceParser {
|
|||
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 @@ public class HistogramParser extends NumericValuesSourceParser {
|
|||
}
|
||||
}
|
||||
|
||||
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);
|
||||
|
|
|
@ -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<InternalDateHistogram, InternalDateHistogram.Bucket>
|
||||
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<Bucket> 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<Bucket> 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 class InternalDateHistogram {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DateTime getKey() {
|
||||
public Object getKey() {
|
||||
return new DateTime(key, DateTimeZone.UTC);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getKeyAsString();
|
||||
public long getDocCount() {
|
||||
return docCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Aggregations getAggregations() {
|
||||
return aggregations;
|
||||
}
|
||||
|
||||
Bucket reduce(List<Bucket> buckets, ReduceContext context) {
|
||||
List<InternalAggregations> 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 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;
|
||||
}
|
||||
|
||||
public DocValueFormat getFormatter() {
|
||||
return format;
|
||||
}
|
||||
|
||||
public boolean getKeyed() {
|
||||
return keyed;
|
||||
}
|
||||
}
|
||||
|
||||
static class Factory extends InternalHistogram.Factory<InternalDateHistogram.Bucket> {
|
||||
static class EmptyBucketInfo {
|
||||
|
||||
Factory() {
|
||||
final Rounding rounding;
|
||||
final InternalAggregations subAggregations;
|
||||
final ExtendedBounds bounds;
|
||||
|
||||
EmptyBucketInfo(Rounding rounding, InternalAggregations subAggregations) {
|
||||
this(rounding, subAggregations, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Type type() {
|
||||
return TYPE;
|
||||
EmptyBucketInfo(Rounding rounding, InternalAggregations subAggregations, ExtendedBounds bounds) {
|
||||
this.rounding = rounding;
|
||||
this.subAggregations = subAggregations;
|
||||
this.bounds = bounds;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType valueType() {
|
||||
return ValueType.DATE;
|
||||
EmptyBucketInfo(StreamInput in) throws IOException {
|
||||
rounding = Rounding.Streams.read(in);
|
||||
subAggregations = InternalAggregations.readAggregations(in);
|
||||
bounds = in.readOptionalWriteable(ExtendedBounds::new);
|
||||
}
|
||||
|
||||
@Override
|
||||
public InternalDateHistogram.Bucket createBucket(InternalAggregations aggregations, InternalDateHistogram.Bucket prototype) {
|
||||
return new Bucket(prototype.key, prototype.docCount, aggregations, prototype.getKeyed(), prototype.format, this);
|
||||
void writeTo(StreamOutput out) throws IOException {
|
||||
Rounding.Streams.write(rounding, out);
|
||||
subAggregations.writeTo(out);
|
||||
out.writeOptionalWriteable(bounds);
|
||||
}
|
||||
|
||||
@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);
|
||||
} else {
|
||||
throw new AggregationExecutionException("Expected key of type Number or DateTime but got [" + key + "]");
|
||||
}
|
||||
|
||||
private final List<Bucket> 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<Bucket> buckets, InternalOrder order, long minDocCount, EmptyBucketInfo emptyBucketInfo,
|
||||
DocValueFormat formatter, boolean keyed, List<PipelineAggregator> pipelineAggregators,
|
||||
Map<String, Object> 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<Histogram.Bucket> getBuckets() {
|
||||
return Collections.unmodifiableList(buckets);
|
||||
}
|
||||
|
||||
@Override
|
||||
public InternalDateHistogram create(List<Bucket> 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<Bucket> iterator;
|
||||
private Bucket current;
|
||||
|
||||
IteratorAndCurrent(Iterator<Bucket> iterator) {
|
||||
this.iterator = iterator;
|
||||
current = iterator.next();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private List<Bucket> reduceBuckets(List<InternalAggregation> aggregations, ReduceContext reduceContext) {
|
||||
|
||||
final PriorityQueue<IteratorAndCurrent> pq = new PriorityQueue<IteratorAndCurrent>(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()));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Bucket readBucket(StreamInput in, boolean keyed, DocValueFormat format) throws IOException {
|
||||
return new Bucket(in, keyed, format, this);
|
||||
List<Bucket> reducedBuckets = new ArrayList<>();
|
||||
if (pq.size() > 0) {
|
||||
// list of buckets coming from different shards that have the same key
|
||||
List<Bucket> 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<Bucket> list, ReduceContext reduceContext) {
|
||||
Bucket lastBucket = null;
|
||||
ExtendedBounds bounds = emptyBucketInfo.bounds;
|
||||
ListIterator<Bucket> 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 {
|
||||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private InternalDateHistogram() {}
|
||||
@Override
|
||||
public InternalAggregation doReduce(List<InternalAggregation> aggregations, ReduceContext reduceContext) {
|
||||
List<Bucket> 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<Bucket> 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());
|
||||
}
|
||||
|
||||
@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<MultiBucketsAggregation.Bucket> buckets) {
|
||||
// convert buckets to the right type
|
||||
List<Bucket> 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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,16 +22,14 @@ 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.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.ListIterator;
|
|||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* TODO should be renamed to InternalNumericHistogram (see comment on {@link Histogram})?
|
||||
* Imelementation of {@link Histogram}.
|
||||
*/
|
||||
public class InternalHistogram<B extends InternalHistogram.Bucket> extends InternalMultiBucketAggregation<InternalHistogram<B>, B>
|
||||
implements Histogram {
|
||||
public final class InternalHistogram extends InternalMultiBucketAggregation<InternalHistogram, InternalHistogram.Bucket>
|
||||
implements Histogram, HistogramFactory {
|
||||
|
||||
public static final Factory<Bucket> HISTOGRAM_FACTORY = new Factory<Bucket>();
|
||||
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 class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
|
|||
/**
|
||||
* 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 class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
|
|||
return aggregations;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
<B extends Bucket> B reduce(List<B> buckets, ReduceContext context) {
|
||||
Bucket reduce(List<Bucket> buckets, ReduceContext context) {
|
||||
List<InternalAggregations> aggregations = new ArrayList<>(buckets.size());
|
||||
long docCount = 0;
|
||||
for (Bucket bucket : buckets) {
|
||||
|
@ -121,7 +110,7 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
|
|||
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 class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
|
|||
|
||||
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);
|
||||
EmptyBucketInfo(StreamInput in) throws IOException {
|
||||
this(in.readDouble(), in.readDouble(), in.readDouble(), in.readDouble(), InternalAggregations.readAggregations(in));
|
||||
}
|
||||
|
||||
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 void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeDouble(interval);
|
||||
out.writeDouble(offset);
|
||||
out.writeDouble(minBound);
|
||||
out.writeDouble(maxBound);
|
||||
subAggregations.writeTo(out);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public static class Factory<B extends InternalHistogram.Bucket> {
|
||||
|
||||
protected Factory() {
|
||||
}
|
||||
|
||||
public Type type() {
|
||||
return TYPE;
|
||||
}
|
||||
|
||||
public ValueType valueType() {
|
||||
return ValueType.NUMERIC;
|
||||
}
|
||||
|
||||
public InternalHistogram<B> create(String name, List<B> buckets, InternalOrder order, long minDocCount,
|
||||
EmptyBucketInfo emptyBucketInfo, DocValueFormat formatter, boolean keyed,
|
||||
List<PipelineAggregator> pipelineAggregators,
|
||||
Map<String, Object> metaData) {
|
||||
return new InternalHistogram<>(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed, this, pipelineAggregators,
|
||||
metaData);
|
||||
}
|
||||
|
||||
public InternalHistogram<B> create(List<B> buckets, InternalHistogram<B> 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);
|
||||
}
|
||||
|
||||
@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 + "]");
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
protected B readBucket(StreamInput in, boolean keyed, DocValueFormat format) throws IOException {
|
||||
return (B) new Bucket(in, keyed, format, this);
|
||||
}
|
||||
}
|
||||
|
||||
private final List<B> buckets;
|
||||
private final List<Bucket> buckets;
|
||||
private final InternalOrder order;
|
||||
private final DocValueFormat format;
|
||||
private final boolean keyed;
|
||||
private final long minDocCount;
|
||||
private final EmptyBucketInfo emptyBucketInfo;
|
||||
private final Factory<B> factory;
|
||||
|
||||
InternalHistogram(String name, List<B> buckets, InternalOrder order, long minDocCount, EmptyBucketInfo emptyBucketInfo,
|
||||
DocValueFormat formatter, boolean keyed, Factory<B> factory, List<PipelineAggregator> pipelineAggregators,
|
||||
InternalHistogram(String name, List<Bucket> buckets, InternalOrder order, long minDocCount, EmptyBucketInfo emptyBucketInfo,
|
||||
DocValueFormat formatter, boolean keyed, List<PipelineAggregator> pipelineAggregators,
|
||||
Map<String, Object> metaData) {
|
||||
super(name, pipelineAggregators, metaData);
|
||||
this.buckets = buckets;
|
||||
|
@ -252,7 +185,6 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
|
|||
this.emptyBucketInfo = emptyBucketInfo;
|
||||
this.format = formatter;
|
||||
this.keyed = keyed;
|
||||
this.factory = factory;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -260,37 +192,24 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
|
|||
*/
|
||||
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 <B extends InternalHistogram.Bucket> Factory<B> resolveFactory(String factoryType) {
|
||||
if (factoryType.equals(InternalDateHistogram.TYPE.name())) {
|
||||
return (Factory<B>) 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 class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<B> getBuckets() {
|
||||
return buckets;
|
||||
}
|
||||
|
||||
public Factory<B> getFactory() {
|
||||
return factory;
|
||||
}
|
||||
|
||||
public Rounding getRounding() {
|
||||
return emptyBucketInfo.rounding;
|
||||
public List<Histogram.Bucket> getBuckets() {
|
||||
return Collections.unmodifiableList(buckets);
|
||||
}
|
||||
|
||||
@Override
|
||||
public InternalHistogram<B> create(List<B> buckets) {
|
||||
return getFactory().create(buckets, this);
|
||||
public InternalHistogram create(List<Bucket> 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<B> {
|
||||
private static class IteratorAndCurrent {
|
||||
|
||||
private final Iterator<B> iterator;
|
||||
private B current;
|
||||
private final Iterator<Bucket> iterator;
|
||||
private Bucket current;
|
||||
|
||||
IteratorAndCurrent(Iterator<B> iterator) {
|
||||
IteratorAndCurrent(Iterator<Bucket> iterator) {
|
||||
this.iterator = iterator;
|
||||
current = iterator.next();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private List<B> reduceBuckets(List<InternalAggregation> aggregations, ReduceContext reduceContext) {
|
||||
private List<Bucket> reduceBuckets(List<InternalAggregation> aggregations, ReduceContext reduceContext) {
|
||||
|
||||
final PriorityQueue<IteratorAndCurrent<B>> pq = new PriorityQueue<IteratorAndCurrent<B>>(aggregations.size()) {
|
||||
final PriorityQueue<IteratorAndCurrent> pq = new PriorityQueue<IteratorAndCurrent>(aggregations.size()) {
|
||||
@Override
|
||||
protected boolean lessThan(IteratorAndCurrent<B> a, IteratorAndCurrent<B> b) {
|
||||
protected boolean lessThan(IteratorAndCurrent a, IteratorAndCurrent b) {
|
||||
return a.current.key < b.current.key;
|
||||
}
|
||||
};
|
||||
for (InternalAggregation aggregation : aggregations) {
|
||||
@SuppressWarnings("unchecked")
|
||||
InternalHistogram<B> histogram = (InternalHistogram<B>) 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<B> reducedBuckets = new ArrayList<>();
|
||||
List<Bucket> reducedBuckets = new ArrayList<>();
|
||||
if (pq.size() > 0) {
|
||||
// list of buckets coming from different shards that have the same key
|
||||
List<B> currentBuckets = new ArrayList<>();
|
||||
long key = pq.top().current.key;
|
||||
List<Bucket> currentBuckets = new ArrayList<>();
|
||||
double key = pq.top().current.key;
|
||||
|
||||
do {
|
||||
final IteratorAndCurrent<B> 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 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
|
|||
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 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
|
|||
} 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 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
|
|||
return reducedBuckets;
|
||||
}
|
||||
|
||||
private void addEmptyBuckets(List<B> list, ReduceContext reduceContext) {
|
||||
B lastBucket = null;
|
||||
ExtendedBounds bounds = emptyBucketInfo.bounds;
|
||||
ListIterator<B> 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<Bucket> list, ReduceContext reduceContext) {
|
||||
ListIterator<Bucket> 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<InternalAggregation> aggregations, ReduceContext reduceContext) {
|
||||
List<B> reducedBuckets = reduceBuckets(aggregations, reduceContext);
|
||||
List<Bucket> reducedBuckets = reduceBuckets(aggregations, reduceContext);
|
||||
|
||||
// adding empty buckets if needed
|
||||
if (minDocCount == 0) {
|
||||
|
@ -476,7 +373,7 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
|
|||
// maintains order
|
||||
} else if (order == InternalOrder.KEY_DESC) {
|
||||
// we just need to reverse here...
|
||||
List<B> reverse = new ArrayList<>(reducedBuckets);
|
||||
List<Bucket> reverse = new ArrayList<>(reducedBuckets);
|
||||
Collections.reverse(reverse);
|
||||
reducedBuckets = reverse;
|
||||
} else {
|
||||
|
@ -484,7 +381,7 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
|
|||
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 class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
|
|||
} else {
|
||||
builder.startArray(CommonFields.BUCKETS);
|
||||
}
|
||||
for (B bucket : buckets) {
|
||||
for (Bucket bucket : buckets) {
|
||||
bucket.toXContent(builder, params);
|
||||
}
|
||||
if (keyed) {
|
||||
|
@ -506,4 +403,33 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
|
|||
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<MultiBucketsAggregation.Bucket> buckets) {
|
||||
// convert buckets to the right type
|
||||
List<Bucket> 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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -35,9 +35,9 @@ class InternalOrder extends Histogram.Order {
|
|||
final byte id;
|
||||
final String key;
|
||||
final boolean asc;
|
||||
final Comparator<InternalHistogram.Bucket> comparator;
|
||||
final Comparator<Histogram.Bucket> comparator;
|
||||
|
||||
InternalOrder(byte id, String key, boolean asc, Comparator<InternalHistogram.Bucket> comparator) {
|
||||
InternalOrder(byte id, String key, boolean asc, Comparator<Histogram.Bucket> comparator) {
|
||||
this.id = id;
|
||||
this.key = key;
|
||||
this.asc = asc;
|
||||
|
@ -57,7 +57,7 @@ class InternalOrder extends Histogram.Order {
|
|||
}
|
||||
|
||||
@Override
|
||||
Comparator<InternalHistogram.Bucket> comparator() {
|
||||
Comparator<Histogram.Bucket> comparator() {
|
||||
return comparator;
|
||||
}
|
||||
|
||||
|
@ -90,11 +90,7 @@ class InternalOrder extends Histogram.Order {
|
|||
static final byte ID = 0;
|
||||
|
||||
Aggregation(String key, boolean asc) {
|
||||
super(ID, key, asc, new MultiBucketsAggregation.Bucket.SubAggregationComparator<InternalHistogram.Bucket>(key, asc));
|
||||
}
|
||||
|
||||
private static String key(String aggName, String valueName) {
|
||||
return (valueName == null) ? aggName : aggName + "." + valueName;
|
||||
super(ID, key, asc, new MultiBucketsAggregation.Bucket.SubAggregationComparator<Histogram.Bucket>(key, asc));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.elasticsearch.index.query.QueryParseContext;
|
|||
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 class BucketHelpers {
|
|||
* @return The value extracted from <code>bucket</code> found at
|
||||
* <code>aggPath</code>
|
||||
*/
|
||||
public static Double resolveBucketValue(InternalMultiBucketAggregation<?, ? extends InternalMultiBucketAggregation.Bucket> agg,
|
||||
public static Double resolveBucketValue(MultiBucketsAggregation agg,
|
||||
InternalMultiBucketAggregation.Bucket bucket, String aggPath, GapPolicy gapPolicy) {
|
||||
List<String> 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<String> aggPathAsList, GapPolicy gapPolicy) {
|
||||
try {
|
||||
Object propertyValue = bucket.getProperty(agg.getName(), aggPathAsList);
|
||||
|
|
|
@ -29,7 +29,8 @@ import org.elasticsearch.index.query.QueryParseContext;
|
|||
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 class CumulativeSumPipelineAggregationBuilder extends AbstractPipelineAgg
|
|||
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");
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -25,7 +25,9 @@ import org.elasticsearch.search.DocValueFormat;
|
|||
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 class CumulativeSumPipelineAggregator extends PipelineAggregator {
|
|||
|
||||
@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<Bucket> 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<InternalAggregation> aggs = StreamSupport.stream(bucket.getAggregations().spliterator(), false).map((p) -> {
|
||||
return (InternalAggregation) p;
|
||||
}).collect(Collectors.toList());
|
||||
aggs.add(new InternalSimpleValue(name(), sum, formatter, new ArrayList<PipelineAggregator>(), 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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,9 +31,9 @@ import org.elasticsearch.index.query.QueryParseContext;
|
|||
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 class DerivativePipelineAggregationBuilder extends AbstractPipelineAggreg
|
|||
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");
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -22,14 +22,14 @@ package org.elasticsearch.search.aggregations.pipeline.derivative;
|
|||
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 class DerivativePipelineAggregator extends PipelineAggregator {
|
|||
|
||||
@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<Bucket> 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<InternalAggregation> 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<PipelineAggregator>(), 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 class DerivativePipelineAggregator extends PipelineAggregator {
|
|||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,7 +30,8 @@ import org.elasticsearch.index.query.QueryParseContext;
|
|||
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 class MovAvgPipelineAggregationBuilder extends AbstractPipelineAggregatio
|
|||
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");
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -23,16 +23,16 @@ import org.elasticsearch.common.collect.EvictingQueue;
|
|||
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 class MovAvgPipelineAggregator extends PipelineAggregator {
|
|||
|
||||
@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<Bucket> newBuckets = new ArrayList<>();
|
||||
EvictingQueue<Double> values = new EvictingQueue<>(this.window);
|
||||
|
||||
long lastValidKey = 0;
|
||||
Number lastValidKey = 0;
|
||||
int lastValidPosition = 0;
|
||||
int counter = 0;
|
||||
|
||||
|
@ -110,12 +110,12 @@ public class MovAvgPipelineAggregator extends PipelineAggregator {
|
|||
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 class MovAvgPipelineAggregator extends PipelineAggregator {
|
|||
return (InternalAggregation) p;
|
||||
}).collect(Collectors.toList());
|
||||
aggs.add(new InternalSimpleValue(name(), movavg, formatter, new ArrayList<PipelineAggregator>(), 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 class MovAvgPipelineAggregator extends PipelineAggregator {
|
|||
}
|
||||
|
||||
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<InternalAggregation> 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 class MovAvgPipelineAggregator extends PipelineAggregator {
|
|||
}).collect(Collectors.toList());
|
||||
aggs.add(new InternalSimpleValue(name(), predictions[i], formatter, new ArrayList<PipelineAggregator>(), 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 class MovAvgPipelineAggregator extends PipelineAggregator {
|
|||
aggs = new ArrayList<>();
|
||||
aggs.add(new InternalSimpleValue(name(), predictions[i], formatter, new ArrayList<PipelineAggregator>(), 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 class MovAvgPipelineAggregator extends PipelineAggregator {
|
|||
}
|
||||
}
|
||||
|
||||
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<Double> 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) {
|
||||
|
|
|
@ -26,8 +26,10 @@ import org.elasticsearch.common.io.stream.StreamOutput;
|
|||
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 class SerialDiffPipelineAggregator extends PipelineAggregator {
|
|||
|
||||
@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<Bucket> newBuckets = new ArrayList<>();
|
||||
EvictingQueue<Double> 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 class SerialDiffPipelineAggregator extends PipelineAggregator {
|
|||
return (InternalAggregation) p;
|
||||
}).collect(Collectors.toList());
|
||||
aggs.add(new InternalSimpleValue(name(), diff, formatter, new ArrayList<PipelineAggregator>(), 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 class SerialDiffPipelineAggregator extends PipelineAggregator {
|
|||
lagWindow.add(thisBucketValue);
|
||||
|
||||
}
|
||||
return factory.create(newBuckets, histo);
|
||||
return factory.createAggregation(newBuckets);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,10 +33,8 @@ public class RoundingTests extends ESTestCase {
|
|||
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 class RoundingTests extends ESTestCase {
|
|||
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 class RoundingTests extends ESTestCase {
|
|||
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 class RoundingTests extends ESTestCase {
|
|||
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));
|
||||
}
|
||||
|
|
|
@ -65,9 +65,6 @@ public class TimeZoneRoundingTests extends ESTestCase {
|
|||
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 class TimeZoneRoundingTests extends ESTestCase {
|
|||
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 class TimeZoneRoundingTests extends ESTestCase {
|
|||
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 class TimeZoneRoundingTests extends ESTestCase {
|
|||
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));
|
||||
}
|
||||
|
||||
|
|
|
@ -124,7 +124,7 @@ public class MissingValueIT extends ESIntegTestCase {
|
|||
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 class MissingValueIT extends ESIntegTestCase {
|
|||
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());
|
||||
}
|
||||
|
||||
|
|
|
@ -841,7 +841,7 @@ public class DateHistogramIT extends ESIntegTestCase {
|
|||
|
||||
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());
|
||||
|
|
|
@ -97,7 +97,7 @@ public class DateHistogramOffsetIT extends ESIntegTestCase {
|
|||
assertThat(response.getHits().getTotalHits(), equalTo(5L));
|
||||
|
||||
Histogram histo = response.getAggregations().get("date_histo");
|
||||
List<? extends Histogram.Bucket> buckets = histo.getBuckets();
|
||||
List<Histogram.Bucket> buckets = histo.getBuckets();
|
||||
assertThat(buckets.size(), equalTo(2));
|
||||
|
||||
checkBucketFor(buckets.get(0), new DateTime(2014, 3, 10, 2, 0, DateTimeZone.UTC), 2L);
|
||||
|
|
|
@ -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<DateHistogramAggregationBuilder> {
|
||||
|
||||
|
|
|
@ -27,7 +27,6 @@ import org.elasticsearch.script.MockScriptPlugin;
|
|||
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 @@ public class HistogramIT extends ESIntegTestCase {
|
|||
Map<String, Function<Map<String, Object>, 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 class HistogramIT extends ESIntegTestCase {
|
|||
}
|
||||
}
|
||||
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 class HistogramIT extends ESIntegTestCase {
|
|||
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 class HistogramIT extends ESIntegTestCase {
|
|||
.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 class HistogramIT extends ESIntegTestCase {
|
|||
.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 class HistogramIT extends ESIntegTestCase {
|
|||
.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<Bucket> 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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<HistogramAggregationBuilder> {
|
||||
|
||||
|
@ -31,9 +29,9 @@ public class HistogramTests extends BaseAggregationTestCase<HistogramAggregation
|
|||
protected HistogramAggregationBuilder createTestAggregatorBuilder() {
|
||||
HistogramAggregationBuilder factory = new HistogramAggregationBuilder("foo");
|
||||
factory.field(INT_FIELD_NAME);
|
||||
factory.interval(randomIntBetween(1, 100000));
|
||||
factory.interval(randomDouble() * 1000);
|
||||
if (randomBoolean()) {
|
||||
factory.extendedBounds(ExtendedBoundsTests.randomExtendedBounds());
|
||||
factory.extendedBounds(randomDouble(), randomDouble());
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
factory.format("###.##");
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.elasticsearch.common.xcontent.XContentParser;
|
|||
import org.elasticsearch.common.xcontent.json.JsonXContent;
|
||||
import org.elasticsearch.search.DocValueFormat;
|
||||
import org.elasticsearch.search.SearchParseException;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBounds;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.joda.time.DateTimeZone;
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.elasticsearch.search.aggregations.pipeline;
|
|||
|
||||
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 class AvgBucketIT extends ESIntegTestCase {
|
|||
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(avgBucket("avg_bucket", "histo>_count")).execute().actionGet();
|
||||
|
||||
assertSearchResponse(response);
|
||||
|
@ -133,7 +132,7 @@ public class AvgBucketIT extends ESIntegTestCase {
|
|||
.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 class AvgBucketIT extends ESIntegTestCase {
|
|||
.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 class AvgBucketIT extends ESIntegTestCase {
|
|||
.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 class AvgBucketIT extends ESIntegTestCase {
|
|||
.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();
|
||||
|
||||
|
|
|
@ -28,8 +28,6 @@ import org.elasticsearch.script.MockScriptPlugin;
|
|||
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 class BucketScriptIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class BucketScriptIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class BucketScriptIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class BucketScriptIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class BucketScriptIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class BucketScriptIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class BucketScriptIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class BucketScriptIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class BucketScriptIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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);
|
||||
|
|
|
@ -29,8 +29,7 @@ import org.elasticsearch.script.Script;
|
|||
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 class BucketSelectorIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class BucketSelectorIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class BucketSelectorIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class BucketSelectorIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class BucketSelectorIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class BucketSelectorIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class BucketSelectorIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class BucketSelectorIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class BucketSelectorIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class BucketSelectorIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class BucketSelectorIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class BucketSelectorIT extends ESIntegTestCase {
|
|||
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 class BucketSelectorIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class BucketSelectorIT extends ESIntegTestCase {
|
|||
|
||||
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 class BucketSelectorIT extends ESIntegTestCase {
|
|||
|
||||
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 class BucketSelectorIT extends ESIntegTestCase {
|
|||
}
|
||||
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();
|
||||
|
|
|
@ -89,7 +89,7 @@ public class CumulativeSumIT extends ESIntegTestCase {
|
|||
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 class CumulativeSumIT extends ESIntegTestCase {
|
|||
.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();
|
||||
|
||||
|
|
|
@ -25,7 +25,6 @@ import org.elasticsearch.index.mapper.core.DateFieldMapper;
|
|||
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 class DateDerivativeIT extends ESIntegTestCase {
|
|||
|
||||
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 class DateDerivativeIT extends ESIntegTestCase {
|
|||
|
||||
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 class DateDerivativeIT extends ESIntegTestCase {
|
|||
|
||||
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 class DateDerivativeIT extends ESIntegTestCase {
|
|||
|
||||
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 class DateDerivativeIT extends ESIntegTestCase {
|
|||
|
||||
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 class DateDerivativeIT extends ESIntegTestCase {
|
|||
|
||||
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 class DateDerivativeIT extends ESIntegTestCase {
|
|||
|
||||
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 class DateDerivativeIT extends ESIntegTestCase {
|
|||
|
||||
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 class DateDerivativeIT extends ESIntegTestCase {
|
|||
|
||||
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();
|
||||
|
|
|
@ -25,10 +25,8 @@ import org.elasticsearch.action.search.SearchPhaseExecutionException;
|
|||
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 class DerivativeIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class DerivativeIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class DerivativeIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class DerivativeIT extends ESIntegTestCase {
|
|||
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 class DerivativeIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class DerivativeIT extends ESIntegTestCase {
|
|||
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 class DerivativeIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class DerivativeIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class DerivativeIT extends ESIntegTestCase {
|
|||
|
||||
assertThat(searchResponse.getHits().getTotalHits(), equalTo(numDocsEmptyIdx));
|
||||
|
||||
InternalHistogram<Bucket> deriv = searchResponse.getAggregations().get("histo");
|
||||
Histogram deriv = searchResponse.getAggregations().get("histo");
|
||||
assertThat(deriv, Matchers.notNullValue());
|
||||
assertThat(deriv.getName(), equalTo("histo"));
|
||||
List<Bucket> 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 class DerivativeIT extends ESIntegTestCase {
|
|||
.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<Bucket> deriv = searchResponse.getAggregations().get("histo");
|
||||
Histogram deriv = searchResponse.getAggregations().get("histo");
|
||||
assertThat(deriv, Matchers.notNullValue());
|
||||
assertThat(deriv.getName(), equalTo("histo"));
|
||||
List<Bucket> 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 class DerivativeIT extends ESIntegTestCase {
|
|||
|
||||
assertThat(searchResponse.getHits().getTotalHits(), equalTo(numDocsEmptyIdx));
|
||||
|
||||
InternalHistogram<Bucket> deriv = searchResponse.getAggregations().get("histo");
|
||||
Histogram deriv = searchResponse.getAggregations().get("histo");
|
||||
assertThat(deriv, Matchers.notNullValue());
|
||||
assertThat(deriv.getName(), equalTo("histo"));
|
||||
List<Bucket> buckets = deriv.getBuckets();
|
||||
|
@ -478,7 +476,7 @@ public class DerivativeIT extends ESIntegTestCase {
|
|||
|
||||
assertThat(searchResponse.getHits().getTotalHits(), equalTo(numDocsEmptyIdx));
|
||||
|
||||
InternalHistogram<Bucket> deriv = searchResponse.getAggregations().get("histo");
|
||||
Histogram deriv = searchResponse.getAggregations().get("histo");
|
||||
assertThat(deriv, Matchers.notNullValue());
|
||||
assertThat(deriv.getName(), equalTo("histo"));
|
||||
List<Bucket> buckets = deriv.getBuckets();
|
||||
|
@ -520,7 +518,7 @@ public class DerivativeIT extends ESIntegTestCase {
|
|||
|
||||
assertThat(searchResponse.getHits().getTotalHits(), equalTo(numDocsEmptyIdx));
|
||||
|
||||
InternalHistogram<Bucket> deriv = searchResponse.getAggregations().get("histo");
|
||||
Histogram deriv = searchResponse.getAggregations().get("histo");
|
||||
assertThat(deriv, Matchers.notNullValue());
|
||||
assertThat(deriv.getName(), equalTo("histo"));
|
||||
List<Bucket> buckets = deriv.getBuckets();
|
||||
|
@ -553,13 +551,13 @@ public class DerivativeIT extends ESIntegTestCase {
|
|||
.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<Bucket> deriv = searchResponse.getAggregations().get("histo");
|
||||
Histogram deriv = searchResponse.getAggregations().get("histo");
|
||||
assertThat(deriv, Matchers.notNullValue());
|
||||
assertThat(deriv.getName(), equalTo("histo"));
|
||||
List<Bucket> buckets = deriv.getBuckets();
|
||||
|
|
|
@ -23,7 +23,6 @@ import org.elasticsearch.ExceptionsHelper;
|
|||
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 class ExtendedStatsBucketIT extends ESIntegTestCase {
|
|||
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 class ExtendedStatsBucketIT extends ESIntegTestCase {
|
|||
.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 class ExtendedStatsBucketIT extends ESIntegTestCase {
|
|||
.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 class ExtendedStatsBucketIT extends ESIntegTestCase {
|
|||
.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 class ExtendedStatsBucketIT extends ESIntegTestCase {
|
|||
.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 class ExtendedStatsBucketIT extends ESIntegTestCase {
|
|||
.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();
|
||||
|
||||
|
|
|
@ -22,7 +22,6 @@ package org.elasticsearch.search.aggregations.pipeline;
|
|||
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 class MaxBucketIT extends ESIntegTestCase {
|
|||
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 class MaxBucketIT extends ESIntegTestCase {
|
|||
.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 class MaxBucketIT extends ESIntegTestCase {
|
|||
.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 class MaxBucketIT extends ESIntegTestCase {
|
|||
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 class MaxBucketIT extends ESIntegTestCase {
|
|||
.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 class MaxBucketIT extends ESIntegTestCase {
|
|||
.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();
|
||||
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.elasticsearch.search.aggregations.pipeline;
|
|||
|
||||
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 class MinBucketIT extends ESIntegTestCase {
|
|||
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 class MinBucketIT extends ESIntegTestCase {
|
|||
.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 class MinBucketIT extends ESIntegTestCase {
|
|||
.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 class MinBucketIT extends ESIntegTestCase {
|
|||
.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 class MinBucketIT extends ESIntegTestCase {
|
|||
.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();
|
||||
|
||||
|
|
|
@ -23,7 +23,6 @@ import org.elasticsearch.ExceptionsHelper;
|
|||
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 class PercentilesBucketIT extends ESIntegTestCase {
|
|||
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 class PercentilesBucketIT extends ESIntegTestCase {
|
|||
.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 class PercentilesBucketIT extends ESIntegTestCase {
|
|||
.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 class PercentilesBucketIT extends ESIntegTestCase {
|
|||
.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 class PercentilesBucketIT extends ESIntegTestCase {
|
|||
.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 class PercentilesBucketIT extends ESIntegTestCase {
|
|||
.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 class PercentilesBucketIT extends ESIntegTestCase {
|
|||
.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]")
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.elasticsearch.search.aggregations.pipeline;
|
|||
|
||||
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 class StatsBucketIT extends ESIntegTestCase {
|
|||
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 class StatsBucketIT extends ESIntegTestCase {
|
|||
.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 class StatsBucketIT extends ESIntegTestCase {
|
|||
.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 class StatsBucketIT extends ESIntegTestCase {
|
|||
.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 class StatsBucketIT extends ESIntegTestCase {
|
|||
.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();
|
||||
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.elasticsearch.search.aggregations.pipeline;
|
|||
|
||||
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 class SumBucketIT extends ESIntegTestCase {
|
|||
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 class SumBucketIT extends ESIntegTestCase {
|
|||
.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 class SumBucketIT extends ESIntegTestCase {
|
|||
.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 class SumBucketIT extends ESIntegTestCase {
|
|||
.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 class SumBucketIT extends ESIntegTestCase {
|
|||
.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();
|
||||
|
||||
|
|
|
@ -23,10 +23,8 @@ import org.elasticsearch.action.index.IndexRequestBuilder;
|
|||
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 class MovAvgIT extends ESIntegTestCase {
|
|||
.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 class MovAvgIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class MovAvgIT extends ESIntegTestCase {
|
|||
.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 class MovAvgIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class MovAvgIT extends ESIntegTestCase {
|
|||
.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 class MovAvgIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class MovAvgIT extends ESIntegTestCase {
|
|||
.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 class MovAvgIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class MovAvgIT extends ESIntegTestCase {
|
|||
.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 class MovAvgIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class MovAvgIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class MovAvgIT extends ESIntegTestCase {
|
|||
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 class MovAvgIT extends ESIntegTestCase {
|
|||
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 class MovAvgIT extends ESIntegTestCase {
|
|||
.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 class MovAvgIT extends ESIntegTestCase {
|
|||
.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 class MovAvgIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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 class MovAvgIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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();
|
||||
|
@ -808,7 +806,7 @@ public class MovAvgIT extends ESIntegTestCase {
|
|||
.prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(
|
||||
histogram("histo").field(INTERVAL_FIELD).interval(interval)
|
||||
.extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1))))
|
||||
.extendedBounds(0L, (long) (interval * (numBuckets - 1)))
|
||||
.subAggregation(randomMetric("the_metric", VALUE_FIELD))
|
||||
.subAggregation(movingAvg("movavg_counts", "the_metric")
|
||||
.window(windowSize)
|
||||
|
@ -829,7 +827,7 @@ public class MovAvgIT extends ESIntegTestCase {
|
|||
.prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(
|
||||
histogram("histo").field(INTERVAL_FIELD).interval(interval)
|
||||
.extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1))))
|
||||
.extendedBounds(0L, (long) (interval * (numBuckets - 1)))
|
||||
.subAggregation(randomMetric("the_metric", VALUE_FIELD))
|
||||
.subAggregation(movingAvg("movavg_counts", "the_metric")
|
||||
.window(windowSize)
|
||||
|
@ -850,7 +848,7 @@ public class MovAvgIT extends ESIntegTestCase {
|
|||
.prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(
|
||||
histogram("histo").field(INTERVAL_FIELD).interval(interval)
|
||||
.extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1))))
|
||||
.extendedBounds(0L, (long) (interval * (numBuckets - 1)))
|
||||
.subAggregation(metric)
|
||||
.subAggregation(movingAvg("movavg_counts", "_count")
|
||||
.window(10)
|
||||
|
@ -889,7 +887,7 @@ public class MovAvgIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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();
|
||||
|
@ -897,7 +895,7 @@ public class MovAvgIT extends ESIntegTestCase {
|
|||
|
||||
Bucket bucket = buckets.get(0);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat((long) bucket.getKey(), equalTo((long) 0));
|
||||
assertThat(bucket.getKey(), equalTo(0d));
|
||||
assertThat(bucket.getDocCount(), equalTo(1L));
|
||||
|
||||
Avg avgAgg = bucket.getAggregations().get("avg");
|
||||
|
@ -916,7 +914,7 @@ public class MovAvgIT extends ESIntegTestCase {
|
|||
// Second bucket
|
||||
bucket = buckets.get(1);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat((long) bucket.getKey(), equalTo(1L));
|
||||
assertThat(bucket.getKey(), equalTo(1d));
|
||||
assertThat(bucket.getDocCount(), equalTo(1L));
|
||||
|
||||
avgAgg = bucket.getAggregations().get("avg");
|
||||
|
@ -937,7 +935,7 @@ public class MovAvgIT extends ESIntegTestCase {
|
|||
for (int i = 2; i < 12; i++) {
|
||||
bucket = buckets.get(i);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat((long) bucket.getKey(), equalTo((long) i));
|
||||
assertThat(bucket.getKey(), equalTo((double) i));
|
||||
assertThat(bucket.getDocCount(), equalTo(1L));
|
||||
|
||||
avgAgg = bucket.getAggregations().get("avg");
|
||||
|
@ -961,7 +959,7 @@ public class MovAvgIT extends ESIntegTestCase {
|
|||
for (int i = 12; i < 24; i++) {
|
||||
bucket = buckets.get(i);
|
||||
assertThat(bucket, notNullValue());
|
||||
assertThat((long) bucket.getKey(), equalTo((long) i));
|
||||
assertThat(bucket.getKey(), equalTo((double) i));
|
||||
assertThat(bucket.getDocCount(), equalTo(0L));
|
||||
|
||||
avgAgg = bucket.getAggregations().get("avg");
|
||||
|
@ -986,7 +984,7 @@ public class MovAvgIT extends ESIntegTestCase {
|
|||
.prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(
|
||||
histogram("histo").field(INTERVAL_FIELD).interval(interval)
|
||||
.extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1))))
|
||||
.extendedBounds(0L, (long) (interval * (numBuckets - 1)))
|
||||
.subAggregation(metric)
|
||||
.subAggregation(movingAvg("movavg_counts", "_count")
|
||||
.window(10)
|
||||
|
@ -1004,7 +1002,7 @@ public class MovAvgIT extends ESIntegTestCase {
|
|||
.prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(
|
||||
histogram("histo").field(INTERVAL_FIELD).interval(interval)
|
||||
.extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1))))
|
||||
.extendedBounds(0L, (long) (interval * (numBuckets - 1)))
|
||||
.subAggregation(metric)
|
||||
.subAggregation(movingAvg("movavg_counts", "_count")
|
||||
.window(windowSize)
|
||||
|
@ -1022,7 +1020,7 @@ public class MovAvgIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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();
|
||||
|
@ -1090,7 +1088,7 @@ public class MovAvgIT extends ESIntegTestCase {
|
|||
.prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(
|
||||
histogram("histo").field(INTERVAL_FIELD).interval(interval)
|
||||
.extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1))))
|
||||
.extendedBounds(0L, (long) (interval * (numBuckets - 1)))
|
||||
.subAggregation(metric)
|
||||
.subAggregation(movingAvg("movavg_counts", "_count")
|
||||
.window(numBuckets)
|
||||
|
@ -1106,7 +1104,7 @@ public class MovAvgIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<Bucket> 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();
|
||||
|
@ -1144,7 +1142,7 @@ public class MovAvgIT extends ESIntegTestCase {
|
|||
.prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(
|
||||
histogram("histo").field(INTERVAL_FIELD).interval(interval)
|
||||
.extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1))))
|
||||
.extendedBounds(0L, (long) (interval * (numBuckets - 1)))
|
||||
.subAggregation(metric)
|
||||
.subAggregation(movingAvg("movavg_counts", "_count")
|
||||
.window(numBuckets)
|
||||
|
@ -1162,7 +1160,7 @@ public class MovAvgIT extends ESIntegTestCase {
|
|||
.prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(
|
||||
histogram("histo").field(INTERVAL_FIELD).interval(interval)
|
||||
.extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1))))
|
||||
.extendedBounds(0L, (long) (interval * (numBuckets - 1)))
|
||||
.subAggregation(metric)
|
||||
.subAggregation(movingAvg("movavg_counts", "_count")
|
||||
.window(numBuckets)
|
||||
|
@ -1192,7 +1190,7 @@ public class MovAvgIT extends ESIntegTestCase {
|
|||
.prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(
|
||||
histogram("histo").field(INTERVAL_FIELD).interval(interval)
|
||||
.extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1))))
|
||||
.extendedBounds(0L, (long) (interval * (numBuckets - 1)))
|
||||
.subAggregation(metric)
|
||||
.subAggregation(movingAvg("movavg_counts", "_count")
|
||||
.window(numBuckets)
|
||||
|
|
|
@ -22,9 +22,8 @@ package org.elasticsearch.search.aggregations.pipeline.serialdiff;
|
|||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.common.collect.EvictingQueue;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBounds;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket;
|
||||
import org.elasticsearch.search.aggregations.pipeline.BucketHelpers;
|
||||
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregationHelperTests;
|
||||
import org.elasticsearch.search.aggregations.pipeline.SimpleValue;
|
||||
|
@ -232,7 +231,7 @@ public class SerialDiffIT extends ESIntegTestCase {
|
|||
.prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(
|
||||
histogram("histo").field(INTERVAL_FIELD).interval(interval)
|
||||
.extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1))))
|
||||
.extendedBounds(0L, (long) (interval * (numBuckets - 1)))
|
||||
.subAggregation(metric)
|
||||
.subAggregation(diff("diff_counts", "_count")
|
||||
.lag(lag)
|
||||
|
@ -244,10 +243,10 @@ public class SerialDiffIT extends ESIntegTestCase {
|
|||
|
||||
assertSearchResponse(response);
|
||||
|
||||
InternalHistogram<InternalHistogram.Bucket> histo = response.getAggregations().get("histo");
|
||||
Histogram histo = response.getAggregations().get("histo");
|
||||
assertThat(histo, notNullValue());
|
||||
assertThat(histo.getName(), equalTo("histo"));
|
||||
List<? extends InternalHistogram.Bucket> buckets = histo.getBuckets();
|
||||
List<? extends Bucket> buckets = histo.getBuckets();
|
||||
assertThat("Size of buckets array is not correct.", buckets.size(), equalTo(mockHisto.size()));
|
||||
|
||||
List<Double> expectedCounts = testValues.get(MetricTarget.COUNT.toString());
|
||||
|
@ -279,7 +278,7 @@ public class SerialDiffIT extends ESIntegTestCase {
|
|||
.prepareSearch("idx").setTypes("type")
|
||||
.addAggregation(
|
||||
histogram("histo").field(INTERVAL_FIELD).interval(interval)
|
||||
.extendedBounds(new ExtendedBounds(0L, (long) (interval * (numBuckets - 1))))
|
||||
.extendedBounds(0L, (long) (interval * (numBuckets - 1)))
|
||||
.subAggregation(metric)
|
||||
.subAggregation(diff("diff_counts", "_count")
|
||||
.lag(-1)
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.elasticsearch.search.profile.aggregation;
|
|||
|
||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregator;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.GlobalOrdinalsStringTermsAggregator;
|
||||
import org.elasticsearch.search.aggregations.metrics.avg.AvgAggregator;
|
||||
import org.elasticsearch.search.aggregations.metrics.max.MaxAggregator;
|
||||
|
@ -99,7 +98,8 @@ public class AggregationProfilerIT extends ESIntegTestCase {
|
|||
assertThat(aggProfileResultsList.size(), equalTo(1));
|
||||
ProfileResult histoAggResult = aggProfileResultsList.get(0);
|
||||
assertThat(histoAggResult, notNullValue());
|
||||
assertThat(histoAggResult.getQueryName(), equalTo(HistogramAggregator.class.getName()));
|
||||
assertThat(histoAggResult.getQueryName(),
|
||||
equalTo("org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregator"));
|
||||
assertThat(histoAggResult.getLuceneDescription(), equalTo("histo"));
|
||||
assertThat(histoAggResult.getProfiledChildren().size(), equalTo(0));
|
||||
assertThat(histoAggResult.getTime(), greaterThan(0L));
|
||||
|
@ -135,7 +135,8 @@ public class AggregationProfilerIT extends ESIntegTestCase {
|
|||
assertThat(aggProfileResultsList.size(), equalTo(1));
|
||||
ProfileResult histoAggResult = aggProfileResultsList.get(0);
|
||||
assertThat(histoAggResult, notNullValue());
|
||||
assertThat(histoAggResult.getQueryName(), equalTo(HistogramAggregator.class.getName()));
|
||||
assertThat(histoAggResult.getQueryName(),
|
||||
equalTo("org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregator"));
|
||||
assertThat(histoAggResult.getLuceneDescription(), equalTo("histo"));
|
||||
assertThat(histoAggResult.getTime(), greaterThan(0L));
|
||||
Map<String, Long> histoBreakdown = histoAggResult.getTimeBreakdown();
|
||||
|
@ -212,7 +213,8 @@ public class AggregationProfilerIT extends ESIntegTestCase {
|
|||
assertThat(aggProfileResultsList.size(), equalTo(1));
|
||||
ProfileResult histoAggResult = aggProfileResultsList.get(0);
|
||||
assertThat(histoAggResult, notNullValue());
|
||||
assertThat(histoAggResult.getQueryName(), equalTo(HistogramAggregator.class.getName()));
|
||||
assertThat(histoAggResult.getQueryName(),
|
||||
equalTo("org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregator"));
|
||||
assertThat(histoAggResult.getLuceneDescription(), equalTo("histo"));
|
||||
assertThat(histoAggResult.getTime(), greaterThan(0L));
|
||||
Map<String, Long> histoBreakdown = histoAggResult.getTimeBreakdown();
|
||||
|
|
|
@ -11,19 +11,10 @@ To make this more formal, here is the rounding function that is used:
|
|||
|
||||
[source,java]
|
||||
--------------------------------------------------
|
||||
rem = value % interval
|
||||
if (rem < 0) {
|
||||
rem += interval
|
||||
}
|
||||
bucket_key = value - rem
|
||||
bucket_key = Math.floor((value - offset) / interval) * interval + offset
|
||||
--------------------------------------------------
|
||||
|
||||
From the rounding function above it can be seen that the intervals themselves **must** be integers.
|
||||
|
||||
WARNING: Currently, values are cast to integers before being bucketed, which
|
||||
might cause negative floating-point values to fall into the wrong bucket. For
|
||||
instance, `-4.5` with an interval of `2` would be cast to `-4`, and so would
|
||||
end up in the `-4 <= val < -2` bucket instead of the `-6 <= val < -4` bucket.
|
||||
The `interval` must be a positive decimal, while the `offset` must be a decimal in `[0, interval[`.
|
||||
|
||||
The following snippet "buckets" the products based on their `price` by interval of `50`:
|
||||
|
||||
|
|
|
@ -19,23 +19,14 @@
|
|||
|
||||
package org.elasticsearch.script.expression;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.expressions.Expression;
|
||||
import org.apache.lucene.expressions.js.JavascriptCompiler;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.search.SearchPhaseExecutionException;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.SearchType;
|
||||
import org.elasticsearch.action.update.UpdateRequestBuilder;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.common.lucene.search.function.CombineFunction;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||
import org.elasticsearch.index.query.QueryBuilders;
|
||||
|
@ -43,22 +34,25 @@ import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilder;
|
|||
import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.script.CompiledScript;
|
||||
import org.elasticsearch.script.Script;
|
||||
import org.elasticsearch.script.GeneralScriptException;
|
||||
import org.elasticsearch.script.Script;
|
||||
import org.elasticsearch.script.ScriptService.ScriptType;
|
||||
import org.elasticsearch.search.SearchHits;
|
||||
import org.elasticsearch.search.aggregations.AggregationBuilders;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram.Bucket;
|
||||
import org.elasticsearch.search.aggregations.metrics.stats.Stats;
|
||||
import org.elasticsearch.search.aggregations.pipeline.SimpleValue;
|
||||
import org.elasticsearch.search.sort.SortBuilders;
|
||||
import org.elasticsearch.search.sort.SortOrder;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
import org.elasticsearch.test.VersionUtils;
|
||||
import org.elasticsearch.test.hamcrest.ElasticsearchAssertions;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE;
|
||||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
|
||||
|
@ -600,10 +594,10 @@ public class MoreExpressionTests extends ESIntegTestCase {
|
|||
"twoSum", "threeSum", "fourSum")))
|
||||
.execute().actionGet();
|
||||
|
||||
InternalHistogram<Bucket> histogram = response.getAggregations().get("histogram");
|
||||
Histogram histogram = response.getAggregations().get("histogram");
|
||||
assertThat(histogram, notNullValue());
|
||||
assertThat(histogram.getName(), equalTo("histogram"));
|
||||
List<Bucket> buckets = histogram.getBuckets();
|
||||
List<Histogram.Bucket> buckets = histogram.getBuckets();
|
||||
|
||||
for (int bucketCount = 0; bucketCount < buckets.size(); ++bucketCount) {
|
||||
Histogram.Bucket bucket = buckets.get(bucketCount);
|
||||
|
|
Loading…
Reference in New Issue