cleanup of aggregations api

- add javadocs
- remove Iterable from all multi-bucket aggregations
- all single-bucket aggregations should have getDocCount() and getAggregations()
- all multi-bucket aggregations should have getBuckets() that returns Collection
- every multi-bucket aggregation should have these methods:
 - getBuckets() : Collection
 - getBucketByKey(String) : Bucket
 - getBucketByKey(Number) : Bucket (only for numeric buckets)
 - getBucketByKey(DateTime) : Bucket (only for date buckets)
 - getBucketByKey(GeoPoint) : Bucket (only for geohash_grid)
- every bucket in all multi-bucket aggregations should have these methods:
 - getKey() : String
 - getKeyAsText() : Text
 - getKeyAsNumber() : Number (if the key can be numeric value, eg. range & histograms)
 - getKeyAsGeoPoint() : GeoPoint (in case of the geohash_grid agg)

 Closes 
This commit is contained in:
uboness 2014-01-25 04:55:00 +01:00
parent b61ca9932a
commit fc6bc4c477
72 changed files with 2515 additions and 2460 deletions
src
main/java/org/elasticsearch/search/aggregations/bucket
test/java/org/elasticsearch

@ -1,100 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket;
import org.elasticsearch.ElasticsearchIllegalArgumentException;
import org.elasticsearch.search.aggregations.Aggregations;
import org.elasticsearch.search.aggregations.metrics.MetricsAggregation;
/**
*
*/
public interface Bucket {
/**
* @return The number of documents that fall within this bucket
*/
long getDocCount();
Aggregations getAggregations();
static class SubAggregationComparator<B extends Bucket> implements java.util.Comparator<B> {
private final String aggName;
private final String valueName;
private final boolean asc;
public SubAggregationComparator(String expression, boolean asc) {
this.asc = asc;
int i = expression.indexOf('.');
if (i < 0) {
this.aggName = expression;
this.valueName = null;
} else {
this.aggName = expression.substring(0, i);
this.valueName = expression.substring(i+1);
}
}
public SubAggregationComparator(String aggName, String valueName, boolean asc) {
this.aggName = aggName;
this.valueName = valueName;
this.asc = asc;
}
public boolean asc() {
return asc;
}
public String aggName() {
return aggName;
}
public String valueName() {
return valueName;
}
@Override
public int compare(B b1, B b2) {
double v1 = value(b1);
double v2 = value(b2);
return asc ? Double.compare(v1, v2) : Double.compare(v2, v1);
}
private double value(B bucket) {
MetricsAggregation aggregation = bucket.getAggregations().get(aggName);
if (aggregation == null) {
throw new ElasticsearchIllegalArgumentException("Unknown aggregation named [" + aggName + "]");
}
if (aggregation instanceof MetricsAggregation.SingleValue) {
//TODO should we throw an exception if the value name is specified?
return ((MetricsAggregation.SingleValue) aggregation).value();
}
if (aggregation instanceof MetricsAggregation.MultiValue) {
if (valueName == null) {
throw new ElasticsearchIllegalArgumentException("Cannot sort on multi valued aggregation [" + aggName + "]. A value name is required");
}
return ((MetricsAggregation.MultiValue) aggregation).value(valueName);
}
throw new ElasticsearchIllegalArgumentException("A mal attempt to sort terms by aggregation [" + aggregation.getName() +
"]. Terms can only be ordered by either standard order or direct calc aggregators of the terms");
}
}
}

@ -0,0 +1,107 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregations;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
/**
* A base class for all the single bucket aggregations.
*/
public abstract class InternalSingleBucketAggregation extends InternalAggregation implements SingleBucketAggregation {
protected long docCount;
protected InternalAggregations aggregations;
protected InternalSingleBucketAggregation() {} // for serialization
/**
* Creates a single bucket aggregation.
*
* @param name The aggregation name.
* @param docCount The document count in the single bucket.
* @param aggregations The already built sub-aggregations that are associated with the bucket.
*/
protected InternalSingleBucketAggregation(String name, long docCount, InternalAggregations aggregations) {
super(name);
this.docCount = docCount;
this.aggregations = aggregations;
}
@Override
public long getDocCount() {
return docCount;
}
@Override
public InternalAggregations getAggregations() {
return aggregations;
}
@Override
public InternalAggregation reduce(ReduceContext reduceContext) {
List<InternalAggregation> aggregations = reduceContext.aggregations();
if (aggregations.size() == 1) {
InternalSingleBucketAggregation reduced = ((InternalSingleBucketAggregation) aggregations.get(0));
reduced.aggregations.reduce(reduceContext.cacheRecycler());
return reduced;
}
InternalSingleBucketAggregation reduced = null;
List<InternalAggregations> subAggregationsList = new ArrayList<InternalAggregations>(aggregations.size());
for (InternalAggregation aggregation : aggregations) {
if (reduced == null) {
reduced = (InternalSingleBucketAggregation) aggregation;
} else {
this.docCount += ((InternalSingleBucketAggregation) aggregation).docCount;
}
subAggregationsList.add(((InternalSingleBucketAggregation) aggregation).aggregations);
}
reduced.aggregations = InternalAggregations.reduce(subAggregationsList, reduceContext.cacheRecycler());
return reduced;
}
@Override
public void readFrom(StreamInput in) throws IOException {
name = in.readString();
docCount = in.readVLong();
aggregations = InternalAggregations.readAggregations(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
out.writeVLong(docCount);
aggregations.writeTo(out);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(name);
builder.field(CommonFields.DOC_COUNT, docCount);
aggregations.toXContentInternal(builder, params);
return builder.endObject();
}
}

@ -0,0 +1,139 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket;
import org.elasticsearch.ElasticsearchIllegalArgumentException;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.search.aggregations.Aggregation;
import org.elasticsearch.search.aggregations.Aggregations;
import org.elasticsearch.search.aggregations.metrics.MetricsAggregation;
import java.util.Collection;
/**
* An aggregation that returns multiple buckets
*/
public interface MultiBucketsAggregation extends Aggregation {
/**
* A bucket represents a criteria to which all documents that fall in it adhere to. It is also uniquely identified
* by a key, and can potentially hold sub-aggregations computed over all documents in it.
*/
public interface Bucket {
/**
* @return The key associated with the bucket as a string
*/
String getKey();
/**
* @return The key associated with the bucket as text (ideal for further streaming this instance)
*/
Text getKeyAsText();
/**
* @return The number of documents that fall within this bucket
*/
long getDocCount();
/**
* @return The sub-aggregations of this bucket
*/
Aggregations getAggregations();
static class SubAggregationComparator<B extends Bucket> implements java.util.Comparator<B> {
private final String aggName;
private final String valueName;
private final boolean asc;
public SubAggregationComparator(String expression, boolean asc) {
this.asc = asc;
int i = expression.indexOf('.');
if (i < 0) {
this.aggName = expression;
this.valueName = null;
} else {
this.aggName = expression.substring(0, i);
this.valueName = expression.substring(i+1);
}
}
public SubAggregationComparator(String aggName, String valueName, boolean asc) {
this.aggName = aggName;
this.valueName = valueName;
this.asc = asc;
}
public boolean asc() {
return asc;
}
public String aggName() {
return aggName;
}
public String valueName() {
return valueName;
}
@Override
public int compare(B b1, B b2) {
double v1 = value(b1);
double v2 = value(b2);
return asc ? Double.compare(v1, v2) : Double.compare(v2, v1);
}
private double value(B bucket) {
MetricsAggregation aggregation = bucket.getAggregations().get(aggName);
if (aggregation == null) {
throw new ElasticsearchIllegalArgumentException("Unknown aggregation named [" + aggName + "]");
}
if (aggregation instanceof MetricsAggregation.SingleValue) {
//TODO should we throw an exception if the value name is specified?
return ((MetricsAggregation.SingleValue) aggregation).value();
}
if (aggregation instanceof MetricsAggregation.MultiValue) {
if (valueName == null) {
throw new ElasticsearchIllegalArgumentException("Cannot sort on multi valued aggregation [" + aggName + "]. A value name is required");
}
return ((MetricsAggregation.MultiValue) aggregation).value(valueName);
}
throw new ElasticsearchIllegalArgumentException("A mal attempt to sort terms by aggregation [" + aggregation.getName() +
"]. Terms can only be ordered by either standard order or direct calc aggregators of the terms");
}
}
}
/**
* @return The buckets of this aggregation.
*/
Collection<? extends Bucket> getBuckets();
/**
* The bucket that is associated with the given key.
*
* @param key The key of the requested bucket.
* @return The bucket
*/
<B extends Bucket> B getBucketByKey(String key);
}

@ -16,91 +16,24 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket;
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.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregations;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.elasticsearch.search.aggregations.Aggregation;
import org.elasticsearch.search.aggregations.Aggregations;
/**
* A base class for all the single bucket aggregations.
* A single bucket aggregation
*/
@SuppressWarnings("unchecked")
public abstract class SingleBucketAggregation<B extends SingleBucketAggregation<B>> extends InternalAggregation {
protected long docCount;
protected InternalAggregations aggregations;
protected SingleBucketAggregation() {} // for serialization
public interface SingleBucketAggregation extends Aggregation {
/**
* Creates a single bucket aggregation.
*
* @param name The aggregation name.
* @param docCount The document count in the single bucket.
* @param aggregations The already built sub-aggregations that are associated with the bucket.
* @return The number of documents in this bucket
*/
protected SingleBucketAggregation(String name, long docCount, InternalAggregations aggregations) {
super(name);
this.docCount = docCount;
this.aggregations = aggregations;
}
long getDocCount();
public long getDocCount() {
return docCount;
}
public InternalAggregations getAggregations() {
return aggregations;
}
@Override
public InternalAggregation reduce(ReduceContext reduceContext) {
List<InternalAggregation> aggregations = reduceContext.aggregations();
if (aggregations.size() == 1) {
B reduced = ((B) aggregations.get(0));
reduced.aggregations.reduce(reduceContext.cacheRecycler());
return reduced;
}
B reduced = null;
List<InternalAggregations> subAggregationsList = new ArrayList<InternalAggregations>(aggregations.size());
for (InternalAggregation aggregation : aggregations) {
if (reduced == null) {
reduced = (B) aggregation;
} else {
this.docCount += ((B) aggregation).docCount;
}
subAggregationsList.add(((B) aggregation).aggregations);
}
reduced.aggregations = InternalAggregations.reduce(subAggregationsList, reduceContext.cacheRecycler());
return reduced;
}
@Override
public void readFrom(StreamInput in) throws IOException {
name = in.readString();
docCount = in.readVLong();
aggregations = InternalAggregations.readAggregations(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
out.writeVLong(docCount);
aggregations.writeTo(out);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(name);
builder.field(CommonFields.DOC_COUNT, docCount);
aggregations.toXContentInternal(builder, params);
return builder.endObject();
}
/**
* @return The sub-aggregations of this bucket
*/
Aggregations getAggregations();
}

@ -19,8 +19,8 @@
package org.elasticsearch.search.aggregations.bucket;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.support.AggregationContext;
/**
* A bucket aggregator that doesn't create new buckets.

@ -18,16 +18,10 @@
*/
package org.elasticsearch.search.aggregations.bucket.filter;
import org.elasticsearch.search.aggregations.Aggregation;
import org.elasticsearch.search.aggregations.Aggregations;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregation;
/**
*
* A {@code filter} aggregation. Defines a single bucket that holds all documents that match a specific filter.
*/
public interface Filter extends Aggregation {
long getDocCount();
Aggregations getAggregations();
public interface Filter extends SingleBucketAggregation {
}

@ -21,14 +21,14 @@ package org.elasticsearch.search.aggregations.bucket.filter;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.search.aggregations.AggregationStreams;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregation;
import org.elasticsearch.search.aggregations.bucket.InternalSingleBucketAggregation;
import java.io.IOException;
/**
*
*/
public class InternalFilter extends SingleBucketAggregation<InternalFilter> implements Filter {
public class InternalFilter extends InternalSingleBucketAggregation implements Filter {
public final static Type TYPE = new Type("filter");

@ -20,22 +20,44 @@ package org.elasticsearch.search.aggregations.bucket.geogrid;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.search.aggregations.Aggregation;
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
import java.util.Collection;
/**
* Represents the results of a GeoHashGrid aggregation
* A {@code geohash_grid} aggregation. Defines multiple buckets, each representing a cell in a geo-grid of a specific
* precision.
*/
public interface GeoHashGrid extends Iterable<GeoHashGrid.Bucket>, Aggregation {
public interface GeoHashGrid extends MultiBucketsAggregation {
public static interface Bucket extends org.elasticsearch.search.aggregations.bucket.Bucket {
/**
* A bucket that is associated with a {@code geohash_grid} cell. The key of the bucket is the {@cod geohash} of the cell
*/
public static interface Bucket extends MultiBucketsAggregation.Bucket {
String getGeoHash();
/**
* @return The geohash of the cell as a geo point
*/
GeoPoint getKeyAsGeoPoint();
GeoPoint getGeoPoint();
long getInternalKey();
/**
* @return A numeric representation of the geohash of the cell
*/
Number getKeyAsNumber();
}
int getNumberOfBuckets();
/**
* @return The buckets of this aggregation (each bucket representing a geohash grid cell)
*/
@Override
Collection<Bucket> getBuckets();
@Override
Bucket getBucketByKey(String key);
Bucket getBucketByKey(Number key);
Bucket getBucketByKey(GeoPoint key);
}

@ -35,10 +35,9 @@ import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
/**
* Aggregates Geo information into cells determined by geohashes of a given precision.
* WARNING - for high-precision geohashes it may prove necessary to use a {@link GeoBoundingBoxFilterBuilder}
* aggregation to focus in on a smaller area to avoid generating too many buckets and using too much RAM
*
* Aggregates Geo information into cells determined by geohashes of a given precision.
* WARNING - for high-precision geohashes it may prove necessary to use a {@link GeoBoundingBoxFilterBuilder}
* aggregation to focus in on a smaller area to avoid generating too many buckets and using too much RAM
*/
public class GeoHashGridParser implements Aggregator.Parser {
@ -46,20 +45,18 @@ public class GeoHashGridParser implements Aggregator.Parser {
public String type() {
return InternalGeoHashGrid.TYPE.name();
}
public static final int DEFAULT_PRECISION=5;
public static final int DEFAULT_MAX_NUM_CELLS=10000;
public static final int DEFAULT_PRECISION = 5;
public static final int DEFAULT_MAX_NUM_CELLS = 10000;
@Override
public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException {
String field = null;
int precision=DEFAULT_PRECISION;
int requiredSize=DEFAULT_MAX_NUM_CELLS;
int shardSize=0;
int precision = DEFAULT_PRECISION;
int requiredSize = DEFAULT_MAX_NUM_CELLS;
int shardSize = 0;
XContentParser.Token token;
String currentFieldName = null;
@ -69,44 +66,39 @@ public class GeoHashGridParser implements Aggregator.Parser {
} else if (token == XContentParser.Token.VALUE_STRING) {
if ("field".equals(currentFieldName)) {
field = parser.text();
}
}
} else if (token == XContentParser.Token.VALUE_NUMBER) {
if ("precision".equals(currentFieldName)) {
precision = parser.intValue();
}else if ("size".equals(currentFieldName)) {
} else if ("size".equals(currentFieldName)) {
requiredSize = parser.intValue();
}else if ("shard_size".equals(currentFieldName)) {
} else if ("shard_size".equals(currentFieldName) || "shardSize".equals(currentFieldName)) {
shardSize = parser.intValue();
}else if ("shardSize".equals(currentFieldName)) {
shardSize = parser.intValue();
}
}
}
}
}
if(shardSize==0)
{
if (shardSize == 0) {
//Use default heuristic to avoid any wrong-ranking caused by distributed counting
shardSize=BucketUtils.suggestShardSideQueueSize(requiredSize,context.numberOfShards());
shardSize = BucketUtils.suggestShardSideQueueSize(requiredSize, context.numberOfShards());
}
ValuesSourceConfig<GeoPointValuesSource> config = new ValuesSourceConfig<GeoPointValuesSource>(GeoPointValuesSource.class);
if (field == null) {
return new GeoGridFactory(aggregationName, config,precision,requiredSize,shardSize);
return new GeoGridFactory(aggregationName, config, precision, requiredSize, shardSize);
}
FieldMapper<?> mapper = context.smartNameFieldMapper(field);
if (mapper == null) {
config.unmapped(true);
return new GeoGridFactory(aggregationName, config,precision,requiredSize,shardSize);
return new GeoGridFactory(aggregationName, config, precision, requiredSize, shardSize);
}
IndexFieldData<?> indexFieldData = context.fieldData().getForField(mapper);
config.fieldContext(new FieldContext(field, indexFieldData));
return new GeoGridFactory(aggregationName, config,precision,requiredSize,shardSize);
return new GeoGridFactory(aggregationName, config, precision, requiredSize, shardSize);
}
private static class GeoGridFactory extends ValueSourceAggregatorFactory<GeoPointValuesSource> {
@ -115,28 +107,28 @@ public class GeoHashGridParser implements Aggregator.Parser {
private int shardSize;
public GeoGridFactory(String name, ValuesSourceConfig<GeoPointValuesSource> valueSourceConfig,
int precision,int requiredSize,int shardSize) {
int precision, int requiredSize, int shardSize) {
super(name, InternalGeoHashGrid.TYPE.name(), valueSourceConfig);
this.precision=precision;
this.requiredSize=requiredSize;
this.shardSize=shardSize;
this.precision = precision;
this.requiredSize = requiredSize;
this.shardSize = shardSize;
}
@Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent) {
return new GeoHashGridAggregator.Unmapped(name, requiredSize, aggregationContext, parent);
return new GeoHashGridAggregator.Unmapped(name, requiredSize, aggregationContext, parent);
}
@Override
protected Aggregator create(final GeoPointValuesSource valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent) {
final CellValues cellIdValues = new CellValues(valuesSource,precision);
final CellValues cellIdValues = new CellValues(valuesSource, precision);
FieldDataSource.Numeric cellIdSource = new CellIdSource(cellIdValues, valuesSource.metaData());
if (cellIdSource.metaData().multiValued()) {
// we need to wrap to ensure uniqueness
cellIdSource = new FieldDataSource.Numeric.SortedAndUnique(cellIdSource);
}
final NumericValuesSource geohashIdSource = new NumericValuesSource(cellIdSource,null,null);
return new GeoHashGridAggregator(name, factories, geohashIdSource, requiredSize,
final NumericValuesSource geohashIdSource = new NumericValuesSource(cellIdSource, null, null);
return new GeoHashGridAggregator(name, factories, geohashIdSource, requiredSize,
shardSize, aggregationContext, parent);
}
@ -147,11 +139,12 @@ public class GeoHashGridParser implements Aggregator.Parser {
private GeoPointValues geoValues;
private int precision;
protected CellValues(GeoPointValuesSource geoPointValues, int precision ) {
protected CellValues(GeoPointValuesSource geoPointValues, int precision) {
super(true);
this.geoPointValues = geoPointValues;
this.precision=precision;
}
this.precision = precision;
}
@Override
public int setDocument(int docId) {
geoValues = geoPointValues.values();
@ -163,7 +156,7 @@ public class GeoHashGridParser implements Aggregator.Parser {
GeoPoint target = geoValues.nextValue();
return GeoHashUtils.encodeAsLong(target.getLat(), target.getLon(), precision);
}
}
private static class CellIdSource extends FieldDataSource.Numeric {
@ -188,7 +181,7 @@ public class GeoHashGridParser implements Aggregator.Parser {
@Override
public DoubleValues doubleValues() {
throw new UnsupportedOperationException();
throw new UnsupportedOperationException();
}
@Override

@ -26,6 +26,8 @@ import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.recycler.Recycler;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.AggregationStreams;
import org.elasticsearch.search.aggregations.Aggregations;
@ -40,10 +42,9 @@ import java.util.*;
* All geohashes in a grid are of the same precision and held internally as a single long
* for efficiency's sake.
*/
public class InternalGeoHashGrid extends InternalAggregation implements GeoHashGrid{
public class InternalGeoHashGrid extends InternalAggregation implements GeoHashGrid {
public static final Type TYPE = new Type("geohashgrid", "ghcells");
protected Map<String, Bucket> bucketMap;
public static final Type TYPE = new Type("geohash_grid", "ghcells");
public static AggregationStreams.Stream STREAM = new AggregationStreams.Stream() {
@Override
@ -59,27 +60,31 @@ public class InternalGeoHashGrid extends InternalAggregation implements GeoHashG
}
static class Bucket implements GeoHashGrid.Bucket , Comparable<Bucket> {
static class Bucket implements GeoHashGrid.Bucket, Comparable<Bucket> {
protected long geohashAsLong;
protected long docCount;
protected InternalAggregations aggregations;
protected InternalAggregations aggregations;
public Bucket(long geohashAsLong, long docCount, InternalAggregations aggregations) {
this.docCount=docCount;
this.aggregations=aggregations;
this.docCount = docCount;
this.aggregations = aggregations;
this.geohashAsLong = geohashAsLong;
}
public GeoPoint getGeoPoint() {
public String getKey() {
return GeoHashUtils.toString(geohashAsLong);
}
@Override
public Text getKeyAsText() {
return new StringText(getKey());
}
public GeoPoint getKeyAsGeoPoint() {
return GeoHashUtils.decode(geohashAsLong);
}
public String getGeoHash() {
return GeoHashUtils.toString(geohashAsLong);
}
@Override
public long getDocCount() {
return docCount;
@ -100,6 +105,7 @@ public class InternalGeoHashGrid extends InternalAggregation implements GeoHashG
}
return 0;
}
public Bucket reduce(List<? extends Bucket> buckets, CacheRecycler cacheRecycler) {
if (buckets.size() == 1) {
// we still need to reduce the sub aggs
@ -122,47 +128,56 @@ public class InternalGeoHashGrid extends InternalAggregation implements GeoHashG
}
@Override
public long getInternalKey() {
public Number getKeyAsNumber() {
return geohashAsLong;
}
}
}
private int requiredSize;
private Collection<Bucket> buckets;
InternalGeoHashGrid() {} // for serialization
protected Map<String, Bucket> bucketMap;
public InternalGeoHashGrid(String name, int requiredSize, Collection<Bucket> buckets) {
InternalGeoHashGrid() {
} // for serialization
public InternalGeoHashGrid(String name, int requiredSize, Collection<Bucket> buckets) {
super(name);
this.requiredSize=requiredSize;
this.buckets=buckets;
this.requiredSize = requiredSize;
this.buckets = buckets;
}
@Override
public Type type() {
return TYPE;
}
static class BucketPriorityQueue extends PriorityQueue<Bucket> {
public BucketPriorityQueue(int size) {
super(size);
}
@Override
public Collection<GeoHashGrid.Bucket> getBuckets() {
Object o = buckets;
return (Collection<GeoHashGrid.Bucket>) o;
}
@Override
protected boolean lessThan(Bucket o1, Bucket o2) {
long i = o2.getDocCount() - o1.getDocCount();
if (i == 0) {
i = o2.compareTo(o1);
if (i == 0) {
i = System.identityHashCode(o2) - System.identityHashCode(o1);
}
@Override
public GeoHashGrid.Bucket getBucketByKey(String geohash) {
if (bucketMap == null) {
bucketMap = new HashMap<String, Bucket>(buckets.size());
for (Bucket bucket : buckets) {
bucketMap.put(bucket.getKey(), bucket);
}
return i > 0 ? true : false;
}
}
return bucketMap.get(geohash);
}
@Override
public GeoHashGrid.Bucket getBucketByKey(Number key) {
return getBucketByKey(GeoHashUtils.toString(key.longValue()));
}
@Override
public GeoHashGrid.Bucket getBucketByKey(GeoPoint key) {
return getBucketByKey(key.geohash());
}
@Override
public InternalGeoHashGrid reduce(ReduceContext reduceContext) {
@ -212,24 +227,12 @@ public class InternalGeoHashGrid extends InternalAggregation implements GeoHashG
buckets.release();
Bucket[] list = new Bucket[ordered.size()];
for (int i = ordered.size() - 1; i >= 0; i--) {
list[i] = (Bucket) ordered.pop();
list[i] = ordered.pop();
}
reduced.buckets = Arrays.asList(list);
return reduced;
}
@Override
public Iterator<GeoHashGrid.Bucket> iterator() {
Object o = buckets.iterator();
return (Iterator<GeoHashGrid.Bucket>) o;
}
@Override
public int getNumberOfBuckets() {
return buckets.size();
}
protected void reduceAndTrimBuckets(CacheRecycler cacheRecycler) {
if (requiredSize > buckets.size()) { // nothing to trim
@ -269,7 +272,7 @@ public class InternalGeoHashGrid extends InternalAggregation implements GeoHashG
out.writeVInt(requiredSize);
out.writeVInt(buckets.size());
for (Bucket bucket : buckets) {
out.writeLong(((Bucket) bucket).geohashAsLong);
out.writeLong(bucket.geohashAsLong);
out.writeVLong(bucket.getDocCount());
((InternalAggregations) bucket.getAggregations()).writeTo(out);
}
@ -281,7 +284,7 @@ public class InternalGeoHashGrid extends InternalAggregation implements GeoHashG
builder.startArray(CommonFields.BUCKETS);
for (Bucket bucket : buckets) {
builder.startObject();
builder.field(CommonFields.KEY, ((Bucket) bucket).getGeoHash());
builder.field(CommonFields.KEY, bucket.getKeyAsText());
builder.field(CommonFields.DOC_COUNT, bucket.getDocCount());
((InternalAggregations) bucket.getAggregations()).toXContentInternal(builder, params);
builder.endObject();
@ -291,5 +294,23 @@ public class InternalGeoHashGrid extends InternalAggregation implements GeoHashG
return builder;
}
static class BucketPriorityQueue extends PriorityQueue<Bucket> {
public BucketPriorityQueue(int size) {
super(size);
}
@Override
protected boolean lessThan(Bucket o1, Bucket o2) {
long i = o2.getDocCount() - o1.getDocCount();
if (i == 0) {
i = o2.compareTo(o1);
if (i == 0) {
i = System.identityHashCode(o2) - System.identityHashCode(o1);
}
}
return i > 0;
}
}
}

@ -18,16 +18,10 @@
*/
package org.elasticsearch.search.aggregations.bucket.global;
import org.elasticsearch.search.aggregations.Aggregation;
import org.elasticsearch.search.aggregations.Aggregations;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregation;
/**
*
* A {@code global} aggregation. Defines a single bucket the holds all the documents in the search context.
*/
public interface Global extends Aggregation {
long getDocCount();
Aggregations getAggregations();
public interface Global extends SingleBucketAggregation {
}

@ -21,7 +21,7 @@ package org.elasticsearch.search.aggregations.bucket.global;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.search.aggregations.AggregationStreams;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregation;
import org.elasticsearch.search.aggregations.bucket.InternalSingleBucketAggregation;
import java.io.IOException;
@ -29,7 +29,7 @@ import java.io.IOException;
* A global scope get (the document set on which we aggregate is all documents in the search context (ie. index + type)
* regardless the query.
*/
public class InternalGlobal extends SingleBucketAggregation<InternalGlobal> implements Global {
public class InternalGlobal extends InternalSingleBucketAggregation implements Global {
public final static Type TYPE = new Type("global");
@ -46,9 +46,9 @@ public class InternalGlobal extends SingleBucketAggregation<InternalGlobal> impl
AggregationStreams.registerStream(STREAM, TYPE.stream());
}
public InternalGlobal() {} // for serialization
InternalGlobal() {} // for serialization
public InternalGlobal(String name, long docCount, InternalAggregations aggregations) {
InternalGlobal(String name, long docCount, InternalAggregations aggregations) {
super(name, docCount, aggregations);
}

@ -1,366 +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 com.carrotsearch.hppc.LongObjectOpenHashMap;
import com.google.common.collect.Lists;
import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.cache.recycler.CacheRecycler;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.recycler.Recycler;
import org.elasticsearch.common.rounding.Rounding;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.Aggregations;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.support.numeric.ValueFormatter;
import org.elasticsearch.search.aggregations.support.numeric.ValueFormatterStreams;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.ListIterator;
/**
* An internal implementation of {@link HistogramBase}
*/
abstract class AbstractHistogramBase<B extends HistogramBase.Bucket> extends InternalAggregation implements HistogramBase<B>, ToXContent, Streamable {
public static class Bucket implements HistogramBase.Bucket {
private long key;
private long docCount;
private InternalAggregations aggregations;
public Bucket(long key, long docCount, InternalAggregations aggregations) {
this.key = key;
this.docCount = docCount;
this.aggregations = aggregations;
}
@Override
public long getKey() {
return key;
}
@Override
public long getDocCount() {
return docCount;
}
@Override
public Aggregations getAggregations() {
return aggregations;
}
Bucket reduce(List<Bucket> buckets, CacheRecycler cacheRecycler) {
if (buckets.size() == 1) {
// we only need to reduce the sub aggregations
Bucket bucket = buckets.get(0);
bucket.aggregations.reduce(cacheRecycler);
return bucket;
}
List<InternalAggregations> aggregations = new ArrayList<InternalAggregations>(buckets.size());
Bucket reduced = null;
for (Bucket bucket : buckets) {
if (reduced == null) {
reduced = bucket;
} else {
reduced.docCount += bucket.docCount;
}
aggregations.add((InternalAggregations) bucket.getAggregations());
}
reduced.aggregations = InternalAggregations.reduce(aggregations, cacheRecycler);
return reduced;
}
}
static class EmptyBucketInfo {
final Rounding rounding;
final InternalAggregations subAggregations;
EmptyBucketInfo(Rounding rounding, InternalAggregations subAggregations) {
this.rounding = rounding;
this.subAggregations = subAggregations;
}
public static EmptyBucketInfo readFrom(StreamInput in) throws IOException {
return new EmptyBucketInfo(Rounding.Streams.read(in), InternalAggregations.readAggregations(in));
}
public static void writeTo(EmptyBucketInfo info, StreamOutput out) throws IOException {
Rounding.Streams.write(info.rounding, out);
info.subAggregations.writeTo(out);
}
}
public static interface Factory<B extends HistogramBase.Bucket> {
String type();
AbstractHistogramBase create(String name, List<B> buckets, InternalOrder order, long minDocCount, EmptyBucketInfo emptyBucketInfo, ValueFormatter formatter, boolean keyed);
Bucket createBucket(long key, long docCount, InternalAggregations aggregations);
}
private List<B> buckets;
private LongObjectOpenHashMap<HistogramBase.Bucket> bucketsMap;
private InternalOrder order;
private ValueFormatter formatter;
private boolean keyed;
private long minDocCount;
private EmptyBucketInfo emptyBucketInfo;
protected AbstractHistogramBase() {} // for serialization
protected AbstractHistogramBase(String name, List<B> buckets, InternalOrder order, long minDocCount, EmptyBucketInfo emptyBucketInfo, ValueFormatter formatter, boolean keyed) {
super(name);
this.buckets = buckets;
this.order = order;
assert (minDocCount == 0) == (emptyBucketInfo != null);
this.minDocCount = minDocCount;
this.emptyBucketInfo = emptyBucketInfo;
this.formatter = formatter;
this.keyed = keyed;
}
@Override
public Iterator<B> iterator() {
return buckets.iterator();
}
@Override
public List<B> buckets() {
return buckets;
}
@Override
public B getByKey(long key) {
if (bucketsMap == null) {
bucketsMap = new LongObjectOpenHashMap<HistogramBase.Bucket>(buckets.size());
for (HistogramBase.Bucket bucket : buckets) {
bucketsMap.put(bucket.getKey(), bucket);
}
}
return (B) bucketsMap.get(key);
}
// TODO extract the reduce logic to a strategy class and have it configurable at request time (two possible strategies - total & delta)
@Override
public InternalAggregation reduce(ReduceContext reduceContext) {
List<InternalAggregation> aggregations = reduceContext.aggregations();
if (aggregations.size() == 1) {
AbstractHistogramBase<B> histo = (AbstractHistogramBase<B>) aggregations.get(0);
if (minDocCount == 1) {
for (B bucket : histo.buckets) {
((Bucket) bucket).aggregations.reduce(reduceContext.cacheRecycler());
}
return histo;
}
CollectionUtil.introSort(histo.buckets, order.asc ? InternalOrder.KEY_ASC.comparator() : InternalOrder.KEY_DESC.comparator());
List<B> list = order.asc ? histo.buckets : Lists.reverse(histo.buckets);
HistogramBase.Bucket prevBucket = null;
ListIterator<B> iter = list.listIterator();
if (minDocCount == 0) {
// we need to fill the gaps with empty buckets
while (iter.hasNext()) {
// look ahead on the next bucket without advancing the iter
// so we'll be able to insert elements at the right position
HistogramBase.Bucket nextBucket = list.get(iter.nextIndex());
((Bucket) nextBucket).aggregations.reduce(reduceContext.cacheRecycler());
if (prevBucket != null) {
long key = emptyBucketInfo.rounding.nextRoundingValue(prevBucket.getKey());
while (key != nextBucket.getKey()) {
iter.add(createBucket(key, 0, emptyBucketInfo.subAggregations));
key = emptyBucketInfo.rounding.nextRoundingValue(key);
}
}
prevBucket = iter.next();
}
} else {
while (iter.hasNext()) {
Bucket bucket = (Bucket) iter.next();
if (bucket.getDocCount() < minDocCount) {
iter.remove();
} else {
bucket.aggregations.reduce(reduceContext.cacheRecycler());
}
}
}
if (order != InternalOrder.KEY_ASC && order != InternalOrder.KEY_DESC) {
CollectionUtil.introSort(histo.buckets, order.comparator());
}
return histo;
}
AbstractHistogramBase reduced = (AbstractHistogramBase) aggregations.get(0);
Recycler.V<LongObjectOpenHashMap<List<Bucket>>> bucketsByKey = reduceContext.cacheRecycler().longObjectMap(-1);
for (InternalAggregation aggregation : aggregations) {
AbstractHistogramBase<B> histogram = (AbstractHistogramBase) aggregation;
for (B bucket : histogram.buckets) {
List<Bucket> bucketList = bucketsByKey.v().get(((Bucket) bucket).key);
if (bucketList == null) {
bucketList = new ArrayList<Bucket>(aggregations.size());
bucketsByKey.v().put(((Bucket) bucket).key, bucketList);
}
bucketList.add((Bucket) bucket);
}
}
List<HistogramBase.Bucket> reducedBuckets = new ArrayList<HistogramBase.Bucket>(bucketsByKey.v().size());
Object[] buckets = bucketsByKey.v().values;
boolean[] allocated = bucketsByKey.v().allocated;
for (int i = 0; i < allocated.length; i++) {
if (allocated[i]) {
Bucket bucket = ((List<Bucket>) buckets[i]).get(0).reduce(((List<Bucket>) buckets[i]), reduceContext.cacheRecycler());
if (bucket.getDocCount() >= minDocCount) {
reducedBuckets.add(bucket);
}
}
}
bucketsByKey.release();
// adding empty buckets in needed
if (minDocCount == 0) {
CollectionUtil.introSort(reducedBuckets, order.asc ? InternalOrder.KEY_ASC.comparator() : InternalOrder.KEY_DESC.comparator());
List<HistogramBase.Bucket> list = order.asc ? reducedBuckets : Lists.reverse(reducedBuckets);
HistogramBase.Bucket prevBucket = null;
ListIterator<HistogramBase.Bucket> iter = list.listIterator();
while (iter.hasNext()) {
HistogramBase.Bucket nextBucket = list.get(iter.nextIndex());
if (prevBucket != null) {
long key = emptyBucketInfo.rounding.nextRoundingValue(prevBucket.getKey());
while (key != nextBucket.getKey()) {
iter.add(createBucket(key, 0, emptyBucketInfo.subAggregations));
key = emptyBucketInfo.rounding.nextRoundingValue(key);
}
}
prevBucket = iter.next();
}
if (order != InternalOrder.KEY_ASC && order != InternalOrder.KEY_DESC) {
CollectionUtil.introSort(reducedBuckets, order.comparator());
}
} else {
CollectionUtil.introSort(reducedBuckets, order.comparator());
}
reduced.buckets = reducedBuckets;
return reduced;
}
protected B createBucket(long key, long docCount, InternalAggregations aggregations) {
return (B) new Bucket(key, docCount, aggregations);
}
@Override
public void readFrom(StreamInput in) throws IOException {
name = in.readString();
order = InternalOrder.Streams.readOrder(in);
minDocCount = in.readVLong();
if (minDocCount == 0) {
emptyBucketInfo = EmptyBucketInfo.readFrom(in);
}
formatter = ValueFormatterStreams.readOptional(in);
keyed = in.readBoolean();
int size = in.readVInt();
List<B> buckets = new ArrayList<B>(size);
for (int i = 0; i < size; i++) {
buckets.add(createBucket(in.readLong(), in.readVLong(), InternalAggregations.readAggregations(in)));
}
this.buckets = buckets;
this.bucketsMap = null;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
InternalOrder.Streams.writeOrder(order, out);
out.writeVLong(minDocCount);
if (minDocCount == 0) {
EmptyBucketInfo.writeTo(emptyBucketInfo, out);
}
ValueFormatterStreams.writeOptional(formatter, out);
out.writeBoolean(keyed);
out.writeVInt(buckets.size());
for (HistogramBase.Bucket bucket : buckets) {
out.writeLong(((Bucket) bucket).key);
out.writeVLong(((Bucket) bucket).docCount);
((Bucket) bucket).aggregations.writeTo(out);
}
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
if (keyed) {
builder.startObject(name);
} else {
builder.startArray(name);
}
for (HistogramBase.Bucket bucket : buckets) {
if (formatter != null) {
Text keyTxt = new StringText(formatter.format(bucket.getKey()));
if (keyed) {
builder.startObject(keyTxt.string());
} else {
builder.startObject();
}
builder.field(CommonFields.KEY_AS_STRING, keyTxt);
} else {
if (keyed) {
builder.startObject(String.valueOf(bucket.getKey()));
} else {
builder.startObject();
}
}
builder.field(CommonFields.KEY, ((Bucket) bucket).key);
builder.field(CommonFields.DOC_COUNT, ((Bucket) bucket).docCount);
((Bucket) bucket).aggregations.toXContentInternal(builder, params);
builder.endObject();
}
if (keyed) {
builder.endObject();
} else {
builder.endArray();
}
return builder;
}
}

@ -20,17 +20,36 @@ package org.elasticsearch.search.aggregations.bucket.histogram;
import org.joda.time.DateTime;
import java.util.Collection;
/**
*
* A {@code date_histogram} aggregation.
*/
public interface DateHistogram extends HistogramBase<DateHistogram.Bucket> {
public interface DateHistogram extends Histogram {
static interface Bucket extends HistogramBase.Bucket {
static interface Bucket extends Histogram.Bucket {
/**
* @return the key as a date construct.
*/
DateTime getKeyAsDate();
}
@Override
Collection<? extends DateHistogram.Bucket> getBuckets();
@Override
Bucket getBucketByKey(String key);
@Override
Bucket getBucketByKey(Number key);
Bucket getBucketByKey(DateTime key);
/**
* The interval the date histogram is based on.
*/
static class Interval {
public static final Interval SECOND = new Interval("1s");
@ -58,7 +77,7 @@ public interface DateHistogram extends HistogramBase<DateHistogram.Bucket> {
return new Interval(days + "d");
}
public static Interval week(int weeks) {
public static Interval weeks(int weeks) {
return new Interval(weeks + "w");
}

@ -32,7 +32,7 @@ import java.io.IOException;
public class DateHistogramBuilder extends ValuesSourceAggregationBuilder<DateHistogramBuilder> {
private Object interval;
private HistogramBase.Order order;
private Histogram.Order order;
private Long minDocCount;
private String preZone;
private String postZone;

@ -234,9 +234,9 @@ public class DateHistogramParser implements Aggregator.Parser {
}
int i = key.indexOf('.');
if (i < 0) {
return HistogramBase.Order.aggregation(key, asc);
return new InternalOrder.Aggregation(key, null, asc);
}
return HistogramBase.Order.aggregation(key.substring(0, i), key.substring(i + 1), asc);
return new InternalOrder.Aggregation(key.substring(0, i), key.substring(i + 1), asc);
}
private long parseOffset(String offset) throws IOException {

@ -18,12 +18,120 @@
*/
package org.elasticsearch.search.aggregations.bucket.histogram;
/**
* TODO should be renamed to NumericHistogram? This would also clean up the code and make it less confusing
*/
public interface Histogram extends HistogramBase<Histogram.Bucket> {
import com.google.common.primitives.Longs;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.search.aggregations.Aggregation;
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
import java.util.Collection;
import java.util.Comparator;
/**
* A {@code histogram} aggregation. Defines multiple buckets, each representing an interval in a histogram.
*/
public interface Histogram extends MultiBucketsAggregation {
/**
* A bucket in the histogram where documents fall in
*/
static interface Bucket extends MultiBucketsAggregation.Bucket {
/**
* @return The key associated with the bucket (all documents that fall in this bucket were rounded to this key)
*/
Number getKeyAsNumber();
static interface Bucket extends HistogramBase.Bucket {
}
/**
* @return The buckets of this histogram (each bucket representing an interval in the histogram)
*/
Collection<? extends Bucket> getBuckets();
/**
* Returns a bucket by the key associated with it.
*
* @param key The key of the bucket.
* @return The bucket that is associated with the given key.
*/
Bucket getBucketByKey(String key);
/**
* Returns a bucket by the key associated with it.
*
* @param key The key of the bucket.
* @return The bucket that is associated with the given key.
*/
Bucket getBucketByKey(Number key);
/**
* A strategy defining the order in which the buckets in this histogram are ordered.
*/
static abstract class Order implements ToXContent {
public static final Order KEY_ASC = new InternalOrder((byte) 1, "_key", true, new Comparator<InternalHistogram.Bucket>() {
@Override
public int compare(InternalHistogram.Bucket b1, InternalHistogram.Bucket b2) {
return Longs.compare(b1.key, b2.key);
}
});
public static final Order KEY_DESC = new InternalOrder((byte) 2, "_key", false, new Comparator<InternalHistogram.Bucket>() {
@Override
public int compare(InternalHistogram.Bucket b1, InternalHistogram.Bucket b2) {
return -Longs.compare(b1.key, b2.key);
}
});
public static final Order COUNT_ASC = new InternalOrder((byte) 3, "_count", true, new Comparator<InternalHistogram.Bucket>() {
@Override
public int compare(InternalHistogram.Bucket b1, InternalHistogram.Bucket b2) {
int cmp = Longs.compare(b1.getDocCount(), b2.getDocCount());
if (cmp == 0) {
cmp = Longs.compare(b1.key, b2.key);
}
return cmp;
}
});
public static final Order COUNT_DESC = new InternalOrder((byte) 4, "_count", false, new Comparator<InternalHistogram.Bucket>() {
@Override
public int compare(InternalHistogram.Bucket b1, InternalHistogram.Bucket b2) {
int cmp = -Longs.compare(b1.getDocCount(), b2.getDocCount());
if (cmp == 0) {
cmp = Longs.compare(b1.key, b2.key);
}
return cmp;
}
});
/**
* Creates a bucket ordering strategy that sorts buckets based on a single-valued calc sug-aggregation
*
* @param aggregationName the name of the aggregation
* @param asc The direction of the order (ascending or descending)
*/
public static Order aggregation(String aggregationName, boolean asc) {
return new InternalOrder.Aggregation(aggregationName, null, asc);
}
/**
* Creates a bucket ordering strategy that sorts buckets based on a multi-valued calc sug-aggregation
*
* @param aggregationName the name of the aggregation
* @param valueName The name of the value of the multi-value get by which the sorting will be applied
* @param asc The direction of the order (ascending or descending)
*/
public static Order aggregation(String aggregationName, String valueName, boolean asc) {
return new InternalOrder.Aggregation(aggregationName, valueName, asc);
}
/**
* @return The bucket comparator by which the order will be applied.
*/
abstract Comparator<InternalHistogram.Bucket> comparator();
}
}

@ -47,7 +47,7 @@ public class HistogramAggregator extends BucketsAggregator {
private final InternalOrder order;
private final boolean keyed;
private final long minDocCount;
private final AbstractHistogramBase.Factory histogramFactory;
private final InternalHistogram.Factory histogramFactory;
private final LongHash bucketOrds;
private LongValues values;
@ -60,7 +60,7 @@ public class HistogramAggregator extends BucketsAggregator {
long minDocCount,
@Nullable NumericValuesSource valuesSource,
long initialCapacity,
AbstractHistogramBase.Factory<?> histogramFactory,
InternalHistogram.Factory<?> histogramFactory,
AggregationContext aggregationContext,
Aggregator parent) {
@ -110,28 +110,27 @@ public class HistogramAggregator extends BucketsAggregator {
@Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
assert owningBucketOrdinal == 0;
List<HistogramBase.Bucket> buckets = new ArrayList<HistogramBase.Bucket>((int) bucketOrds.size());
List<InternalHistogram.Bucket> buckets = new ArrayList<InternalHistogram.Bucket>((int) bucketOrds.size());
for (long i = 0; i < bucketOrds.capacity(); ++i) {
final long ord = bucketOrds.id(i);
if (ord < 0) {
continue; // slot is not allocated
}
buckets.add(histogramFactory.createBucket(rounding.valueForKey(bucketOrds.key(i)), bucketDocCount(ord), bucketAggregations(ord)));
buckets.add(histogramFactory.createBucket(rounding.valueForKey(bucketOrds.key(i)), bucketDocCount(ord), bucketAggregations(ord), valuesSource.formatter()));
}
CollectionUtil.introSort(buckets, order.comparator());
// value source will be null for unmapped fields
ValueFormatter formatter = valuesSource != null ? valuesSource.formatter() : null;
AbstractHistogramBase.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 ? new AbstractHistogramBase.EmptyBucketInfo(rounding, buildEmptySubAggregations()) : null;
InternalHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 ? new InternalHistogram.EmptyBucketInfo(rounding, buildEmptySubAggregations()) : null;
return histogramFactory.create(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed);
}
@Override
public InternalAggregation buildEmptyAggregation() {
ValueFormatter formatter = valuesSource != null ? valuesSource.formatter() : null;
AbstractHistogramBase.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 ? new AbstractHistogramBase.EmptyBucketInfo(rounding, buildEmptySubAggregations()) : null;
InternalHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 ? new InternalHistogram.EmptyBucketInfo(rounding, buildEmptySubAggregations()) : null;
return histogramFactory.create(name, Collections.emptyList(), order, minDocCount, emptyBucketInfo, formatter, keyed);
}
@ -146,10 +145,10 @@ public class HistogramAggregator extends BucketsAggregator {
private final InternalOrder order;
private final boolean keyed;
private final long minDocCount;
private final AbstractHistogramBase.Factory<?> histogramFactory;
private final InternalHistogram.Factory<?> histogramFactory;
public Factory(String name, ValuesSourceConfig<NumericValuesSource> valueSourceConfig,
Rounding rounding, InternalOrder order, boolean keyed, long minDocCount, AbstractHistogramBase.Factory<?> histogramFactory) {
Rounding rounding, InternalOrder order, boolean keyed, long minDocCount, InternalHistogram.Factory<?> histogramFactory) {
super(name, histogramFactory.type(), valueSourceConfig);
this.rounding = rounding;
this.order = order;

@ -1,125 +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 com.google.common.primitives.Longs;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.search.aggregations.Aggregation;
import java.util.Comparator;
import java.util.List;
/**
* A histogram get result
*/
interface HistogramBase<B extends HistogramBase.Bucket> extends Aggregation, Iterable<B> {
/**
* A bucket in the histogram where documents fall in
*/
static interface Bucket extends org.elasticsearch.search.aggregations.bucket.Bucket {
/**
* @return The key associated with the bucket (all documents that fall in this bucket were rounded to this key)
*/
long getKey();
}
List<B> buckets();
/**
* Returns a bucket by the key associated with it.
*
* @param key The key of the bucket.
* @return The bucket that is associated with the given key.
*/
B getByKey(long key);
/**
* A strategy defining the order in which the buckets in this histogram are ordered.
*/
static abstract class Order implements ToXContent {
public static final Order KEY_ASC = new InternalOrder((byte) 1, "_key", true, new Comparator<HistogramBase.Bucket>() {
@Override
public int compare(HistogramBase.Bucket b1, HistogramBase.Bucket b2) {
return Longs.compare(b1.getKey(), b2.getKey());
}
});
public static final Order KEY_DESC = new InternalOrder((byte) 2, "_key", false, new Comparator<HistogramBase.Bucket>() {
@Override
public int compare(HistogramBase.Bucket b1, HistogramBase.Bucket b2) {
return - Longs.compare(b1.getKey(), b2.getKey());
}
});
public static final Order COUNT_ASC = new InternalOrder((byte) 3, "_count", true, new Comparator<HistogramBase.Bucket>() {
@Override
public int compare(HistogramBase.Bucket b1, HistogramBase.Bucket b2) {
int cmp = Longs.compare(b1.getDocCount(), b2.getDocCount());
if (cmp == 0) {
cmp = Longs.compare(b1.getKey(), b2.getKey());
}
return cmp;
}
});
public static final Order COUNT_DESC = new InternalOrder((byte) 4, "_count", false, new Comparator<HistogramBase.Bucket>() {
@Override
public int compare(HistogramBase.Bucket b1, HistogramBase.Bucket b2) {
int cmp = - Longs.compare(b1.getDocCount(), b2.getDocCount());
if (cmp == 0) {
cmp = Longs.compare(b1.getKey(), b2.getKey());
}
return cmp;
}
});
/**
* Creates a bucket ordering strategy which sorts buckets based on a single-valued calc get
*
* @param aggregationName the name of the get
* @param asc The direction of the order (ascending or descending)
*/
public static InternalOrder aggregation(String aggregationName, boolean asc) {
return new InternalOrder.Aggregation(aggregationName, null, asc);
}
/**
* Creates a bucket ordering strategy which sorts buckets based on a multi-valued calc get
*
* @param aggregationName the name of the get
* @param valueName The name of the value of the multi-value get by which the sorting will be applied
* @param asc The direction of the order (ascending or descending)
*/
public static InternalOrder aggregation(String aggregationName, String valueName, boolean asc) {
return new InternalOrder.Aggregation(aggregationName, valueName, asc);
}
/**
* @return The bucket comparator by which the order will be applied.
*/
abstract Comparator<Bucket> comparator();
}
}

@ -26,28 +26,51 @@ import org.elasticsearch.search.builder.SearchSourceBuilderException;
import java.io.IOException;
/**
*
* A builder for a histogram aggregation.
*/
public class HistogramBuilder extends ValuesSourceAggregationBuilder<HistogramBuilder> {
private Long interval;
private HistogramBase.Order order;
private Histogram.Order order;
private Long minDocCount;
/**
* Constructs a new histogram aggregation builder.
*
* @param name The name of the aggregation (will serve as the unique identifier for the aggregation result in the response)
*/
public HistogramBuilder(String name) {
super(name, InternalHistogram.TYPE.name());
}
/**
* Sets the interval for the histogram.
*
* @param interval The interval for the histogram
* @return This builder
*/
public HistogramBuilder interval(long interval) {
this.interval = interval;
return this;
}
/**
* Sets the order by which the buckets will be returned.
*
* @param order The order by which the buckets will be returned
* @return This builder
*/
public HistogramBuilder order(Histogram.Order order) {
this.order = order;
return this;
}
/**
* Sets the minimum document count per bucket. Buckets with less documents than this min value will not be returned.
*
* @param minDocCount The minimum document count per bucket
* @return This builder
*/
public HistogramBuilder minDocCount(long minDocCount) {
this.minDocCount = minDocCount;
return this;

@ -75,7 +75,7 @@ public class HistogramParser implements Aggregator.Parser {
} else if ("format".equals(currentFieldName)) {
format = parser.text();
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
throw new SearchParseException(context, "Unknown key for a " + token + " in aggregation [" + aggregationName + "]: [" + currentFieldName + "].");
}
} else if (token == XContentParser.Token.VALUE_NUMBER) {
if ("interval".equals(currentFieldName)) {
@ -83,7 +83,7 @@ public class HistogramParser implements Aggregator.Parser {
} else if ("min_doc_count".equals(currentFieldName) || "minDocCount".equals(currentFieldName)) {
minDocCount = parser.longValue();
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
throw new SearchParseException(context, "Unknown key for a " + token + " in aggregation [" + aggregationName + "]: [" + currentFieldName + "].");
}
} else if (token == XContentParser.Token.VALUE_BOOLEAN) {
if ("keyed".equals(currentFieldName)) {
@ -91,7 +91,7 @@ public class HistogramParser implements Aggregator.Parser {
} else if ("script_values_sorted".equals(currentFieldName)) {
assumeSorted = parser.booleanValue();
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
throw new SearchParseException(context, "Unknown key for a " + token + " in aggregation [" + aggregationName + "]: [" + currentFieldName + "].");
}
} else if (token == XContentParser.Token.START_OBJECT) {
if ("params".equals(currentFieldName)) {
@ -103,15 +103,17 @@ public class HistogramParser implements Aggregator.Parser {
} else if (token == XContentParser.Token.VALUE_STRING) {
String dir = parser.text();
boolean asc = "asc".equals(dir);
if (!asc && !"desc".equals(dir)) {
throw new SearchParseException(context, "Unknown order direction [" + dir + "] in aggregation [" + aggregationName + "]. Should be either [asc] or [desc]");
}
order = resolveOrder(currentFieldName, asc);
//TODO should we throw an error if the value is not "asc" or "desc"???
}
}
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
throw new SearchParseException(context, "Unknown key for a " + token + " in aggregation [" + aggregationName + "]: [" + currentFieldName + "].");
}
} else {
throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].");
throw new SearchParseException(context, "Unexpected token " + token + " in aggregation [" + aggregationName + "].");
}
}
@ -159,8 +161,8 @@ public class HistogramParser implements Aggregator.Parser {
}
int i = key.indexOf('.');
if (i < 0) {
return HistogramBase.Order.aggregation(key, asc);
return new InternalOrder.Aggregation(key, null, asc);
}
return HistogramBase.Order.aggregation(key.substring(0, i), key.substring(i + 1), asc);
return new InternalOrder.Aggregation(key.substring(0, i), key.substring(i + 1), asc);
}
}

@ -19,8 +19,8 @@
package org.elasticsearch.search.aggregations.bucket.histogram;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.index.mapper.core.DateFieldMapper;
import org.elasticsearch.search.aggregations.AggregationStreams;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.support.numeric.ValueFormatter;
import org.joda.time.DateTime;
@ -31,10 +31,10 @@ import java.util.List;
/**
*
*/
public class InternalDateHistogram extends AbstractHistogramBase<DateHistogram.Bucket> implements DateHistogram {
public class InternalDateHistogram extends InternalHistogram<InternalDateHistogram.Bucket> implements DateHistogram {
public final static Type TYPE = new Type("date_histogram", "dhisto");
public final static Factory FACTORY = new Factory();
final static Type TYPE = new Type("date_histogram", "dhisto");
final static Factory FACTORY = new Factory();
private final static AggregationStreams.Stream STREAM = new AggregationStreams.Stream() {
@Override
@ -49,23 +49,27 @@ public class InternalDateHistogram extends AbstractHistogramBase<DateHistogram.B
AggregationStreams.registerStream(STREAM, TYPE.stream());
}
static class Bucket extends AbstractHistogramBase.Bucket implements DateHistogram.Bucket {
static class Bucket extends InternalHistogram.Bucket implements DateHistogram.Bucket {
Bucket(long key, long docCount, InternalAggregations aggregations) {
private final ValueFormatter formatter;
Bucket(long key, long docCount, InternalAggregations aggregations, ValueFormatter formatter) {
super(key, docCount, aggregations);
this.formatter = formatter;
}
Bucket(long key, long docCount, List<InternalAggregation> aggregations) {
super(key, docCount, new InternalAggregations(aggregations));
@Override
public String getKey() {
return formatter != null ? formatter.format(key) : DateFieldMapper.Defaults.DATE_TIME_FORMATTER.printer().print(key);
}
@Override
public DateTime getKeyAsDate() {
return new DateTime(getKey());
return new DateTime(key);
}
}
static class Factory implements AbstractHistogramBase.Factory<DateHistogram.Bucket> {
static class Factory extends InternalHistogram.Factory<InternalDateHistogram.Bucket> {
private Factory() {
}
@ -76,19 +80,21 @@ public class InternalDateHistogram extends AbstractHistogramBase<DateHistogram.B
}
@Override
public AbstractHistogramBase<?> create(String name, List<DateHistogram.Bucket> buckets, InternalOrder order, long minDocCount, EmptyBucketInfo emptyBucketInfo, ValueFormatter formatter, boolean keyed) {
public InternalDateHistogram create(String name, List<InternalDateHistogram.Bucket> buckets, InternalOrder order,
long minDocCount, EmptyBucketInfo emptyBucketInfo, ValueFormatter formatter, boolean keyed) {
return new InternalDateHistogram(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed);
}
@Override
public AbstractHistogramBase.Bucket createBucket(long key, long docCount, InternalAggregations aggregations) {
return new Bucket(key, docCount, aggregations);
public InternalDateHistogram.Bucket createBucket(long key, long docCount, InternalAggregations aggregations, ValueFormatter formatter) {
return new Bucket(key, docCount, aggregations, formatter);
}
}
InternalDateHistogram() {} // for serialization
InternalDateHistogram(String name, List<DateHistogram.Bucket> buckets, InternalOrder order, long minDocCount, EmptyBucketInfo emptyBucketInfo, ValueFormatter formatter, boolean keyed) {
InternalDateHistogram(String name, List<InternalDateHistogram.Bucket> buckets, InternalOrder order, long minDocCount,
EmptyBucketInfo emptyBucketInfo, ValueFormatter formatter, boolean keyed) {
super(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed);
}
@ -98,7 +104,12 @@ public class InternalDateHistogram extends AbstractHistogramBase<DateHistogram.B
}
@Override
protected DateHistogram.Bucket createBucket(long key, long docCount, InternalAggregations aggregations) {
return new Bucket(key, docCount, aggregations);
public DateHistogram.Bucket getBucketByKey(DateTime key) {
return getBucketByKey(key.getMillis());
}
@Override
protected InternalDateHistogram.Bucket createBucket(long key, long docCount, InternalAggregations aggregations, ValueFormatter formatter) {
return new Bucket(key, docCount, aggregations, formatter);
}
}

@ -18,21 +18,37 @@
*/
package org.elasticsearch.search.aggregations.bucket.histogram;
import com.carrotsearch.hppc.LongObjectOpenHashMap;
import com.google.common.collect.Lists;
import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.cache.recycler.CacheRecycler;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.recycler.Recycler;
import org.elasticsearch.common.rounding.Rounding;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.AggregationStreams;
import org.elasticsearch.search.aggregations.Aggregations;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.support.numeric.ValueFormatter;
import org.elasticsearch.search.aggregations.support.numeric.ValueFormatterStreams;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.ListIterator;
/**
* TODO should be renamed to InternalNumericHistogram (see comment on {@link Histogram})?
*/
public class InternalHistogram extends AbstractHistogramBase<Histogram.Bucket> implements Histogram {
public class InternalHistogram<B extends InternalHistogram.Bucket> extends InternalAggregation implements Histogram {
public final static Type TYPE = new Type("histogram", "histo");
public final static Factory FACTORY = new Factory();
final static Type TYPE = new Type("histogram", "histo");
final static Factory FACTORY = new Factory();
private final static AggregationStreams.Stream STREAM = new AggregationStreams.Stream() {
@Override
@ -47,38 +63,123 @@ public class InternalHistogram extends AbstractHistogramBase<Histogram.Bucket> i
AggregationStreams.registerStream(STREAM, TYPE.stream());
}
static class Bucket extends AbstractHistogramBase.Bucket implements Histogram.Bucket {
public static class Bucket implements Histogram.Bucket {
Bucket(long key, long docCount, InternalAggregations aggregations) {
super(key, docCount, aggregations);
}
long key;
private long docCount;
private InternalAggregations aggregations;
}
static class Factory implements AbstractHistogramBase.Factory<Histogram.Bucket> {
private Factory() {
public Bucket(long key, long docCount, InternalAggregations aggregations) {
this.key = key;
this.docCount = docCount;
this.aggregations = aggregations;
}
@Override
public String getKey() {
return String.valueOf(key);
}
@Override
public Text getKeyAsText() {
return new StringText(getKey());
}
@Override
public Number getKeyAsNumber() {
return key;
}
@Override
public long getDocCount() {
return docCount;
}
@Override
public Aggregations getAggregations() {
return aggregations;
}
<B extends Bucket> B reduce(List<B> buckets, CacheRecycler cacheRecycler) {
if (buckets.size() == 1) {
// we only need to reduce the sub aggregations
Bucket bucket = buckets.get(0);
bucket.aggregations.reduce(cacheRecycler);
return (B) bucket;
}
List<InternalAggregations> aggregations = new ArrayList<InternalAggregations>(buckets.size());
Bucket reduced = null;
for (Bucket bucket : buckets) {
if (reduced == null) {
reduced = bucket;
} else {
reduced.docCount += bucket.docCount;
}
aggregations.add((InternalAggregations) bucket.getAggregations());
}
reduced.aggregations = InternalAggregations.reduce(aggregations, cacheRecycler);
return (B) reduced;
}
}
static class EmptyBucketInfo {
final Rounding rounding;
final InternalAggregations subAggregations;
EmptyBucketInfo(Rounding rounding, InternalAggregations subAggregations) {
this.rounding = rounding;
this.subAggregations = subAggregations;
}
public static EmptyBucketInfo readFrom(StreamInput in) throws IOException {
return new EmptyBucketInfo(Rounding.Streams.read(in), InternalAggregations.readAggregations(in));
}
public static void writeTo(EmptyBucketInfo info, StreamOutput out) throws IOException {
Rounding.Streams.write(info.rounding, out);
info.subAggregations.writeTo(out);
}
}
static class Factory<B extends InternalHistogram.Bucket> {
protected Factory() {
}
public String type() {
return TYPE.name();
}
public AbstractHistogramBase<?> create(String name, List<Histogram.Bucket> buckets, InternalOrder order, long minDocCount, EmptyBucketInfo emptyBucketInfo, ValueFormatter formatter, boolean keyed) {
return new InternalHistogram(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed);
public InternalHistogram<B> create(String name, List<B> buckets, InternalOrder order, long minDocCount,
EmptyBucketInfo emptyBucketInfo, ValueFormatter formatter, boolean keyed) {
return new InternalHistogram<B>(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed);
}
public Bucket createBucket(long key, long docCount, InternalAggregations aggregations) {
return new Bucket(key, docCount, aggregations);
public B createBucket(long key, long docCount, InternalAggregations aggregations, ValueFormatter formatter) {
return (B) new Bucket(key, docCount, aggregations);
}
}
public InternalHistogram() {} // for serialization
private List<B> buckets;
private LongObjectOpenHashMap<B> bucketsMap;
private InternalOrder order;
private ValueFormatter formatter;
private boolean keyed;
private long minDocCount;
private EmptyBucketInfo emptyBucketInfo;
public InternalHistogram(String name, List<Histogram.Bucket> buckets, InternalOrder order, long minDocCount, EmptyBucketInfo emptyBucketInfo, ValueFormatter formatter, boolean keyed) {
super(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed);
InternalHistogram() {} // for serialization
InternalHistogram(String name, List<B> buckets, InternalOrder order, long minDocCount, EmptyBucketInfo emptyBucketInfo, ValueFormatter formatter, boolean keyed) {
super(name);
this.buckets = buckets;
this.order = order;
assert (minDocCount == 0) == (emptyBucketInfo != null);
this.minDocCount = minDocCount;
this.emptyBucketInfo = emptyBucketInfo;
this.formatter = formatter;
this.keyed = keyed;
}
@Override
@ -86,7 +187,216 @@ public class InternalHistogram extends AbstractHistogramBase<Histogram.Bucket> i
return TYPE;
}
protected Bucket createBucket(long key, long docCount, InternalAggregations aggregations) {
return new Bucket(key, docCount, aggregations);
@Override
public Collection<B> getBuckets() {
return buckets;
}
@Override
public B getBucketByKey(String key) {
return getBucketByKey(Long.valueOf(key));
}
@Override
public B getBucketByKey(Number key) {
if (bucketsMap == null) {
bucketsMap = new LongObjectOpenHashMap<B>(buckets.size());
for (B bucket : buckets) {
bucketsMap.put(bucket.key, bucket);
}
}
return bucketsMap.get(key.longValue());
}
@Override
public InternalAggregation reduce(ReduceContext reduceContext) {
List<InternalAggregation> aggregations = reduceContext.aggregations();
if (aggregations.size() == 1) {
InternalHistogram<B> histo = (InternalHistogram<B>) aggregations.get(0);
if (minDocCount == 1) {
for (B bucket : histo.buckets) {
bucket.aggregations.reduce(reduceContext.cacheRecycler());
}
return histo;
}
CollectionUtil.introSort(histo.buckets, order.asc ? InternalOrder.KEY_ASC.comparator() : InternalOrder.KEY_DESC.comparator());
List<B> list = order.asc ? histo.buckets : Lists.reverse(histo.buckets);
B prevBucket = null;
ListIterator<B> iter = list.listIterator();
if (minDocCount == 0) {
// we need to fill the gaps with empty buckets
while (iter.hasNext()) {
// look ahead on the next bucket without advancing the iter
// so we'll be able to insert elements at the right position
B nextBucket = list.get(iter.nextIndex());
nextBucket.aggregations.reduce(reduceContext.cacheRecycler());
if (prevBucket != null) {
long key = emptyBucketInfo.rounding.nextRoundingValue(prevBucket.key);
while (key != nextBucket.key) {
iter.add(createBucket(key, 0, emptyBucketInfo.subAggregations, formatter));
key = emptyBucketInfo.rounding.nextRoundingValue(key);
}
}
prevBucket = iter.next();
}
} else {
while (iter.hasNext()) {
InternalHistogram.Bucket bucket = iter.next();
if (bucket.getDocCount() < minDocCount) {
iter.remove();
} else {
bucket.aggregations.reduce(reduceContext.cacheRecycler());
}
}
}
if (order != InternalOrder.KEY_ASC && order != InternalOrder.KEY_DESC) {
CollectionUtil.introSort(histo.buckets, order.comparator());
}
return histo;
}
InternalHistogram reduced = (InternalHistogram) aggregations.get(0);
Recycler.V<LongObjectOpenHashMap<List<Histogram.Bucket>>> bucketsByKey = reduceContext.cacheRecycler().longObjectMap(-1);
for (InternalAggregation aggregation : aggregations) {
InternalHistogram<B> histogram = (InternalHistogram) aggregation;
for (B bucket : histogram.buckets) {
List<Histogram.Bucket> bucketList = bucketsByKey.v().get(bucket.key);
if (bucketList == null) {
bucketList = new ArrayList<Histogram.Bucket>(aggregations.size());
bucketsByKey.v().put(bucket.key, bucketList);
}
bucketList.add(bucket);
}
}
List<B> reducedBuckets = new ArrayList<B>(bucketsByKey.v().size());
Object[] buckets = bucketsByKey.v().values;
boolean[] allocated = bucketsByKey.v().allocated;
for (int i = 0; i < allocated.length; i++) {
if (allocated[i]) {
B bucket = ((List<B>) buckets[i]).get(0).reduce(((List<B>) buckets[i]), reduceContext.cacheRecycler());
if (bucket.getDocCount() >= minDocCount) {
reducedBuckets.add(bucket);
}
}
}
bucketsByKey.release();
// adding empty buckets in needed
if (minDocCount == 0) {
CollectionUtil.introSort(reducedBuckets, order.asc ? InternalOrder.KEY_ASC.comparator() : InternalOrder.KEY_DESC.comparator());
List<B> list = order.asc ? reducedBuckets : Lists.reverse(reducedBuckets);
B prevBucket = null;
ListIterator<B> iter = list.listIterator();
while (iter.hasNext()) {
B nextBucket = list.get(iter.nextIndex());
if (prevBucket != null) {
long key = emptyBucketInfo.rounding.nextRoundingValue(prevBucket.key);
while (key != nextBucket.key) {
iter.add(createBucket(key, 0, emptyBucketInfo.subAggregations, formatter));
key = emptyBucketInfo.rounding.nextRoundingValue(key);
}
}
prevBucket = iter.next();
}
if (order != InternalOrder.KEY_ASC && order != InternalOrder.KEY_DESC) {
CollectionUtil.introSort(reducedBuckets, order.comparator());
}
} else {
CollectionUtil.introSort(reducedBuckets, order.comparator());
}
reduced.buckets = reducedBuckets;
return reduced;
}
protected B createBucket(long key, long docCount, InternalAggregations aggregations, ValueFormatter formatter) {
return (B) new InternalHistogram.Bucket(key, docCount, aggregations);
}
@Override
public void readFrom(StreamInput in) throws IOException {
name = in.readString();
order = InternalOrder.Streams.readOrder(in);
minDocCount = in.readVLong();
if (minDocCount == 0) {
emptyBucketInfo = EmptyBucketInfo.readFrom(in);
}
formatter = ValueFormatterStreams.readOptional(in);
keyed = in.readBoolean();
int size = in.readVInt();
List<B> buckets = new ArrayList<B>(size);
for (int i = 0; i < size; i++) {
buckets.add(createBucket(in.readLong(), in.readVLong(), InternalAggregations.readAggregations(in), formatter));
}
this.buckets = buckets;
this.bucketsMap = null;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
InternalOrder.Streams.writeOrder(order, out);
out.writeVLong(minDocCount);
if (minDocCount == 0) {
EmptyBucketInfo.writeTo(emptyBucketInfo, out);
}
ValueFormatterStreams.writeOptional(formatter, out);
out.writeBoolean(keyed);
out.writeVInt(buckets.size());
for (B bucket : buckets) {
out.writeLong(bucket.key);
out.writeVLong(bucket.docCount);
bucket.aggregations.writeTo(out);
}
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
if (keyed) {
builder.startObject(name);
} else {
builder.startArray(name);
}
for (B bucket : buckets) {
if (formatter != null) {
Text keyTxt = new StringText(formatter.format(bucket.key));
if (keyed) {
builder.startObject(keyTxt.string());
} else {
builder.startObject();
}
builder.field(CommonFields.KEY_AS_STRING, keyTxt);
} else {
if (keyed) {
builder.startObject(String.valueOf(bucket.getKeyAsNumber()));
} else {
builder.startObject();
}
}
builder.field(CommonFields.KEY, bucket.key);
builder.field(CommonFields.DOC_COUNT, bucket.docCount);
bucket.aggregations.toXContentInternal(builder, params);
builder.endObject();
}
if (keyed) {
builder.endObject();
} else {
builder.endArray();
}
return builder;
}
}

@ -21,22 +21,22 @@ 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.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.bucket.Bucket;
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
import java.io.IOException;
import java.util.Comparator;
/**
* An internal {@link HistogramBase.Order} strategy which is identified by a unique id.
* An internal {@link Histogram.Order} strategy which is identified by a unique id.
*/
class InternalOrder extends HistogramBase.Order {
class InternalOrder extends Histogram.Order {
final byte id;
final String key;
final boolean asc;
final Comparator<HistogramBase.Bucket> comparator;
final Comparator<InternalHistogram.Bucket> comparator;
InternalOrder(byte id, String key, boolean asc, Comparator<HistogramBase.Bucket> comparator) {
InternalOrder(byte id, String key, boolean asc, Comparator<InternalHistogram.Bucket> comparator) {
this.id = id;
this.key = key;
this.asc = asc;
@ -56,7 +56,7 @@ class InternalOrder extends HistogramBase.Order {
}
@Override
Comparator<HistogramBase.Bucket> comparator() {
Comparator<InternalHistogram.Bucket> comparator() {
return comparator;
}
@ -70,11 +70,11 @@ class InternalOrder extends HistogramBase.Order {
static final byte ID = 0;
Aggregation(String key, boolean asc) {
super(ID, key, asc, new Bucket.SubAggregationComparator<HistogramBase.Bucket>(key, asc));
super(ID, key, asc, new MultiBucketsAggregation.Bucket.SubAggregationComparator<InternalHistogram.Bucket>(key, asc));
}
Aggregation(String aggName, String valueName, boolean asc) {
super(ID, key(aggName, valueName), asc, new Bucket.SubAggregationComparator<HistogramBase.Bucket>(aggName, valueName, asc));
super(ID, key(aggName, valueName), asc, new MultiBucketsAggregation.Bucket.SubAggregationComparator<InternalHistogram.Bucket>(aggName, valueName, asc));
}
private static String key(String aggName, String valueName) {

@ -21,14 +21,14 @@ package org.elasticsearch.search.aggregations.bucket.missing;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.search.aggregations.AggregationStreams;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregation;
import org.elasticsearch.search.aggregations.bucket.InternalSingleBucketAggregation;
import java.io.IOException;
/**
*
*/
public class InternalMissing extends SingleBucketAggregation<InternalMissing> implements Missing {
public class InternalMissing extends InternalSingleBucketAggregation implements Missing {
public final static Type TYPE = new Type("missing");

@ -18,16 +18,10 @@
*/
package org.elasticsearch.search.aggregations.bucket.missing;
import org.elasticsearch.search.aggregations.Aggregation;
import org.elasticsearch.search.aggregations.Aggregations;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregation;
/**
*
* A {@code missing} aggregation. Defines a single bucket of all documents that are missing a specific field.
*/
public interface Missing extends Aggregation {
long getDocCount();
Aggregations getAggregations();
public interface Missing extends SingleBucketAggregation {
}

@ -21,14 +21,14 @@ package org.elasticsearch.search.aggregations.bucket.nested;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.search.aggregations.AggregationStreams;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregation;
import org.elasticsearch.search.aggregations.bucket.InternalSingleBucketAggregation;
import java.io.IOException;
/**
*
*/
public class InternalNested extends SingleBucketAggregation<InternalNested> implements Nested {
public class InternalNested extends InternalSingleBucketAggregation implements Nested {
public static final Type TYPE = new Type("nested");
@ -57,5 +57,4 @@ public class InternalNested extends SingleBucketAggregation<InternalNested> impl
return TYPE;
}
}

@ -18,16 +18,10 @@
*/
package org.elasticsearch.search.aggregations.bucket.nested;
import org.elasticsearch.search.aggregations.Aggregation;
import org.elasticsearch.search.aggregations.Aggregations;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregation;
/**
*
* A {@code nested} aggregation. Defines a single bucket that holds all the nested documents of a specific path.
*/
public interface Nested extends Aggregation {
long getDocCount();
Aggregations getAggregations();
public interface Nested extends SingleBucketAggregation {
}

@ -1,292 +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.range;
import com.google.common.collect.Lists;
import org.elasticsearch.cache.recycler.CacheRecycler;
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.Aggregations;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.support.numeric.ValueFormatter;
import org.elasticsearch.search.aggregations.support.numeric.ValueFormatterStreams;
import java.io.IOException;
import java.util.*;
/**
*
*/
public abstract class AbstractRangeBase<B extends RangeBase.Bucket> extends InternalAggregation implements RangeBase<B> {
public abstract static class Bucket implements RangeBase.Bucket {
private double from = Double.NEGATIVE_INFINITY;
private double to = Double.POSITIVE_INFINITY;
private long docCount;
private InternalAggregations aggregations;
private String key;
private boolean explicitKey;
public Bucket(String key, double from, double to, long docCount, InternalAggregations aggregations, ValueFormatter formatter) {
if (key != null) {
this.key = key;
explicitKey = true;
} else {
this.key = key(from, to, formatter);
explicitKey = false;
}
this.from = from;
this.to = to;
this.docCount = docCount;
this.aggregations = aggregations;
}
public String getKey() {
return key;
}
@Override
public double getFrom() {
return from;
}
@Override
public double getTo() {
return to;
}
@Override
public long getDocCount() {
return docCount;
}
@Override
public Aggregations getAggregations() {
return aggregations;
}
Bucket reduce(List<Bucket> ranges, CacheRecycler cacheRecycler) {
if (ranges.size() == 1) {
// we stil need to call reduce on all the sub aggregations
Bucket bucket = ranges.get(0);
bucket.aggregations.reduce(cacheRecycler);
return bucket;
}
Bucket reduced = null;
List<InternalAggregations> aggregationsList = Lists.newArrayListWithCapacity(ranges.size());
for (Bucket range : ranges) {
if (reduced == null) {
reduced = range;
} else {
reduced.docCount += range.docCount;
}
aggregationsList.add(range.aggregations);
}
reduced.aggregations = InternalAggregations.reduce(aggregationsList, cacheRecycler);
return reduced;
}
void toXContent(XContentBuilder builder, Params params, ValueFormatter formatter, boolean keyed) throws IOException {
if (keyed) {
builder.startObject(key);
} else {
builder.startObject();
if (explicitKey) {
builder.field(CommonFields.KEY, key);
}
}
if (!Double.isInfinite(from)) {
builder.field(CommonFields.FROM, from);
if (formatter != null) {
builder.field(CommonFields.FROM_AS_STRING, formatter.format(from));
}
}
if (!Double.isInfinite(to)) {
builder.field(CommonFields.TO, to);
if (formatter != null) {
builder.field(CommonFields.TO_AS_STRING, formatter.format(to));
}
}
builder.field(CommonFields.DOC_COUNT, docCount);
aggregations.toXContentInternal(builder, params);
builder.endObject();
}
private static String key(double from, double to, ValueFormatter formatter) {
StringBuilder sb = new StringBuilder();
sb.append(Double.isInfinite(from) ? "*" : formatter != null ? formatter.format(from) : from);
sb.append("-");
sb.append(Double.isInfinite(to) ? "*" : formatter != null ? formatter.format(to) : to);
return sb.toString();
}
}
public static interface Factory<B extends RangeBase.Bucket> {
public String type();
public AbstractRangeBase<B> create(String name, List<B> buckets, ValueFormatter formatter, boolean keyed);
public B createBucket(String key, double from, double to, long docCount, InternalAggregations aggregations, ValueFormatter formatter);
}
private List<B> ranges;
private Map<String, B> rangeMap;
private ValueFormatter formatter;
private boolean keyed;
private boolean unmapped;
public AbstractRangeBase() {} // for serialization
public AbstractRangeBase(String name, List<B> ranges, ValueFormatter formatter, boolean keyed) {
this(name, ranges, formatter, keyed, false);
}
public AbstractRangeBase(String name, List<B> ranges, ValueFormatter formatter, boolean keyed, boolean unmapped) {
super(name);
this.ranges = ranges;
this.formatter = formatter;
this.keyed = keyed;
this.unmapped = unmapped;
}
@Override
@SuppressWarnings("unchecked")
public Iterator<B> iterator() {
Object iter = ranges.iterator();
return (Iterator<B>) iter;
}
@Override
public B getByKey(String key) {
if (rangeMap == null) {
rangeMap = new HashMap<String, B>();
for (RangeBase.Bucket bucket : ranges) {
rangeMap.put(bucket.getKey(), (B) bucket);
}
}
return (B) rangeMap.get(key);
}
@Override
public List<B> buckets() {
return ranges;
}
@Override
public AbstractRangeBase reduce(ReduceContext reduceContext) {
List<InternalAggregation> aggregations = reduceContext.aggregations();
if (aggregations.size() == 1) {
AbstractRangeBase<B> reduced = (AbstractRangeBase<B>) aggregations.get(0);
for (B bucket : reduced.buckets()) {
((Bucket) bucket).aggregations.reduce(reduceContext.cacheRecycler());
}
return reduced;
}
List<List<Bucket>> rangesList = null;
for (InternalAggregation aggregation : aggregations) {
AbstractRangeBase<Bucket> ranges = (AbstractRangeBase) aggregation;
if (ranges.unmapped) {
continue;
}
if (rangesList == null) {
rangesList = new ArrayList<List<Bucket>>(ranges.ranges.size());
for (Bucket bucket : ranges.ranges) {
List<Bucket> sameRangeList = new ArrayList<Bucket>(aggregations.size());
sameRangeList.add(bucket);
rangesList.add(sameRangeList);
}
} else {
int i = 0;
for (Bucket range : ranges.ranges) {
rangesList.get(i++).add(range);
}
}
}
if (rangesList == null) {
// unmapped, we can just take the first one
return (AbstractRangeBase<?>) aggregations.get(0);
}
AbstractRangeBase reduced = (AbstractRangeBase) aggregations.get(0);
int i = 0;
for (List<Bucket> sameRangeList : rangesList) {
reduced.ranges.set(i++, (sameRangeList.get(0)).reduce(sameRangeList, reduceContext.cacheRecycler()));
}
return reduced;
}
protected abstract B createBucket(String key, double from, double to, long docCount, InternalAggregations aggregations, ValueFormatter formatter);
@Override
public void readFrom(StreamInput in) throws IOException {
name = in.readString();
formatter = ValueFormatterStreams.readOptional(in);
keyed = in.readBoolean();
int size = in.readVInt();
List<B> ranges = Lists.newArrayListWithCapacity(size);
for (int i = 0; i < size; i++) {
String key = in.readOptionalString();
ranges.add(createBucket(key, in.readDouble(), in.readDouble(), in.readVLong(), InternalAggregations.readAggregations(in), formatter));
}
this.ranges = ranges;
this.rangeMap = null;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
ValueFormatterStreams.writeOptional(formatter, out);
out.writeBoolean(keyed);
out.writeVInt(ranges.size());
for (B bucket : ranges) {
out.writeOptionalString(((Bucket) bucket).key);
out.writeDouble(((Bucket) bucket).from);
out.writeDouble(((Bucket) bucket).to);
out.writeVLong(((Bucket) bucket).docCount);
((Bucket) bucket).aggregations.writeTo(out);
}
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
if (keyed) {
builder.startObject(name);
} else {
builder.startArray(name);
}
for (B range : ranges) {
((Bucket) range).toXContent(builder, params, formatter, keyed);
}
if (keyed) {
builder.endObject();
} else {
builder.endArray();
}
return builder;
}
}

@ -18,18 +18,27 @@
*/
package org.elasticsearch.search.aggregations.bucket.range;
import com.google.common.collect.Lists;
import org.elasticsearch.cache.recycler.CacheRecycler;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.AggregationStreams;
import org.elasticsearch.search.aggregations.Aggregations;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.support.numeric.ValueFormatter;
import org.elasticsearch.search.aggregations.support.numeric.ValueFormatterStreams;
import java.io.IOException;
import java.util.List;
import java.util.*;
/**
*
*/
public class InternalRange extends AbstractRangeBase<Range.Bucket> implements Range {
public class InternalRange<B extends InternalRange.Bucket> extends InternalAggregation implements Range {
static final Factory FACTORY = new Factory();
@ -37,7 +46,7 @@ public class InternalRange extends AbstractRangeBase<Range.Bucket> implements Ra
private final static AggregationStreams.Stream STREAM = new AggregationStreams.Stream() {
@Override
public AbstractRangeBase readResult(StreamInput in) throws IOException {
public InternalRange readResult(StreamInput in) throws IOException {
InternalRange ranges = new InternalRange();
ranges.readFrom(in);
return ranges;
@ -48,40 +57,146 @@ public class InternalRange extends AbstractRangeBase<Range.Bucket> implements Ra
AggregationStreams.registerStream(STREAM, TYPE.stream());
}
public static class Bucket extends AbstractRangeBase.Bucket implements Range.Bucket {
public static class Bucket implements Range.Bucket {
private double from = Double.NEGATIVE_INFINITY;
private double to = Double.POSITIVE_INFINITY;
private long docCount;
private InternalAggregations aggregations;
private String key;
private boolean explicitKey;
public Bucket(String key, double from, double to, long docCount, InternalAggregations aggregations, ValueFormatter formatter) {
super(key, from, to, docCount, aggregations, formatter);
if (key != null) {
this.key = key;
explicitKey = true;
} else {
this.key = key(from, to, formatter);
explicitKey = false;
}
this.from = from;
this.to = to;
this.docCount = docCount;
this.aggregations = aggregations;
}
}
public static class Factory implements AbstractRangeBase.Factory<Range.Bucket> {
public String getKey() {
return key;
}
@Override
public Text getKeyAsText() {
return new StringText(getKey());
}
@Override
public Number getFrom() {
return from;
}
@Override
public Number getTo() {
return to;
}
@Override
public long getDocCount() {
return docCount;
}
@Override
public Aggregations getAggregations() {
return aggregations;
}
Bucket reduce(List<Bucket> ranges, CacheRecycler cacheRecycler) {
if (ranges.size() == 1) {
// we stil need to call reduce on all the sub aggregations
Bucket bucket = ranges.get(0);
bucket.aggregations.reduce(cacheRecycler);
return bucket;
}
Bucket reduced = null;
List<InternalAggregations> aggregationsList = Lists.newArrayListWithCapacity(ranges.size());
for (Bucket range : ranges) {
if (reduced == null) {
reduced = range;
} else {
reduced.docCount += range.docCount;
}
aggregationsList.add(range.aggregations);
}
reduced.aggregations = InternalAggregations.reduce(aggregationsList, cacheRecycler);
return reduced;
}
void toXContent(XContentBuilder builder, Params params, ValueFormatter formatter, boolean keyed) throws IOException {
if (keyed) {
builder.startObject(key);
} else {
builder.startObject();
if (explicitKey) {
builder.field(CommonFields.KEY, key);
}
}
if (!Double.isInfinite(from)) {
builder.field(CommonFields.FROM, from);
if (formatter != null) {
builder.field(CommonFields.FROM_AS_STRING, formatter.format(from));
}
}
if (!Double.isInfinite(to)) {
builder.field(CommonFields.TO, to);
if (formatter != null) {
builder.field(CommonFields.TO_AS_STRING, formatter.format(to));
}
}
builder.field(CommonFields.DOC_COUNT, docCount);
aggregations.toXContentInternal(builder, params);
builder.endObject();
}
private static String key(double from, double to, ValueFormatter formatter) {
StringBuilder sb = new StringBuilder();
sb.append(Double.isInfinite(from) ? "*" : formatter != null ? formatter.format(from) : from);
sb.append("-");
sb.append(Double.isInfinite(to) ? "*" : formatter != null ? formatter.format(to) : to);
return sb.toString();
}
}
public static class Factory<B extends Bucket, R extends InternalRange<B>> {
public String type() {
return TYPE.name();
}
@Override
public AbstractRangeBase<Range.Bucket> create(String name, List<Range.Bucket> ranges, ValueFormatter formatter, boolean keyed) {
return new InternalRange(name, ranges, formatter, keyed);
public R create(String name, List<B> ranges, ValueFormatter formatter, boolean keyed, boolean unmapped) {
return (R) new InternalRange<B>(name, ranges, formatter, keyed, unmapped);
}
public Range.Bucket createBucket(String key, double from, double to, long docCount, InternalAggregations aggregations, ValueFormatter formatter) {
return new Bucket(key, from, to, docCount, aggregations, formatter);
public B createBucket(String key, double from, double to, long docCount, InternalAggregations aggregations, ValueFormatter formatter) {
return (B) new Bucket(key, from, to, docCount, aggregations, formatter);
}
}
protected Range.Bucket createBucket(String key, double from, double to, long docCount, InternalAggregations aggregations, ValueFormatter formatter) {
return new Bucket(key, from, to, docCount, aggregations, formatter);
}
private List<B> ranges;
private Map<String, B> rangeMap;
private ValueFormatter formatter;
private boolean keyed;
private boolean unmapped;
public InternalRange() {} // for serialization
public InternalRange(String name, List<Range.Bucket> ranges, ValueFormatter formatter, boolean keyed) {
super(name, ranges, formatter, keyed);
public InternalRange(String name, List<B> ranges, ValueFormatter formatter, boolean keyed, boolean unmapped) {
super(name);
this.ranges = ranges;
this.formatter = formatter;
this.keyed = keyed;
this.unmapped = unmapped;
}
@Override
@ -89,4 +204,116 @@ public class InternalRange extends AbstractRangeBase<Range.Bucket> implements Ra
return TYPE;
}
@Override
public Collection<B> getBuckets() {
return ranges;
}
@Override
public B getBucketByKey(String key) {
if (rangeMap == null) {
rangeMap = new HashMap<String, B>(ranges.size());
for (Range.Bucket bucket : ranges) {
rangeMap.put(bucket.getKey(), (B) bucket);
}
}
return rangeMap.get(key);
}
@Override
public InternalAggregation reduce(ReduceContext reduceContext) {
List<InternalAggregation> aggregations = reduceContext.aggregations();
if (aggregations.size() == 1) {
InternalRange<B> reduced = (InternalRange<B>) aggregations.get(0);
for (B bucket : reduced.ranges) {
bucket.aggregations.reduce(reduceContext.cacheRecycler());
}
return reduced;
}
List<List<Bucket>> rangesList = null;
for (InternalAggregation aggregation : aggregations) {
InternalRange<Bucket> ranges = (InternalRange) aggregation;
if (ranges.unmapped) {
continue;
}
if (rangesList == null) {
rangesList = new ArrayList<List<Bucket>>(ranges.ranges.size());
for (Bucket bucket : ranges.ranges) {
List<Bucket> sameRangeList = new ArrayList<Bucket>(aggregations.size());
sameRangeList.add(bucket);
rangesList.add(sameRangeList);
}
} else {
int i = 0;
for (Bucket range : ranges.ranges) {
rangesList.get(i++).add(range);
}
}
}
if (rangesList == null) {
// unmapped, we can just take the first one
return aggregations.get(0);
}
InternalRange reduced = (InternalRange) aggregations.get(0);
int i = 0;
for (List<Bucket> sameRangeList : rangesList) {
reduced.ranges.set(i++, (sameRangeList.get(0)).reduce(sameRangeList, reduceContext.cacheRecycler()));
}
return reduced;
}
protected B createBucket(String key, double from, double to, long docCount, InternalAggregations aggregations, ValueFormatter formatter) {
return (B) new Bucket(key, from, to, docCount, aggregations, formatter);
}
@Override
public void readFrom(StreamInput in) throws IOException {
name = in.readString();
formatter = ValueFormatterStreams.readOptional(in);
keyed = in.readBoolean();
int size = in.readVInt();
List<B> ranges = Lists.newArrayListWithCapacity(size);
for (int i = 0; i < size; i++) {
String key = in.readOptionalString();
ranges.add(createBucket(key, in.readDouble(), in.readDouble(), in.readVLong(), InternalAggregations.readAggregations(in), formatter));
}
this.ranges = ranges;
this.rangeMap = null;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
ValueFormatterStreams.writeOptional(formatter, out);
out.writeBoolean(keyed);
out.writeVInt(ranges.size());
for (B bucket : ranges) {
out.writeOptionalString(((Bucket) bucket).key);
out.writeDouble(((Bucket) bucket).from);
out.writeDouble(((Bucket) bucket).to);
out.writeVLong(((Bucket) bucket).docCount);
((Bucket) bucket).aggregations.writeTo(out);
}
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
if (keyed) {
builder.startObject(name);
} else {
builder.startArray(name);
}
for (B range : ranges) {
((Bucket) range).toXContent(builder, params, formatter, keyed);
}
if (keyed) {
builder.endObject();
} else {
builder.endArray();
}
return builder;
}
}

@ -18,11 +18,36 @@
*/
package org.elasticsearch.search.aggregations.bucket.range;
/**
*
*/
public interface Range extends RangeBase<Range.Bucket> {
import org.elasticsearch.search.aggregations.Aggregation;
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
static interface Bucket extends RangeBase.Bucket {
import java.util.Collection;
/**
* A {@code range} aggregation. Defines multiple buckets, each associated with a pre-defined value range of a field,
* and where the value of that fields in all documents in each bucket fall in the bucket's range.
*/
public interface Range extends MultiBucketsAggregation {
/**
* A bucket associated with a specific range
*/
public static interface Bucket extends MultiBucketsAggregation.Bucket {
/**
* @return The lower bound of the range
*/
Number getFrom();
/**
* @return The upper bound of the range (excluding)
*/
Number getTo();
}
Collection<? extends Bucket> getBuckets();
@Override
Bucket getBucketByKey(String key);
}

@ -81,7 +81,7 @@ public class RangeAggregator extends BucketsAggregator {
private final NumericValuesSource valuesSource;
private final Range[] ranges;
private final boolean keyed;
private final AbstractRangeBase.Factory rangeFactory;
private final InternalRange.Factory rangeFactory;
private DoubleValues values;
final double[] maxTo;
@ -89,7 +89,7 @@ public class RangeAggregator extends BucketsAggregator {
public RangeAggregator(String name,
AggregatorFactories factories,
NumericValuesSource valuesSource,
AbstractRangeBase.Factory rangeFactory,
InternalRange.Factory rangeFactory,
List<Range> ranges,
boolean keyed,
AggregationContext aggregationContext,
@ -188,31 +188,32 @@ public class RangeAggregator extends BucketsAggregator {
@Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
List<RangeBase.Bucket> buckets = Lists.newArrayListWithCapacity(ranges.length);
List<org.elasticsearch.search.aggregations.bucket.range.Range.Bucket> buckets = Lists.newArrayListWithCapacity(ranges.length);
for (int i = 0; i < ranges.length; i++) {
Range range = ranges[i];
final long bucketOrd = subBucketOrdinal(owningBucketOrdinal, i);
RangeBase.Bucket bucket = rangeFactory.createBucket(range.key, range.from, range.to, bucketDocCount(bucketOrd),
bucketAggregations(bucketOrd), valuesSource.formatter());
org.elasticsearch.search.aggregations.bucket.range.Range.Bucket bucket = rangeFactory.createBucket(
range.key, range.from, range.to, bucketDocCount(bucketOrd),bucketAggregations(bucketOrd), valuesSource.formatter());
buckets.add(bucket);
}
// value source can be null in the case of unmapped fields
ValueFormatter formatter = valuesSource != null ? valuesSource.formatter() : null;
return rangeFactory.create(name, buckets, formatter, keyed);
return rangeFactory.create(name, buckets, formatter, keyed, false);
}
@Override
public InternalAggregation buildEmptyAggregation() {
InternalAggregations subAggs = buildEmptySubAggregations();
List<RangeBase.Bucket> buckets = Lists.newArrayListWithCapacity(ranges.length);
List<org.elasticsearch.search.aggregations.bucket.range.Range.Bucket> buckets = Lists.newArrayListWithCapacity(ranges.length);
for (int i = 0; i < ranges.length; i++) {
Range range = ranges[i];
RangeBase.Bucket bucket = rangeFactory.createBucket(range.key, range.from, range.to, 0, subAggs, valuesSource.formatter());
org.elasticsearch.search.aggregations.bucket.range.Range.Bucket bucket = rangeFactory.createBucket(
range.key, range.from, range.to, 0, subAggs, valuesSource.formatter());
buckets.add(bucket);
}
// value source can be null in the case of unmapped fields
ValueFormatter formatter = valuesSource != null ? valuesSource.formatter() : null;
return rangeFactory.create(name, buckets, formatter, keyed);
return rangeFactory.create(name, buckets, formatter, keyed, false);
}
private static final void sortRanges(final Range[] ranges) {
@ -240,7 +241,7 @@ public class RangeAggregator extends BucketsAggregator {
private final List<RangeAggregator.Range> ranges;
private final boolean keyed;
private final AbstractRangeBase.Factory factory;
private final InternalRange.Factory factory;
private final ValueFormatter formatter;
private final ValueParser parser;
@ -251,7 +252,7 @@ public class RangeAggregator extends BucketsAggregator {
ValueParser parser,
AggregationContext aggregationContext,
Aggregator parent,
AbstractRangeBase.Factory factory) {
InternalRange.Factory factory) {
super(name, BucketAggregationMode.MULTI_BUCKETS, AggregatorFactories.EMPTY, 0, aggregationContext, parent);
this.ranges = ranges;
@ -278,28 +279,29 @@ public class RangeAggregator extends BucketsAggregator {
}
@Override
public AbstractRangeBase buildAggregation(long owningBucketOrdinal) {
return (AbstractRangeBase) buildEmptyAggregation();
public InternalRange buildAggregation(long owningBucketOrdinal) {
return buildEmptyAggregation();
}
@Override
public AbstractRangeBase buildEmptyAggregation() {
public InternalRange buildEmptyAggregation() {
InternalAggregations subAggs = buildEmptySubAggregations();
List<RangeBase.Bucket> buckets = new ArrayList<RangeBase.Bucket>(ranges.size());
List<org.elasticsearch.search.aggregations.bucket.range.Range.Bucket> buckets =
new ArrayList<org.elasticsearch.search.aggregations.bucket.range.Range.Bucket>(ranges.size());
for (RangeAggregator.Range range : ranges) {
buckets.add(factory.createBucket(range.key, range.from, range.to, 0, subAggs, formatter));
}
return factory.create(name, buckets, formatter, keyed);
return factory.create(name, buckets, formatter, keyed, true);
}
}
public static class Factory extends ValueSourceAggregatorFactory<NumericValuesSource> {
private final AbstractRangeBase.Factory rangeFactory;
private final InternalRange.Factory rangeFactory;
private final List<Range> ranges;
private final boolean keyed;
public Factory(String name, ValuesSourceConfig<NumericValuesSource> valueSourceConfig, AbstractRangeBase.Factory rangeFactory, List<Range> ranges, boolean keyed) {
public Factory(String name, ValuesSourceConfig<NumericValuesSource> valueSourceConfig, InternalRange.Factory rangeFactory, List<Range> ranges, boolean keyed) {
super(name, rangeFactory.type(), valueSourceConfig);
this.rangeFactory = rangeFactory;
this.ranges = ranges;

@ -1,43 +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.range;
import org.elasticsearch.search.aggregations.Aggregation;
import java.util.List;
/**
*
*/
public interface RangeBase<B extends RangeBase.Bucket> extends Aggregation, Iterable<B> {
public static interface Bucket extends org.elasticsearch.search.aggregations.bucket.Bucket {
String getKey();
double getFrom();
double getTo();
}
List<B> buckets();
B getByKey(String key);
}

@ -18,19 +18,27 @@
*/
package org.elasticsearch.search.aggregations.bucket.range.date;
import org.elasticsearch.search.aggregations.bucket.range.RangeBase;
import org.elasticsearch.search.aggregations.bucket.range.Range;
import org.joda.time.DateTime;
import java.util.Collection;
/**
*
*/
public interface DateRange extends RangeBase<DateRange.Bucket> {
public interface DateRange extends Range {
static interface Bucket extends RangeBase.Bucket {
static interface Bucket extends Range.Bucket {
DateTime getFromAsDate();
DateTime getToAsDate();
}
@Override
Collection<? extends DateRange.Bucket> getBuckets();
@Override
DateRange.Bucket getBucketByKey(String key);
}

@ -22,7 +22,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.search.aggregations.AggregationStreams;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.bucket.range.AbstractRangeBase;
import org.elasticsearch.search.aggregations.bucket.range.InternalRange;
import org.elasticsearch.search.aggregations.support.numeric.ValueFormatter;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
@ -33,13 +33,13 @@ import java.util.List;
/**
*
*/
public class InternalDateRange extends AbstractRangeBase<DateRange.Bucket> implements DateRange {
public class InternalDateRange extends InternalRange<InternalDateRange.Bucket> implements DateRange {
public final static Type TYPE = new Type("date_range", "drange");
private final static AggregationStreams.Stream STREAM = new AggregationStreams.Stream() {
@Override
public AbstractRangeBase<?> readResult(StreamInput in) throws IOException {
public InternalDateRange readResult(StreamInput in) throws IOException {
InternalDateRange ranges = new InternalDateRange();
ranges.readFrom(in);
return ranges;
@ -52,7 +52,7 @@ public class InternalDateRange extends AbstractRangeBase<DateRange.Bucket> imple
public static final Factory FACTORY = new Factory();
public static class Bucket extends AbstractRangeBase.Bucket implements DateRange.Bucket {
public static class Bucket extends InternalRange.Bucket implements DateRange.Bucket {
public Bucket(String key, double from, double to, long docCount, List<InternalAggregation> aggregations, ValueFormatter formatter) {
super(key, from, to, docCount, new InternalAggregations(aggregations), formatter);
@ -64,16 +64,16 @@ public class InternalDateRange extends AbstractRangeBase<DateRange.Bucket> imple
@Override
public DateTime getFromAsDate() {
return Double.isInfinite(getFrom()) ? null : new DateTime((long) getFrom(), DateTimeZone.UTC);
return Double.isInfinite(getFrom().doubleValue()) ? null : new DateTime(getFrom().longValue(), DateTimeZone.UTC);
}
@Override
public DateTime getToAsDate() {
return Double.isInfinite(getTo()) ? null : new DateTime((long) getTo(), DateTimeZone.UTC);
return Double.isInfinite(getTo().doubleValue()) ? null : new DateTime(getTo().longValue(), DateTimeZone.UTC);
}
}
private static class Factory implements AbstractRangeBase.Factory<DateRange.Bucket> {
private static class Factory extends InternalRange.Factory<InternalDateRange.Bucket, InternalDateRange> {
@Override
public String type() {
@ -81,8 +81,8 @@ public class InternalDateRange extends AbstractRangeBase<DateRange.Bucket> imple
}
@Override
public AbstractRangeBase<DateRange.Bucket> create(String name, List<DateRange.Bucket> buckets, ValueFormatter formatter, boolean keyed) {
return new InternalDateRange(name, buckets, formatter, keyed);
public InternalDateRange create(String name, List<InternalDateRange.Bucket> ranges, ValueFormatter formatter, boolean keyed, boolean unmapped) {
return new InternalDateRange(name, ranges, formatter, keyed, unmapped);
}
@Override
@ -91,20 +91,19 @@ public class InternalDateRange extends AbstractRangeBase<DateRange.Bucket> imple
}
}
public InternalDateRange() {
}
InternalDateRange() {} // for serialization
public InternalDateRange(String name, List<DateRange.Bucket> ranges, ValueFormatter formatter, boolean keyed) {
super(name, ranges, formatter, keyed);
}
@Override
protected Bucket createBucket(String key, double from, double to, long docCount, InternalAggregations aggregations, ValueFormatter formatter) {
return new Bucket(key, from, to, docCount, aggregations, formatter);
InternalDateRange(String name, List<InternalDateRange.Bucket> ranges, ValueFormatter formatter, boolean keyed, boolean unmapped) {
super(name, ranges, formatter, keyed, unmapped);
}
@Override
public Type type() {
return TYPE;
}
@Override
protected Bucket createBucket(String key, double from, double to, long docCount, InternalAggregations aggregations, ValueFormatter formatter) {
return new Bucket(key, from, to, docCount, aggregations, formatter);
}
}

@ -18,13 +18,21 @@
*/
package org.elasticsearch.search.aggregations.bucket.range.geodistance;
import org.elasticsearch.search.aggregations.bucket.range.RangeBase;
import org.elasticsearch.search.aggregations.bucket.range.Range;
import java.util.Collection;
/**
*
*/
public interface GeoDistance extends RangeBase<GeoDistance.Bucket> {
public interface GeoDistance extends Range {
public static interface Bucket extends RangeBase.Bucket {}
public static interface Bucket extends Range.Bucket {
}
@Override
Collection<? extends GeoDistance.Bucket> getBuckets();
@Override
GeoDistance.Bucket getBucketByKey(String key);
}

@ -27,7 +27,7 @@ import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.bucket.range.AbstractRangeBase;
import org.elasticsearch.search.aggregations.bucket.range.InternalRange;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Unmapped;
import org.elasticsearch.search.aggregations.support.*;
@ -200,12 +200,12 @@ public class GeoDistanceParser implements Aggregator.Parser {
private final GeoPoint origin;
private final DistanceUnit unit;
private final GeoDistance distanceType;
private final AbstractRangeBase.Factory rangeFactory;
private final InternalRange.Factory rangeFactory;
private final List<RangeAggregator.Range> ranges;
private final boolean keyed;
public GeoDistanceFactory(String name, ValuesSourceConfig<GeoPointValuesSource> valueSourceConfig,
AbstractRangeBase.Factory rangeFactory, GeoPoint origin, DistanceUnit unit, GeoDistance distanceType,
InternalRange.Factory rangeFactory, GeoPoint origin, DistanceUnit unit, GeoDistance distanceType,
List<RangeAggregator.Range> ranges, boolean keyed) {
super(name, rangeFactory.type(), valueSourceConfig);
this.origin = origin;

@ -22,7 +22,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.search.aggregations.AggregationStreams;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.bucket.range.AbstractRangeBase;
import org.elasticsearch.search.aggregations.bucket.range.InternalRange;
import org.elasticsearch.search.aggregations.support.numeric.ValueFormatter;
import java.io.IOException;
@ -31,7 +31,7 @@ import java.util.List;
/**
*
*/
public class InternalGeoDistance extends AbstractRangeBase<GeoDistance.Bucket> implements GeoDistance {
public class InternalGeoDistance extends InternalRange<InternalGeoDistance.Bucket> implements GeoDistance {
public static final Type TYPE = new Type("geo_distance", "gdist");
@ -50,7 +50,7 @@ public class InternalGeoDistance extends AbstractRangeBase<GeoDistance.Bucket> i
public static final Factory FACTORY = new Factory();
static class Bucket extends AbstractRangeBase.Bucket implements GeoDistance.Bucket {
static class Bucket extends InternalRange.Bucket implements GeoDistance.Bucket {
Bucket(String key, double from, double to, long docCount, List<InternalAggregation> aggregations, ValueFormatter formatter) {
this(key, from, to, docCount, new InternalAggregations(aggregations), formatter);
@ -62,7 +62,7 @@ public class InternalGeoDistance extends AbstractRangeBase<GeoDistance.Bucket> i
}
private static class Factory implements AbstractRangeBase.Factory<GeoDistance.Bucket> {
private static class Factory extends InternalRange.Factory<InternalGeoDistance.Bucket, InternalGeoDistance> {
@Override
public String type() {
@ -70,8 +70,8 @@ public class InternalGeoDistance extends AbstractRangeBase<GeoDistance.Bucket> i
}
@Override
public AbstractRangeBase<GeoDistance.Bucket> create(String name, List<GeoDistance.Bucket> buckets, ValueFormatter formatter, boolean keyed) {
return new InternalGeoDistance(name, buckets, formatter, keyed);
public InternalGeoDistance create(String name, List<Bucket> ranges, ValueFormatter formatter, boolean keyed, boolean unmapped) {
return new InternalGeoDistance(name, ranges, formatter, keyed, unmapped);
}
@Override
@ -82,17 +82,17 @@ public class InternalGeoDistance extends AbstractRangeBase<GeoDistance.Bucket> i
InternalGeoDistance() {} // for serialization
public InternalGeoDistance(String name, List<GeoDistance.Bucket> ranges, ValueFormatter formatter, boolean keyed) {
super(name, ranges, formatter, keyed);
}
@Override
protected Bucket createBucket(String key, double from, double to, long docCount, InternalAggregations aggregations, ValueFormatter formatter) {
return new Bucket(key, from, to, docCount, aggregations, formatter);
public InternalGeoDistance(String name, List<Bucket> ranges, ValueFormatter formatter, boolean keyed, boolean unmapped) {
super(name, ranges, formatter, keyed, unmapped);
}
@Override
public Type type() {
return TYPE;
}
@Override
protected Bucket createBucket(String key, double from, double to, long docCount, InternalAggregations aggregations, ValueFormatter formatter) {
return new Bucket(key, from, to, docCount, aggregations, formatter);
}
}

@ -18,18 +18,26 @@
*/
package org.elasticsearch.search.aggregations.bucket.range.ipv4;
import org.elasticsearch.search.aggregations.bucket.range.RangeBase;
import org.elasticsearch.search.aggregations.bucket.range.Range;
import java.util.Collection;
/**
*
*/
public interface IPv4Range extends RangeBase<IPv4Range.Bucket> {
public interface IPv4Range extends Range {
static interface Bucket extends RangeBase.Bucket {
static interface Bucket extends Range.Bucket {
String getFromAsString();
String getToAsString();
}
@Override
Collection<? extends IPv4Range.Bucket> getBuckets();
@Override
IPv4Range.Bucket getBucketByKey(String key);
}

@ -29,7 +29,6 @@ import java.util.regex.Pattern;
*/
public class IPv4RangeBuilder extends AbstractRangeBuilder<IPv4RangeBuilder> {
public static final long MAX_IP = 4294967296l;
private static final Pattern MASK_PATTERN = Pattern.compile("[\\.|/]");
public IPv4RangeBuilder(String name) {
@ -113,8 +112,8 @@ public class IPv4RangeBuilder extends AbstractRangeBuilder<IPv4RangeBuilder> {
}
int to = from + (~mask);
long longTo = intIpToLongIp(to) + 1; // we have to +1 the here as the range is non-inclusive on the "to" side
if (longTo == MAX_IP) {
long longTo = intIpToLongIp(to) + 1; // we have to +1 here as the range is non-inclusive on the "to" side
if (longTo == InternalIPv4Range.MAX_IP) {
longTo = -1;
}

@ -22,7 +22,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.search.aggregations.AggregationStreams;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.bucket.range.AbstractRangeBase;
import org.elasticsearch.search.aggregations.bucket.range.InternalRange;
import org.elasticsearch.search.aggregations.support.numeric.ValueFormatter;
import java.io.IOException;
@ -31,7 +31,7 @@ import java.util.List;
/**
*
*/
public class InternalIPv4Range extends AbstractRangeBase<IPv4Range.Bucket> implements IPv4Range {
public class InternalIPv4Range extends InternalRange<InternalIPv4Range.Bucket> implements IPv4Range {
public static final long MAX_IP = 4294967296l;
@ -39,7 +39,7 @@ public class InternalIPv4Range extends AbstractRangeBase<IPv4Range.Bucket> imple
private final static AggregationStreams.Stream STREAM = new AggregationStreams.Stream() {
@Override
public AbstractRangeBase<?> readResult(StreamInput in) throws IOException {
public InternalIPv4Range readResult(StreamInput in) throws IOException {
InternalIPv4Range range = new InternalIPv4Range();
range.readFrom(in);
return range;
@ -52,7 +52,7 @@ public class InternalIPv4Range extends AbstractRangeBase<IPv4Range.Bucket> imple
public static final Factory FACTORY = new Factory();
public static class Bucket extends AbstractRangeBase.Bucket implements IPv4Range.Bucket {
public static class Bucket extends InternalRange.Bucket implements IPv4Range.Bucket {
public Bucket(String key, double from, double to, long docCount, List<InternalAggregation> aggregations, ValueFormatter formatter) {
super(key, from, to, docCount, new InternalAggregations(aggregations), formatter);
@ -64,16 +64,18 @@ public class InternalIPv4Range extends AbstractRangeBase<IPv4Range.Bucket> imple
@Override
public String getFromAsString() {
return Double.isInfinite(getFrom()) ? null : getFrom() == 0 ? null : ValueFormatter.IPv4.format(getFrom());
double from = getFrom().doubleValue();
return Double.isInfinite(from) ? null : from == 0 ? null : ValueFormatter.IPv4.format(from);
}
@Override
public String getToAsString() {
return Double.isInfinite(getTo()) ? null : MAX_IP == getTo() ? null : ValueFormatter.IPv4.format(getTo());
double to = getTo().doubleValue();
return Double.isInfinite(to) ? null : MAX_IP == to ? null : ValueFormatter.IPv4.format(to);
}
}
private static class Factory implements AbstractRangeBase.Factory<IPv4Range.Bucket> {
private static class Factory extends InternalRange.Factory<InternalIPv4Range.Bucket, InternalIPv4Range> {
@Override
public String type() {
@ -81,8 +83,8 @@ public class InternalIPv4Range extends AbstractRangeBase<IPv4Range.Bucket> imple
}
@Override
public AbstractRangeBase<IPv4Range.Bucket> create(String name, List<IPv4Range.Bucket> buckets, ValueFormatter formatter, boolean keyed) {
return new InternalIPv4Range(name, buckets, keyed);
public InternalIPv4Range create(String name, List<Bucket> ranges, ValueFormatter formatter, boolean keyed, boolean unmapped) {
return new InternalIPv4Range(name, ranges, keyed, unmapped);
}
@Override
@ -91,16 +93,10 @@ public class InternalIPv4Range extends AbstractRangeBase<IPv4Range.Bucket> imple
}
}
public InternalIPv4Range() {
}
public InternalIPv4Range() {} // for serialization
public InternalIPv4Range(String name, List<IPv4Range.Bucket> ranges, boolean keyed) {
super(name, ranges, ValueFormatter.IPv4, keyed);
}
@Override
protected Bucket createBucket(String key, double from, double to, long docCount, InternalAggregations aggregations, ValueFormatter formatter) {
return new Bucket(key, from, to, docCount, aggregations, formatter);
public InternalIPv4Range(String name, List<InternalIPv4Range.Bucket> ranges, boolean keyed, boolean unmapped) {
super(name, ranges, ValueFormatter.IPv4, keyed, unmapped);
}
@Override
@ -108,4 +104,9 @@ public class InternalIPv4Range extends AbstractRangeBase<IPv4Range.Bucket> imple
return TYPE;
}
@Override
protected Bucket createBucket(String key, double from, double to, long docCount, InternalAggregations aggregations, ValueFormatter formatter) {
return new Bucket(key, from, to, docCount, aggregations, formatter);
}
}

@ -68,7 +68,12 @@ public class DoubleTerms extends InternalTerms {
}
@Override
public Text getKey() {
public String getKey() {
return String.valueOf(term);
}
@Override
public Text getKeyAsText() {
return new StringText(String.valueOf(term));
}

@ -24,7 +24,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.bucket.Bucket;
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
import java.io.IOException;
@ -165,11 +165,11 @@ class InternalOrder extends Terms.Order {
static final byte ID = 0;
Aggregation(String key, boolean asc) {
super(ID, key, asc, new Bucket.SubAggregationComparator<Terms.Bucket>(key, asc));
super(ID, key, asc, new MultiBucketsAggregation.Bucket.SubAggregationComparator<Terms.Bucket>(key, asc));
}
Aggregation(String aggName, String metricName, boolean asc) {
super(ID, key(aggName, metricName), asc, new Bucket.SubAggregationComparator<Terms.Bucket>(aggName, metricName, asc));
super(ID, key(aggName, metricName), asc, new MultiBucketsAggregation.Bucket.SubAggregationComparator<Terms.Bucket>(aggName, metricName, asc));
}
String aggName() {
@ -251,12 +251,12 @@ class InternalOrder extends Terms.Order {
public static void writeOrder(InternalOrder order, StreamOutput out) throws IOException {
out.writeByte(order.id());
if (order instanceof Aggregation) {
out.writeBoolean(((Bucket.SubAggregationComparator) order.comparator).asc());
out.writeString(((Bucket.SubAggregationComparator) order.comparator).aggName());
boolean hasValueName = ((Bucket.SubAggregationComparator) order.comparator).valueName() != null;
out.writeBoolean(((MultiBucketsAggregation.Bucket.SubAggregationComparator) order.comparator).asc());
out.writeString(((MultiBucketsAggregation.Bucket.SubAggregationComparator) order.comparator).aggName());
boolean hasValueName = ((MultiBucketsAggregation.Bucket.SubAggregationComparator) order.comparator).valueName() != null;
out.writeBoolean(hasValueName);
if (hasValueName) {
out.writeString(((Bucket.SubAggregationComparator) order.comparator).valueName());
out.writeString(((MultiBucketsAggregation.Bucket.SubAggregationComparator) order.comparator).valueName());
}
}
}

@ -99,23 +99,17 @@ public abstract class InternalTerms extends InternalAggregation implements Terms
}
@Override
public Iterator<Terms.Bucket> iterator() {
Object o = buckets.iterator();
return (Iterator<Terms.Bucket>) o;
}
@Override
public Collection<Terms.Bucket> buckets() {
public Collection<Terms.Bucket> getBuckets() {
Object o = buckets;
return (Collection<Terms.Bucket>) o;
}
@Override
public Terms.Bucket getByTerm(String term) {
public Terms.Bucket getBucketByKey(String term) {
if (bucketMap == null) {
bucketMap = Maps.newHashMapWithExpectedSize(buckets.size());
for (Bucket bucket : buckets) {
bucketMap.put(bucket.getKey().string(), bucket);
bucketMap.put(bucket.getKey(), bucket);
}
}
return bucketMap.get(term);
@ -145,10 +139,10 @@ public abstract class InternalTerms extends InternalAggregation implements Terms
buckets = new HashMap<Text, List<Bucket>>(terms.buckets.size());
}
for (Bucket bucket : terms.buckets) {
List<Bucket> existingBuckets = buckets.get(bucket.getKey());
List<Bucket> existingBuckets = buckets.get(bucket.getKeyAsText());
if (existingBuckets == null) {
existingBuckets = new ArrayList<Bucket>(aggregations.size());
buckets.put(bucket.getKey(), existingBuckets);
buckets.put(bucket.getKeyAsText(), existingBuckets);
}
existingBuckets.add(bucket);
}

@ -70,7 +70,12 @@ public class LongTerms extends InternalTerms {
}
@Override
public Text getKey() {
public String getKey() {
return String.valueOf(term);
}
@Override
public Text getKeyAsText() {
return new StringText(String.valueOf(term));
}

@ -65,7 +65,12 @@ public class StringTerms extends InternalTerms {
}
@Override
public Text getKey() {
public String getKey() {
return termBytes.utf8ToString();
}
@Override
public Text getKeyAsText() {
return new BytesText(new BytesArray(termBytes));
}

@ -18,19 +18,19 @@
*/
package org.elasticsearch.search.aggregations.bucket.terms;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.search.aggregations.Aggregation;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
import org.elasticsearch.search.aggregations.support.ScriptValueType;
import java.util.Collection;
import java.util.Comparator;
/**
*
* A {@code terms} aggregation. Defines multiple bucket, each associated with a unique term for a specific field.
* All documents in a bucket has the bucket's term in that field.
*/
public interface Terms extends Aggregation, Iterable<Terms.Bucket> {
public interface Terms extends MultiBucketsAggregation {
static enum ValueType {
@ -58,9 +58,10 @@ public interface Terms extends Aggregation, Iterable<Terms.Bucket> {
}
}
static abstract class Bucket implements org.elasticsearch.search.aggregations.bucket.Bucket {
public abstract Text getKey();
/**
* A bucket that is associated with a single term
*/
static abstract class Bucket implements MultiBucketsAggregation.Bucket {
public abstract Number getKeyAsNumber();
@ -68,13 +69,12 @@ public interface Terms extends Aggregation, Iterable<Terms.Bucket> {
}
Collection<Bucket> buckets();
Bucket getByTerm(String term);
Collection<Bucket> getBuckets();
Bucket getBucketByKey(String term);
/**
*
* Determines the order by which the term buckets will be sorted
*/
static abstract class Order implements ToXContent {

@ -62,8 +62,8 @@ public class CopyToMapperIntegrationTests extends ElasticsearchIntegrationTest {
assertThat(response.getHits().totalHits(), equalTo((long) recordCount));
assertThat(((Terms) response.getAggregations().get("test")).buckets().size(), equalTo(recordCount + 1));
assertThat(((Terms) response.getAggregations().get("test_raw")).buckets().size(), equalTo(recordCount));
assertThat(((Terms) response.getAggregations().get("test")).getBuckets().size(), equalTo(recordCount + 1));
assertThat(((Terms) response.getAggregations().get("test_raw")).getBuckets().size(), equalTo(recordCount));
}

@ -108,9 +108,9 @@ public class PercolatorFacetsAndAggregationsTests extends ElasticsearchIntegrati
List<Aggregation> aggregations = response.getAggregations().asList();
assertThat(aggregations.size(), equalTo(1));
assertThat(aggregations.get(0).getName(), equalTo("a"));
List<Terms.Bucket> buckets = new ArrayList<Terms.Bucket>(((Terms) aggregations.get(0)).buckets());
List<Terms.Bucket> buckets = new ArrayList<Terms.Bucket>(((Terms) aggregations.get(0)).getBuckets());
assertThat(buckets.size(), equalTo(1));
assertThat(buckets.get(0).getKey().string(), equalTo("b"));
assertThat(buckets.get(0).getKeyAsText().string(), equalTo("b"));
assertThat(buckets.get(0).getDocCount(), equalTo((long) expectedCount[i % values.length]));
} else {
assertThat(response.getFacets().facets().size(), equalTo(1));

@ -53,6 +53,8 @@ public class RandomTests extends ElasticsearchIntegrationTest {
.build();
}
// Make sure that unordered, reversed, disjoint and/or overlapping ranges are supported
// Duel with filters
public void testRandomRanges() throws Exception {
@ -136,7 +138,7 @@ public class RandomTests extends ElasticsearchIntegrationTest {
}
}
final Range.Bucket bucket = range.getByKey(Integer.toString(i));
final Range.Bucket bucket = range.getBucketByKey(Integer.toString(i));
assertEquals(bucket.getKey(), count, bucket.getDocCount());
final Filter filter = resp.getAggregations().get("filter" + i);
@ -204,14 +206,14 @@ public class RandomTests extends ElasticsearchIntegrationTest {
final Terms stringMapTerms = resp.getAggregations().get("string_map");
final Terms stringOrdinalsTerms = resp.getAggregations().get("string_ordinals");
assertEquals(valuesSet.size(), longTerms.buckets().size());
assertEquals(valuesSet.size(), doubleTerms.buckets().size());
assertEquals(valuesSet.size(), stringMapTerms.buckets().size());
assertEquals(valuesSet.size(), stringOrdinalsTerms.buckets().size());
for (Terms.Bucket bucket : longTerms.buckets()) {
final Terms.Bucket doubleBucket = doubleTerms.getByTerm(Double.toString(Long.parseLong(bucket.getKey().string())));
final Terms.Bucket stringMapBucket = stringMapTerms.getByTerm(bucket.getKey().string());
final Terms.Bucket stringOrdinalsBucket = stringOrdinalsTerms.getByTerm(bucket.getKey().string());
assertEquals(valuesSet.size(), longTerms.getBuckets().size());
assertEquals(valuesSet.size(), doubleTerms.getBuckets().size());
assertEquals(valuesSet.size(), stringMapTerms.getBuckets().size());
assertEquals(valuesSet.size(), stringOrdinalsTerms.getBuckets().size());
for (Terms.Bucket bucket : longTerms.getBuckets()) {
final Terms.Bucket doubleBucket = doubleTerms.getBucketByKey(Double.toString(Long.parseLong(bucket.getKeyAsText().string())));
final Terms.Bucket stringMapBucket = stringMapTerms.getBucketByKey(bucket.getKeyAsText().string());
final Terms.Bucket stringOrdinalsBucket = stringOrdinalsTerms.getBucketByKey(bucket.getKeyAsText().string());
assertNotNull(doubleBucket);
assertNotNull(stringMapBucket);
assertNotNull(stringOrdinalsBucket);
@ -259,10 +261,10 @@ public class RandomTests extends ElasticsearchIntegrationTest {
assertThat(terms, notNullValue());
Histogram histo = resp.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(terms.buckets().size(), equalTo(histo.buckets().size()));
for (Terms.Bucket bucket : terms) {
assertThat(terms.getBuckets().size(), equalTo(histo.getBuckets().size()));
for (Terms.Bucket bucket : terms.getBuckets()) {
final long key = bucket.getKeyAsNumber().longValue() * interval;
final Histogram.Bucket histoBucket = histo.getByKey(key);
final Histogram.Bucket histoBucket = histo.getBucketByKey(key);
assertEquals(bucket.getDocCount(), histoBucket.getDocCount());
}
}

@ -97,24 +97,24 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
DateHistogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(3));
assertThat(histo.getBuckets().size(), equalTo(3));
long key = new DateTime(2012, 1, 1, 0, 0, DateTimeZone.UTC).getMillis();
DateHistogram.Bucket bucket = histo.getByKey(key);
DateHistogram.Bucket bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(1l));
key = new DateTime(2012, 2, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(2l));
key = new DateTime(2012, 3, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(3l));
}
@ -133,11 +133,11 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
DateHistogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(3));
assertThat(histo.getBuckets().size(), equalTo(3));
int i = 0;
for (DateHistogram.Bucket bucket : histo.buckets()) {
assertThat(bucket.getKey(), equalTo(new DateTime(2012, i+1, 1, 0, 0, DateTimeZone.UTC).getMillis()));
for (DateHistogram.Bucket bucket : histo.getBuckets()) {
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(new DateTime(2012, i+1, 1, 0, 0, DateTimeZone.UTC).getMillis()));
i++;
}
}
@ -157,11 +157,11 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
DateHistogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(3));
assertThat(histo.getBuckets().size(), equalTo(3));
int i = 2;
for (DateHistogram.Bucket bucket : histo.buckets()) {
assertThat(bucket.getKey(), equalTo(new DateTime(2012, i+1, 1, 0, 0, DateTimeZone.UTC).getMillis()));
for (DateHistogram.Bucket bucket : histo.getBuckets()) {
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(new DateTime(2012, i+1, 1, 0, 0, DateTimeZone.UTC).getMillis()));
i--;
}
}
@ -181,11 +181,11 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
DateHistogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(3));
assertThat(histo.getBuckets().size(), equalTo(3));
int i = 0;
for (DateHistogram.Bucket bucket : histo.buckets()) {
assertThat(bucket.getKey(), equalTo(new DateTime(2012, i+1, 1, 0, 0, DateTimeZone.UTC).getMillis()));
for (DateHistogram.Bucket bucket : histo.getBuckets()) {
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(new DateTime(2012, i+1, 1, 0, 0, DateTimeZone.UTC).getMillis()));
i++;
}
}
@ -205,11 +205,11 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
DateHistogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(3));
assertThat(histo.getBuckets().size(), equalTo(3));
int i = 2;
for (DateHistogram.Bucket bucket : histo.buckets()) {
assertThat(bucket.getKey(), equalTo(new DateTime(2012, i+1, 1, 0, 0, DateTimeZone.UTC).getMillis()));
for (DateHistogram.Bucket bucket : histo.getBuckets()) {
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(new DateTime(2012, i+1, 1, 0, 0, DateTimeZone.UTC).getMillis()));
i--;
}
}
@ -227,30 +227,30 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
DateHistogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(3));
assertThat(histo.getBuckets().size(), equalTo(3));
long key = new DateTime(2012, 1, 1, 0, 0, DateTimeZone.UTC).getMillis();
DateHistogram.Bucket bucket = histo.getByKey(key);
DateHistogram.Bucket bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(1l));
Sum sum = bucket.getAggregations().get("sum");
assertThat(sum, notNullValue());
assertThat(sum.getValue(), equalTo(1.0));
key = new DateTime(2012, 2, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(2l));
sum = bucket.getAggregations().get("sum");
assertThat(sum, notNullValue());
assertThat(sum.getValue(), equalTo(5.0));
key = new DateTime(2012, 3, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(3l));
sum = bucket.getAggregations().get("sum");
assertThat(sum, notNullValue());
@ -270,30 +270,30 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
DateHistogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(3));
assertThat(histo.getBuckets().size(), equalTo(3));
long key = new DateTime(2012, 1, 1, 0, 0, DateTimeZone.UTC).getMillis();
DateHistogram.Bucket bucket = histo.getByKey(key);
DateHistogram.Bucket bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(1l));
Max max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
assertThat(max.getValue(), equalTo((double) new DateTime(2012, 1, 2, 0, 0, DateTimeZone.UTC).getMillis()));
key = new DateTime(2012, 2, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(2l));
max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
assertThat(max.getValue(), equalTo((double) new DateTime(2012, 2, 15, 0, 0, DateTimeZone.UTC).getMillis()));
key = new DateTime(2012, 3, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(3l));
max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
@ -316,11 +316,11 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
DateHistogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(3));
assertThat(histo.getBuckets().size(), equalTo(3));
int i = 0;
for (DateHistogram.Bucket bucket : histo.buckets()) {
assertThat(bucket.getKey(), equalTo(new DateTime(2012, i+1, 1, 0, 0, DateTimeZone.UTC).getMillis()));
for (DateHistogram.Bucket bucket : histo.getBuckets()) {
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(new DateTime(2012, i+1, 1, 0, 0, DateTimeZone.UTC).getMillis()));
i++;
}
}
@ -341,11 +341,11 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
DateHistogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(3));
assertThat(histo.getBuckets().size(), equalTo(3));
int i = 2;
for (DateHistogram.Bucket bucket : histo.buckets()) {
assertThat(bucket.getKey(), equalTo(new DateTime(2012, i+1, 1, 0, 0, DateTimeZone.UTC).getMillis()));
for (DateHistogram.Bucket bucket : histo.getBuckets()) {
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(new DateTime(2012, i+1, 1, 0, 0, DateTimeZone.UTC).getMillis()));
i--;
}
}
@ -366,11 +366,11 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
DateHistogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(3));
assertThat(histo.getBuckets().size(), equalTo(3));
int i = 0;
for (DateHistogram.Bucket bucket : histo.buckets()) {
assertThat(bucket.getKey(), equalTo(new DateTime(2012, i+1, 1, 0, 0, DateTimeZone.UTC).getMillis()));
for (DateHistogram.Bucket bucket : histo.getBuckets()) {
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(new DateTime(2012, i+1, 1, 0, 0, DateTimeZone.UTC).getMillis()));
i++;
}
}
@ -391,11 +391,11 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
DateHistogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(3));
assertThat(histo.getBuckets().size(), equalTo(3));
int i = 2;
for (DateHistogram.Bucket bucket : histo.buckets()) {
assertThat(bucket.getKey(), equalTo(new DateTime(2012, i+1, 1, 0, 0, DateTimeZone.UTC).getMillis()));
for (DateHistogram.Bucket bucket : histo.getBuckets()) {
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(new DateTime(2012, i+1, 1, 0, 0, DateTimeZone.UTC).getMillis()));
i--;
}
}
@ -416,24 +416,24 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(3));
assertThat(histo.getBuckets().size(), equalTo(3));
long key = new DateTime(2012, 2, 1, 0, 0, DateTimeZone.UTC).getMillis();
DateHistogram.Bucket bucket = histo.getByKey(key);
DateHistogram.Bucket bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(1l));
key = new DateTime(2012, 3, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(2l));
key = new DateTime(2012, 4, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(3l));
}
@ -458,30 +458,30 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
DateHistogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(4));
assertThat(histo.getBuckets().size(), equalTo(4));
long key = new DateTime(2012, 1, 1, 0, 0, DateTimeZone.UTC).getMillis();
DateHistogram.Bucket bucket = histo.getByKey(key);
DateHistogram.Bucket bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(1l));
key = new DateTime(2012, 2, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(3l));
key = new DateTime(2012, 3, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(5l));
key = new DateTime(2012, 4, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(3l));
}
@ -500,21 +500,23 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
DateHistogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(4));
assertThat(histo.getBuckets().size(), equalTo(4));
DateHistogram.Bucket bucket = histo.buckets().get(0);
List<DateHistogram.Bucket> buckets = new ArrayList<DateHistogram.Bucket>(histo.getBuckets());
DateHistogram.Bucket bucket = buckets.get(0);
assertThat(bucket, notNullValue());
assertThat(bucket.getDocCount(), equalTo(5l));
bucket = histo.buckets().get(1);
bucket = buckets.get(1);
assertThat(bucket, notNullValue());
assertThat(bucket.getDocCount(), equalTo(3l));
bucket = histo.buckets().get(2);
bucket = buckets.get(2);
assertThat(bucket, notNullValue());
assertThat(bucket.getDocCount(), equalTo(3l));
bucket = histo.buckets().get(3);
bucket = buckets.get(3);
assertThat(bucket, notNullValue());
assertThat(bucket.getDocCount(), equalTo(1l));
}
@ -545,30 +547,30 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
DateHistogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(4));
assertThat(histo.getBuckets().size(), equalTo(4));
long key = new DateTime(2012, 2, 1, 0, 0, DateTimeZone.UTC).getMillis();
DateHistogram.Bucket bucket = histo.getByKey(key);
DateHistogram.Bucket bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(1l));
key = new DateTime(2012, 3, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(3l));
key = new DateTime(2012, 4, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(5l));
key = new DateTime(2012, 5, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(3l));
}
@ -599,39 +601,39 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
DateHistogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(4));
assertThat(histo.getBuckets().size(), equalTo(4));
long key = new DateTime(2012, 2, 1, 0, 0, DateTimeZone.UTC).getMillis();
DateHistogram.Bucket bucket = histo.getByKey(key);
DateHistogram.Bucket bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(1l));
Max max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
assertThat((long) max.getValue(), equalTo(new DateTime(2012, 3, 3, 0, 0, DateTimeZone.UTC).getMillis()));
key = new DateTime(2012, 3, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(3l));
max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
assertThat((long) max.getValue(), equalTo(new DateTime(2012, 4, 16, 0, 0, DateTimeZone.UTC).getMillis()));
key = new DateTime(2012, 4, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(5l));
max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
assertThat((long) max.getValue(), equalTo(new DateTime(2012, 5, 24, 0, 0, DateTimeZone.UTC).getMillis()));
key = new DateTime(2012, 5, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(3l));
max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
@ -658,24 +660,24 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
DateHistogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(3));
assertThat(histo.getBuckets().size(), equalTo(3));
long key = new DateTime(2012, 1, 1, 0, 0, DateTimeZone.UTC).getMillis();
DateHistogram.Bucket bucket = histo.getByKey(key);
DateHistogram.Bucket bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(1l));
key = new DateTime(2012, 2, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(2l));
key = new DateTime(2012, 3, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(3l));
}
@ -694,30 +696,30 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
DateHistogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(3));
assertThat(histo.getBuckets().size(), equalTo(3));
long key = new DateTime(2012, 1, 1, 0, 0, DateTimeZone.UTC).getMillis();
DateHistogram.Bucket bucket = histo.getByKey(key);
DateHistogram.Bucket bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(1l));
Max max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
assertThat(max.getValue(), equalTo((double) new DateTime(2012, 1, 2, 0, 0, DateTimeZone.UTC).getMillis()));
key = new DateTime(2012, 2, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(2l));
max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
assertThat(max.getValue(), equalTo((double) new DateTime(2012, 2, 15, 0, 0, DateTimeZone.UTC).getMillis()));
key = new DateTime(2012, 3, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(3l));
max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
@ -736,30 +738,30 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
DateHistogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(4));
assertThat(histo.getBuckets().size(), equalTo(4));
long key = new DateTime(2012, 1, 1, 0, 0, DateTimeZone.UTC).getMillis();
DateHistogram.Bucket bucket = histo.getByKey(key);
DateHistogram.Bucket bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(1l));
key = new DateTime(2012, 2, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(3l));
key = new DateTime(2012, 3, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(5l));
key = new DateTime(2012, 4, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(3l));
}
@ -787,39 +789,39 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
DateHistogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(4));
assertThat(histo.getBuckets().size(), equalTo(4));
long key = new DateTime(2012, 1, 1, 0, 0, DateTimeZone.UTC).getMillis();
DateHistogram.Bucket bucket = histo.getByKey(key);
DateHistogram.Bucket bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(1l));
Max max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
assertThat((long) max.getValue(), equalTo(new DateTime(2012, 2, 3, 0, 0, DateTimeZone.UTC).getMillis()));
key = new DateTime(2012, 2, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(3l));
max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
assertThat((long) max.getValue(), equalTo(new DateTime(2012, 3, 16, 0, 0, DateTimeZone.UTC).getMillis()));
key = new DateTime(2012, 3, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(5l));
max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
assertThat((long) max.getValue(), equalTo(new DateTime(2012, 4, 24, 0, 0, DateTimeZone.UTC).getMillis()));
key = new DateTime(2012, 4, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(3l));
max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
@ -838,7 +840,7 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
DateHistogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(0));
assertThat(histo.getBuckets().size(), equalTo(0));
}
@Test
@ -853,24 +855,24 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
DateHistogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(3));
assertThat(histo.getBuckets().size(), equalTo(3));
long key = new DateTime(2012, 1, 1, 0, 0, DateTimeZone.UTC).getMillis();
DateHistogram.Bucket bucket = histo.getByKey(key);
DateHistogram.Bucket bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(1l));
key = new DateTime(2012, 2, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(2l));
key = new DateTime(2012, 3, 1, 0, 0, DateTimeZone.UTC).getMillis();
bucket = histo.getByKey(key);
bucket = histo.getBucketByKey(key);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo(key));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
assertThat(bucket.getDocCount(), equalTo(3l));
}
@ -894,13 +896,13 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
assertThat(searchResponse.getHits().getTotalHits(), equalTo(2l));
Histogram histo = searchResponse.getAggregations().get("histo");
assertThat(histo, Matchers.notNullValue());
Histogram.Bucket bucket = histo.getByKey(1l);
Histogram.Bucket bucket = histo.getBucketByKey(1l);
assertThat(bucket, Matchers.notNullValue());
DateHistogram dateHisto = bucket.getAggregations().get("date_histo");
assertThat(dateHisto, Matchers.notNullValue());
assertThat(dateHisto.getName(), equalTo("date_histo"));
assertThat(dateHisto.buckets().isEmpty(), is(true));
assertThat(dateHisto.getBuckets().isEmpty(), is(true));
}
}

@ -61,10 +61,6 @@ public class DateRangeTests extends ElasticsearchIntegrationTest {
.build();
}
private static DateTime date(int month, int day) {
return new DateTime(2012, month, day, 0, 0, DateTimeZone.UTC);
}
private static IndexRequestBuilder indexDoc(int month, int day, int value) throws Exception {
return client().prepareIndex("idx", "type").setSource(jsonBuilder()
.startObject()
@ -74,6 +70,10 @@ public class DateRangeTests extends ElasticsearchIntegrationTest {
.endObject());
}
private static DateTime date(int month, int day) {
return new DateTime(2012, month, day, 0, 0, DateTimeZone.UTC);
}
int numDocs;
@Before
@ -121,17 +121,19 @@ public class DateRangeTests extends ElasticsearchIntegrationTest {
DateRange range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
DateRange.Bucket bucket = range.buckets().get(0);
List<DateRange.Bucket> buckets = new ArrayList<DateRange.Bucket>(range.getBuckets());
DateRange.Bucket bucket = buckets.get(0);
assertThat(bucket.getKey(), equalTo("a long time ago"));
assertThat(bucket.getDocCount(), equalTo(0L));
bucket = range.buckets().get(1);
bucket = buckets.get(1);
assertThat(bucket.getKey(), equalTo("recently"));
assertThat(bucket.getDocCount(), equalTo((long) numDocs));
bucket = range.buckets().get(2);
bucket = buckets.get(2);
assertThat(bucket.getKey(), equalTo("last year"));
assertThat(bucket.getDocCount(), equalTo(0L));
}
@ -152,32 +154,32 @@ public class DateRangeTests extends ElasticsearchIntegrationTest {
DateRange range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
DateRange.Bucket bucket = range.getByKey("*-2012-02-15T00:00:00.000Z");
DateRange.Bucket bucket = range.getBucketByKey("*-2012-02-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-2012-02-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsDate(), nullValue());
assertThat(bucket.getTo(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
bucket = range.getBucketByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getTo(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("2012-03-15T00:00:00.000Z-*");
bucket = range.getBucketByKey("2012-03-15T00:00:00.000Z-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-03-15T00:00:00.000Z-*"));
assertThat(bucket.getFrom(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsDate(), nullValue());
assertThat(bucket.getDocCount(), equalTo(numDocs - 4l));
}
@ -198,32 +200,32 @@ public class DateRangeTests extends ElasticsearchIntegrationTest {
DateRange range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
DateRange.Bucket bucket = range.getByKey("*-2012-02-15T00:00:00.000Z");
DateRange.Bucket bucket = range.getBucketByKey("*-2012-02-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-2012-02-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsDate(), nullValue());
assertThat(bucket.getTo(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
bucket = range.getBucketByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getTo(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("2012-03-15T00:00:00.000Z-*");
bucket = range.getBucketByKey("2012-03-15T00:00:00.000Z-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-03-15T00:00:00.000Z-*"));
assertThat(bucket.getFrom(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsDate(), nullValue());
assertThat(bucket.getDocCount(), equalTo(numDocs - 4l));
}
@ -245,32 +247,32 @@ public class DateRangeTests extends ElasticsearchIntegrationTest {
DateRange range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
DateRange.Bucket bucket = range.getByKey("*-2012-02-15");
DateRange.Bucket bucket = range.getBucketByKey("*-2012-02-15");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-2012-02-15"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsDate(), nullValue());
assertThat(bucket.getTo(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("2012-02-15-2012-03-15");
bucket = range.getBucketByKey("2012-02-15-2012-03-15");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-02-15-2012-03-15"));
assertThat(bucket.getFrom(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getTo(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("2012-03-15-*");
bucket = range.getBucketByKey("2012-03-15-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-03-15-*"));
assertThat(bucket.getFrom(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsDate(), nullValue());
assertThat(bucket.getDocCount(), equalTo(numDocs - 4l));
}
@ -291,32 +293,32 @@ public class DateRangeTests extends ElasticsearchIntegrationTest {
DateRange range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
DateRange.Bucket bucket = range.getByKey("*-2012-02-15T00:00:00.000Z");
DateRange.Bucket bucket = range.getBucketByKey("*-2012-02-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-2012-02-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsDate(), nullValue());
assertThat(bucket.getTo(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
bucket = range.getBucketByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getTo(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("2012-03-15T00:00:00.000Z-*");
bucket = range.getBucketByKey("2012-03-15T00:00:00.000Z-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-03-15T00:00:00.000Z-*"));
assertThat(bucket.getFrom(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsDate(), nullValue());
assertThat(bucket.getDocCount(), equalTo(numDocs - 4l));
}
@ -337,32 +339,32 @@ public class DateRangeTests extends ElasticsearchIntegrationTest {
DateRange range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
DateRange.Bucket bucket = range.getByKey("r1");
DateRange.Bucket bucket = range.getBucketByKey("r1");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("r1"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsDate(), nullValue());
assertThat(bucket.getTo(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("r2");
bucket = range.getBucketByKey("r2");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("r2"));
assertThat(bucket.getFrom(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getTo(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("r3");
bucket = range.getBucketByKey("r3");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("r3"));
assertThat(bucket.getFrom(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsDate(), nullValue());
assertThat(bucket.getDocCount(), equalTo(numDocs - 4l));
}
@ -393,38 +395,38 @@ public class DateRangeTests extends ElasticsearchIntegrationTest {
DateRange range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
DateRange.Bucket bucket = range.getByKey("r1");
DateRange.Bucket bucket = range.getBucketByKey("r1");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("r1"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsDate(), nullValue());
assertThat(bucket.getTo(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
Sum sum = bucket.getAggregations().get("sum");
assertThat(sum, notNullValue());
assertThat(sum.getValue(), equalTo((double) 1 + 2));
bucket = range.getByKey("r2");
bucket = range.getBucketByKey("r2");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("r2"));
assertThat(bucket.getFrom(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getTo(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
sum = bucket.getAggregations().get("sum");
assertThat(sum, notNullValue());
assertThat(sum.getValue(), equalTo((double) 3 + 4));
bucket = range.getByKey("r3");
bucket = range.getBucketByKey("r3");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("r3"));
assertThat(bucket.getFrom(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsDate(), nullValue());
assertThat(bucket.getDocCount(), equalTo(numDocs - 4l));
sum = bucket.getAggregations().get("sum");
@ -448,38 +450,38 @@ public class DateRangeTests extends ElasticsearchIntegrationTest {
DateRange range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
DateRange.Bucket bucket = range.getByKey("r1");
DateRange.Bucket bucket = range.getBucketByKey("r1");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("r1"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsDate(), nullValue());
assertThat(bucket.getTo(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
Min min = bucket.getAggregations().get("min");
assertThat(min, notNullValue());
assertThat(min.getValue(), equalTo((double) date(1, 2).getMillis()));
bucket = range.getByKey("r2");
bucket = range.getBucketByKey("r2");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("r2"));
assertThat(bucket.getFrom(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getTo(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
min = bucket.getAggregations().get("min");
assertThat(min, notNullValue());
assertThat(min.getValue(), equalTo((double) date(2, 15).getMillis()));
bucket = range.getByKey("r3");
bucket = range.getBucketByKey("r3");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("r3"));
assertThat(bucket.getFrom(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsDate(), nullValue());
assertThat(bucket.getDocCount(), equalTo(numDocs - 4l));
min = bucket.getAggregations().get("min");
@ -512,32 +514,32 @@ public class DateRangeTests extends ElasticsearchIntegrationTest {
DateRange range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
DateRange.Bucket bucket = range.getByKey("*-2012-02-15T00:00:00.000Z");
DateRange.Bucket bucket = range.getBucketByKey("*-2012-02-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-2012-02-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsDate(), nullValue());
assertThat(bucket.getTo(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
bucket = range.getBucketByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getTo(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getDocCount(), equalTo(3l));
bucket = range.getByKey("2012-03-15T00:00:00.000Z-*");
bucket = range.getBucketByKey("2012-03-15T00:00:00.000Z-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-03-15T00:00:00.000Z-*"));
assertThat(bucket.getFrom(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsDate(), nullValue());
assertThat(bucket.getDocCount(), equalTo(numDocs - 2l));
}
@ -569,32 +571,32 @@ public class DateRangeTests extends ElasticsearchIntegrationTest {
DateRange range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
DateRange.Bucket bucket = range.getByKey("*-2012-02-15T00:00:00.000Z");
DateRange.Bucket bucket = range.getBucketByKey("*-2012-02-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-2012-02-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsDate(), nullValue());
assertThat(bucket.getTo(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getDocCount(), equalTo(1l));
bucket = range.getByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
bucket = range.getBucketByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getTo(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("2012-03-15T00:00:00.000Z-*");
bucket = range.getBucketByKey("2012-03-15T00:00:00.000Z-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-03-15T00:00:00.000Z-*"));
assertThat(bucket.getFrom(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsDate(), nullValue());
assertThat(bucket.getDocCount(), equalTo(numDocs - 1l));
}
@ -626,38 +628,38 @@ public class DateRangeTests extends ElasticsearchIntegrationTest {
DateRange range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
DateRange.Bucket bucket = range.getByKey("*-2012-02-15T00:00:00.000Z");
DateRange.Bucket bucket = range.getBucketByKey("*-2012-02-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-2012-02-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsDate(), nullValue());
assertThat(bucket.getTo(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getDocCount(), equalTo(1l));
Max max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
assertThat(max.getValue(), equalTo((double) date(3, 3).getMillis()));
bucket = range.getByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
bucket = range.getBucketByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getTo(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
assertThat(max.getValue(), equalTo((double) date(4, 3).getMillis()));
bucket = range.getByKey("2012-03-15T00:00:00.000Z-*");
bucket = range.getBucketByKey("2012-03-15T00:00:00.000Z-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-03-15T00:00:00.000Z-*"));
assertThat(bucket.getFrom(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsDate(), nullValue());
assertThat(bucket.getDocCount(), equalTo(numDocs - 1l));
max = bucket.getAggregations().get("max");
@ -680,32 +682,32 @@ public class DateRangeTests extends ElasticsearchIntegrationTest {
DateRange range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
DateRange.Bucket bucket = range.getByKey("*-2012-02-15T00:00:00.000Z");
DateRange.Bucket bucket = range.getBucketByKey("*-2012-02-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-2012-02-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsDate(), nullValue());
assertThat(bucket.getTo(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
bucket = range.getBucketByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getTo(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("2012-03-15T00:00:00.000Z-*");
bucket = range.getBucketByKey("2012-03-15T00:00:00.000Z-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-03-15T00:00:00.000Z-*"));
assertThat(bucket.getFrom(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsDate(), nullValue());
assertThat(bucket.getDocCount(), equalTo(numDocs - 4l));
}
@ -727,38 +729,38 @@ public class DateRangeTests extends ElasticsearchIntegrationTest {
DateRange range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
DateRange.Bucket bucket = range.getByKey("*-2012-02-15T00:00:00.000Z");
DateRange.Bucket bucket = range.getBucketByKey("*-2012-02-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-2012-02-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsDate(), nullValue());
assertThat(bucket.getTo(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
Max max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
assertThat(max.getValue(), equalTo((double) date(2, 2).getMillis()));
bucket = range.getByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
bucket = range.getBucketByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getTo(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
assertThat(max.getValue(), equalTo((double) date(3, 2).getMillis()));
bucket = range.getByKey("2012-03-15T00:00:00.000Z-*");
bucket = range.getBucketByKey("2012-03-15T00:00:00.000Z-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-03-15T00:00:00.000Z-*"));
assertThat(bucket.getFrom(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsDate(), nullValue());
assertThat(bucket.getDocCount(), equalTo(numDocs - 4l));
max = bucket.getAggregations().get("max");
@ -790,32 +792,32 @@ public class DateRangeTests extends ElasticsearchIntegrationTest {
DateRange range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
DateRange.Bucket bucket = range.getByKey("*-2012-02-15T00:00:00.000Z");
DateRange.Bucket bucket = range.getBucketByKey("*-2012-02-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-2012-02-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsDate(), nullValue());
assertThat(bucket.getTo(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
bucket = range.getBucketByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getTo(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getDocCount(), equalTo(3l));
bucket = range.getByKey("2012-03-15T00:00:00.000Z-*");
bucket = range.getBucketByKey("2012-03-15T00:00:00.000Z-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-03-15T00:00:00.000Z-*"));
assertThat(bucket.getFrom(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsDate(), nullValue());
assertThat(bucket.getDocCount(), equalTo(numDocs - 2l));
}
@ -837,38 +839,38 @@ public class DateRangeTests extends ElasticsearchIntegrationTest {
DateRange range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
DateRange.Bucket bucket = range.getByKey("*-2012-02-15T00:00:00.000Z");
DateRange.Bucket bucket = range.getBucketByKey("*-2012-02-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-2012-02-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsDate(), nullValue());
assertThat(bucket.getTo(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
Min min = bucket.getAggregations().get("min");
assertThat(min, notNullValue());
assertThat(min.getValue(), equalTo((double) date(1, 2).getMillis()));
bucket = range.getByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
bucket = range.getBucketByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getTo(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getDocCount(), equalTo(3l));
min = bucket.getAggregations().get("min");
assertThat(min, notNullValue());
assertThat(min.getValue(), equalTo((double) date(2, 2).getMillis()));
bucket = range.getByKey("2012-03-15T00:00:00.000Z-*");
bucket = range.getBucketByKey("2012-03-15T00:00:00.000Z-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-03-15T00:00:00.000Z-*"));
assertThat(bucket.getFrom(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsDate(), nullValue());
assertThat(bucket.getDocCount(), equalTo(numDocs - 2l));
min = bucket.getAggregations().get("min");
@ -894,32 +896,32 @@ public class DateRangeTests extends ElasticsearchIntegrationTest {
DateRange range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
DateRange.Bucket bucket = range.getByKey("*-2012-02-15T00:00:00.000Z");
DateRange.Bucket bucket = range.getBucketByKey("*-2012-02-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-2012-02-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsDate(), nullValue());
assertThat(bucket.getTo(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getDocCount(), equalTo(0l));
bucket = range.getByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
bucket = range.getBucketByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getTo(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getDocCount(), equalTo(0l));
bucket = range.getByKey("2012-03-15T00:00:00.000Z-*");
bucket = range.getBucketByKey("2012-03-15T00:00:00.000Z-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-03-15T00:00:00.000Z-*"));
assertThat(bucket.getFrom(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsDate(), nullValue());
assertThat(bucket.getDocCount(), equalTo(0l));
}
@ -940,32 +942,32 @@ public class DateRangeTests extends ElasticsearchIntegrationTest {
DateRange range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
DateRange.Bucket bucket = range.getByKey("*-2012-02-15T00:00:00.000Z");
DateRange.Bucket bucket = range.getBucketByKey("*-2012-02-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-2012-02-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsDate(), nullValue());
assertThat(bucket.getTo(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getDocCount(), equalTo(0l));
bucket = range.getByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
bucket = range.getBucketByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getTo(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getDocCount(), equalTo(0l));
bucket = range.getByKey("2012-03-15T00:00:00.000Z-*");
bucket = range.getBucketByKey("2012-03-15T00:00:00.000Z-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-03-15T00:00:00.000Z-*"));
assertThat(bucket.getFrom(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsDate(), nullValue());
assertThat(bucket.getDocCount(), equalTo(0l));
}
@ -986,32 +988,32 @@ public class DateRangeTests extends ElasticsearchIntegrationTest {
DateRange range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
DateRange.Bucket bucket = range.getByKey("*-2012-02-15T00:00:00.000Z");
DateRange.Bucket bucket = range.getBucketByKey("*-2012-02-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-2012-02-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsDate(), nullValue());
assertThat(bucket.getTo(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
bucket = range.getBucketByKey("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-02-15T00:00:00.000Z-2012-03-15T00:00:00.000Z"));
assertThat(bucket.getFrom(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(2, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(2, 15)));
assertThat(bucket.getTo(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getTo().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getToAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("2012-03-15T00:00:00.000Z-*");
bucket = range.getBucketByKey("2012-03-15T00:00:00.000Z-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("2012-03-15T00:00:00.000Z-*"));
assertThat(bucket.getFrom(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) date(3, 15).getMillis()));
assertThat(bucket.getFromAsDate(), equalTo(date(3, 15)));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsDate(), nullValue());
assertThat(bucket.getDocCount(), equalTo(numDocs - 4l));
}
@ -1036,18 +1038,19 @@ public class DateRangeTests extends ElasticsearchIntegrationTest {
assertThat(searchResponse.getHits().getTotalHits(), equalTo(2l));
Histogram histo = searchResponse.getAggregations().get("histo");
assertThat(histo, Matchers.notNullValue());
Histogram.Bucket bucket = histo.getByKey(1l);
Histogram.Bucket bucket = histo.getBucketByKey(1l);
assertThat(bucket, Matchers.notNullValue());
DateRange dateRange = bucket.getAggregations().get("date_range");
List<DateRange.Bucket> buckets = new ArrayList<DateRange.Bucket>(dateRange.getBuckets());
assertThat(dateRange, Matchers.notNullValue());
assertThat(dateRange.getName(), equalTo("date_range"));
assertThat(dateRange.buckets().size(), is(1));
assertThat(dateRange.buckets().get(0).getKey(), equalTo("0-1"));
assertThat(dateRange.buckets().get(0).getFrom(), equalTo(0.0));
assertThat(dateRange.buckets().get(0).getTo(), equalTo(1.0));
assertThat(dateRange.buckets().get(0).getDocCount(), equalTo(0l));
assertThat(dateRange.buckets().get(0).getAggregations().asList().isEmpty(), is(true));
assertThat(buckets.size(), is(1));
assertThat(buckets.get(0).getKey(), equalTo("0-1"));
assertThat(buckets.get(0).getFrom().doubleValue(), equalTo(0.0));
assertThat(buckets.get(0).getTo().doubleValue(), equalTo(1.0));
assertThat(buckets.get(0).getDocCount(), equalTo(0l));
assertThat(buckets.get(0).getAggregations().asList().isEmpty(), is(true));
}

@ -91,6 +91,10 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
ensureSearchable();
}
private String key(Terms.Bucket bucket) {
return randomBoolean() ? bucket.getKey() : bucket.getKeyAsText().string();
}
@Test
// the main purpose of this test is to make sure we're not allocating 2GB of memory per shard
public void sizeIsZero() {
@ -106,7 +110,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(100));
assertThat(terms.getBuckets().size(), equalTo(100));
}
@Test
@ -122,12 +126,12 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("" + (double)i);
Terms.Bucket bucket = terms.getBucketByKey("" + (double) i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (double)i));
assertThat(key(bucket), equalTo("" + (double)i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
assertThat(bucket.getDocCount(), equalTo(1l));
}
@ -148,12 +152,12 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(20));
assertThat(terms.getBuckets().size(), equalTo(20));
for (int i = 0; i < 20; i++) {
Terms.Bucket bucket = terms.getByTerm("" + (double) i);
Terms.Bucket bucket = terms.getBucketByKey("" + (double) i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (double) i));
assertThat(key(bucket), equalTo("" + (double) i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
assertThat(bucket.getDocCount(), equalTo(1l));
}
@ -173,12 +177,12 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
int i = 0;
for (Terms.Bucket bucket : terms.buckets()) {
for (Terms.Bucket bucket : terms.getBuckets()) {
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (double)i));
assertThat(key(bucket), equalTo("" + (double)i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
assertThat(bucket.getDocCount(), equalTo(1l));
i++;
@ -199,12 +203,12 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
int i = 4;
for (Terms.Bucket bucket : terms.buckets()) {
for (Terms.Bucket bucket : terms.getBuckets()) {
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (double) i));
assertThat(key(bucket), equalTo("" + (double) i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
assertThat(bucket.getDocCount(), equalTo(1l));
i--;
@ -225,12 +229,12 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("" + (double) i);
Terms.Bucket bucket = terms.getBucketByKey("" + (double) i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (double) i));
assertThat(key(bucket), equalTo("" + (double) i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
assertThat(bucket.getDocCount(), equalTo(1l));
Sum sum = bucket.getAggregations().get("sum");
@ -253,12 +257,12 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("" + (double) i);
Terms.Bucket bucket = terms.getBucketByKey("" + (double) i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (double) i));
assertThat(key(bucket), equalTo("" + (double) i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
assertThat(bucket.getDocCount(), equalTo(1l));
Sum sum = bucket.getAggregations().get("sum");
@ -281,12 +285,12 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("" + (i+1d));
Terms.Bucket bucket = terms.getBucketByKey("" + (i + 1d));
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (i+1d)));
assertThat(key(bucket), equalTo("" + (i+1d)));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i+1));
assertThat(bucket.getDocCount(), equalTo(1l));
}
@ -305,12 +309,12 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(6));
assertThat(terms.getBuckets().size(), equalTo(6));
for (int i = 0; i < 6; i++) {
Terms.Bucket bucket = terms.getByTerm("" + (double) i);
Terms.Bucket bucket = terms.getBucketByKey("" + (double) i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (double) i));
assertThat(key(bucket), equalTo("" + (double) i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
if (i == 0 || i == 5) {
assertThat(bucket.getDocCount(), equalTo(1l));
@ -334,12 +338,12 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(6));
assertThat(terms.getBuckets().size(), equalTo(6));
for (int i = 0; i < 6; i++) {
Terms.Bucket bucket = terms.getByTerm("" + (i+1d));
Terms.Bucket bucket = terms.getBucketByKey("" + (i + 1d));
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (i+1d)));
assertThat(key(bucket), equalTo("" + (i+1d)));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i+1));
if (i == 0 || i == 5) {
assertThat(bucket.getDocCount(), equalTo(1l));
@ -363,11 +367,11 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(1));
assertThat(terms.getBuckets().size(), equalTo(1));
Terms.Bucket bucket = terms.getByTerm("1.0");
Terms.Bucket bucket = terms.getBucketByKey("1.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("1.0"));
assertThat(key(bucket), equalTo("1.0"));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(1));
assertThat(bucket.getDocCount(), equalTo(5l));
}
@ -404,12 +408,12 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(6));
assertThat(terms.getBuckets().size(), equalTo(6));
for (int i = 0; i < 6; i++) {
Terms.Bucket bucket = terms.getByTerm("" + (i+1d));
Terms.Bucket bucket = terms.getBucketByKey("" + (i + 1d));
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (i+1d)));
assertThat(key(bucket), equalTo("" + (i+1d)));
assertThat(bucket.getKeyAsNumber().doubleValue(), equalTo(i+1d));
final long count = i == 0 || i == 5 ? 1 : 2;
double s = 0;
@ -439,12 +443,12 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("" + (double) i);
Terms.Bucket bucket = terms.getBucketByKey("" + (double) i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (double) i));
assertThat(key(bucket), equalTo("" + (double) i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
assertThat(bucket.getDocCount(), equalTo(1l));
}
@ -464,12 +468,12 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("" + (double) i);
Terms.Bucket bucket = terms.getBucketByKey("" + (double) i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (double) i));
assertThat(key(bucket), equalTo("" + (double) i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
assertThat(bucket.getDocCount(), equalTo(1l));
Sum sum = bucket.getAggregations().get("sum");
@ -491,12 +495,12 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(6));
assertThat(terms.getBuckets().size(), equalTo(6));
for (int i = 0; i < 6; i++) {
Terms.Bucket bucket = terms.getByTerm("" + (double) i);
Terms.Bucket bucket = terms.getBucketByKey("" + (double) i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (double) i));
assertThat(key(bucket), equalTo("" + (double) i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
if (i == 0 || i == 5) {
assertThat(bucket.getDocCount(), equalTo(1l));
@ -544,12 +548,12 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(6));
assertThat(terms.getBuckets().size(), equalTo(6));
for (int i = 0; i < 6; i++) {
Terms.Bucket bucket = terms.getByTerm("" + i + ".0");
Terms.Bucket bucket = terms.getBucketByKey("" + i + ".0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + i + ".0"));
assertThat(key(bucket), equalTo("" + i + ".0"));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
final long count = i == 0 || i == 5 ? 1 : 2;
double s = 0;
@ -580,7 +584,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(0));
assertThat(terms.getBuckets().size(), equalTo(0));
}
@Test
@ -596,12 +600,12 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("" + (double) i);
Terms.Bucket bucket = terms.getBucketByKey("" + (double) i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (double) i));
assertThat(key(bucket), equalTo("" + (double) i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
assertThat(bucket.getDocCount(), equalTo(1l));
}
@ -628,13 +632,13 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
assertThat(searchResponse.getHits().getTotalHits(), equalTo(2l));
Histogram histo = searchResponse.getAggregations().get("histo");
assertThat(histo, Matchers.notNullValue());
Histogram.Bucket bucket = histo.getByKey(1l);
Histogram.Bucket bucket = histo.getBucketByKey(1l);
assertThat(bucket, Matchers.notNullValue());
Terms terms = bucket.getAggregations().get("terms");
assertThat(terms, Matchers.notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().isEmpty(), is(true));
assertThat(terms.getBuckets().isEmpty(), is(true));
}
@Test
@ -653,12 +657,12 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("" + (double)i);
Terms.Bucket bucket = terms.getBucketByKey("" + (double) i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (double)i));
assertThat(key(bucket), equalTo("" + (double)i));
assertThat(bucket.getDocCount(), equalTo(1l));
Avg avg = bucket.getAggregations().get("avg_i");
assertThat(avg, notNullValue());
@ -759,13 +763,13 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 4; i >= 0; i--) {
Terms.Bucket bucket = terms.getByTerm("" + (double)i);
Terms.Bucket bucket = terms.getBucketByKey("" + (double) i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (double)i));
assertThat(key(bucket), equalTo("" + (double)i));
assertThat(bucket.getDocCount(), equalTo(1l));
Avg avg = bucket.getAggregations().get("avg_i");
@ -790,12 +794,12 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("" + (double)i);
Terms.Bucket bucket = terms.getBucketByKey("" + (double) i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (double)i));
assertThat(key(bucket), equalTo("" + (double)i));
assertThat(bucket.getDocCount(), equalTo(1l));
Stats stats = bucket.getAggregations().get("stats");
@ -820,12 +824,12 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 4; i >= 0; i--) {
Terms.Bucket bucket = terms.getByTerm("" + (double)i);
Terms.Bucket bucket = terms.getBucketByKey("" + (double) i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (double)i));
assertThat(key(bucket), equalTo("" + (double)i));
assertThat(bucket.getDocCount(), equalTo(1l));
Stats stats = bucket.getAggregations().get("stats");
@ -850,12 +854,12 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("" + (double)i);
Terms.Bucket bucket = terms.getBucketByKey("" + (double) i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (double)i));
assertThat(key(bucket), equalTo("" + (double)i));
assertThat(bucket.getDocCount(), equalTo(1l));
ExtendedStats stats = bucket.getAggregations().get("stats");

@ -166,7 +166,7 @@ public class FilterTests extends ElasticsearchIntegrationTest {
assertThat(searchResponse.getHits().getTotalHits(), equalTo(2l));
Histogram histo = searchResponse.getAggregations().get("histo");
assertThat(histo, Matchers.notNullValue());
Histogram.Bucket bucket = histo.getByKey(1l);
Histogram.Bucket bucket = histo.getBucketByKey(1l);
assertThat(bucket, Matchers.notNullValue());
Filter filter = bucket.getAggregations().get("filter");

@ -136,27 +136,27 @@ public class GeoDistanceTests extends ElasticsearchIntegrationTest {
GeoDistance geoDist = response.getAggregations().get("amsterdam_rings");
assertThat(geoDist, notNullValue());
assertThat(geoDist.getName(), equalTo("amsterdam_rings"));
assertThat(geoDist.buckets().size(), equalTo(3));
assertThat(geoDist.getBuckets().size(), equalTo(3));
GeoDistance.Bucket bucket = geoDist.getByKey("*-500.0");
GeoDistance.Bucket bucket = geoDist.getBucketByKey("*-500.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-500.0"));
assertThat(bucket.getFrom(), equalTo(0.0));
assertThat(bucket.getTo(), equalTo(500.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(0.0));
assertThat(bucket.getTo().doubleValue(), equalTo(500.0));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = geoDist.getByKey("500.0-1000.0");
bucket = geoDist.getBucketByKey("500.0-1000.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("500.0-1000.0"));
assertThat(bucket.getFrom(), equalTo(500.0));
assertThat(bucket.getTo(), equalTo(1000.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(500.0));
assertThat(bucket.getTo().doubleValue(), equalTo(1000.0));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = geoDist.getByKey("1000.0-*");
bucket = geoDist.getBucketByKey("1000.0-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("1000.0-*"));
assertThat(bucket.getFrom(), equalTo(1000.0));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(1000.0));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(1l));
}
@ -178,27 +178,27 @@ public class GeoDistanceTests extends ElasticsearchIntegrationTest {
GeoDistance geoDist = response.getAggregations().get("amsterdam_rings");
assertThat(geoDist, notNullValue());
assertThat(geoDist.getName(), equalTo("amsterdam_rings"));
assertThat(geoDist.buckets().size(), equalTo(3));
assertThat(geoDist.getBuckets().size(), equalTo(3));
GeoDistance.Bucket bucket = geoDist.getByKey("ring1");
GeoDistance.Bucket bucket = geoDist.getBucketByKey("ring1");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("ring1"));
assertThat(bucket.getFrom(), equalTo(0.0));
assertThat(bucket.getTo(), equalTo(500.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(0.0));
assertThat(bucket.getTo().doubleValue(), equalTo(500.0));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = geoDist.getByKey("ring2");
bucket = geoDist.getBucketByKey("ring2");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("ring2"));
assertThat(bucket.getFrom(), equalTo(500.0));
assertThat(bucket.getTo(), equalTo(1000.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(500.0));
assertThat(bucket.getTo().doubleValue(), equalTo(1000.0));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = geoDist.getByKey("ring3");
bucket = geoDist.getBucketByKey("ring3");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("ring3"));
assertThat(bucket.getFrom(), equalTo(1000.0));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(1000.0));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(1l));
}
@ -222,27 +222,27 @@ public class GeoDistanceTests extends ElasticsearchIntegrationTest {
GeoDistance geoDist = response.getAggregations().get("amsterdam_rings");
assertThat(geoDist, notNullValue());
assertThat(geoDist.getName(), equalTo("amsterdam_rings"));
assertThat(geoDist.buckets().size(), equalTo(3));
assertThat(geoDist.getBuckets().size(), equalTo(3));
GeoDistance.Bucket bucket = geoDist.getByKey("*-500.0");
GeoDistance.Bucket bucket = geoDist.getBucketByKey("*-500.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-500.0"));
assertThat(bucket.getFrom(), equalTo(0.0));
assertThat(bucket.getTo(), equalTo(500.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(0.0));
assertThat(bucket.getTo().doubleValue(), equalTo(500.0));
assertThat(bucket.getDocCount(), equalTo(0l));
bucket = geoDist.getByKey("500.0-1000.0");
bucket = geoDist.getBucketByKey("500.0-1000.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("500.0-1000.0"));
assertThat(bucket.getFrom(), equalTo(500.0));
assertThat(bucket.getTo(), equalTo(1000.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(500.0));
assertThat(bucket.getTo().doubleValue(), equalTo(1000.0));
assertThat(bucket.getDocCount(), equalTo(0l));
bucket = geoDist.getByKey("1000.0-*");
bucket = geoDist.getBucketByKey("1000.0-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("1000.0-*"));
assertThat(bucket.getFrom(), equalTo(1000.0));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(1000.0));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(0l));
}
@ -264,27 +264,27 @@ public class GeoDistanceTests extends ElasticsearchIntegrationTest {
GeoDistance geoDist = response.getAggregations().get("amsterdam_rings");
assertThat(geoDist, notNullValue());
assertThat(geoDist.getName(), equalTo("amsterdam_rings"));
assertThat(geoDist.buckets().size(), equalTo(3));
assertThat(geoDist.getBuckets().size(), equalTo(3));
GeoDistance.Bucket bucket = geoDist.getByKey("*-500.0");
GeoDistance.Bucket bucket = geoDist.getBucketByKey("*-500.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-500.0"));
assertThat(bucket.getFrom(), equalTo(0.0));
assertThat(bucket.getTo(), equalTo(500.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(0.0));
assertThat(bucket.getTo().doubleValue(), equalTo(500.0));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = geoDist.getByKey("500.0-1000.0");
bucket = geoDist.getBucketByKey("500.0-1000.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("500.0-1000.0"));
assertThat(bucket.getFrom(), equalTo(500.0));
assertThat(bucket.getTo(), equalTo(1000.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(500.0));
assertThat(bucket.getTo().doubleValue(), equalTo(1000.0));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = geoDist.getByKey("1000.0-*");
bucket = geoDist.getBucketByKey("1000.0-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("1000.0-*"));
assertThat(bucket.getFrom(), equalTo(1000.0));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(1000.0));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(1l));
}
@ -308,50 +308,50 @@ public class GeoDistanceTests extends ElasticsearchIntegrationTest {
GeoDistance geoDist = response.getAggregations().get("amsterdam_rings");
assertThat(geoDist, notNullValue());
assertThat(geoDist.getName(), equalTo("amsterdam_rings"));
assertThat(geoDist.buckets().size(), equalTo(3));
assertThat(geoDist.getBuckets().size(), equalTo(3));
GeoDistance.Bucket bucket = geoDist.getByKey("*-500.0");
GeoDistance.Bucket bucket = geoDist.getBucketByKey("*-500.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-500.0"));
assertThat(bucket.getFrom(), equalTo(0.0));
assertThat(bucket.getTo(), equalTo(500.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(0.0));
assertThat(bucket.getTo().doubleValue(), equalTo(500.0));
assertThat(bucket.getDocCount(), equalTo(2l));
assertThat(bucket.getAggregations().asList().isEmpty(), is(false));
Terms cities = bucket.getAggregations().get("cities");
assertThat(cities, Matchers.notNullValue());
Set<String> names = Sets.newHashSet();
for (Terms.Bucket city : cities) {
names.add(city.getKey().string());
for (Terms.Bucket city : cities.getBuckets()) {
names.add(city.getKey());
}
assertThat(names.contains("utrecht") && names.contains("haarlem"), is(true));
bucket = geoDist.getByKey("500.0-1000.0");
bucket = geoDist.getBucketByKey("500.0-1000.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("500.0-1000.0"));
assertThat(bucket.getFrom(), equalTo(500.0));
assertThat(bucket.getTo(), equalTo(1000.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(500.0));
assertThat(bucket.getTo().doubleValue(), equalTo(1000.0));
assertThat(bucket.getDocCount(), equalTo(2l));
assertThat(bucket.getAggregations().asList().isEmpty(), is(false));
cities = bucket.getAggregations().get("cities");
assertThat(cities, Matchers.notNullValue());
names = Sets.newHashSet();
for (Terms.Bucket city : cities) {
names.add(city.getKey().string());
for (Terms.Bucket city : cities.getBuckets()) {
names.add(city.getKey());
}
assertThat(names.contains("berlin") && names.contains("prague"), is(true));
bucket = geoDist.getByKey("1000.0-*");
bucket = geoDist.getBucketByKey("1000.0-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("1000.0-*"));
assertThat(bucket.getFrom(), equalTo(1000.0));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(1000.0));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(1l));
assertThat(bucket.getAggregations().asList().isEmpty(), is(false));
cities = bucket.getAggregations().get("cities");
assertThat(cities, Matchers.notNullValue());
names = Sets.newHashSet();
for (Terms.Bucket city : cities) {
names.add(city.getKey().string());
for (Terms.Bucket city : cities.getBuckets()) {
names.add(city.getKey());
}
assertThat(names.contains("tel-aviv"), is(true));
}
@ -378,17 +378,18 @@ public class GeoDistanceTests extends ElasticsearchIntegrationTest {
assertThat(searchResponse.getHits().getTotalHits(), equalTo(2l));
Histogram histo = searchResponse.getAggregations().get("histo");
assertThat(histo, Matchers.notNullValue());
Histogram.Bucket bucket = histo.getByKey(1l);
Histogram.Bucket bucket = histo.getBucketByKey(1l);
assertThat(bucket, Matchers.notNullValue());
GeoDistance geoDistance = bucket.getAggregations().get("geo_dist");
List<GeoDistance.Bucket> buckets = new ArrayList<GeoDistance.Bucket>(geoDistance.getBuckets());
assertThat(geoDistance, Matchers.notNullValue());
assertThat(geoDistance.getName(), equalTo("geo_dist"));
assertThat(geoDistance.buckets().size(), is(1));
assertThat(geoDistance.buckets().get(0).getKey(), equalTo("0-100"));
assertThat(geoDistance.buckets().get(0).getFrom(), equalTo(0.0));
assertThat(geoDistance.buckets().get(0).getTo(), equalTo(100.0));
assertThat(geoDistance.buckets().get(0).getDocCount(), equalTo(0l));
assertThat(buckets.size(), is(1));
assertThat(buckets.get(0).getKey(), equalTo("0-100"));
assertThat(buckets.get(0).getFrom().doubleValue(), equalTo(0.0));
assertThat(buckets.get(0).getTo().doubleValue(), equalTo(100.0));
assertThat(buckets.get(0).getDocCount(), equalTo(0l));
}
@Test
@ -409,27 +410,27 @@ public class GeoDistanceTests extends ElasticsearchIntegrationTest {
GeoDistance geoDist = response.getAggregations().get("amsterdam_rings");
assertThat(geoDist, notNullValue());
assertThat(geoDist.getName(), equalTo("amsterdam_rings"));
assertThat(geoDist.buckets().size(), equalTo(3));
assertThat(geoDist.getBuckets().size(), equalTo(3));
GeoDistance.Bucket bucket = geoDist.getByKey("*-500.0");
GeoDistance.Bucket bucket = geoDist.getBucketByKey("*-500.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-500.0"));
assertThat(bucket.getFrom(), equalTo(0.0));
assertThat(bucket.getTo(), equalTo(500.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(0.0));
assertThat(bucket.getTo().doubleValue(), equalTo(500.0));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = geoDist.getByKey("500.0-1000.0");
bucket = geoDist.getBucketByKey("500.0-1000.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("500.0-1000.0"));
assertThat(bucket.getFrom(), equalTo(500.0));
assertThat(bucket.getTo(), equalTo(1000.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(500.0));
assertThat(bucket.getTo().doubleValue(), equalTo(1000.0));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = geoDist.getByKey("1000.0-*");
bucket = geoDist.getBucketByKey("1000.0-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("1000.0-*"));
assertThat(bucket.getFrom(), equalTo(1000.0));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(1000.0));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(1l));
}

@ -117,8 +117,8 @@ public class GeoHashGridTests extends ElasticsearchIntegrationTest {
assertThat(response.getFailedShards(), equalTo(0));
GeoHashGrid geoGrid = response.getAggregations().get("geohashgrid");
for (GeoHashGrid.Bucket cell : geoGrid) {
String geohash = cell.getGeoHash();
for (GeoHashGrid.Bucket cell : geoGrid.getBuckets()) {
String geohash = cell.getKey();
long bucketCount = cell.getDocCount();
int expectedBucketCount = expectedDocCountsForGeoHash.get(geohash);
@ -151,8 +151,8 @@ public class GeoHashGridTests extends ElasticsearchIntegrationTest {
Filter filter = response.getAggregations().get("filtered");
GeoHashGrid geoGrid = filter.getAggregations().get("geohashgrid");
for (GeoHashGrid.Bucket cell : geoGrid) {
String geohash = cell.getGeoHash();
for (GeoHashGrid.Bucket cell : geoGrid.getBuckets()) {
String geohash = cell.getKey();
long bucketCount = cell.getDocCount();
int expectedBucketCount = expectedDocCountsForGeoHash.get(geohash);
assertNotSame(bucketCount, 0);
@ -180,7 +180,7 @@ public class GeoHashGridTests extends ElasticsearchIntegrationTest {
assertThat(response.getFailedShards(), equalTo(0));
GeoHashGrid geoGrid = response.getAggregations().get("geohashgrid");
assertThat(geoGrid.getNumberOfBuckets(), equalTo(0));
assertThat(geoGrid.getBuckets().size(), equalTo(0));
}
}
@ -198,8 +198,8 @@ public class GeoHashGridTests extends ElasticsearchIntegrationTest {
assertThat(response.getFailedShards(), equalTo(0));
GeoHashGrid geoGrid = response.getAggregations().get("geohashgrid");
for (GeoHashGrid.Bucket cell : geoGrid) {
String geohash = cell.getGeoHash();
for (GeoHashGrid.Bucket cell : geoGrid.getBuckets()) {
String geohash = cell.getKey();
long bucketCount = cell.getDocCount();
int expectedBucketCount = expectedDocCountsForGeoHash.get(geohash);
@ -226,9 +226,9 @@ public class GeoHashGridTests extends ElasticsearchIntegrationTest {
GeoHashGrid geoGrid = response.getAggregations().get("geohashgrid");
//Check we only have one bucket with the best match for that resolution
assertThat(geoGrid.getNumberOfBuckets(), equalTo(1));
for (GeoHashGrid.Bucket cell : geoGrid) {
String geohash = cell.getGeoHash();
assertThat(geoGrid.getBuckets().size(), equalTo(1));
for (GeoHashGrid.Bucket cell : geoGrid.getBuckets()) {
String geohash = cell.getKey();
long bucketCount = cell.getDocCount();
int expectedBucketCount = 0;
for (ObjectIntCursor<String> cursor : expectedDocCountsForGeoHash) {

@ -116,12 +116,12 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(numValueBuckets));
assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
for (int i = 0; i < numValueBuckets; ++i) {
Histogram.Bucket bucket = histo.getByKey(i * interval);
Histogram.Bucket bucket = histo.getBucketByKey(i * interval);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo((long) i * interval));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo((long) i * interval));
assertThat(bucket.getDocCount(), equalTo(valueCounts[i]));
}
}
@ -138,12 +138,13 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(numValueBuckets));
assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
List<Histogram.Bucket> buckets = new ArrayList<Histogram.Bucket>(histo.getBuckets());
for (int i = 0; i < numValueBuckets; ++i) {
Histogram.Bucket bucket = histo.buckets().get(i);
Histogram.Bucket bucket = buckets.get(i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo((long) i * interval));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo((long) i * interval));
assertThat(bucket.getDocCount(), equalTo(valueCounts[i]));
}
}
@ -160,12 +161,13 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(numValueBuckets));
assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
List<Histogram.Bucket> buckets = new ArrayList<Histogram.Bucket>(histo.getBuckets());
for (int i = 0; i < numValueBuckets; ++i) {
Histogram.Bucket bucket = histo.buckets().get(numValueBuckets -i - 1);
Histogram.Bucket bucket = buckets.get(numValueBuckets -i - 1);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo((long) i * interval));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo((long) i * interval));
assertThat(bucket.getDocCount(), equalTo(valueCounts[i]));
}
}
@ -182,14 +184,15 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(numValueBuckets));
assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
LongOpenHashSet buckets = new LongOpenHashSet();
List<Histogram.Bucket> histoBuckets = new ArrayList<Histogram.Bucket>(histo.getBuckets());
long previousCount = Long.MIN_VALUE;
for (int i = 0; i < numValueBuckets; ++i) {
Histogram.Bucket bucket = histo.buckets().get(i);
Histogram.Bucket bucket = histoBuckets.get(i);
assertThat(bucket, notNullValue());
long key = bucket.getKey();
long key = bucket.getKeyAsNumber().longValue();
assertEquals(0, key % interval);
assertTrue(buckets.add(key));
assertThat(bucket.getDocCount(), equalTo(valueCounts[(int) (key / interval)]));
@ -210,14 +213,15 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(numValueBuckets));
assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
LongOpenHashSet buckets = new LongOpenHashSet();
List<Histogram.Bucket> histoBuckets = new ArrayList<Histogram.Bucket>(histo.getBuckets());
long previousCount = Long.MAX_VALUE;
for (int i = 0; i < numValueBuckets; ++i) {
Histogram.Bucket bucket = histo.buckets().get(i);
Histogram.Bucket bucket = histoBuckets.get(i);
assertThat(bucket, notNullValue());
long key = bucket.getKey();
long key = bucket.getKeyAsNumber().longValue();
assertEquals(0, key % interval);
assertTrue(buckets.add(key));
assertThat(bucket.getDocCount(), equalTo(valueCounts[(int) (key / interval)]));
@ -239,12 +243,13 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(numValueBuckets));
assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
List<Histogram.Bucket> buckets = new ArrayList<Histogram.Bucket>(histo.getBuckets());
for (int i = 0; i < numValueBuckets; ++i) {
Histogram.Bucket bucket = histo.buckets().get(i);
Histogram.Bucket bucket = buckets.get(i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo((long) i * interval));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo((long) i * interval));
assertThat(bucket.getDocCount(), equalTo(valueCounts[i]));
assertThat(bucket.getAggregations().asList().isEmpty(), is(false));
Sum sum = bucket.getAggregations().get("sum");
@ -272,12 +277,13 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(numValueBuckets));
assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
List<Histogram.Bucket> buckets = new ArrayList<Histogram.Bucket>(histo.getBuckets());
for (int i = 0; i < numValueBuckets; ++i) {
Histogram.Bucket bucket = histo.buckets().get(i);
Histogram.Bucket bucket = buckets.get(i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo((long) i * interval));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo((long) i * interval));
assertThat(bucket.getDocCount(), equalTo(valueCounts[i]));
assertThat(bucket.getAggregations().asList().isEmpty(), is(false));
Sum sum = bucket.getAggregations().get("sum");
@ -305,14 +311,15 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(numValueBuckets));
assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
LongOpenHashSet visited = new LongOpenHashSet();
double previousSum = Double.NEGATIVE_INFINITY;
List<Histogram.Bucket> buckets = new ArrayList<Histogram.Bucket>(histo.getBuckets());
for (int i = 0; i < numValueBuckets; ++i) {
Histogram.Bucket bucket = histo.buckets().get(i);
Histogram.Bucket bucket = buckets.get(i);
assertThat(bucket, notNullValue());
long key = bucket.getKey();
long key = bucket.getKeyAsNumber().longValue();
assertTrue(visited.add(key));
int b = (int) (key / interval);
assertThat(bucket.getDocCount(), equalTo(valueCounts[b]));
@ -344,14 +351,15 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(numValueBuckets));
assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
LongOpenHashSet visited = new LongOpenHashSet();
double previousSum = Double.POSITIVE_INFINITY;
List<Histogram.Bucket> buckets = new ArrayList<Histogram.Bucket>(histo.getBuckets());
for (int i = 0; i < numValueBuckets; ++i) {
Histogram.Bucket bucket = histo.buckets().get(i);
Histogram.Bucket bucket = buckets.get(i);
assertThat(bucket, notNullValue());
long key = bucket.getKey();
long key = bucket.getKeyAsNumber().longValue();
assertTrue(visited.add(key));
int b = (int) (key / interval);
assertThat(bucket.getDocCount(), equalTo(valueCounts[b]));
@ -383,14 +391,15 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(numValueBuckets));
assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
LongOpenHashSet visited = new LongOpenHashSet();
double previousSum = Double.NEGATIVE_INFINITY;
List<Histogram.Bucket> buckets = new ArrayList<Histogram.Bucket>(histo.getBuckets());
for (int i = 0; i < numValueBuckets; ++i) {
Histogram.Bucket bucket = histo.buckets().get(i);
Histogram.Bucket bucket = buckets.get(i);
assertThat(bucket, notNullValue());
long key = bucket.getKey();
long key = bucket.getKeyAsNumber().longValue();
assertTrue(visited.add(key));
int b = (int) (key / interval);
assertThat(bucket.getDocCount(), equalTo(valueCounts[b]));
@ -422,14 +431,15 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(numValueBuckets));
assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
LongOpenHashSet visited = new LongOpenHashSet();
double previousSum = Double.POSITIVE_INFINITY;
List<Histogram.Bucket> buckets = new ArrayList<Histogram.Bucket>(histo.getBuckets());
for (int i = 0; i < numValueBuckets; ++i) {
Histogram.Bucket bucket = histo.buckets().get(i);
Histogram.Bucket bucket = buckets.get(i);
assertThat(bucket, notNullValue());
long key = bucket.getKey();
long key = bucket.getKeyAsNumber().longValue();
assertTrue(visited.add(key));
int b = (int) (key / interval);
assertThat(bucket.getDocCount(), equalTo(valueCounts[b]));
@ -466,12 +476,12 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(numBuckets));
assertThat(histo.getBuckets().size(), equalTo(numBuckets));
for (int i = 2 / interval; i <= (numDocs + 1) / interval; ++i) {
Histogram.Bucket bucket = histo.getByKey(i * interval);
Histogram.Bucket bucket = histo.getBucketByKey(i * interval);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo((long) i * interval));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo((long) i * interval));
assertThat(bucket.getDocCount(), equalTo(counts[i]));
}
}
@ -488,12 +498,12 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(numValuesBuckets));
assertThat(histo.getBuckets().size(), equalTo(numValuesBuckets));
for (int i = 0; i < numValuesBuckets; ++i) {
Histogram.Bucket bucket = histo.getByKey(i * interval);
Histogram.Bucket bucket = histo.getBucketByKey(i * interval);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo((long) i * interval));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo((long) i * interval));
assertThat(bucket.getDocCount(), equalTo(valuesCounts[i]));
}
}
@ -510,12 +520,13 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(numValuesBuckets));
assertThat(histo.getBuckets().size(), equalTo(numValuesBuckets));
List<Histogram.Bucket> buckets = new ArrayList<Histogram.Bucket>(histo.getBuckets());
for (int i = 0; i < numValuesBuckets; ++i) {
Histogram.Bucket bucket = histo.buckets().get(numValuesBuckets -i - 1);
Histogram.Bucket bucket = buckets.get(numValuesBuckets -i - 1);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo((long) i * interval));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo((long) i * interval));
assertThat(bucket.getDocCount(), equalTo(valuesCounts[i]));
}
}
@ -543,12 +554,12 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(numBuckets));
assertThat(histo.getBuckets().size(), equalTo(numBuckets));
for (int i = 2 / interval; i <= (numDocs + 2) / interval; ++i) {
Histogram.Bucket bucket = histo.getByKey(i * interval);
Histogram.Bucket bucket = histo.getBucketByKey(i * interval);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo((long) i * interval));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo((long) i * interval));
assertThat(bucket.getDocCount(), equalTo(counts[i]));
}
}
@ -577,20 +588,20 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(numBuckets));
assertThat(histo.getBuckets().size(), equalTo(numBuckets));
for (int i = 2 / interval; i < (numDocs + 2) / interval; ++i) {
Histogram.Bucket bucket = histo.getByKey(i * interval);
Histogram.Bucket bucket = histo.getBucketByKey(i * interval);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo((long) i * interval));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo((long) i * interval));
assertThat(bucket.getDocCount(), equalTo(counts[i]));
Terms terms = bucket.getAggregations().get(MULTI_VALUED_FIELD_NAME);
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo(MULTI_VALUED_FIELD_NAME));
int minTerm = Math.max(2, i * interval - 1);
int maxTerm = Math.min(numDocs + 2, (i + 1) * interval);
assertThat(terms.buckets().size(), equalTo(maxTerm - minTerm + 1));
Iterator<Terms.Bucket> iter = terms.iterator();
assertThat(terms.getBuckets().size(), equalTo(maxTerm - minTerm + 1));
Iterator<Terms.Bucket> iter = terms.getBuckets().iterator();
for (int j = minTerm; j <= maxTerm; ++j) {
assertThat(iter.next().getKeyAsNumber().longValue(), equalTo((long) j));
}
@ -609,12 +620,12 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(numValueBuckets));
assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
for (int i = 0; i < numValueBuckets; ++i) {
Histogram.Bucket bucket = histo.getByKey(i * interval);
Histogram.Bucket bucket = histo.getBucketByKey(i * interval);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo((long) i * interval));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo((long) i * interval));
assertThat(bucket.getDocCount(), equalTo(valueCounts[i]));
}
}
@ -632,12 +643,13 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(numValueBuckets));
assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
List<Histogram.Bucket> buckets = new ArrayList<Histogram.Bucket>(histo.getBuckets());
for (int i = 0; i < numValueBuckets; ++i) {
Histogram.Bucket bucket = histo.buckets().get(i);
Histogram.Bucket bucket = buckets.get(i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo((long) i * interval));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo((long) i * interval));
assertThat(bucket.getDocCount(), equalTo(valueCounts[i]));
assertThat(bucket.getAggregations().asList().isEmpty(), is(false));
Sum sum = bucket.getAggregations().get("sum");
@ -664,12 +676,12 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(numValuesBuckets));
assertThat(histo.getBuckets().size(), equalTo(numValuesBuckets));
for (int i = 0; i < numValuesBuckets; ++i) {
Histogram.Bucket bucket = histo.getByKey(i * interval);
Histogram.Bucket bucket = histo.getBucketByKey(i * interval);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo((long) i * interval));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo((long) i * interval));
assertThat(bucket.getDocCount(), equalTo(valuesCounts[i]));
}
}
@ -687,12 +699,12 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(numValuesBuckets));
assertThat(histo.getBuckets().size(), equalTo(numValuesBuckets));
for (int i = 0; i < numValuesBuckets; ++i) {
Histogram.Bucket bucket = histo.getByKey(i * interval);
Histogram.Bucket bucket = histo.getBucketByKey(i * interval);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo((long) i * interval));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo((long) i * interval));
assertThat(bucket.getDocCount(), equalTo(valuesCounts[i]));
assertThat(bucket.getAggregations().asList().isEmpty(), is(false));
Sum sum = bucket.getAggregations().get("sum");
@ -720,7 +732,7 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(0));
assertThat(histo.getBuckets().size(), equalTo(0));
}
@Test
@ -735,12 +747,12 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
assertThat(histo.buckets().size(), equalTo(numValueBuckets));
assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
for (int i = 0; i < numValueBuckets; ++i) {
Histogram.Bucket bucket = histo.getByKey(i * interval);
Histogram.Bucket bucket = histo.getBucketByKey(i * interval);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo((long) i * interval));
assertThat(bucket.getKeyAsNumber().longValue(), equalTo((long) i * interval));
assertThat(bucket.getDocCount(), equalTo(valueCounts[i]));
}
}
@ -766,13 +778,13 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
assertThat(searchResponse.getHits().getTotalHits(), equalTo(2l));
Histogram histo = searchResponse.getAggregations().get("histo");
assertThat(histo, Matchers.notNullValue());
Histogram.Bucket bucket = histo.getByKey(1l);
Histogram.Bucket bucket = histo.getBucketByKey(1l);
assertThat(bucket, Matchers.notNullValue());
histo = bucket.getAggregations().get("sub_histo");
assertThat(histo, Matchers.notNullValue());
assertThat(histo.getName(), equalTo("sub_histo"));
assertThat(histo.buckets().isEmpty(), is(true));
assertThat(histo.getBuckets().isEmpty(), is(true));
}
}

@ -93,32 +93,32 @@ public class IPv4RangeTests extends ElasticsearchIntegrationTest {
IPv4Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
IPv4Range.Bucket bucket = range.getByKey("*-10.0.0.100");
IPv4Range.Bucket bucket = range.getBucketByKey("*-10.0.0.100");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-10.0.0.100"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsString(), nullValue());
assertThat(bucket.getToAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getDocCount(), equalTo(100l));
bucket = range.getByKey("10.0.0.100-10.0.0.200");
bucket = range.getBucketByKey("10.0.0.100-10.0.0.200");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.100-10.0.0.200"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getToAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getDocCount(), equalTo(100l));
bucket = range.getByKey("10.0.0.200-*");
bucket = range.getBucketByKey("10.0.0.200-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.200-*"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsString(), nullValue());
assertThat(bucket.getDocCount(), equalTo(55l));
}
@ -138,23 +138,23 @@ public class IPv4RangeTests extends ElasticsearchIntegrationTest {
IPv4Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(2));
assertThat(range.getBuckets().size(), equalTo(2));
IPv4Range.Bucket bucket = range.getByKey("10.0.0.0/25");
IPv4Range.Bucket bucket = range.getBucketByKey("10.0.0.0/25");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.0/25"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.0")));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.0")));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.0"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.128")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.128")));
assertThat(bucket.getToAsString(), equalTo("10.0.0.128"));
assertThat(bucket.getDocCount(), equalTo(128l));
bucket = range.getByKey("10.0.0.128/25");
bucket = range.getBucketByKey("10.0.0.128/25");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.128/25"));
assertThat((long) bucket.getFrom(), equalTo(IpFieldMapper.ipToLong("10.0.0.128")));
assertThat((long) bucket.getFrom().doubleValue(), equalTo(IpFieldMapper.ipToLong("10.0.0.128")));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.128"));
assertThat((long) bucket.getTo(), equalTo(IpFieldMapper.ipToLong("10.0.1.0"))); // range is exclusive on the to side
assertThat((long) bucket.getTo().doubleValue(), equalTo(IpFieldMapper.ipToLong("10.0.1.0"))); // range is exclusive on the to side
assertThat(bucket.getToAsString(), equalTo("10.0.1.0"));
assertThat(bucket.getDocCount(), equalTo(127l)); // include 10.0.0.128
}
@ -175,32 +175,32 @@ public class IPv4RangeTests extends ElasticsearchIntegrationTest {
IPv4Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
IPv4Range.Bucket bucket = range.getByKey("r1");
IPv4Range.Bucket bucket = range.getBucketByKey("r1");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("r1"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsString(), nullValue());
assertThat(bucket.getToAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getDocCount(), equalTo(100l));
bucket = range.getByKey("r2");
bucket = range.getBucketByKey("r2");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("r2"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getToAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getDocCount(), equalTo(100l));
bucket = range.getByKey("r3");
bucket = range.getBucketByKey("r3");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("r3"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsString(), nullValue());
assertThat(bucket.getDocCount(), equalTo(55l));
}
@ -222,38 +222,38 @@ public class IPv4RangeTests extends ElasticsearchIntegrationTest {
IPv4Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
IPv4Range.Bucket bucket = range.getByKey("*-10.0.0.100");
IPv4Range.Bucket bucket = range.getBucketByKey("*-10.0.0.100");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-10.0.0.100"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsString(), nullValue());
assertThat(bucket.getToAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getDocCount(), equalTo(100l));
Sum sum = bucket.getAggregations().get("sum");
assertThat(sum, notNullValue());
assertThat(sum.getValue(), equalTo((double) 100));
bucket = range.getByKey("10.0.0.100-10.0.0.200");
bucket = range.getBucketByKey("10.0.0.100-10.0.0.200");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.100-10.0.0.200"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getToAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getDocCount(), equalTo(100l));
sum = bucket.getAggregations().get("sum");
assertThat(sum, notNullValue());
assertThat(sum.getValue(), equalTo((double) 200));
bucket = range.getByKey("10.0.0.200-*");
bucket = range.getBucketByKey("10.0.0.200-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.200-*"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsString(), nullValue());
assertThat(bucket.getDocCount(), equalTo(55l));
sum = bucket.getAggregations().get("sum");
@ -278,38 +278,38 @@ public class IPv4RangeTests extends ElasticsearchIntegrationTest {
IPv4Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
IPv4Range.Bucket bucket = range.getByKey("*-10.0.0.100");
IPv4Range.Bucket bucket = range.getBucketByKey("*-10.0.0.100");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-10.0.0.100"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsString(), nullValue());
assertThat(bucket.getToAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getDocCount(), equalTo(100l));
Max max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
assertThat(max.getValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.99")));
bucket = range.getByKey("10.0.0.100-10.0.0.200");
bucket = range.getBucketByKey("10.0.0.100-10.0.0.200");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.100-10.0.0.200"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getToAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getDocCount(), equalTo(100l));
max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
assertThat(max.getValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.199")));
bucket = range.getByKey("10.0.0.200-*");
bucket = range.getBucketByKey("10.0.0.200-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.200-*"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsString(), nullValue());
assertThat(bucket.getDocCount(), equalTo(55l));
max = bucket.getAggregations().get("max");
@ -334,32 +334,32 @@ public class IPv4RangeTests extends ElasticsearchIntegrationTest {
IPv4Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
IPv4Range.Bucket bucket = range.getByKey("*-10.0.0.100");
IPv4Range.Bucket bucket = range.getBucketByKey("*-10.0.0.100");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-10.0.0.100"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsString(), nullValue());
assertThat(bucket.getToAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getDocCount(), equalTo(100l));
bucket = range.getByKey("10.0.0.100-10.0.0.200");
bucket = range.getBucketByKey("10.0.0.100-10.0.0.200");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.100-10.0.0.200"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getToAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getDocCount(), equalTo(100l));
bucket = range.getByKey("10.0.0.200-*");
bucket = range.getBucketByKey("10.0.0.200-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.200-*"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsString(), nullValue());
assertThat(bucket.getDocCount(), equalTo(55l));
}
@ -397,32 +397,32 @@ public class IPv4RangeTests extends ElasticsearchIntegrationTest {
IPv4Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
IPv4Range.Bucket bucket = range.getByKey("*-10.0.0.100");
IPv4Range.Bucket bucket = range.getBucketByKey("*-10.0.0.100");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-10.0.0.100"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsString(), nullValue());
assertThat(bucket.getToAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getDocCount(), equalTo(100l));
bucket = range.getByKey("10.0.0.100-10.0.0.200");
bucket = range.getBucketByKey("10.0.0.100-10.0.0.200");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.100-10.0.0.200"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getToAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getDocCount(), equalTo(101l));
bucket = range.getByKey("10.0.0.200-*");
bucket = range.getBucketByKey("10.0.0.200-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.200-*"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsString(), nullValue());
assertThat(bucket.getDocCount(), equalTo(56l));
}
@ -444,32 +444,32 @@ public class IPv4RangeTests extends ElasticsearchIntegrationTest {
IPv4Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
IPv4Range.Bucket bucket = range.getByKey("*-10.0.0.100");
IPv4Range.Bucket bucket = range.getBucketByKey("*-10.0.0.100");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-10.0.0.100"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsString(), nullValue());
assertThat(bucket.getToAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getDocCount(), equalTo(100l));
bucket = range.getByKey("10.0.0.100-10.0.0.200");
bucket = range.getBucketByKey("10.0.0.100-10.0.0.200");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.100-10.0.0.200"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getToAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getDocCount(), equalTo(101l));
bucket = range.getByKey("10.0.0.200-*");
bucket = range.getBucketByKey("10.0.0.200-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.200-*"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsString(), nullValue());
assertThat(bucket.getDocCount(), equalTo(56l));
}
@ -492,38 +492,38 @@ public class IPv4RangeTests extends ElasticsearchIntegrationTest {
IPv4Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
IPv4Range.Bucket bucket = range.getByKey("*-10.0.0.100");
IPv4Range.Bucket bucket = range.getBucketByKey("*-10.0.0.100");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-10.0.0.100"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsString(), nullValue());
assertThat(bucket.getToAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getDocCount(), equalTo(100l));
Max max = bucket.getAggregations().get("max");
assertThat(max, Matchers.notNullValue());
assertThat((long) max.getValue(), equalTo(IpFieldMapper.ipToLong("10.0.0.100")));
bucket = range.getByKey("10.0.0.100-10.0.0.200");
bucket = range.getBucketByKey("10.0.0.100-10.0.0.200");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.100-10.0.0.200"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getToAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getDocCount(), equalTo(101l));
max = bucket.getAggregations().get("max");
assertThat(max, Matchers.notNullValue());
assertThat((long) max.getValue(), equalTo(IpFieldMapper.ipToLong("10.0.0.200")));
bucket = range.getByKey("10.0.0.200-*");
bucket = range.getBucketByKey("10.0.0.200-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.200-*"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsString(), nullValue());
assertThat(bucket.getDocCount(), equalTo(56l));
max = bucket.getAggregations().get("max");
@ -547,32 +547,32 @@ public class IPv4RangeTests extends ElasticsearchIntegrationTest {
IPv4Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
IPv4Range.Bucket bucket = range.getByKey("*-10.0.0.100");
IPv4Range.Bucket bucket = range.getBucketByKey("*-10.0.0.100");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-10.0.0.100"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsString(), nullValue());
assertThat(bucket.getToAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getDocCount(), equalTo(100l));
bucket = range.getByKey("10.0.0.100-10.0.0.200");
bucket = range.getBucketByKey("10.0.0.100-10.0.0.200");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.100-10.0.0.200"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getToAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getDocCount(), equalTo(100l));
bucket = range.getByKey("10.0.0.200-*");
bucket = range.getBucketByKey("10.0.0.200-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.200-*"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsString(), nullValue());
assertThat(bucket.getDocCount(), equalTo(55l));
}
@ -594,38 +594,38 @@ public class IPv4RangeTests extends ElasticsearchIntegrationTest {
IPv4Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
IPv4Range.Bucket bucket = range.getByKey("*-10.0.0.100");
IPv4Range.Bucket bucket = range.getBucketByKey("*-10.0.0.100");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-10.0.0.100"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsString(), nullValue());
assertThat(bucket.getToAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getDocCount(), equalTo(100l));
Max max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
assertThat(max.getValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.99")));
bucket = range.getByKey("10.0.0.100-10.0.0.200");
bucket = range.getBucketByKey("10.0.0.100-10.0.0.200");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.100-10.0.0.200"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getToAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getDocCount(), equalTo(100l));
max = bucket.getAggregations().get("max");
assertThat(max, notNullValue());
assertThat(max.getValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.199")));
bucket = range.getByKey("10.0.0.200-*");
bucket = range.getBucketByKey("10.0.0.200-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.200-*"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsString(), nullValue());
assertThat(bucket.getDocCount(), equalTo(55l));
max = bucket.getAggregations().get("max");
@ -649,32 +649,32 @@ public class IPv4RangeTests extends ElasticsearchIntegrationTest {
IPv4Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
IPv4Range.Bucket bucket = range.getByKey("*-10.0.0.100");
IPv4Range.Bucket bucket = range.getBucketByKey("*-10.0.0.100");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-10.0.0.100"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsString(), nullValue());
assertThat(bucket.getToAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getDocCount(), equalTo(100l));
bucket = range.getByKey("10.0.0.100-10.0.0.200");
bucket = range.getBucketByKey("10.0.0.100-10.0.0.200");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.100-10.0.0.200"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getToAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getDocCount(), equalTo(101l));
bucket = range.getByKey("10.0.0.200-*");
bucket = range.getBucketByKey("10.0.0.200-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.200-*"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsString(), nullValue());
assertThat(bucket.getDocCount(), equalTo(56l));
}
@ -696,38 +696,38 @@ public class IPv4RangeTests extends ElasticsearchIntegrationTest {
IPv4Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
IPv4Range.Bucket bucket = range.getByKey("*-10.0.0.100");
IPv4Range.Bucket bucket = range.getBucketByKey("*-10.0.0.100");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-10.0.0.100"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsString(), nullValue());
assertThat(bucket.getToAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getDocCount(), equalTo(100l));
Max max = bucket.getAggregations().get("max");
assertThat(max, Matchers.notNullValue());
assertThat((long) max.getValue(), equalTo(IpFieldMapper.ipToLong("10.0.0.100")));
bucket = range.getByKey("10.0.0.100-10.0.0.200");
bucket = range.getBucketByKey("10.0.0.100-10.0.0.200");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.100-10.0.0.200"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getToAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getDocCount(), equalTo(101l));
max = bucket.getAggregations().get("max");
assertThat(max, Matchers.notNullValue());
assertThat((long) max.getValue(), equalTo(IpFieldMapper.ipToLong("10.0.0.200")));
bucket = range.getByKey("10.0.0.200-*");
bucket = range.getBucketByKey("10.0.0.200-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.200-*"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsString(), nullValue());
assertThat(bucket.getDocCount(), equalTo(56l));
max = bucket.getAggregations().get("max");
@ -751,32 +751,32 @@ public class IPv4RangeTests extends ElasticsearchIntegrationTest {
IPv4Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
IPv4Range.Bucket bucket = range.getByKey("*-10.0.0.100");
IPv4Range.Bucket bucket = range.getBucketByKey("*-10.0.0.100");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-10.0.0.100"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsString(), nullValue());
assertThat(bucket.getToAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getDocCount(), equalTo(0l));
bucket = range.getByKey("10.0.0.100-10.0.0.200");
bucket = range.getBucketByKey("10.0.0.100-10.0.0.200");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.100-10.0.0.200"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getToAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getDocCount(), equalTo(0l));
bucket = range.getByKey("10.0.0.200-*");
bucket = range.getBucketByKey("10.0.0.200-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.200-*"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsString(), nullValue());
assertThat(bucket.getDocCount(), equalTo(0l));
}
@ -797,32 +797,32 @@ public class IPv4RangeTests extends ElasticsearchIntegrationTest {
IPv4Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
IPv4Range.Bucket bucket = range.getByKey("*-10.0.0.100");
IPv4Range.Bucket bucket = range.getBucketByKey("*-10.0.0.100");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-10.0.0.100"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getFromAsString(), nullValue());
assertThat(bucket.getToAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getDocCount(), equalTo(100l));
bucket = range.getByKey("10.0.0.100-10.0.0.200");
bucket = range.getBucketByKey("10.0.0.100-10.0.0.200");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.100-10.0.0.200"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.100"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.100")));
assertThat(bucket.getToAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getTo(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getDocCount(), equalTo(100l));
bucket = range.getByKey("10.0.0.200-*");
bucket = range.getBucketByKey("10.0.0.200-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("10.0.0.200-*"));
assertThat(bucket.getFromAsString(), equalTo("10.0.0.200"));
assertThat(bucket.getFrom(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo((double) IpFieldMapper.ipToLong("10.0.0.200")));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getToAsString(), nullValue());
assertThat(bucket.getDocCount(), equalTo(55l));
}
@ -849,16 +849,17 @@ public class IPv4RangeTests extends ElasticsearchIntegrationTest {
assertThat(searchResponse.getHits().getTotalHits(), equalTo(2l));
Histogram histo = searchResponse.getAggregations().get("histo");
assertThat(histo, Matchers.notNullValue());
Histogram.Bucket bucket = histo.getByKey(1l);
Histogram.Bucket bucket = histo.getBucketByKey(1l);
assertThat(bucket, Matchers.notNullValue());
IPv4Range range = bucket.getAggregations().get("ip_range");
List<IPv4Range.Bucket> buckets = new ArrayList<IPv4Range.Bucket>(range.getBuckets());
assertThat(range, Matchers.notNullValue());
assertThat(range.getName(), equalTo("ip_range"));
assertThat(range.buckets().size(), is(1));
assertThat(range.buckets().get(0).getKey(), equalTo("r1"));
assertThat(range.buckets().get(0).getFromAsString(), equalTo("10.0.0.1"));
assertThat(range.buckets().get(0).getToAsString(), equalTo("10.0.0.10"));
assertThat(range.buckets().get(0).getDocCount(), equalTo(0l));
assertThat(buckets.size(), is(1));
assertThat(buckets.get(0).getKey(), equalTo("r1"));
assertThat(buckets.get(0).getFromAsString(), equalTo("10.0.0.1"));
assertThat(buckets.get(0).getToAsString(), equalTo("10.0.0.10"));
assertThat(buckets.get(0).getDocCount(), equalTo(0l));
}
}

@ -88,6 +88,10 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
createIndex("idx_unmapped");
ensureSearchable();
}
private String key(Terms.Bucket bucket) {
return randomBoolean() ? bucket.getKey() : key(bucket);
}
@Test
// the main purpose of this test is to make sure we're not allocating 2GB of memory per shard
@ -104,7 +108,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(100));
assertThat(terms.getBuckets().size(), equalTo(100));
}
@Test
@ -120,12 +124,12 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("" + i);
Terms.Bucket bucket = terms.getBucketByKey("" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + i));
assertThat(key(bucket), equalTo("" + i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
assertThat(bucket.getDocCount(), equalTo(1l));
}
@ -146,12 +150,12 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(20));
assertThat(terms.getBuckets().size(), equalTo(20));
for (int i = 0; i < 20; i++) {
Terms.Bucket bucket = terms.getByTerm("" + i);
Terms.Bucket bucket = terms.getBucketByKey("" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + i));
assertThat(key(bucket), equalTo("" + i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
assertThat(bucket.getDocCount(), equalTo(1l));
}
@ -169,12 +173,12 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
int i = 0;
for (Terms.Bucket bucket : terms.buckets()) {
for (Terms.Bucket bucket : terms.getBuckets()) {
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + i));
assertThat(key(bucket), equalTo("" + i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
assertThat(bucket.getDocCount(), equalTo(1l));
i++;
@ -195,12 +199,12 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
int i = 4;
for (Terms.Bucket bucket : terms.buckets()) {
for (Terms.Bucket bucket : terms.getBuckets()) {
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + i));
assertThat(key(bucket), equalTo("" + i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
assertThat(bucket.getDocCount(), equalTo(1l));
i--;
@ -221,12 +225,12 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("" + i);
Terms.Bucket bucket = terms.getBucketByKey("" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + i));
assertThat(key(bucket), equalTo("" + i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
assertThat(bucket.getDocCount(), equalTo(1l));
Sum sum = bucket.getAggregations().get("sum");
@ -249,12 +253,12 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("" + i);
Terms.Bucket bucket = terms.getBucketByKey("" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + i));
assertThat(key(bucket), equalTo("" + i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
assertThat(bucket.getDocCount(), equalTo(1l));
Sum sum = bucket.getAggregations().get("sum");
@ -277,12 +281,12 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("" + (i+1d));
Terms.Bucket bucket = terms.getBucketByKey("" + (i + 1d));
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (i+1d)));
assertThat(key(bucket), equalTo("" + (i+1d)));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i+1));
assertThat(bucket.getDocCount(), equalTo(1l));
}
@ -301,12 +305,12 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(6));
assertThat(terms.getBuckets().size(), equalTo(6));
for (int i = 0; i < 6; i++) {
Terms.Bucket bucket = terms.getByTerm("" + i);
Terms.Bucket bucket = terms.getBucketByKey("" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + i));
assertThat(key(bucket), equalTo("" + i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
if (i == 0 || i == 5) {
assertThat(bucket.getDocCount(), equalTo(1l));
@ -330,12 +334,12 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(6));
assertThat(terms.getBuckets().size(), equalTo(6));
for (int i = 0; i < 6; i++) {
Terms.Bucket bucket = terms.getByTerm("" + (i-1d));
Terms.Bucket bucket = terms.getBucketByKey("" + (i - 1d));
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (i-1d)));
assertThat(key(bucket), equalTo("" + (i-1d)));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i-1));
if (i == 0 || i == 5) {
assertThat(bucket.getDocCount(), equalTo(1l));
@ -359,11 +363,11 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(1));
assertThat(terms.getBuckets().size(), equalTo(1));
Terms.Bucket bucket = terms.getByTerm("1.0");
Terms.Bucket bucket = terms.getBucketByKey("1.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("1.0"));
assertThat(key(bucket), equalTo("1.0"));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(1));
assertThat(bucket.getDocCount(), equalTo(5l));
}
@ -400,12 +404,12 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(6));
assertThat(terms.getBuckets().size(), equalTo(6));
for (int i = 0; i < 6; i++) {
Terms.Bucket bucket = terms.getByTerm("" + (i+1d));
Terms.Bucket bucket = terms.getBucketByKey("" + (i + 1d));
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + (i+1d)));
assertThat(key(bucket), equalTo("" + (i+1d)));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i+1));
final long count = i == 0 || i == 5 ? 1 : 2;
double s = 0;
@ -435,12 +439,12 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("" + i);
Terms.Bucket bucket = terms.getBucketByKey("" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + i));
assertThat(key(bucket), equalTo("" + i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
assertThat(bucket.getDocCount(), equalTo(1l));
}
@ -460,12 +464,12 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("" + i);
Terms.Bucket bucket = terms.getBucketByKey("" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + i));
assertThat(key(bucket), equalTo("" + i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
assertThat(bucket.getDocCount(), equalTo(1l));
Sum sum = bucket.getAggregations().get("sum");
@ -487,12 +491,12 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(6));
assertThat(terms.getBuckets().size(), equalTo(6));
for (int i = 0; i < 6; i++) {
Terms.Bucket bucket = terms.getByTerm("" + i);
Terms.Bucket bucket = terms.getBucketByKey("" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + i));
assertThat(key(bucket), equalTo("" + i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
if (i == 0 || i == 5) {
assertThat(bucket.getDocCount(), equalTo(1l));
@ -538,12 +542,12 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(6));
assertThat(terms.getBuckets().size(), equalTo(6));
for (int i = 0; i < 6; i++) {
Terms.Bucket bucket = terms.getByTerm("" + i);
Terms.Bucket bucket = terms.getBucketByKey("" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + i));
assertThat(key(bucket), equalTo("" + i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
final long count = i == 0 || i == 5 ? 1 : 2;
double s = 0;
@ -574,7 +578,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(0));
assertThat(terms.getBuckets().size(), equalTo(0));
}
@Test
@ -590,12 +594,12 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("" + i);
Terms.Bucket bucket = terms.getBucketByKey("" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + i));
assertThat(key(bucket), equalTo("" + i));
assertThat(bucket.getKeyAsNumber().intValue(), equalTo(i));
assertThat(bucket.getDocCount(), equalTo(1l));
}
@ -622,13 +626,13 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
assertThat(searchResponse.getHits().getTotalHits(), equalTo(2l));
Histogram histo = searchResponse.getAggregations().get("histo");
assertThat(histo, Matchers.notNullValue());
Histogram.Bucket bucket = histo.getByKey(1l);
Histogram.Bucket bucket = histo.getBucketByKey(1l);
assertThat(bucket, Matchers.notNullValue());
Terms terms = bucket.getAggregations().get("terms");
assertThat(terms, Matchers.notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().isEmpty(), is(true));
assertThat(terms.getBuckets().isEmpty(), is(true));
}
@Test
@ -647,12 +651,12 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("" + i);
Terms.Bucket bucket = terms.getBucketByKey("" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + i));
assertThat(key(bucket), equalTo("" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
Avg avg = bucket.getAggregations().get("avg_i");
assertThat(avg, notNullValue());
@ -753,13 +757,13 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 4; i >= 0; i--) {
Terms.Bucket bucket = terms.getByTerm("" + i);
Terms.Bucket bucket = terms.getBucketByKey("" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + i));
assertThat(key(bucket), equalTo("" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
Avg avg = bucket.getAggregations().get("avg_i");
@ -784,12 +788,12 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("" + i);
Terms.Bucket bucket = terms.getBucketByKey("" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + i));
assertThat(key(bucket), equalTo("" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
Stats stats = bucket.getAggregations().get("stats");
@ -814,12 +818,12 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 4; i >= 0; i--) {
Terms.Bucket bucket = terms.getByTerm("" + i);
Terms.Bucket bucket = terms.getBucketByKey("" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + i));
assertThat(key(bucket), equalTo("" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
Stats stats = bucket.getAggregations().get("stats");
@ -844,12 +848,12 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("" + i);
Terms.Bucket bucket = terms.getBucketByKey("" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("" + i));
assertThat(key(bucket), equalTo("" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
ExtendedStats stats = bucket.getAggregations().get("stats");

@ -116,18 +116,18 @@ public class MinDocCountTests extends ElasticsearchIntegrationTest {
// check that terms2 is a subset of terms1
private void assertSubset(Terms terms1, Terms terms2, long minDocCount, int size, String include) {
final Matcher matcher = include == null ? null : Pattern.compile(include).matcher("");;
final Iterator<Terms.Bucket> it1 = terms1.iterator();
final Iterator<Terms.Bucket> it2 = terms2.iterator();
final Iterator<Terms.Bucket> it1 = terms1.getBuckets().iterator();
final Iterator<Terms.Bucket> it2 = terms2.getBuckets().iterator();
int size2 = 0;
while (it1.hasNext()) {
final Terms.Bucket bucket1 = it1.next();
if (bucket1.getDocCount() >= minDocCount && (matcher == null || matcher.reset(bucket1.getKey().string()).matches())) {
if (bucket1.getDocCount() >= minDocCount && (matcher == null || matcher.reset(bucket1.getKey()).matches())) {
if (size2++ == size) {
break;
}
assertTrue(it2.hasNext());
final Terms.Bucket bucket2 = it2.next();
assertEquals(bucket1.getKey(), bucket2.getKey());
assertEquals(bucket1.getKeyAsText(), bucket2.getKeyAsText());
assertEquals(bucket1.getDocCount(), bucket2.getDocCount());
}
}
@ -135,22 +135,22 @@ public class MinDocCountTests extends ElasticsearchIntegrationTest {
}
private void assertSubset(Histogram histo1, Histogram histo2, long minDocCount) {
final Iterator<Histogram.Bucket> it2 = histo2.iterator();
for (Histogram.Bucket b1 : histo1) {
final Iterator<? extends Histogram.Bucket> it2 = histo2.getBuckets().iterator();
for (Histogram.Bucket b1 : histo1.getBuckets()) {
if (b1.getDocCount() >= minDocCount) {
final Histogram.Bucket b2 = it2.next();
assertEquals(b1.getKey(), b2.getKey());
assertEquals(b1.getKeyAsNumber(), b2.getKeyAsNumber());
assertEquals(b1.getDocCount(), b2.getDocCount());
}
}
}
private void assertSubset(DateHistogram histo1, DateHistogram histo2, long minDocCount) {
final Iterator<DateHistogram.Bucket> it2 = histo2.iterator();
for (DateHistogram.Bucket b1 : histo1) {
final Iterator<? extends DateHistogram.Bucket> it2 = histo2.getBuckets().iterator();
for (DateHistogram.Bucket b1 : histo1.getBuckets()) {
if (b1.getDocCount() >= minDocCount) {
final DateHistogram.Bucket b2 = it2.next();
assertEquals(b1.getKey(), b2.getKey());
assertEquals(b1.getKeyAsNumber(), b2.getKeyAsNumber());
assertEquals(b1.getDocCount(), b2.getDocCount());
}
}
@ -284,7 +284,7 @@ public class MinDocCountTests extends ElasticsearchIntegrationTest {
.minDocCount(0))
.execute().actionGet();
final Terms allTerms = allTermsResponse.getAggregations().get("terms");
assertEquals(cardinality, allTerms.buckets().size());
assertEquals(cardinality, allTerms.getBuckets().size());
for (long minDocCount = 0; minDocCount < 20; ++minDocCount) {
final int size = randomIntBetween(1, cardinality + 2);

@ -209,7 +209,7 @@ public class MissingTests extends ElasticsearchIntegrationTest {
assertThat(searchResponse.getHits().getTotalHits(), equalTo(2l));
Histogram histo = searchResponse.getAggregations().get("histo");
assertThat(histo, Matchers.notNullValue());
Histogram.Bucket bucket = histo.getByKey(1l);
Histogram.Bucket bucket = histo.getBucketByKey(1l);
assertThat(bucket, Matchers.notNullValue());
Missing missing = bucket.getAggregations().get("missing");

@ -188,14 +188,14 @@ public class NestedTests extends ElasticsearchIntegrationTest {
LongTerms values = nested.getAggregations().get("values");
assertThat(values, notNullValue());
assertThat(values.getName(), equalTo("values"));
assertThat(values.buckets(), notNullValue());
assertThat(values.buckets().size(), equalTo(uniqueValues));
assertThat(values.getBuckets(), notNullValue());
assertThat(values.getBuckets().size(), equalTo(uniqueValues));
for (int i = 0; i < counts.length; ++i) {
final String key = Long.toString(i);
if (counts[i] == 0) {
assertNull(values.getByTerm(key));
assertNull(values.getBucketByKey(key));
} else {
Bucket bucket = values.getByTerm(key);
Bucket bucket = values.getBucketByKey(key);
assertNotNull(bucket);
assertEquals(counts[i], bucket.getDocCount());
}
@ -216,13 +216,13 @@ public class NestedTests extends ElasticsearchIntegrationTest {
LongTerms values = response.getAggregations().get("top_values");
assertThat(values, notNullValue());
assertThat(values.getName(), equalTo("top_values"));
assertThat(values.buckets(), notNullValue());
assertThat(values.buckets().size(), equalTo(numParents));
assertThat(values.getBuckets(), notNullValue());
assertThat(values.getBuckets().size(), equalTo(numParents));
for (int i = 0; i < numParents; i++) {
String topValue = "" + (i + 1);
assertThat(values.getByTerm(topValue), notNullValue());
Nested nested = values.getByTerm(topValue).getAggregations().get("nested");
assertThat(values.getBucketByKey(topValue), notNullValue());
Nested nested = values.getBucketByKey(topValue).getAggregations().get("nested");
assertThat(nested, notNullValue());
Max max = nested.getAggregations().get("max_value");
assertThat(max, notNullValue());
@ -258,7 +258,7 @@ public class NestedTests extends ElasticsearchIntegrationTest {
assertThat(searchResponse.getHits().getTotalHits(), equalTo(2l));
Histogram histo = searchResponse.getAggregations().get("histo");
assertThat(histo, Matchers.notNullValue());
Histogram.Bucket bucket = histo.getByKey(1l);
Histogram.Bucket bucket = histo.getBucketByKey(1l);
assertThat(bucket, Matchers.notNullValue());
Nested nested = bucket.getAggregations().get("nested");

@ -91,14 +91,14 @@ public class RangeTests extends ElasticsearchIntegrationTest {
assertSearchResponse(response);
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.buckets().size(), equalTo(numDocs + 1));
assertThat(terms.getBuckets().size(), equalTo(numDocs + 1));
for (int i = 1; i < numDocs + 2; ++i) {
Terms.Bucket bucket = terms.getByTerm("" + i);
Terms.Bucket bucket = terms.getBucketByKey("" + i);
assertThat(bucket, notNullValue());
final long docCount = i == 1 || i == numDocs + 1 ? 1 : 2;
assertThat(bucket.getDocCount(), equalTo(docCount));
Range range = bucket.getAggregations().get("range");
Range.Bucket rangeBucket = range.getByKey("*-3.0");
Range.Bucket rangeBucket = range.getBucketByKey("*-3.0");
assertThat(rangeBucket, notNullValue());
if (i == 1 || i == 3) {
assertThat(rangeBucket.getDocCount(), equalTo(1L));
@ -107,7 +107,7 @@ public class RangeTests extends ElasticsearchIntegrationTest {
} else {
assertThat(rangeBucket.getDocCount(), equalTo(0L));
}
rangeBucket = range.getByKey("3.0-6.0");
rangeBucket = range.getBucketByKey("3.0-6.0");
assertThat(rangeBucket, notNullValue());
if (i == 3 || i == 6) {
assertThat(rangeBucket.getDocCount(), equalTo(1L));
@ -116,7 +116,7 @@ public class RangeTests extends ElasticsearchIntegrationTest {
} else {
assertThat(rangeBucket.getDocCount(), equalTo(0L));
}
rangeBucket = range.getByKey("6.0-*");
rangeBucket = range.getBucketByKey("6.0-*");
assertThat(rangeBucket, notNullValue());
if (i == 6 || i == numDocs + 1) {
assertThat(rangeBucket.getDocCount(), equalTo(1L));
@ -144,27 +144,27 @@ public class RangeTests extends ElasticsearchIntegrationTest {
Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
Range.Bucket bucket = range.getByKey("*-3.0");
Range.Bucket bucket = range.getBucketByKey("*-3.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-3.0"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo(), equalTo(3.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(3.0));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("3.0-6.0");
bucket = range.getBucketByKey("3.0-6.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("3.0-6.0"));
assertThat(bucket.getFrom(), equalTo(3.0));
assertThat(bucket.getTo(), equalTo(6.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(3.0));
assertThat(bucket.getTo().doubleValue(), equalTo(6.0));
assertThat(bucket.getDocCount(), equalTo(3l));
bucket = range.getByKey("6.0-*");
bucket = range.getBucketByKey("6.0-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("6.0-*"));
assertThat(bucket.getFrom(), equalTo(6.0));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(6.0));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(numDocs - 5L));
}
@ -184,27 +184,27 @@ public class RangeTests extends ElasticsearchIntegrationTest {
Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
Range.Bucket bucket = range.getByKey("r1");
Range.Bucket bucket = range.getBucketByKey("r1");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("r1"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo(), equalTo(3.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(3.0));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("r2");
bucket = range.getBucketByKey("r2");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("r2"));
assertThat(bucket.getFrom(), equalTo(3.0));
assertThat(bucket.getTo(), equalTo(6.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(3.0));
assertThat(bucket.getTo().doubleValue(), equalTo(6.0));
assertThat(bucket.getDocCount(), equalTo(3l));
bucket = range.getByKey("r3");
bucket = range.getBucketByKey("r3");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("r3"));
assertThat(bucket.getFrom(), equalTo(6.0));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(6.0));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(numDocs - 5L));
}
@ -225,33 +225,33 @@ public class RangeTests extends ElasticsearchIntegrationTest {
Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
Range.Bucket bucket = range.getByKey("*-3.0");
Range.Bucket bucket = range.getBucketByKey("*-3.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-3.0"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo(), equalTo(3.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(3.0));
assertThat(bucket.getDocCount(), equalTo(2l));
Sum sum = bucket.getAggregations().get("sum");
assertThat(sum, notNullValue());
assertThat(sum.getValue(), equalTo(3.0)); // 1 + 2
bucket = range.getByKey("3.0-6.0");
bucket = range.getBucketByKey("3.0-6.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("3.0-6.0"));
assertThat(bucket.getFrom(), equalTo(3.0));
assertThat(bucket.getTo(), equalTo(6.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(3.0));
assertThat(bucket.getTo().doubleValue(), equalTo(6.0));
assertThat(bucket.getDocCount(), equalTo(3l));
sum = bucket.getAggregations().get("sum");
assertThat(sum, notNullValue());
assertThat(sum.getValue(), equalTo(12.0)); // 3 + 4 + 5
bucket = range.getByKey("6.0-*");
bucket = range.getBucketByKey("6.0-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("6.0-*"));
assertThat(bucket.getFrom(), equalTo(6.0));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(6.0));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(numDocs - 5l));
sum = bucket.getAggregations().get("sum");
assertThat(sum, notNullValue());
@ -279,33 +279,33 @@ public class RangeTests extends ElasticsearchIntegrationTest {
Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
Range.Bucket bucket = range.getByKey("*-3.0");
Range.Bucket bucket = range.getBucketByKey("*-3.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-3.0"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo(), equalTo(3.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(3.0));
assertThat(bucket.getDocCount(), equalTo(2l));
Avg avg = bucket.getAggregations().get("avg");
assertThat(avg, notNullValue());
assertThat(avg.getValue(), equalTo(1.5)); // (1 + 2) / 2
bucket = range.getByKey("3.0-6.0");
bucket = range.getBucketByKey("3.0-6.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("3.0-6.0"));
assertThat(bucket.getFrom(), equalTo(3.0));
assertThat(bucket.getTo(), equalTo(6.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(3.0));
assertThat(bucket.getTo().doubleValue(), equalTo(6.0));
assertThat(bucket.getDocCount(), equalTo(3l));
avg = bucket.getAggregations().get("avg");
assertThat(avg, notNullValue());
assertThat(avg.getValue(), equalTo(4.0)); // (3 + 4 + 5) / 3
bucket = range.getByKey("6.0-*");
bucket = range.getBucketByKey("6.0-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("6.0-*"));
assertThat(bucket.getFrom(), equalTo(6.0));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(6.0));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(numDocs - 5l));
avg = bucket.getAggregations().get("avg");
assertThat(avg, notNullValue());
@ -333,27 +333,27 @@ public class RangeTests extends ElasticsearchIntegrationTest {
Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
Range.Bucket bucket = range.getByKey("*-3.0");
Range.Bucket bucket = range.getBucketByKey("*-3.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-3.0"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo(), equalTo(3.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(3.0));
assertThat(bucket.getDocCount(), equalTo(1l)); // 2
bucket = range.getByKey("3.0-6.0");
bucket = range.getBucketByKey("3.0-6.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("3.0-6.0"));
assertThat(bucket.getFrom(), equalTo(3.0));
assertThat(bucket.getTo(), equalTo(6.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(3.0));
assertThat(bucket.getTo().doubleValue(), equalTo(6.0));
assertThat(bucket.getDocCount(), equalTo(3l)); // 3, 4, 5
bucket = range.getByKey("6.0-*");
bucket = range.getBucketByKey("6.0-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("6.0-*"));
assertThat(bucket.getFrom(), equalTo(6.0));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(6.0));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(numDocs - 4l));
}
@ -386,27 +386,27 @@ public class RangeTests extends ElasticsearchIntegrationTest {
Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
Range.Bucket bucket = range.getByKey("*-3.0");
Range.Bucket bucket = range.getBucketByKey("*-3.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-3.0"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo(), equalTo(3.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(3.0));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("3.0-6.0");
bucket = range.getBucketByKey("3.0-6.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("3.0-6.0"));
assertThat(bucket.getFrom(), equalTo(3.0));
assertThat(bucket.getTo(), equalTo(6.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(3.0));
assertThat(bucket.getTo().doubleValue(), equalTo(6.0));
assertThat(bucket.getDocCount(), equalTo(4l));
bucket = range.getByKey("6.0-*");
bucket = range.getBucketByKey("6.0-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("6.0-*"));
assertThat(bucket.getFrom(), equalTo(6.0));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(6.0));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(numDocs - 4l));
}
@ -440,27 +440,27 @@ public class RangeTests extends ElasticsearchIntegrationTest {
Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
Range.Bucket bucket = range.getByKey("*-3.0");
Range.Bucket bucket = range.getBucketByKey("*-3.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-3.0"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo(), equalTo(3.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(3.0));
assertThat(bucket.getDocCount(), equalTo(1l));
bucket = range.getByKey("3.0-6.0");
bucket = range.getBucketByKey("3.0-6.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("3.0-6.0"));
assertThat(bucket.getFrom(), equalTo(3.0));
assertThat(bucket.getTo(), equalTo(6.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(3.0));
assertThat(bucket.getTo().doubleValue(), equalTo(6.0));
assertThat(bucket.getDocCount(), equalTo(4l));
bucket = range.getByKey("6.0-*");
bucket = range.getBucketByKey("6.0-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("6.0-*"));
assertThat(bucket.getFrom(), equalTo(6.0));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(6.0));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(numDocs - 3l));
}
@ -499,35 +499,35 @@ public class RangeTests extends ElasticsearchIntegrationTest {
Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
Range.Bucket bucket = range.getByKey("*-3.0");
Range.Bucket bucket = range.getBucketByKey("*-3.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-3.0"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo(), equalTo(3.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(3.0));
assertThat(bucket.getDocCount(), equalTo(1l));
Sum sum = bucket.getAggregations().get("sum");
assertThat(sum, notNullValue());
assertThat(sum.getName(), equalTo("sum"));
assertThat(sum.getValue(), equalTo(2d+3d));
bucket = range.getByKey("3.0-6.0");
bucket = range.getBucketByKey("3.0-6.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("3.0-6.0"));
assertThat(bucket.getFrom(), equalTo(3.0));
assertThat(bucket.getTo(), equalTo(6.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(3.0));
assertThat(bucket.getTo().doubleValue(), equalTo(6.0));
assertThat(bucket.getDocCount(), equalTo(4l));
sum = bucket.getAggregations().get("sum");
assertThat(sum, notNullValue());
assertThat(sum.getName(), equalTo("sum"));
assertThat(sum.getValue(), equalTo((double) 2+3+3+4+4+5+5+6));
bucket = range.getByKey("6.0-*");
bucket = range.getBucketByKey("6.0-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("6.0-*"));
assertThat(bucket.getFrom(), equalTo(6.0));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(6.0));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(numDocs - 3L));
sum = bucket.getAggregations().get("sum");
assertThat(sum, notNullValue());
@ -555,27 +555,27 @@ public class RangeTests extends ElasticsearchIntegrationTest {
Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
Range.Bucket bucket = range.getByKey("*-3.0");
Range.Bucket bucket = range.getBucketByKey("*-3.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-3.0"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo(), equalTo(3.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(3.0));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("3.0-6.0");
bucket = range.getBucketByKey("3.0-6.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("3.0-6.0"));
assertThat(bucket.getFrom(), equalTo(3.0));
assertThat(bucket.getTo(), equalTo(6.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(3.0));
assertThat(bucket.getTo().doubleValue(), equalTo(6.0));
assertThat(bucket.getDocCount(), equalTo(3l));
bucket = range.getByKey("6.0-*");
bucket = range.getBucketByKey("6.0-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("6.0-*"));
assertThat(bucket.getFrom(), equalTo(6.0));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(6.0));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(numDocs - 5l));
}
@ -596,33 +596,33 @@ public class RangeTests extends ElasticsearchIntegrationTest {
Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
Range.Bucket bucket = range.getByKey("*-3.0");
Range.Bucket bucket = range.getBucketByKey("*-3.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-3.0"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo(), equalTo(3.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(3.0));
assertThat(bucket.getDocCount(), equalTo(2l));
Avg avg = bucket.getAggregations().get("avg");
assertThat(avg, notNullValue());
assertThat(avg.getValue(), equalTo(1.5)); // (1 + 2) / 2
bucket = range.getByKey("3.0-6.0");
bucket = range.getBucketByKey("3.0-6.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("3.0-6.0"));
assertThat(bucket.getFrom(), equalTo(3.0));
assertThat(bucket.getTo(), equalTo(6.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(3.0));
assertThat(bucket.getTo().doubleValue(), equalTo(6.0));
assertThat(bucket.getDocCount(), equalTo(3l));
avg = bucket.getAggregations().get("avg");
assertThat(avg, notNullValue());
assertThat(avg.getValue(), equalTo(4.0)); // (3 + 4 + 5) / 3
bucket = range.getByKey("6.0-*");
bucket = range.getBucketByKey("6.0-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("6.0-*"));
assertThat(bucket.getFrom(), equalTo(6.0));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(6.0));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(numDocs - 5l));
avg = bucket.getAggregations().get("avg");
assertThat(avg, notNullValue());
@ -648,20 +648,20 @@ public class RangeTests extends ElasticsearchIntegrationTest {
Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(2));
assertThat(range.getBuckets().size(), equalTo(2));
Range.Bucket bucket = range.getByKey("*--1.0");
Range.Bucket bucket = range.getBucketByKey("*--1.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*--1.0"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo(), equalTo(-1.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(-1.0));
assertThat(bucket.getDocCount(), equalTo(0l));
bucket = range.getByKey("1000.0-*");
bucket = range.getBucketByKey("1000.0-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("1000.0-*"));
assertThat(bucket.getFrom(), equalTo(1000d));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(1000d));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(0l));
}
@ -681,27 +681,27 @@ public class RangeTests extends ElasticsearchIntegrationTest {
Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
Range.Bucket bucket = range.getByKey("*-3.0");
Range.Bucket bucket = range.getBucketByKey("*-3.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-3.0"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo(), equalTo(3.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(3.0));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("3.0-6.0");
bucket = range.getBucketByKey("3.0-6.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("3.0-6.0"));
assertThat(bucket.getFrom(), equalTo(3.0));
assertThat(bucket.getTo(), equalTo(6.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(3.0));
assertThat(bucket.getTo().doubleValue(), equalTo(6.0));
assertThat(bucket.getDocCount(), equalTo(4l));
bucket = range.getByKey("6.0-*");
bucket = range.getBucketByKey("6.0-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("6.0-*"));
assertThat(bucket.getFrom(), equalTo(6.0));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(6.0));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(numDocs - 4l));
}
@ -739,35 +739,35 @@ public class RangeTests extends ElasticsearchIntegrationTest {
Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
Range.Bucket bucket = range.getByKey("r1");
Range.Bucket bucket = range.getBucketByKey("r1");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("r1"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo(), equalTo(3.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(3.0));
assertThat(bucket.getDocCount(), equalTo(2l));
Sum sum = bucket.getAggregations().get("sum");
assertThat(sum, notNullValue());
assertThat(sum.getName(), equalTo("sum"));
assertThat(sum.getValue(), equalTo((double) 1+2+2+3));
bucket = range.getByKey("r2");
bucket = range.getBucketByKey("r2");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("r2"));
assertThat(bucket.getFrom(), equalTo(3.0));
assertThat(bucket.getTo(), equalTo(6.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(3.0));
assertThat(bucket.getTo().doubleValue(), equalTo(6.0));
assertThat(bucket.getDocCount(), equalTo(4l));
sum = bucket.getAggregations().get("sum");
assertThat(sum, notNullValue());
assertThat(sum.getName(), equalTo("sum"));
assertThat(sum.getValue(), equalTo((double) 2+3+3+4+4+5+5+6));
bucket = range.getByKey("r3");
bucket = range.getBucketByKey("r3");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("r3"));
assertThat(bucket.getFrom(), equalTo(6.0));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(6.0));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(numDocs - 4l));
sum = bucket.getAggregations().get("sum");
assertThat(sum, notNullValue());
@ -795,27 +795,27 @@ public class RangeTests extends ElasticsearchIntegrationTest {
Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
Range.Bucket bucket = range.getByKey("*-3.0");
Range.Bucket bucket = range.getBucketByKey("*-3.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-3.0"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo(), equalTo(3.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(3.0));
assertThat(bucket.getDocCount(), equalTo(0l));
bucket = range.getByKey("3.0-6.0");
bucket = range.getBucketByKey("3.0-6.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("3.0-6.0"));
assertThat(bucket.getFrom(), equalTo(3.0));
assertThat(bucket.getTo(), equalTo(6.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(3.0));
assertThat(bucket.getTo().doubleValue(), equalTo(6.0));
assertThat(bucket.getDocCount(), equalTo(0l));
bucket = range.getByKey("6.0-*");
bucket = range.getBucketByKey("6.0-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("6.0-*"));
assertThat(bucket.getFrom(), equalTo(6.0));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(6.0));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(0l));
}
@ -837,27 +837,27 @@ public class RangeTests extends ElasticsearchIntegrationTest {
Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(3));
assertThat(range.getBuckets().size(), equalTo(3));
Range.Bucket bucket = range.getByKey("*-3.0");
Range.Bucket bucket = range.getBucketByKey("*-3.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-3.0"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo(), equalTo(3.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(3.0));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("3.0-6.0");
bucket = range.getBucketByKey("3.0-6.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("3.0-6.0"));
assertThat(bucket.getFrom(), equalTo(3.0));
assertThat(bucket.getTo(), equalTo(6.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(3.0));
assertThat(bucket.getTo().doubleValue(), equalTo(6.0));
assertThat(bucket.getDocCount(), equalTo(3l));
bucket = range.getByKey("6.0-*");
bucket = range.getBucketByKey("6.0-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("6.0-*"));
assertThat(bucket.getFrom(), equalTo(6.0));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(6.0));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(numDocs - 5l));
}
@ -878,34 +878,34 @@ public class RangeTests extends ElasticsearchIntegrationTest {
Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), equalTo(4));
assertThat(range.getBuckets().size(), equalTo(4));
Range.Bucket bucket = range.getByKey("*-5.0");
Range.Bucket bucket = range.getBucketByKey("*-5.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("*-5.0"));
assertThat(bucket.getFrom(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo(), equalTo(5.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(Double.NEGATIVE_INFINITY));
assertThat(bucket.getTo().doubleValue(), equalTo(5.0));
assertThat(bucket.getDocCount(), equalTo(4l));
bucket = range.getByKey("3.0-6.0");
bucket = range.getBucketByKey("3.0-6.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("3.0-6.0"));
assertThat(bucket.getFrom(), equalTo(3.0));
assertThat(bucket.getTo(), equalTo(6.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(3.0));
assertThat(bucket.getTo().doubleValue(), equalTo(6.0));
assertThat(bucket.getDocCount(), equalTo(4l));
bucket = range.getByKey("4.0-5.0");
bucket = range.getBucketByKey("4.0-5.0");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("4.0-5.0"));
assertThat(bucket.getFrom(), equalTo(4.0));
assertThat(bucket.getTo(), equalTo(5.0));
assertThat(bucket.getFrom().doubleValue(), equalTo(4.0));
assertThat(bucket.getTo().doubleValue(), equalTo(5.0));
assertThat(bucket.getDocCount(), equalTo(2l));
bucket = range.getByKey("4.0-*");
bucket = range.getBucketByKey("4.0-*");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey(), equalTo("4.0-*"));
assertThat(bucket.getFrom(), equalTo(4.0));
assertThat(bucket.getTo(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getFrom().doubleValue(), equalTo(4.0));
assertThat(bucket.getTo().doubleValue(), equalTo(Double.POSITIVE_INFINITY));
assertThat(bucket.getDocCount(), equalTo(numDocs - 2l));
}
@ -930,17 +930,18 @@ public class RangeTests extends ElasticsearchIntegrationTest {
assertThat(searchResponse.getHits().getTotalHits(), equalTo(2l));
Histogram histo = searchResponse.getAggregations().get("histo");
assertThat(histo, Matchers.notNullValue());
Histogram.Bucket bucket = histo.getByKey(1l);
Histogram.Bucket bucket = histo.getBucketByKey(1l);
assertThat(bucket, Matchers.notNullValue());
Range range = bucket.getAggregations().get("range");
List<Range.Bucket> buckets = new ArrayList<Range.Bucket>(range.getBuckets());
assertThat(range, Matchers.notNullValue());
assertThat(range.getName(), equalTo("range"));
assertThat(range.buckets().size(), is(1));
assertThat(range.buckets().get(0).getKey(), equalTo("0-2"));
assertThat(range.buckets().get(0).getFrom(), equalTo(0.0));
assertThat(range.buckets().get(0).getTo(), equalTo(2.0));
assertThat(range.buckets().get(0).getDocCount(), equalTo(0l));
assertThat(buckets.size(), is(1));
assertThat(buckets.get(0).getKey(), equalTo("0-2"));
assertThat(buckets.get(0).getFrom().doubleValue(), equalTo(0.0));
assertThat(buckets.get(0).getTo().doubleValue(), equalTo(2.0));
assertThat(buckets.get(0).getDocCount(), equalTo(0l));
}
}

@ -42,8 +42,6 @@ import org.junit.Test;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.search.aggregations.AggregationBuilders.*;
import static org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope;
import static org.elasticsearch.test.ElasticsearchIntegrationTest.Scope;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.equalTo;
@ -106,7 +104,7 @@ public class ShardReduceTests extends ElasticsearchIntegrationTest {
Global global = response.getAggregations().get("global");
DateHistogram histo = global.getAggregations().get("histo");
assertThat(histo.buckets().size(), equalTo(4));
assertThat(histo.getBuckets().size(), equalTo(4));
}
@Test
@ -122,7 +120,7 @@ public class ShardReduceTests extends ElasticsearchIntegrationTest {
Filter filter = response.getAggregations().get("filter");
DateHistogram histo = filter.getAggregations().get("histo");
assertThat(histo.buckets().size(), equalTo(4));
assertThat(histo.getBuckets().size(), equalTo(4));
}
@Test
@ -138,7 +136,7 @@ public class ShardReduceTests extends ElasticsearchIntegrationTest {
Missing missing = response.getAggregations().get("missing");
DateHistogram histo = missing.getAggregations().get("histo");
assertThat(histo.buckets().size(), equalTo(4));
assertThat(histo.getBuckets().size(), equalTo(4));
}
@Test
@ -158,7 +156,7 @@ public class ShardReduceTests extends ElasticsearchIntegrationTest {
Filter filter = global.getAggregations().get("filter");
Missing missing = filter.getAggregations().get("missing");
DateHistogram histo = missing.getAggregations().get("histo");
assertThat(histo.buckets().size(), equalTo(4));
assertThat(histo.getBuckets().size(), equalTo(4));
}
@Test
@ -174,7 +172,7 @@ public class ShardReduceTests extends ElasticsearchIntegrationTest {
Nested nested = response.getAggregations().get("nested");
DateHistogram histo = nested.getAggregations().get("histo");
assertThat(histo.buckets().size(), equalTo(4));
assertThat(histo.getBuckets().size(), equalTo(4));
}
@Test
@ -189,8 +187,8 @@ public class ShardReduceTests extends ElasticsearchIntegrationTest {
assertSearchResponse(response);
Terms terms = response.getAggregations().get("terms");
DateHistogram histo = terms.getByTerm("term").getAggregations().get("histo");
assertThat(histo.buckets().size(), equalTo(4));
DateHistogram histo = terms.getBucketByKey("term").getAggregations().get("histo");
assertThat(histo.getBuckets().size(), equalTo(4));
}
@Test
@ -205,8 +203,8 @@ public class ShardReduceTests extends ElasticsearchIntegrationTest {
assertSearchResponse(response);
Terms terms = response.getAggregations().get("terms");
DateHistogram histo = terms.getByTerm("1").getAggregations().get("histo");
assertThat(histo.buckets().size(), equalTo(4));
DateHistogram histo = terms.getBucketByKey("1").getAggregations().get("histo");
assertThat(histo.getBuckets().size(), equalTo(4));
}
@Test
@ -221,8 +219,8 @@ public class ShardReduceTests extends ElasticsearchIntegrationTest {
assertSearchResponse(response);
Terms terms = response.getAggregations().get("terms");
DateHistogram histo = terms.getByTerm("1.5").getAggregations().get("histo");
assertThat(histo.buckets().size(), equalTo(4));
DateHistogram histo = terms.getBucketByKey("1.5").getAggregations().get("histo");
assertThat(histo.getBuckets().size(), equalTo(4));
}
@Test
@ -237,8 +235,8 @@ public class ShardReduceTests extends ElasticsearchIntegrationTest {
assertSearchResponse(response);
Range range = response.getAggregations().get("range");
DateHistogram histo = range.getByKey("r1").getAggregations().get("histo");
assertThat(histo.buckets().size(), equalTo(4));
DateHistogram histo = range.getBucketByKey("r1").getAggregations().get("histo");
assertThat(histo.getBuckets().size(), equalTo(4));
}
@Test
@ -253,8 +251,8 @@ public class ShardReduceTests extends ElasticsearchIntegrationTest {
assertSearchResponse(response);
DateRange range = response.getAggregations().get("range");
DateHistogram histo = range.getByKey("r1").getAggregations().get("histo");
assertThat(histo.buckets().size(), equalTo(4));
DateHistogram histo = range.getBucketByKey("r1").getAggregations().get("histo");
assertThat(histo.getBuckets().size(), equalTo(4));
}
@Test
@ -269,8 +267,8 @@ public class ShardReduceTests extends ElasticsearchIntegrationTest {
assertSearchResponse(response);
IPv4Range range = response.getAggregations().get("range");
DateHistogram histo = range.getByKey("r1").getAggregations().get("histo");
assertThat(histo.buckets().size(), equalTo(4));
DateHistogram histo = range.getBucketByKey("r1").getAggregations().get("histo");
assertThat(histo.getBuckets().size(), equalTo(4));
}
@Test
@ -285,8 +283,8 @@ public class ShardReduceTests extends ElasticsearchIntegrationTest {
assertSearchResponse(response);
Histogram topHisto = response.getAggregations().get("topHisto");
DateHistogram histo = topHisto.getByKey(0).getAggregations().get("histo");
assertThat(histo.buckets().size(), equalTo(4));
DateHistogram histo = topHisto.getBucketByKey(0).getAggregations().get("histo");
assertThat(histo.getBuckets().size(), equalTo(4));
}
@Test
@ -301,8 +299,9 @@ public class ShardReduceTests extends ElasticsearchIntegrationTest {
assertSearchResponse(response);
DateHistogram topHisto = response.getAggregations().get("topHisto");
DateHistogram histo = topHisto.iterator().next().getAggregations().get("histo");
assertThat(histo.buckets().size(), equalTo(4));
DateHistogram histo = topHisto.getBuckets().iterator().next().getAggregations().get("histo");
assertThat(histo.getBuckets().size(), equalTo(4));
}
@Test
@ -317,8 +316,8 @@ public class ShardReduceTests extends ElasticsearchIntegrationTest {
assertSearchResponse(response);
GeoHashGrid grid = response.getAggregations().get("grid");
DateHistogram histo = grid.iterator().next().getAggregations().get("histo");
assertThat(histo.buckets().size(), equalTo(4));
DateHistogram histo = grid.getBuckets().iterator().next().getAggregations().get("histo");
assertThat(histo.getBuckets().size(), equalTo(4));
}

@ -76,7 +76,7 @@ public class ShardSizeTermsTests extends ElasticsearchIntegrationTest {
.execute().actionGet();
Terms terms = response.getAggregations().get("keys");
Collection<Terms.Bucket> buckets = terms.buckets();
Collection<Terms.Bucket> buckets = terms.getBuckets();
assertThat(buckets.size(), equalTo(3));
Map<String, Long> expected = ImmutableMap.<String, Long>builder()
.put("1", 8l)
@ -84,7 +84,7 @@ public class ShardSizeTermsTests extends ElasticsearchIntegrationTest {
.put("2", 4l)
.build();
for (Terms.Bucket bucket : buckets) {
assertThat(bucket.getDocCount(), equalTo(expected.get(bucket.getKey().string())));
assertThat(bucket.getDocCount(), equalTo(expected.get(bucket.getKeyAsText().string())));
}
}
@ -103,7 +103,7 @@ public class ShardSizeTermsTests extends ElasticsearchIntegrationTest {
.execute().actionGet();
Terms terms = response.getAggregations().get("keys");
Collection<Terms.Bucket> buckets = terms.buckets();
Collection<Terms.Bucket> buckets = terms.getBuckets();
assertThat(buckets.size(), equalTo(3)); // we still only return 3 entries (based on the 'size' param)
Map<String, Long> expected = ImmutableMap.<String, Long>builder()
.put("1", 8l)
@ -111,7 +111,7 @@ public class ShardSizeTermsTests extends ElasticsearchIntegrationTest {
.put("2", 5l) // <-- count is now fixed
.build();
for (Terms.Bucket bucket : buckets) {
assertThat(bucket.getDocCount(), equalTo(expected.get(bucket.getKey().string())));
assertThat(bucket.getDocCount(), equalTo(expected.get(bucket.getKeyAsText().string())));
}
}
@ -130,7 +130,7 @@ public class ShardSizeTermsTests extends ElasticsearchIntegrationTest {
.execute().actionGet();
Terms terms = response.getAggregations().get("keys");
Collection<Terms.Bucket> buckets = terms.buckets();
Collection<Terms.Bucket> buckets = terms.getBuckets();
assertThat(buckets.size(), equalTo(3)); // we still only return 3 entries (based on the 'size' param)
Map<String, Long> expected = ImmutableMap.<String, Long>builder()
.put("1", 5l)
@ -138,7 +138,7 @@ public class ShardSizeTermsTests extends ElasticsearchIntegrationTest {
.put("3", 3l) // <-- count is now fixed
.build();
for (Terms.Bucket bucket: buckets) {
assertThat(bucket.getDocCount(), equalTo(expected.get(bucket.getKey().string())));
assertThat(bucket.getDocCount(), equalTo(expected.get(bucket.getKey())));
}
}
@ -157,7 +157,7 @@ public class ShardSizeTermsTests extends ElasticsearchIntegrationTest {
.execute().actionGet();
Terms terms = response.getAggregations().get("keys");
Collection<Terms.Bucket> buckets = terms.buckets();
Collection<Terms.Bucket> buckets = terms.getBuckets();
assertThat(buckets.size(), equalTo(3));
Map<Integer, Long> expected = ImmutableMap.<Integer, Long>builder()
.put(1, 8l)
@ -184,7 +184,7 @@ public class ShardSizeTermsTests extends ElasticsearchIntegrationTest {
.execute().actionGet();
Terms terms = response.getAggregations().get("keys");
Collection<Terms.Bucket> buckets = terms.buckets();
Collection<Terms.Bucket> buckets = terms.getBuckets();
assertThat(buckets.size(), equalTo(3)); // we still only return 3 entries (based on the 'size' param)
Map<Integer, Long> expected = ImmutableMap.<Integer, Long>builder()
.put(1, 8l)
@ -211,7 +211,7 @@ public class ShardSizeTermsTests extends ElasticsearchIntegrationTest {
.execute().actionGet();
Terms terms = response.getAggregations().get("keys");
Collection<Terms.Bucket> buckets = terms.buckets();
Collection<Terms.Bucket> buckets = terms.getBuckets();
assertThat(buckets.size(), equalTo(3)); // we still only return 3 entries (based on the 'size' param)
Map<Integer, Long> expected = ImmutableMap.<Integer, Long>builder()
.put(1, 5l)
@ -238,7 +238,7 @@ public class ShardSizeTermsTests extends ElasticsearchIntegrationTest {
.execute().actionGet();
Terms terms = response.getAggregations().get("keys");
Collection<Terms.Bucket> buckets = terms.buckets();
Collection<Terms.Bucket> buckets = terms.getBuckets();
assertThat(buckets.size(), equalTo(3));
Map<Integer, Long> expected = ImmutableMap.<Integer, Long>builder()
.put(1, 8l)
@ -265,7 +265,7 @@ public class ShardSizeTermsTests extends ElasticsearchIntegrationTest {
.execute().actionGet();
Terms terms = response.getAggregations().get("keys");
Collection<Terms.Bucket> buckets = terms.buckets();
Collection<Terms.Bucket> buckets = terms.getBuckets();
assertThat(buckets.size(), equalTo(3));
Map<Integer, Long> expected = ImmutableMap.<Integer, Long>builder()
.put(1, 8l)
@ -292,7 +292,7 @@ public class ShardSizeTermsTests extends ElasticsearchIntegrationTest {
.execute().actionGet();
Terms terms = response.getAggregations().get("keys");
Collection<Terms.Bucket> buckets = terms.buckets();
Collection<Terms.Bucket> buckets = terms.getBuckets();
assertThat(buckets.size(), equalTo(3));
Map<Integer, Long> expected = ImmutableMap.<Integer, Long>builder()
.put(1, 5l)

@ -99,6 +99,10 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
ensureSearchable();
}
private String key(Terms.Bucket bucket) {
return randomBoolean() ? bucket.getKey() : bucket.getKeyAsText().string();
}
@Test
// the main purpose of this test is to make sure we're not allocating 2GB of memory per shard
public void sizeIsZero() {
@ -116,7 +120,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(minDocCount == 0 ? 105 : 100)); // 105 because of the other type
assertThat(terms.getBuckets().size(), equalTo(minDocCount == 0 ? 105 : 100)); // 105 because of the other type
}
@Test
@ -132,12 +136,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("val" + i);
Terms.Bucket bucket = terms.getBucketByKey("val" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val" + i));
assertThat(key(bucket), equalTo("val" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
}
}
@ -158,12 +162,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(10));
assertThat(terms.getBuckets().size(), equalTo(10));
for (int i = 0; i < 10; i++) {
Terms.Bucket bucket = terms.getByTerm("val00" + i);
Terms.Bucket bucket = terms.getBucketByKey("val00" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val00" + i));
assertThat(key(bucket), equalTo("val00" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
}
@ -180,12 +184,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(8));
assertThat(terms.getBuckets().size(), equalTo(8));
for (int i = 2; i < 10; i++) {
Terms.Bucket bucket = terms.getByTerm("val00" + i);
Terms.Bucket bucket = terms.getBucketByKey("val00" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val00" + i));
assertThat(key(bucket), equalTo("val00" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
}
@ -202,12 +206,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(10));
assertThat(terms.getBuckets().size(), equalTo(10));
for (int i = 0; i < 10; i++) {
Terms.Bucket bucket = terms.getByTerm("val00" + i);
Terms.Bucket bucket = terms.getBucketByKey("val00" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val00" + i));
assertThat(key(bucket), equalTo("val00" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
}
}
@ -229,12 +233,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(10));
assertThat(terms.getBuckets().size(), equalTo(10));
for (int i = 0; i < 10; i++) {
Terms.Bucket bucket = terms.getByTerm("val00" + i);
Terms.Bucket bucket = terms.getBucketByKey("val00" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val00" + i));
assertThat(key(bucket), equalTo("val00" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
}
@ -252,12 +256,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(8));
assertThat(terms.getBuckets().size(), equalTo(8));
for (int i = 2; i < 10; i++) {
Terms.Bucket bucket = terms.getByTerm("val00" + i);
Terms.Bucket bucket = terms.getBucketByKey("val00" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val00" + i));
assertThat(key(bucket), equalTo("val00" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
}
@ -275,12 +279,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(10));
assertThat(terms.getBuckets().size(), equalTo(10));
for (int i = 0; i < 10; i++) {
Terms.Bucket bucket = terms.getByTerm("val00" + i);
Terms.Bucket bucket = terms.getBucketByKey("val00" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val00" + i));
assertThat(key(bucket), equalTo("val00" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
}
}
@ -301,12 +305,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(20));
assertThat(terms.getBuckets().size(), equalTo(20));
for (int i = 0; i < 20; i++) {
Terms.Bucket bucket = terms.getByTerm("val" + Strings.padStart(i+"", 3, '0'));
Terms.Bucket bucket = terms.getBucketByKey("val" + Strings.padStart(i + "", 3, '0'));
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val" + Strings.padStart(i+"", 3, '0')));
assertThat(key(bucket), equalTo("val" + Strings.padStart(i+"", 3, '0')));
assertThat(bucket.getDocCount(), equalTo(1l));
}
}
@ -325,12 +329,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
int i = 0;
for (Terms.Bucket bucket : terms.buckets()) {
for (Terms.Bucket bucket : terms.getBuckets()) {
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val" + i));
assertThat(key(bucket), equalTo("val" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
i++;
}
@ -350,12 +354,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
int i = 4;
for (Terms.Bucket bucket : terms.buckets()) {
for (Terms.Bucket bucket : terms.getBuckets()) {
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val" + i));
assertThat(key(bucket), equalTo("val" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
i--;
}
@ -375,12 +379,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("val" + i);
Terms.Bucket bucket = terms.getBucketByKey("val" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val" + i));
assertThat(key(bucket), equalTo("val" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
ValueCount valueCount = bucket.getAggregations().get("count");
assertThat(valueCount, notNullValue());
@ -402,12 +406,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("val" + i);
Terms.Bucket bucket = terms.getBucketByKey("val" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val" + i));
assertThat(key(bucket), equalTo("val" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
ValueCount valueCount = bucket.getAggregations().get("count");
assertThat(valueCount, notNullValue());
@ -429,12 +433,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("foo_val" + i);
Terms.Bucket bucket = terms.getBucketByKey("foo_val" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("foo_val" + i));
assertThat(key(bucket), equalTo("foo_val" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
}
}
@ -453,11 +457,11 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(1));
assertThat(terms.getBuckets().size(), equalTo(1));
Terms.Bucket bucket = terms.getByTerm("val");
Terms.Bucket bucket = terms.getBucketByKey("val");
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val"));
assertThat(key(bucket), equalTo("val"));
assertThat(bucket.getDocCount(), equalTo(5l));
}
@ -474,12 +478,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(6));
assertThat(terms.getBuckets().size(), equalTo(6));
for (int i = 0; i < 6; i++) {
Terms.Bucket bucket = terms.getByTerm("val" + i);
Terms.Bucket bucket = terms.getBucketByKey("val" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val" + i));
assertThat(key(bucket), equalTo("val" + i));
if (i == 0 || i == 5) {
assertThat(bucket.getDocCount(), equalTo(1l));
} else {
@ -502,12 +506,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(6));
assertThat(terms.getBuckets().size(), equalTo(6));
for (int i = 0; i < 6; i++) {
Terms.Bucket bucket = terms.getByTerm("foo_val" + i);
Terms.Bucket bucket = terms.getBucketByKey("foo_val" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("foo_val" + i));
assertThat(key(bucket), equalTo("foo_val" + i));
if (i == 0 || i == 5) {
assertThat(bucket.getDocCount(), equalTo(1l));
} else {
@ -549,12 +553,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(6));
assertThat(terms.getBuckets().size(), equalTo(6));
for (int i = 0; i < 6; i++) {
Terms.Bucket bucket = terms.getByTerm("foo_val" + i);
Terms.Bucket bucket = terms.getBucketByKey("foo_val" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("foo_val" + i));
assertThat(key(bucket), equalTo("foo_val" + i));
if (i == 0 | i == 5) {
assertThat(bucket.getDocCount(), equalTo(1l));
ValueCount valueCount = bucket.getAggregations().get("count");
@ -564,7 +568,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
assertThat(bucket.getDocCount(), equalTo(2l));
ValueCount valueCount = bucket.getAggregations().get("count");
assertThat(valueCount, notNullValue());
assertThat("term[" + bucket.getKey().string() + "]", valueCount.getValue(), equalTo(4l));
assertThat("term[" + key(bucket) + "]", valueCount.getValue(), equalTo(4l));
}
}
}
@ -582,12 +586,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("val" + i);
Terms.Bucket bucket = terms.getBucketByKey("val" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val" + i));
assertThat(key(bucket), equalTo("val" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
}
}
@ -605,12 +609,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("val" + i);
Terms.Bucket bucket = terms.getBucketByKey("val" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val" + i));
assertThat(key(bucket), equalTo("val" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
}
}
@ -629,12 +633,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("val" + i);
Terms.Bucket bucket = terms.getBucketByKey("val" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val" + i));
assertThat(key(bucket), equalTo("val" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
ValueCount valueCount = bucket.getAggregations().get("count");
assertThat(valueCount, notNullValue());
@ -655,12 +659,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(6));
assertThat(terms.getBuckets().size(), equalTo(6));
for (int i = 0; i < 6; i++) {
Terms.Bucket bucket = terms.getByTerm("val" + i);
Terms.Bucket bucket = terms.getBucketByKey("val" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val" + i));
assertThat(key(bucket), equalTo("val" + i));
if (i == 0 || i == 5) {
assertThat(bucket.getDocCount(), equalTo(1l));
} else {
@ -683,12 +687,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(6));
assertThat(terms.getBuckets().size(), equalTo(6));
for (int i = 0; i < 6; i++) {
Terms.Bucket bucket = terms.getByTerm("val" + i);
Terms.Bucket bucket = terms.getBucketByKey("val" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val" + i));
assertThat(key(bucket), equalTo("val" + i));
if (i == 0 | i == 5) {
assertThat(bucket.getDocCount(), equalTo(1l));
ValueCount valueCount = bucket.getAggregations().get("count");
@ -717,7 +721,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(0));
assertThat(terms.getBuckets().size(), equalTo(0));
}
@Test
@ -733,12 +737,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("val" + i);
Terms.Bucket bucket = terms.getBucketByKey("val" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val" + i));
assertThat(key(bucket), equalTo("val" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
}
}
@ -757,12 +761,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = filter.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(3));
assertThat(terms.getBuckets().size(), equalTo(3));
for (int i = 2; i <= 4; i++) {
Terms.Bucket bucket = terms.getByTerm("val" + i);
Terms.Bucket bucket = terms.getBucketByKey("val" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val" + i));
assertThat(key(bucket), equalTo("val" + i));
assertThat(bucket.getDocCount(), equalTo(i == 3 ? 2L : 1L));
}
}
@ -788,13 +792,13 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
assertThat(searchResponse.getHits().getTotalHits(), equalTo(2l));
Histogram histo = searchResponse.getAggregations().get("histo");
assertThat(histo, Matchers.notNullValue());
Histogram.Bucket bucket = histo.getByKey(1l);
Histogram.Bucket bucket = histo.getBucketByKey(1l);
assertThat(bucket, Matchers.notNullValue());
Terms terms = bucket.getAggregations().get("terms");
assertThat(terms, Matchers.notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().isEmpty(), is(true));
assertThat(terms.getBuckets().isEmpty(), is(true));
}
@Test
@ -814,12 +818,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("val" + i);
Terms.Bucket bucket = terms.getBucketByKey("val" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val" + i));
assertThat(key(bucket), equalTo("val" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
Avg avg = bucket.getAggregations().get("avg_i");
assertThat(avg, notNullValue());
@ -925,13 +929,13 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 4; i >= 0; i--) {
Terms.Bucket bucket = terms.getByTerm("val" + i);
Terms.Bucket bucket = terms.getBucketByKey("val" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val" + i));
assertThat(key(bucket), equalTo("val" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
Avg avg = bucket.getAggregations().get("avg_i");
@ -957,12 +961,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("val" + i);
Terms.Bucket bucket = terms.getBucketByKey("val" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val" + i));
assertThat(key(bucket), equalTo("val" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
Stats stats = bucket.getAggregations().get("stats");
@ -988,12 +992,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 4; i >= 0; i--) {
Terms.Bucket bucket = terms.getByTerm("val" + i);
Terms.Bucket bucket = terms.getBucketByKey("val" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val" + i));
assertThat(key(bucket), equalTo("val" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
Stats stats = bucket.getAggregations().get("stats");
@ -1019,12 +1023,12 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.buckets().size(), equalTo(5));
assertThat(terms.getBuckets().size(), equalTo(5));
for (int i = 0; i < 5; i++) {
Terms.Bucket bucket = terms.getByTerm("val" + i);
Terms.Bucket bucket = terms.getBucketByKey("val" + i);
assertThat(bucket, notNullValue());
assertThat(bucket.getKey().string(), equalTo("val" + i));
assertThat(key(bucket), equalTo("val" + i));
assertThat(bucket.getDocCount(), equalTo(1l));
ExtendedStats stats = bucket.getAggregations().get("stats");

@ -45,7 +45,7 @@ public class AvgTests extends AbstractNumericTests {
assertThat(searchResponse.getHits().getTotalHits(), equalTo(2l));
Histogram histo = searchResponse.getAggregations().get("histo");
assertThat(histo, notNullValue());
Histogram.Bucket bucket = histo.getByKey(1l);
Histogram.Bucket bucket = histo.getBucketByKey(1l);
assertThat(bucket, notNullValue());
Avg avg = bucket.getAggregations().get("avg");

@ -59,7 +59,7 @@ public class ExtendedStatsTests extends AbstractNumericTests {
assertThat(searchResponse.getHits().getTotalHits(), equalTo(2l));
Histogram histo = searchResponse.getAggregations().get("histo");
assertThat(histo, notNullValue());
Histogram.Bucket bucket = histo.getByKey(1l);
Histogram.Bucket bucket = histo.getBucketByKey(1l);
assertThat(bucket, notNullValue());
ExtendedStats stats = bucket.getAggregations().get("stats");

@ -45,7 +45,7 @@ public class MaxTests extends AbstractNumericTests {
assertThat(searchResponse.getHits().getTotalHits(), equalTo(2l));
Histogram histo = searchResponse.getAggregations().get("histo");
assertThat(histo, notNullValue());
Histogram.Bucket bucket = histo.getByKey(1l);
Histogram.Bucket bucket = histo.getBucketByKey(1l);
assertThat(bucket, notNullValue());
Max max = bucket.getAggregations().get("max");

@ -45,7 +45,7 @@ public class MinTests extends AbstractNumericTests {
assertThat(searchResponse.getHits().getTotalHits(), equalTo(2l));
Histogram histo = searchResponse.getAggregations().get("histo");
assertThat(histo, notNullValue());
Histogram.Bucket bucket = histo.getByKey(1l);
Histogram.Bucket bucket = histo.getBucketByKey(1l);
assertThat(bucket, notNullValue());
Min min = bucket.getAggregations().get("min");

@ -48,7 +48,7 @@ public class StatsTests extends AbstractNumericTests {
assertThat(searchResponse.getHits().getTotalHits(), equalTo(2l));
Histogram histo = searchResponse.getAggregations().get("histo");
assertThat(histo, notNullValue());
Histogram.Bucket bucket = histo.getByKey(1l);
Histogram.Bucket bucket = histo.getBucketByKey(1l);
assertThat(bucket, notNullValue());
Stats stats = bucket.getAggregations().get("stats");

@ -45,7 +45,7 @@ public class SumTests extends AbstractNumericTests {
assertThat(searchResponse.getHits().getTotalHits(), equalTo(2l));
Histogram histo = searchResponse.getAggregations().get("histo");
assertThat(histo, notNullValue());
Histogram.Bucket bucket = histo.getByKey(1l);
Histogram.Bucket bucket = histo.getBucketByKey(1l);
assertThat(bucket, notNullValue());
Sum sum = bucket.getAggregations().get("sum");