Aggregations: Adding Sum Bucket Aggregation

Closes #11007
This commit is contained in:
Colin Goodheart-Smithe 2015-05-06 14:20:25 +01:00
parent e70a8d4ee9
commit cf1251796f
9 changed files with 668 additions and 0 deletions

View File

@ -158,4 +158,5 @@ include::reducer/avg-bucket-aggregation.asciidoc[]
include::reducer/derivative-aggregation.asciidoc[]
include::reducer/max-bucket-aggregation.asciidoc[]
include::reducer/min-bucket-aggregation.asciidoc[]
include::reducer/sum-bucket-aggregation.asciidoc[]
include::reducer/movavg-aggregation.asciidoc[]

View File

@ -0,0 +1,100 @@
[[search-aggregations-reducer-sum-bucket-aggregation]]
=== Sum Bucket Aggregation
A sibling reducer aggregation which calculates the sum across all bucket of a specified metric in a sibling aggregation.
The specified metric must be numeric and the sibling aggregation must be a multi-bucket aggregation.
==== Syntax
A `sum_bucket` aggregation looks like this in isolation:
[source,js]
--------------------------------------------------
{
"sum_bucket": {
"buckets_path": "the_sum"
}
}
--------------------------------------------------
.`sum_bucket` Parameters
|===
|Parameter Name |Description |Required |Default Value
|`buckets_path` |The path to the buckets we wish to find the sum for (see <<bucket-path-syntax>> for more
details) |Required |
|`gap_policy` |The policy to apply when gaps are found in the data (see <<gap-policy>> for more
details)|Optional, defaults to `skip` ||
|`format` |format to apply to the output value of this aggregation |Optional, defaults to `null` |
|===
The following snippet calculates the sum of all the total monthly `sales` buckets:
[source,js]
--------------------------------------------------
{
"aggs" : {
"sales_per_month" : {
"date_histogram" : {
"field" : "date",
"interval" : "month"
},
"aggs": {
"sales": {
"sum": {
"field": "price"
}
}
}
},
"sum_monthly_sales": {
"sum_bucket": {
"buckets_paths": "sales_per_month>sales" <1>
}
}
}
}
--------------------------------------------------
<1> `bucket_paths` instructs this sum_bucket aggregation that we want the sum of the `sales` aggregation in the
`sales_per_month` date histogram.
And the following may be the response:
[source,js]
--------------------------------------------------
{
"aggregations": {
"sales_per_month": {
"buckets": [
{
"key_as_string": "2015/01/01 00:00:00",
"key": 1420070400000,
"doc_count": 3,
"sales": {
"value": 550
}
},
{
"key_as_string": "2015/02/01 00:00:00",
"key": 1422748800000,
"doc_count": 2,
"sales": {
"value": 60
}
},
{
"key_as_string": "2015/03/01 00:00:00",
"key": 1425168000000,
"doc_count": 2,
"sales": {
"value": 375
}
}
]
},
"sum_monthly_sales": {
"value": 985
}
}
}
--------------------------------------------------

View File

@ -60,6 +60,7 @@ import org.elasticsearch.search.aggregations.reducers.Reducer;
import org.elasticsearch.search.aggregations.reducers.bucketmetrics.avg.AvgBucketParser;
import org.elasticsearch.search.aggregations.reducers.bucketmetrics.max.MaxBucketParser;
import org.elasticsearch.search.aggregations.reducers.bucketmetrics.min.MinBucketParser;
import org.elasticsearch.search.aggregations.reducers.bucketmetrics.sum.SumBucketParser;
import org.elasticsearch.search.aggregations.reducers.derivative.DerivativeParser;
import org.elasticsearch.search.aggregations.reducers.movavg.MovAvgParser;
import org.elasticsearch.search.aggregations.reducers.movavg.models.MovAvgModelModule;
@ -111,6 +112,7 @@ public class AggregationModule extends AbstractModule implements SpawnModules{
reducerParsers.add(MaxBucketParser.class);
reducerParsers.add(MinBucketParser.class);
reducerParsers.add(AvgBucketParser.class);
reducerParsers.add(SumBucketParser.class);
reducerParsers.add(MovAvgParser.class);
}

View File

@ -64,6 +64,7 @@ import org.elasticsearch.search.aggregations.reducers.bucketmetrics.InternalBuck
import org.elasticsearch.search.aggregations.reducers.bucketmetrics.avg.AvgBucketReducer;
import org.elasticsearch.search.aggregations.reducers.bucketmetrics.max.MaxBucketReducer;
import org.elasticsearch.search.aggregations.reducers.bucketmetrics.min.MinBucketReducer;
import org.elasticsearch.search.aggregations.reducers.bucketmetrics.sum.SumBucketReducer;
import org.elasticsearch.search.aggregations.reducers.derivative.DerivativeReducer;
import org.elasticsearch.search.aggregations.reducers.derivative.InternalDerivative;
import org.elasticsearch.search.aggregations.reducers.movavg.MovAvgReducer;
@ -124,6 +125,7 @@ public class TransportAggregationModule extends AbstractModule implements SpawnM
MaxBucketReducer.registerStreams();
MinBucketReducer.registerStreams();
AvgBucketReducer.registerStreams();
SumBucketReducer.registerStreams();
MovAvgReducer.registerStreams();
}

View File

@ -22,6 +22,7 @@ package org.elasticsearch.search.aggregations.reducers;
import org.elasticsearch.search.aggregations.reducers.bucketmetrics.avg.AvgBucketBuilder;
import org.elasticsearch.search.aggregations.reducers.bucketmetrics.max.MaxBucketBuilder;
import org.elasticsearch.search.aggregations.reducers.bucketmetrics.min.MinBucketBuilder;
import org.elasticsearch.search.aggregations.reducers.bucketmetrics.sum.SumBucketBuilder;
import org.elasticsearch.search.aggregations.reducers.derivative.DerivativeBuilder;
import org.elasticsearch.search.aggregations.reducers.movavg.MovAvgBuilder;
@ -46,6 +47,10 @@ public final class ReducerBuilders {
return new AvgBucketBuilder(name);
}
public static final SumBucketBuilder sumBucket(String name) {
return new SumBucketBuilder(name);
}
public static final MovAvgBuilder movingAvg(String name) {
return new MovAvgBuilder(name);
}

View File

@ -0,0 +1,30 @@
/*
* 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.reducers.bucketmetrics.sum;
import org.elasticsearch.search.aggregations.reducers.bucketmetrics.BucketMetricsBuilder;
public class SumBucketBuilder extends BucketMetricsBuilder<SumBucketBuilder> {
public SumBucketBuilder(String name) {
super(name, SumBucketReducer.TYPE.name());
}
}

View File

@ -0,0 +1,38 @@
/*
* 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.reducers.bucketmetrics.sum;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.search.aggregations.reducers.BucketHelpers.GapPolicy;
import org.elasticsearch.search.aggregations.reducers.bucketmetrics.BucketMetricsParser;
import org.elasticsearch.search.aggregations.reducers.ReducerFactory;
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
public class SumBucketParser extends BucketMetricsParser {
@Override
public String type() {
return SumBucketReducer.TYPE.name();
}
@Override
protected ReducerFactory buildFactory(String reducerName, String[] bucketsPaths, GapPolicy gapPolicy, @Nullable ValueFormatter formatter) {
return new SumBucketReducer.Factory(reducerName, bucketsPaths, gapPolicy, formatter);
}
}

View File

@ -0,0 +1,112 @@
/*
* 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.reducers.bucketmetrics.sum;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.reducers.BucketHelpers.GapPolicy;
import org.elasticsearch.search.aggregations.reducers.InternalSimpleValue;
import org.elasticsearch.search.aggregations.reducers.Reducer;
import org.elasticsearch.search.aggregations.reducers.ReducerFactory;
import org.elasticsearch.search.aggregations.reducers.ReducerStreams;
import org.elasticsearch.search.aggregations.reducers.bucketmetrics.BucketMetricsReducer;
import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class SumBucketReducer extends BucketMetricsReducer {
public final static Type TYPE = new Type("sum_bucket");
public final static ReducerStreams.Stream STREAM = new ReducerStreams.Stream() {
@Override
public SumBucketReducer readResult(StreamInput in) throws IOException {
SumBucketReducer result = new SumBucketReducer();
result.readFrom(in);
return result;
}
};
public static void registerStreams() {
ReducerStreams.registerStream(STREAM, TYPE.stream());
}
private double sum = 0;
private SumBucketReducer() {
}
protected SumBucketReducer(String name, String[] bucketsPaths, GapPolicy gapPolicy, @Nullable ValueFormatter formatter,
Map<String, Object> metaData) {
super(name, bucketsPaths, gapPolicy, formatter, metaData);
}
@Override
public Type type() {
return TYPE;
}
@Override
protected void preCollection() {
sum = 0;
}
@Override
protected void collectBucketValue(String bucketKey, Double bucketValue) {
sum += bucketValue;
}
@Override
protected InternalAggregation buildAggregation(List<Reducer> reducers, Map<String, Object> metadata) {
return new InternalSimpleValue(name(), sum, formatter, reducers, metadata);
}
public static class Factory extends ReducerFactory {
private final ValueFormatter formatter;
private final GapPolicy gapPolicy;
public Factory(String name, String[] bucketsPaths, GapPolicy gapPolicy, @Nullable ValueFormatter formatter) {
super(name, TYPE.name(), bucketsPaths);
this.gapPolicy = gapPolicy;
this.formatter = formatter;
}
@Override
protected Reducer createInternal(Map<String, Object> metaData) throws IOException {
return new SumBucketReducer(name, bucketsPaths, gapPolicy, formatter, metaData);
}
@Override
public void doValidate(AggregatorFactory parent, AggregatorFactory[] aggFactories, List<ReducerFactory> reducerFactories) {
if (bucketsPaths.length != 1) {
throw new IllegalStateException(Reducer.Parser.BUCKETS_PATH.getPreferredName()
+ " must contain a single entry for reducer [" + name + "]");
}
}
}
}

View File

@ -0,0 +1,378 @@
/*
* 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.reducers;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket;
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
import org.elasticsearch.search.aggregations.metrics.sum.Sum;
import org.elasticsearch.search.aggregations.reducers.BucketHelpers.GapPolicy;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.junit.Test;
import java.util.ArrayList;
import java.util.List;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
import static org.elasticsearch.search.aggregations.AggregationBuilders.sum;
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
import static org.elasticsearch.search.aggregations.reducers.ReducerBuilders.sumBucket;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.core.IsNull.notNullValue;
@ElasticsearchIntegrationTest.SuiteScopeTest
public class SumBucketTests extends ElasticsearchIntegrationTest {
private static final String SINGLE_VALUED_FIELD_NAME = "l_value";
static int numDocs;
static int interval;
static int minRandomValue;
static int maxRandomValue;
static int numValueBuckets;
static long[] valueCounts;
@Override
public void setupSuiteScopeCluster() throws Exception {
createIndex("idx");
createIndex("idx_unmapped");
numDocs = randomIntBetween(6, 20);
interval = randomIntBetween(2, 5);
minRandomValue = 0;
maxRandomValue = 20;
numValueBuckets = ((maxRandomValue - minRandomValue) / interval) + 1;
valueCounts = new long[numValueBuckets];
List<IndexRequestBuilder> builders = new ArrayList<>();
for (int i = 0; i < numDocs; i++) {
int fieldValue = randomIntBetween(minRandomValue, maxRandomValue);
builders.add(client().prepareIndex("idx", "type").setSource(
jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, fieldValue).field("tag", "tag" + (i % interval))
.endObject()));
final int bucket = (fieldValue / interval); // + (fieldValue < 0 ? -1 : 0) - (minRandomValue / interval - 1);
valueCounts[bucket]++;
}
assertAcked(prepareCreate("empty_bucket_idx").addMapping("type", SINGLE_VALUED_FIELD_NAME, "type=integer"));
for (int i = 0; i < 2; i++) {
builders.add(client().prepareIndex("empty_bucket_idx", "type", "" + i).setSource(
jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, i * 2).endObject()));
}
indexRandom(true, builders);
ensureSearchable();
}
@Test
public void testDocCount_topLevel() throws Exception {
SearchResponse response = client().prepareSearch("idx")
.addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
.extendedBounds((long) minRandomValue, (long) maxRandomValue))
.addAggregation(sumBucket("sum_bucket").setBucketsPaths("histo>_count")).execute().actionGet();
assertSearchResponse(response);
Histogram histo = response.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
List<? extends Bucket> buckets = histo.getBuckets();
assertThat(buckets.size(), equalTo(numValueBuckets));
double sum = 0;
for (int i = 0; i < numValueBuckets; ++i) {
Histogram.Bucket bucket = buckets.get(i);
assertThat(bucket, notNullValue());
assertThat(((Number) bucket.getKey()).longValue(), equalTo((long) i * interval));
assertThat(bucket.getDocCount(), equalTo(valueCounts[i]));
sum += bucket.getDocCount();
}
InternalSimpleValue sumBucketValue = response.getAggregations().get("sum_bucket");
assertThat(sumBucketValue, notNullValue());
assertThat(sumBucketValue.getName(), equalTo("sum_bucket"));
assertThat(sumBucketValue.value(), equalTo(sum));
}
@Test
public void testDocCount_asSubAgg() throws Exception {
SearchResponse response = client()
.prepareSearch("idx")
.addAggregation(
terms("terms")
.field("tag")
.order(Order.term(true))
.subAggregation(
histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
.extendedBounds((long) minRandomValue, (long) maxRandomValue))
.subAggregation(sumBucket("sum_bucket").setBucketsPaths("histo>_count"))).execute().actionGet();
assertSearchResponse(response);
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
List<Terms.Bucket> termsBuckets = terms.getBuckets();
assertThat(termsBuckets.size(), equalTo(interval));
for (int i = 0; i < interval; ++i) {
Terms.Bucket termsBucket = termsBuckets.get(i);
assertThat(termsBucket, notNullValue());
assertThat((String) termsBucket.getKey(), equalTo("tag" + (i % interval)));
Histogram histo = termsBucket.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
List<? extends Bucket> buckets = histo.getBuckets();
double sum = 0;
for (int j = 0; j < numValueBuckets; ++j) {
Histogram.Bucket bucket = buckets.get(j);
assertThat(bucket, notNullValue());
assertThat(((Number) bucket.getKey()).longValue(), equalTo((long) j * interval));
sum += bucket.getDocCount();
}
InternalSimpleValue sumBucketValue = termsBucket.getAggregations().get("sum_bucket");
assertThat(sumBucketValue, notNullValue());
assertThat(sumBucketValue.getName(), equalTo("sum_bucket"));
assertThat(sumBucketValue.value(), equalTo(sum));
}
}
@Test
public void testMetric_topLevel() throws Exception {
SearchResponse response = client()
.prepareSearch("idx")
.addAggregation(terms("terms").field("tag").subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME)))
.addAggregation(sumBucket("sum_bucket").setBucketsPaths("terms>sum")).execute().actionGet();
assertSearchResponse(response);
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
List<Terms.Bucket> buckets = terms.getBuckets();
assertThat(buckets.size(), equalTo(interval));
double bucketSum = 0;
for (int i = 0; i < interval; ++i) {
Terms.Bucket bucket = buckets.get(i);
assertThat(bucket, notNullValue());
assertThat((String) bucket.getKey(), equalTo("tag" + (i % interval)));
assertThat(bucket.getDocCount(), greaterThan(0l));
Sum sum = bucket.getAggregations().get("sum");
assertThat(sum, notNullValue());
bucketSum += sum.value();
}
InternalSimpleValue sumBucketValue = response.getAggregations().get("sum_bucket");
assertThat(sumBucketValue, notNullValue());
assertThat(sumBucketValue.getName(), equalTo("sum_bucket"));
assertThat(sumBucketValue.value(), equalTo(bucketSum));
}
@Test
public void testMetric_asSubAgg() throws Exception {
SearchResponse response = client()
.prepareSearch("idx")
.addAggregation(
terms("terms")
.field("tag")
.order(Order.term(true))
.subAggregation(
histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
.extendedBounds((long) minRandomValue, (long) maxRandomValue)
.subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME)))
.subAggregation(sumBucket("sum_bucket").setBucketsPaths("histo>sum"))).execute().actionGet();
assertSearchResponse(response);
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
List<Terms.Bucket> termsBuckets = terms.getBuckets();
assertThat(termsBuckets.size(), equalTo(interval));
for (int i = 0; i < interval; ++i) {
Terms.Bucket termsBucket = termsBuckets.get(i);
assertThat(termsBucket, notNullValue());
assertThat((String) termsBucket.getKey(), equalTo("tag" + (i % interval)));
Histogram histo = termsBucket.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
List<? extends Bucket> buckets = histo.getBuckets();
double bucketSum = 0;
for (int j = 0; j < numValueBuckets; ++j) {
Histogram.Bucket bucket = buckets.get(j);
assertThat(bucket, notNullValue());
assertThat(((Number) bucket.getKey()).longValue(), equalTo((long) j * interval));
if (bucket.getDocCount() != 0) {
Sum sum = bucket.getAggregations().get("sum");
assertThat(sum, notNullValue());
bucketSum += sum.value();
}
}
InternalSimpleValue sumBucketValue = termsBucket.getAggregations().get("sum_bucket");
assertThat(sumBucketValue, notNullValue());
assertThat(sumBucketValue.getName(), equalTo("sum_bucket"));
assertThat(sumBucketValue.value(), equalTo(bucketSum));
}
}
@Test
public void testMetric_asSubAggWithInsertZeros() throws Exception {
SearchResponse response = client()
.prepareSearch("idx")
.addAggregation(
terms("terms")
.field("tag")
.order(Order.term(true))
.subAggregation(
histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
.extendedBounds((long) minRandomValue, (long) maxRandomValue)
.subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME)))
.subAggregation(sumBucket("sum_bucket").setBucketsPaths("histo>sum").gapPolicy(GapPolicy.INSERT_ZEROS)))
.execute().actionGet();
assertSearchResponse(response);
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
List<Terms.Bucket> termsBuckets = terms.getBuckets();
assertThat(termsBuckets.size(), equalTo(interval));
for (int i = 0; i < interval; ++i) {
Terms.Bucket termsBucket = termsBuckets.get(i);
assertThat(termsBucket, notNullValue());
assertThat((String) termsBucket.getKey(), equalTo("tag" + (i % interval)));
Histogram histo = termsBucket.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
List<? extends Bucket> buckets = histo.getBuckets();
double bucketSum = 0;
for (int j = 0; j < numValueBuckets; ++j) {
Histogram.Bucket bucket = buckets.get(j);
assertThat(bucket, notNullValue());
assertThat(((Number) bucket.getKey()).longValue(), equalTo((long) j * interval));
Sum sum = bucket.getAggregations().get("sum");
assertThat(sum, notNullValue());
bucketSum += sum.value();
}
InternalSimpleValue sumBucketValue = termsBucket.getAggregations().get("sum_bucket");
assertThat(sumBucketValue, notNullValue());
assertThat(sumBucketValue.getName(), equalTo("sum_bucket"));
assertThat(sumBucketValue.value(), equalTo(bucketSum));
}
}
@Test
public void testNoBuckets() throws Exception {
SearchResponse response = client().prepareSearch("idx")
.addAggregation(terms("terms").field("tag").exclude("tag.*").subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME)))
.addAggregation(sumBucket("sum_bucket").setBucketsPaths("terms>sum")).execute().actionGet();
assertSearchResponse(response);
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
List<Terms.Bucket> buckets = terms.getBuckets();
assertThat(buckets.size(), equalTo(0));
InternalSimpleValue sumBucketValue = response.getAggregations().get("sum_bucket");
assertThat(sumBucketValue, notNullValue());
assertThat(sumBucketValue.getName(), equalTo("sum_bucket"));
assertThat(sumBucketValue.value(), equalTo(0.0));
}
@Test
public void testNested() throws Exception {
SearchResponse response = client()
.prepareSearch("idx")
.addAggregation(
terms("terms")
.field("tag")
.order(Order.term(true))
.subAggregation(
histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
.extendedBounds((long) minRandomValue, (long) maxRandomValue))
.subAggregation(sumBucket("sum_histo_bucket").setBucketsPaths("histo>_count")))
.addAggregation(sumBucket("sum_terms_bucket").setBucketsPaths("terms>sum_histo_bucket")).execute().actionGet();
assertSearchResponse(response);
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
List<Terms.Bucket> termsBuckets = terms.getBuckets();
assertThat(termsBuckets.size(), equalTo(interval));
double aggTermsSum = 0;
for (int i = 0; i < interval; ++i) {
Terms.Bucket termsBucket = termsBuckets.get(i);
assertThat(termsBucket, notNullValue());
assertThat((String) termsBucket.getKey(), equalTo("tag" + (i % interval)));
Histogram histo = termsBucket.getAggregations().get("histo");
assertThat(histo, notNullValue());
assertThat(histo.getName(), equalTo("histo"));
List<? extends Bucket> buckets = histo.getBuckets();
double aggHistoSum = 0;
for (int j = 0; j < numValueBuckets; ++j) {
Histogram.Bucket bucket = buckets.get(j);
assertThat(bucket, notNullValue());
assertThat(((Number) bucket.getKey()).longValue(), equalTo((long) j * interval));
aggHistoSum += bucket.getDocCount();
}
InternalSimpleValue sumBucketValue = termsBucket.getAggregations().get("sum_histo_bucket");
assertThat(sumBucketValue, notNullValue());
assertThat(sumBucketValue.getName(), equalTo("sum_histo_bucket"));
assertThat(sumBucketValue.value(), equalTo(aggHistoSum));
aggTermsSum += aggHistoSum;
}
InternalSimpleValue sumBucketValue = response.getAggregations().get("sum_terms_bucket");
assertThat(sumBucketValue, notNullValue());
assertThat(sumBucketValue.getName(), equalTo("sum_terms_bucket"));
assertThat(sumBucketValue.value(), equalTo(aggTermsSum));
}
}