From cf1251796f0e06861e4aeccc8234caff2233c458 Mon Sep 17 00:00:00 2001 From: Colin Goodheart-Smithe Date: Wed, 6 May 2015 14:20:25 +0100 Subject: [PATCH] Aggregations: Adding Sum Bucket Aggregation Closes #11007 --- docs/reference/aggregations/reducer.asciidoc | 1 + .../reducer/sum-bucket-aggregation.asciidoc | 100 +++++ .../aggregations/AggregationModule.java | 2 + .../TransportAggregationModule.java | 2 + .../reducers/ReducerBuilders.java | 5 + .../bucketmetrics/sum/SumBucketBuilder.java | 30 ++ .../bucketmetrics/sum/SumBucketParser.java | 38 ++ .../bucketmetrics/sum/SumBucketReducer.java | 112 ++++++ .../aggregations/reducers/SumBucketTests.java | 378 ++++++++++++++++++ 9 files changed, 668 insertions(+) create mode 100644 docs/reference/aggregations/reducer/sum-bucket-aggregation.asciidoc create mode 100644 src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/sum/SumBucketBuilder.java create mode 100644 src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/sum/SumBucketParser.java create mode 100644 src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/sum/SumBucketReducer.java create mode 100644 src/test/java/org/elasticsearch/search/aggregations/reducers/SumBucketTests.java diff --git a/docs/reference/aggregations/reducer.asciidoc b/docs/reference/aggregations/reducer.asciidoc index 525521c4f45..33f82e78523 100644 --- a/docs/reference/aggregations/reducer.asciidoc +++ b/docs/reference/aggregations/reducer.asciidoc @@ -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[] diff --git a/docs/reference/aggregations/reducer/sum-bucket-aggregation.asciidoc b/docs/reference/aggregations/reducer/sum-bucket-aggregation.asciidoc new file mode 100644 index 00000000000..0a6a98cc977 --- /dev/null +++ b/docs/reference/aggregations/reducer/sum-bucket-aggregation.asciidoc @@ -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 <> for more + details) |Required | + |`gap_policy` |The policy to apply when gaps are found in the data (see <> 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 + } + } +} +-------------------------------------------------- + diff --git a/src/main/java/org/elasticsearch/search/aggregations/AggregationModule.java b/src/main/java/org/elasticsearch/search/aggregations/AggregationModule.java index 9cb80770399..9c41dc63379 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/AggregationModule.java +++ b/src/main/java/org/elasticsearch/search/aggregations/AggregationModule.java @@ -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); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/TransportAggregationModule.java b/src/main/java/org/elasticsearch/search/aggregations/TransportAggregationModule.java index cfad175fee2..81051a7d9bb 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/TransportAggregationModule.java +++ b/src/main/java/org/elasticsearch/search/aggregations/TransportAggregationModule.java @@ -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(); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/ReducerBuilders.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/ReducerBuilders.java index ada1eb6f834..71481de17ce 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/reducers/ReducerBuilders.java +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/ReducerBuilders.java @@ -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); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/sum/SumBucketBuilder.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/sum/SumBucketBuilder.java new file mode 100644 index 00000000000..6c3f18f6b99 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/sum/SumBucketBuilder.java @@ -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 { + + public SumBucketBuilder(String name) { + super(name, SumBucketReducer.TYPE.name()); + } + +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/sum/SumBucketParser.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/sum/SumBucketParser.java new file mode 100644 index 00000000000..30e8093afd1 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/sum/SumBucketParser.java @@ -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); + } +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/sum/SumBucketReducer.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/sum/SumBucketReducer.java new file mode 100644 index 00000000000..018a87ffa2d --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/sum/SumBucketReducer.java @@ -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 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 reducers, Map 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 metaData) throws IOException { + return new SumBucketReducer(name, bucketsPaths, gapPolicy, formatter, metaData); + } + + @Override + public void doValidate(AggregatorFactory parent, AggregatorFactory[] aggFactories, List reducerFactories) { + if (bucketsPaths.length != 1) { + throw new IllegalStateException(Reducer.Parser.BUCKETS_PATH.getPreferredName() + + " must contain a single entry for reducer [" + name + "]"); + } + } + + } + +} diff --git a/src/test/java/org/elasticsearch/search/aggregations/reducers/SumBucketTests.java b/src/test/java/org/elasticsearch/search/aggregations/reducers/SumBucketTests.java new file mode 100644 index 00000000000..a4cc26bed79 --- /dev/null +++ b/src/test/java/org/elasticsearch/search/aggregations/reducers/SumBucketTests.java @@ -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 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 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 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 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 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 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 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 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 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 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 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 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)); + } +}