Add helper classes to determine if aggs have a value (#36020)

This adds a set of helper classes to determine if an agg "has a value". 
This is needed because InternalAggs represent "empty" in different 
manners according to convention. Some use `NaN`, `+/- Inf`, `0.0`, etc.

A user can pass the Internal agg type to one of these helper methods
and it will report if the agg contains a value or not, which allows the
user to differentiate "empty" from a real `NaN`.

These helpers are best-effort in some cases.  For example, several
pipeline aggs share a single return class but use different conventions
to mark "empty", so the helper uses the loosest definition that applies
to all the aggs that use the class.

Sums in particular are unreliable.  The InternalSum simply returns 0.0
if the agg is empty (which is correct, no values == sum of zero).  But this
also means the helper cannot differentiate from "empty" and `+1 + -1`.
This commit is contained in:
Zachary Tong 2019-01-22 12:38:55 -05:00 committed by GitHub
parent 715719ee3b
commit 2ba9e361ab
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
49 changed files with 777 additions and 98 deletions

View File

@ -0,0 +1,29 @@
/*
* 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.matrix.stats;
/**
* Counterpart to {@link org.elasticsearch.search.aggregations.support.AggregationInspectionHelper}, providing
* helpers for some aggs in the MatrixStats package
*/
public class MatrixAggregationInspectionHelper {
public static boolean hasValue(InternalMatrixStats agg) {
return agg.getResults() != null;
}
}

View File

@ -53,10 +53,12 @@ public class MatrixStatsAggregatorTests extends AggregatorTestCase {
.fields(Collections.singletonList("field"));
InternalMatrixStats stats = search(searcher, new MatchAllDocsQuery(), aggBuilder, ft);
assertNull(stats.getStats());
assertFalse(MatrixAggregationInspectionHelper.hasValue(stats));
}
}
}
@AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37587")
public void testTwoFields() throws Exception {
String fieldA = "a";
MappedFieldType ftA = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.DOUBLE);
@ -87,8 +89,9 @@ public class MatrixStatsAggregatorTests extends AggregatorTestCase {
IndexSearcher searcher = new IndexSearcher(reader);
MatrixStatsAggregationBuilder aggBuilder = new MatrixStatsAggregationBuilder("my_agg")
.fields(Arrays.asList(fieldA, fieldB));
InternalMatrixStats stats = search(searcher, new MatchAllDocsQuery(), aggBuilder, ftA, ftB);
InternalMatrixStats stats = searchAndReduce(searcher, new MatchAllDocsQuery(), aggBuilder, ftA, ftB);
multiPassStats.assertNearlyEqual(new MatrixStatsResults(stats.getStats()));
assertTrue(MatrixAggregationInspectionHelper.hasValue(stats));
}
}
}

View File

@ -0,0 +1,34 @@
/*
* 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.join.aggregations;
/**
* Counterpart to {@link org.elasticsearch.search.aggregations.support.AggregationInspectionHelper}, providing
* helpers for some aggs in the Join package
*/
public class JoinAggregationInspectionHelper {
public static boolean hasValue(InternalParent agg) {
return agg.getDocCount() > 0;
}
public static boolean hasValue(InternalChildren agg) {
return agg.getDocCount() > 0;
}
}

View File

@ -90,6 +90,7 @@ public class ChildrenToParentAggregatorTests extends AggregatorTestCase {
assertEquals(0, childrenToParent.getDocCount());
assertNotNull("Aggregations: " + childrenToParent.getAggregations().asMap(), parentAggregation);
assertEquals(Double.POSITIVE_INFINITY, ((InternalMin) parentAggregation).getValue(), Double.MIN_VALUE);
assertFalse(JoinAggregationInspectionHelper.hasValue(childrenToParent));
});
indexReader.close();
directory.close();
@ -119,6 +120,7 @@ public class ChildrenToParentAggregatorTests extends AggregatorTestCase {
parent.getAggregations().asMap(),
expectedTotalParents, parent.getDocCount());
assertEquals(expectedMinValue, ((InternalMin) parent.getAggregations().get("in_parent")).getValue(), Double.MIN_VALUE);
assertTrue(JoinAggregationInspectionHelper.hasValue(parent));
});
// verify for each children
@ -170,6 +172,7 @@ public class ChildrenToParentAggregatorTests extends AggregatorTestCase {
// verify a terms-aggregation inside the parent-aggregation
testCaseTerms(new MatchAllDocsQuery(), indexSearcher, parent -> {
assertNotNull(parent);
assertTrue(JoinAggregationInspectionHelper.hasValue(parent));
LongTerms valueTerms = parent.getAggregations().get("value_terms");
assertNotNull(valueTerms);

View File

@ -105,6 +105,7 @@ public class ParentToChildrenAggregatorTests extends AggregatorTestCase {
expectedMinValue = Math.min(expectedMinValue, expectedValues.v2());
}
assertEquals(expectedTotalChildren, child.getDocCount());
assertTrue(JoinAggregationInspectionHelper.hasValue(child));
assertEquals(expectedMinValue, ((InternalMin) child.getAggregations().get("in_child")).getValue(), Double.MIN_VALUE);
});

View File

@ -226,7 +226,7 @@ public class InternalComposite
}
}
static class InternalBucket extends InternalMultiBucketAggregation.InternalBucket
public static class InternalBucket extends InternalMultiBucketAggregation.InternalBucket
implements CompositeAggregation.Bucket, KeyComparable<InternalBucket> {
private final CompositeKey key;

View File

@ -34,7 +34,7 @@ import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
class GeoGridBucket extends InternalMultiBucketAggregation.InternalBucket implements GeoHashGrid.Bucket, Comparable<GeoGridBucket> {
public class GeoGridBucket extends InternalMultiBucketAggregation.InternalBucket implements GeoHashGrid.Bucket, Comparable<GeoGridBucket> {
protected long geohashAsLong;
protected long docCount;

View File

@ -98,6 +98,10 @@ abstract class AbstractInternalHDRPercentiles extends InternalNumericMetricsAggr
return state.getEstimatedFootprintInBytes();
}
DoubleHistogram getState() {
return state;
}
@Override
public AbstractInternalHDRPercentiles doReduce(List<InternalAggregation> aggregations, ReduceContext reduceContext) {
DoubleHistogram merged = null;

View File

@ -82,6 +82,10 @@ abstract class AbstractInternalTDigestPercentiles extends InternalNumericMetrics
return state.byteSize();
}
TDigestState getState() {
return state;
}
@Override
public AbstractInternalTDigestPercentiles doReduce(List<InternalAggregation> aggregations, ReduceContext reduceContext) {
TDigestState merged = null;

View File

@ -79,6 +79,10 @@ public final class InternalCardinality extends InternalNumericMetricsAggregation
return counts == null ? 0 : counts.cardinality(0);
}
HyperLogLogPlusPlus getCounts() {
return counts;
}
@Override
public InternalAggregation doReduce(List<InternalAggregation> aggregations, ReduceContext reduceContext) {
InternalCardinality reduced = null;

View File

@ -121,7 +121,7 @@ public class InternalMedianAbsoluteDeviation extends InternalNumericMetricsAggre
return MedianAbsoluteDeviationAggregationBuilder.NAME;
}
public TDigestState getValuesSketch() {
TDigestState getValuesSketch() {
return valuesSketch;
}

View File

@ -80,6 +80,10 @@ public class InternalScriptedMetric extends InternalAggregation implements Scrip
return aggregation.get(0);
}
List<Object> getAggregation() {
return aggregation;
}
@Override
public InternalAggregation doReduce(List<InternalAggregation> aggregations, ReduceContext reduceContext) {
List<Object> aggregationObjects = new ArrayList<>();

View File

@ -0,0 +1,78 @@
/*
* 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.metrics;
import org.elasticsearch.search.aggregations.pipeline.InternalDerivative;
/**
* Counterpart to {@link org.elasticsearch.search.aggregations.support.AggregationInspectionHelper}, providing
* helpers for some aggs that have package-private getters. AggregationInspectionHelper delegates to these
* helpers when needed, and consumers should prefer to use AggregationInspectionHelper instead of these
* helpers.
*/
public class MetricInspectionHelper {
public static boolean hasValue(InternalAvg agg) {
return agg.getCount() > 0;
}
public static boolean hasValue(InternalCardinality agg) {
return agg.getCounts() != null;
}
public static boolean hasValue(InternalHDRPercentileRanks agg) {
return agg.getState().getTotalCount() > 0;
}
public static boolean hasValue(InternalHDRPercentiles agg) {
return agg.getState().getTotalCount() > 0;
}
public static boolean hasValue(InternalMedianAbsoluteDeviation agg) {
return agg.getValuesSketch().size() > 0;
}
public static boolean hasValue(InternalScriptedMetric agg) {
// TODO better way to know if the scripted metric received documents?
// Could check for null too, but a script might return null on purpose...
return agg.getAggregation().size() > 0 ;
}
public static boolean hasValue(InternalTDigestPercentileRanks agg) {
return agg.getState().size() > 0;
}
public static boolean hasValue(InternalTDigestPercentiles agg) {
return agg.getState().size() > 0;
}
public static boolean hasValue(InternalTopHits agg) {
return (agg.getHits().getTotalHits().value == 0
&& Double.isNaN(agg.getHits().getMaxScore())
&& Double.isNaN(agg.getTopDocs().maxScore)) == false;
}
public static boolean hasValue(InternalWeightedAvg agg) {
return (agg.getSum() == 0.0 && agg.getWeight() == 0L) == false;
}
public static boolean hasValue(InternalDerivative agg) {
return true;
}
}

View File

@ -114,7 +114,10 @@ public class BucketScriptPipelineAggregator extends PipelineAggregator {
} else {
final List<InternalAggregation> aggs = StreamSupport.stream(bucket.getAggregations().spliterator(), false).map(
(p) -> (InternalAggregation) p).collect(Collectors.toList());
aggs.add(new InternalSimpleValue(name(), returned.doubleValue(), formatter, new ArrayList<>(), metaData()));
InternalSimpleValue simpleValue = new InternalSimpleValue(name(), returned.doubleValue(),
formatter, new ArrayList<>(), metaData());
aggs.add(simpleValue);
InternalMultiBucketAggregation.InternalBucket newBucket = originalAgg.createBucket(new InternalAggregations(aggs),
bucket);
newBuckets.add(newBucket);

View File

@ -83,8 +83,7 @@ public class CumulativeSumPipelineAggregator extends PipelineAggregator {
sum += thisBucketValue;
}
List<InternalAggregation> aggs = StreamSupport
.stream(bucket.getAggregations().spliterator(), false)
List<InternalAggregation> aggs = StreamSupport.stream(bucket.getAggregations().spliterator(), false)
.map((p) -> (InternalAggregation) p)
.collect(Collectors.toList());
aggs.add(new InternalSimpleValue(name(), sum, formatter, new ArrayList<>(), metaData()));

View File

@ -117,7 +117,7 @@ public class MovAvgPipelineAggregator extends PipelineAggregator {
// since we only change newBucket if we can add to it
Bucket newBucket = bucket;
if (!(thisBucketValue == null || thisBucketValue.equals(Double.NaN))) {
if ((thisBucketValue == null || thisBucketValue.equals(Double.NaN)) == false) {
// Some models (e.g. HoltWinters) have certain preconditions that must be met
if (model.hasValue(values.size())) {
@ -126,7 +126,7 @@ public class MovAvgPipelineAggregator extends PipelineAggregator {
List<InternalAggregation> aggs = StreamSupport.stream(bucket.getAggregations().spliterator(), false)
.map((p) -> (InternalAggregation) p)
.collect(Collectors.toList());
aggs.add(new InternalSimpleValue(name(), movavg, formatter, new ArrayList<PipelineAggregator>(), metaData()));
aggs.add(new InternalSimpleValue(name(), movavg, formatter, new ArrayList<>(), metaData()));
newBucket = factory.createBucket(factory.getKey(bucket), bucket.getDocCount(), new InternalAggregations(aggs));
}
@ -153,10 +153,10 @@ public class MovAvgPipelineAggregator extends PipelineAggregator {
Bucket bucket = newBuckets.get(lastValidPosition + i + 1);
// Get the existing aggs in the bucket so we don't clobber data
aggs = StreamSupport.stream(bucket.getAggregations().spliterator(), false).map((p) -> {
return (InternalAggregation) p;
}).collect(Collectors.toList());
aggs.add(new InternalSimpleValue(name(), predictions[i], formatter, new ArrayList<PipelineAggregator>(), metaData()));
aggs = StreamSupport.stream(bucket.getAggregations().spliterator(), false)
.map((p) -> (InternalAggregation) p)
.collect(Collectors.toList());
aggs.add(new InternalSimpleValue(name(), predictions[i], formatter, new ArrayList<>(), metaData()));
Bucket newBucket = factory.createBucket(newKey, bucket.getDocCount(), new InternalAggregations(aggs));
@ -166,7 +166,7 @@ public class MovAvgPipelineAggregator extends PipelineAggregator {
} else {
// Not seen before, create fresh
aggs = new ArrayList<>();
aggs.add(new InternalSimpleValue(name(), predictions[i], formatter, new ArrayList<PipelineAggregator>(), metaData()));
aggs.add(new InternalSimpleValue(name(), predictions[i], formatter, new ArrayList<>(), metaData()));
Bucket newBucket = factory.createBucket(newKey, 0, new InternalAggregations(aggs));

View File

@ -0,0 +1,242 @@
/*
* 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.support;
import org.elasticsearch.search.aggregations.bucket.adjacency.InternalAdjacencyMatrix;
import org.elasticsearch.search.aggregations.bucket.composite.InternalComposite;
import org.elasticsearch.search.aggregations.bucket.filter.InternalFilter;
import org.elasticsearch.search.aggregations.bucket.filter.InternalFilters;
import org.elasticsearch.search.aggregations.bucket.geogrid.InternalGeoHashGrid;
import org.elasticsearch.search.aggregations.bucket.global.InternalGlobal;
import org.elasticsearch.search.aggregations.bucket.histogram.InternalAutoDateHistogram;
import org.elasticsearch.search.aggregations.bucket.histogram.InternalDateHistogram;
import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram;
import org.elasticsearch.search.aggregations.bucket.missing.InternalMissing;
import org.elasticsearch.search.aggregations.bucket.nested.InternalNested;
import org.elasticsearch.search.aggregations.bucket.nested.InternalReverseNested;
import org.elasticsearch.search.aggregations.bucket.range.InternalRange;
import org.elasticsearch.search.aggregations.bucket.sampler.InternalSampler;
import org.elasticsearch.search.aggregations.bucket.sampler.UnmappedSampler;
import org.elasticsearch.search.aggregations.bucket.significant.InternalSignificantTerms;
import org.elasticsearch.search.aggregations.bucket.significant.UnmappedSignificantTerms;
import org.elasticsearch.search.aggregations.bucket.terms.InternalTerms;
import org.elasticsearch.search.aggregations.bucket.terms.UnmappedTerms;
import org.elasticsearch.search.aggregations.metrics.InternalAvg;
import org.elasticsearch.search.aggregations.metrics.InternalCardinality;
import org.elasticsearch.search.aggregations.metrics.InternalExtendedStats;
import org.elasticsearch.search.aggregations.metrics.InternalGeoBounds;
import org.elasticsearch.search.aggregations.metrics.InternalGeoCentroid;
import org.elasticsearch.search.aggregations.metrics.InternalHDRPercentileRanks;
import org.elasticsearch.search.aggregations.metrics.InternalHDRPercentiles;
import org.elasticsearch.search.aggregations.metrics.InternalMax;
import org.elasticsearch.search.aggregations.metrics.InternalMedianAbsoluteDeviation;
import org.elasticsearch.search.aggregations.metrics.InternalMin;
import org.elasticsearch.search.aggregations.metrics.InternalScriptedMetric;
import org.elasticsearch.search.aggregations.metrics.InternalStats;
import org.elasticsearch.search.aggregations.metrics.InternalSum;
import org.elasticsearch.search.aggregations.metrics.InternalTDigestPercentileRanks;
import org.elasticsearch.search.aggregations.metrics.InternalTDigestPercentiles;
import org.elasticsearch.search.aggregations.metrics.InternalTopHits;
import org.elasticsearch.search.aggregations.metrics.InternalValueCount;
import org.elasticsearch.search.aggregations.metrics.InternalWeightedAvg;
import org.elasticsearch.search.aggregations.metrics.MetricInspectionHelper;
import org.elasticsearch.search.aggregations.pipeline.InternalBucketMetricValue;
import org.elasticsearch.search.aggregations.pipeline.InternalPercentilesBucket;
import org.elasticsearch.search.aggregations.pipeline.InternalSimpleValue;
import java.util.stream.StreamSupport;
/**
* Provides a set of static helpers to determine if a particular type of InternalAggregation "has a value"
* or not. This can be difficult to determine from an external perspective because each agg uses
* different internal bookkeeping to determine if it is empty or not (NaN, +/-Inf, 0.0, etc).
*
* This set of helpers aim to ease that task by codifying what "empty" is for each agg.
*
* It is not entirely accurate for all aggs, since some do not expose or track the needed state
* (e.g. sum doesn't record count, so it's not clear if the sum is 0.0 because it is empty
* or because of summing to zero). Pipeline aggs in particular are not well supported
* by these helpers since most share InternalSimpleValue and it's not clear which pipeline
* generated the value.
*/
public class AggregationInspectionHelper {
public static <A extends InternalTerms<A, B>, B extends InternalTerms.Bucket<B>> boolean hasValue(InternalTerms<A, B> agg) {
return agg.getBuckets().stream().anyMatch(bucket -> bucket.getDocCount() > 0);
}
public static boolean hasValue(UnmappedTerms agg) {
return false;
}
public static boolean hasValue(UnmappedSignificantTerms agg) {
return false;
}
public static boolean hasValue(UnmappedSampler agg) {
return false;
}
public static boolean hasValue(InternalAdjacencyMatrix agg) {
return agg.getBuckets().stream().anyMatch(bucket -> bucket.getDocCount() > 0);
}
public static boolean hasValue(InternalFilters agg) {
return agg.getBuckets().stream().anyMatch(bucket -> bucket.getDocCount() > 0);
}
public static boolean hasValue(InternalFilter agg) {
return agg.getDocCount() > 0;
}
public static boolean hasValue(InternalGeoHashGrid agg) {
return agg.getBuckets().stream().anyMatch(bucket -> bucket.getDocCount() > 0);
}
public static boolean hasValue(InternalGlobal agg) {
return agg.getDocCount() > 0;
}
public static boolean hasValue(InternalHistogram agg) {
return agg.getBuckets().stream().anyMatch(bucket -> bucket.getDocCount() > 0);
}
public static boolean hasValue(InternalDateHistogram agg) {
return agg.getBuckets().stream().anyMatch(bucket -> bucket.getDocCount() > 0);
}
public static boolean hasValue(InternalAutoDateHistogram agg) {
return agg.getBuckets().stream().anyMatch(bucket -> bucket.getDocCount() > 0);
}
public static boolean hasValue(InternalComposite agg) {
return agg.getBuckets().stream().anyMatch(bucket -> bucket.getDocCount() > 0);
}
public static boolean hasValue(InternalMissing agg) {
return agg.getDocCount() > 0;
}
public static boolean hasValue(InternalNested agg) {
return agg.getDocCount() > 0;
}
public static boolean hasValue(InternalReverseNested agg) {
return agg.getDocCount() > 0;
}
public static <B extends InternalRange.Bucket, R extends InternalRange<B, R>> boolean hasValue(InternalRange<B, R> agg) {
return agg.getBuckets().stream().anyMatch(bucket -> bucket.getDocCount() > 0);
}
public static boolean hasValue(InternalSampler agg) {
return agg.getDocCount() > 0;
}
public static <A extends InternalSignificantTerms<A, B>,
B extends InternalSignificantTerms.Bucket<B>> boolean hasValue(InternalSignificantTerms<A, B> agg) {
return agg.getBuckets().stream().anyMatch(bucket -> bucket.getDocCount() > 0);
}
public static boolean hasValue(InternalAvg agg) {
return MetricInspectionHelper.hasValue(agg);
}
public static boolean hasValue(InternalSum agg) {
// TODO this could be incorrect... e.g. +1 + -1
return agg.getValue() != 0.0;
}
public static boolean hasValue(InternalCardinality agg) {
return MetricInspectionHelper.hasValue(agg);
}
public static boolean hasValue(InternalExtendedStats agg) {
return agg.getCount() > 0;
}
public static boolean hasValue(InternalGeoBounds agg) {
return (agg.topLeft() == null && agg.bottomRight() == null) == false;
}
public static boolean hasValue(InternalGeoCentroid agg) {
return agg.centroid() != null && agg.count() > 0;
}
public static boolean hasValue(InternalHDRPercentileRanks agg) {
return MetricInspectionHelper.hasValue(agg);
}
public static boolean hasValue(InternalHDRPercentiles agg) {
return MetricInspectionHelper.hasValue(agg);
}
public static boolean hasValue(InternalMax agg) {
return agg.getValue() != Double.NEGATIVE_INFINITY;
}
public static boolean hasValue(InternalMedianAbsoluteDeviation agg) {
return MetricInspectionHelper.hasValue(agg);
}
public static boolean hasValue(InternalMin agg) {
return agg.getValue() != Double.POSITIVE_INFINITY;
}
public static boolean hasValue(InternalScriptedMetric agg) {
return MetricInspectionHelper.hasValue(agg);
}
public static boolean hasValue(InternalStats agg) {
return agg.getCount() > 0;
}
public static boolean hasValue(InternalTDigestPercentileRanks agg) {
return MetricInspectionHelper.hasValue(agg);
}
public static boolean hasValue(InternalTDigestPercentiles agg) {
return MetricInspectionHelper.hasValue(agg);
}
public static boolean hasValue(InternalTopHits agg) {
return MetricInspectionHelper.hasValue(agg);
}
public static boolean hasValue(InternalValueCount agg) {
return agg.getValue() > 0;
}
public static boolean hasValue(InternalWeightedAvg agg) {
return MetricInspectionHelper.hasValue(agg);
}
public static boolean hasValue(InternalSimpleValue agg) {
// This is a coarse approximation, since some aggs use positive/negative infinity or NaN
return (Double.isInfinite(agg.getValue()) || Double.isNaN(agg.getValue())) == false;
}
public static boolean hasValue(InternalBucketMetricValue agg) {
return Double.isInfinite(agg.value()) == false;
}
public static boolean hasValue(InternalPercentilesBucket agg) {
return StreamSupport.stream(agg.spliterator(), false).allMatch(p -> Double.isNaN(p.getValue())) == false;
}
}

View File

@ -32,6 +32,7 @@ import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import org.junit.Before;
public class FilterAggregatorTests extends AggregatorTestCase {
@ -57,6 +58,7 @@ public class FilterAggregatorTests extends AggregatorTestCase {
InternalFilter response = search(indexSearcher, new MatchAllDocsQuery(), builder,
fieldType);
assertEquals(response.getDocCount(), 0);
assertFalse(AggregationInspectionHelper.hasValue(response));
indexReader.close();
directory.close();
}
@ -96,6 +98,11 @@ public class FilterAggregatorTests extends AggregatorTestCase {
response = search(indexSearcher, new MatchAllDocsQuery(), builder, fieldType);
}
assertEquals(response.getDocCount(), (long) expectedBucketCount[value]);
if (expectedBucketCount[expectedBucketCount[value]] > 0) {
assertTrue(AggregationInspectionHelper.hasValue(response));
} else {
assertFalse(AggregationInspectionHelper.hasValue(response));
}
}
indexReader.close();
directory.close();

View File

@ -32,6 +32,7 @@ import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import org.junit.Before;
import java.util.HashSet;
@ -68,6 +69,7 @@ public class FiltersAggregatorTests extends AggregatorTestCase {
for (InternalFilters.InternalBucket filter : response.getBuckets()) {
assertEquals(filter.getDocCount(), 0);
}
assertFalse(AggregationInspectionHelper.hasValue(response));
indexReader.close();
directory.close();
}
@ -129,6 +131,7 @@ public class FiltersAggregatorTests extends AggregatorTestCase {
assertEquals(filters.getBucketByKey("bar").getDocCount(), 1);
assertEquals(filters.getBucketByKey("same").getDocCount(), 1);
assertEquals(filters.getBucketByKey("other").getDocCount(), 2);
assertTrue(AggregationInspectionHelper.hasValue(filters));
}
indexReader.close();
@ -185,14 +188,22 @@ public class FiltersAggregatorTests extends AggregatorTestCase {
List<InternalFilters.InternalBucket> buckets = response.getBuckets();
assertEquals(buckets.size(), filters.length+1);
int sum = 0;
for (InternalFilters.InternalBucket bucket : buckets) {
if ("other".equals(bucket.getKey())) {
assertEquals(bucket.getDocCount(), expectedOtherCount);
} else {
int index = Integer.parseInt(bucket.getKey());
assertEquals(bucket.getDocCount(), (long) expectedBucketCount[filterTerms[index]]);
sum += expectedBucketCount[filterTerms[index]];
}
}
if (sum > 0) {
assertTrue(AggregationInspectionHelper.hasValue(response));
} else {
assertFalse(AggregationInspectionHelper.hasValue(response));
}
}
indexReader.close();
directory.close();

View File

@ -31,6 +31,7 @@ import org.elasticsearch.index.mapper.GeoPointFieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import java.io.IOException;
import java.util.ArrayList;
@ -53,6 +54,7 @@ public class GeoHashGridAggregatorTests extends AggregatorTestCase {
// Intentionally not writing any docs
}, geoHashGrid -> {
assertEquals(0, geoHashGrid.getBuckets().size());
assertFalse(AggregationInspectionHelper.hasValue(geoHashGrid));
});
}
@ -61,6 +63,7 @@ public class GeoHashGridAggregatorTests extends AggregatorTestCase {
iw.addDocument(Collections.singleton(new LatLonDocValuesField(FIELD_NAME, 10D, 10D)));
}, geoHashGrid -> {
assertEquals(0, geoHashGrid.getBuckets().size());
assertFalse(AggregationInspectionHelper.hasValue(geoHashGrid));
});
}
@ -94,6 +97,7 @@ public class GeoHashGridAggregatorTests extends AggregatorTestCase {
for (GeoHashGrid.Bucket bucket : geoHashGrid.getBuckets()) {
assertEquals((long) expectedCountPerGeoHash.get(bucket.getKeyAsString()), bucket.getDocCount());
}
assertTrue(AggregationInspectionHelper.hasValue(geoHashGrid));
});
}

View File

@ -38,7 +38,8 @@ import org.elasticsearch.index.mapper.DateFieldMapper;
import org.elasticsearch.search.aggregations.AggregationBuilders;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.MultiBucketConsumerService;
import org.elasticsearch.search.aggregations.metrics.Stats;
import org.elasticsearch.search.aggregations.metrics.InternalStats;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import org.hamcrest.Matchers;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
@ -75,18 +76,27 @@ public class AutoDateHistogramAggregatorTests extends AggregatorTestCase {
public void testMatchNoDocs() throws IOException {
testBothCases(new MatchNoDocsQuery(), DATES_WITH_TIME,
aggregation -> aggregation.setNumBuckets(10).field(DATE_FIELD),
histogram -> assertEquals(0, histogram.getBuckets().size())
histogram -> {
assertEquals(0, histogram.getBuckets().size());
assertFalse(AggregationInspectionHelper.hasValue(histogram));
}
);
}
public void testMatchAllDocs() throws IOException {
testSearchCase(DEFAULT_QUERY, DATES_WITH_TIME,
aggregation -> aggregation.setNumBuckets(6).field(DATE_FIELD),
histogram -> assertEquals(10, histogram.getBuckets().size())
histogram -> {
assertEquals(10, histogram.getBuckets().size());
assertTrue(AggregationInspectionHelper.hasValue(histogram));
}
);
testSearchAndReduceCase(DEFAULT_QUERY, DATES_WITH_TIME,
aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD),
histogram -> assertEquals(8, histogram.getBuckets().size())
histogram -> {
assertEquals(8, histogram.getBuckets().size());
assertTrue(AggregationInspectionHelper.hasValue(histogram));
}
);
}
@ -95,16 +105,18 @@ public class AutoDateHistogramAggregatorTests extends AggregatorTestCase {
aggregation -> aggregation.setNumBuckets(8).field(DATE_FIELD)
.subAggregation(AggregationBuilders.stats("stats").field(DATE_FIELD)),
histogram -> {
assertTrue(AggregationInspectionHelper.hasValue(histogram));
final List<? extends Histogram.Bucket> buckets = histogram.getBuckets();
assertEquals(8, buckets.size());
Histogram.Bucket bucket = buckets.get(0);
assertEquals("2010-01-01T00:00:00.000Z", bucket.getKeyAsString());
assertEquals(2, bucket.getDocCount());
Stats stats = bucket.getAggregations().get("stats");
InternalStats stats = bucket.getAggregations().get("stats");
assertEquals("2010-03-12T01:07:45.000Z", stats.getMinAsString());
assertEquals("2010-04-27T03:43:34.000Z", stats.getMaxAsString());
assertEquals(2L, stats.getCount());
assertTrue(AggregationInspectionHelper.hasValue(stats));
bucket = buckets.get(1);
assertEquals("2011-01-01T00:00:00.000Z", bucket.getKeyAsString());
@ -113,6 +125,7 @@ public class AutoDateHistogramAggregatorTests extends AggregatorTestCase {
assertTrue(Double.isInfinite(stats.getMin()));
assertTrue(Double.isInfinite(stats.getMax()));
assertEquals(0L, stats.getCount());
assertFalse(AggregationInspectionHelper.hasValue(stats));
bucket = buckets.get(2);
assertEquals("2012-01-01T00:00:00.000Z", bucket.getKeyAsString());
@ -121,6 +134,7 @@ public class AutoDateHistogramAggregatorTests extends AggregatorTestCase {
assertEquals("2012-05-18T04:11:00.000Z", stats.getMinAsString());
assertEquals("2012-05-18T04:11:00.000Z", stats.getMaxAsString());
assertEquals(1L, stats.getCount());
assertTrue(AggregationInspectionHelper.hasValue(stats));
bucket = buckets.get(3);
assertEquals("2013-01-01T00:00:00.000Z", bucket.getKeyAsString());
@ -129,6 +143,7 @@ public class AutoDateHistogramAggregatorTests extends AggregatorTestCase {
assertEquals("2013-05-29T05:11:31.000Z", stats.getMinAsString());
assertEquals("2013-10-31T08:24:05.000Z", stats.getMaxAsString());
assertEquals(2L, stats.getCount());
assertTrue(AggregationInspectionHelper.hasValue(stats));
bucket = buckets.get(4);
assertEquals("2014-01-01T00:00:00.000Z", bucket.getKeyAsString());
@ -137,6 +152,7 @@ public class AutoDateHistogramAggregatorTests extends AggregatorTestCase {
assertTrue(Double.isInfinite(stats.getMin()));
assertTrue(Double.isInfinite(stats.getMax()));
assertEquals(0L, stats.getCount());
assertFalse(AggregationInspectionHelper.hasValue(stats));
bucket = buckets.get(5);
assertEquals("2015-01-01T00:00:00.000Z", bucket.getKeyAsString());
@ -145,6 +161,7 @@ public class AutoDateHistogramAggregatorTests extends AggregatorTestCase {
assertEquals("2015-02-13T13:09:32.000Z", stats.getMinAsString());
assertEquals("2015-11-13T16:14:34.000Z", stats.getMaxAsString());
assertEquals(3L, stats.getCount());
assertTrue(AggregationInspectionHelper.hasValue(stats));
bucket = buckets.get(6);
assertEquals("2016-01-01T00:00:00.000Z", bucket.getKeyAsString());
@ -153,6 +170,7 @@ public class AutoDateHistogramAggregatorTests extends AggregatorTestCase {
assertEquals("2016-03-04T17:09:50.000Z", stats.getMinAsString());
assertEquals("2016-03-04T17:09:50.000Z", stats.getMaxAsString());
assertEquals(1L, stats.getCount());
assertTrue(AggregationInspectionHelper.hasValue(stats));
bucket = buckets.get(7);
assertEquals("2017-01-01T00:00:00.000Z", bucket.getKeyAsString());
@ -161,6 +179,7 @@ public class AutoDateHistogramAggregatorTests extends AggregatorTestCase {
assertEquals("2017-12-12T22:55:46.000Z", stats.getMinAsString());
assertEquals("2017-12-12T22:55:46.000Z", stats.getMaxAsString());
assertEquals(1L, stats.getCount());
assertTrue(AggregationInspectionHelper.hasValue(stats));
});
}
@ -169,7 +188,10 @@ public class AutoDateHistogramAggregatorTests extends AggregatorTestCase {
final Consumer<AutoDateHistogramAggregationBuilder> aggregation = agg -> agg.setNumBuckets(10).field(DATE_FIELD);
testSearchCase(DEFAULT_QUERY, dates, aggregation,
histogram -> assertEquals(0, histogram.getBuckets().size())
histogram -> {
assertEquals(0, histogram.getBuckets().size());
assertFalse(AggregationInspectionHelper.hasValue(histogram));
}
);
testSearchAndReduceCase(DEFAULT_QUERY, dates, aggregation,
Assert::assertNull
@ -179,7 +201,10 @@ public class AutoDateHistogramAggregatorTests extends AggregatorTestCase {
public void testAggregateWrongField() throws IOException {
testBothCases(DEFAULT_QUERY, DATES_WITH_TIME,
aggregation -> aggregation.setNumBuckets(10).field("wrong_field"),
histogram -> assertEquals(0, histogram.getBuckets().size())
histogram -> {
assertEquals(0, histogram.getBuckets().size());
assertFalse(AggregationInspectionHelper.hasValue(histogram));
}
);
}
@ -197,6 +222,7 @@ public class AutoDateHistogramAggregatorTests extends AggregatorTestCase {
assertEquals(DATES_WITH_TIME.get(5 + i), bucket.getKey());
assertEquals(1, bucket.getDocCount());
}
assertTrue(AggregationInspectionHelper.hasValue(histogram));
}
);
testSearchAndReduceCase(rangeQuery, DATES_WITH_TIME,
@ -211,6 +237,7 @@ public class AutoDateHistogramAggregatorTests extends AggregatorTestCase {
assertEquals(expectedDocCount.size(), buckets.size());
buckets.forEach(bucket ->
assertEquals(expectedDocCount.getOrDefault(bucket.getKey(), 0).longValue(), bucket.getDocCount()));
assertTrue(AggregationInspectionHelper.hasValue(histogram));
}
);
}
@ -244,6 +271,7 @@ public class AutoDateHistogramAggregatorTests extends AggregatorTestCase {
assertEquals(expectedDocCount.size(), buckets.size());
buckets.forEach(bucket ->
assertEquals(expectedDocCount.getOrDefault(bucket.getKey(), 0).longValue(), bucket.getDocCount()));
assertTrue(AggregationInspectionHelper.hasValue(histogram));
}
);
}
@ -287,6 +315,7 @@ public class AutoDateHistogramAggregatorTests extends AggregatorTestCase {
assertEquals(5, buckets.size());
buckets.forEach(bucket ->
assertEquals(expectedDocCount.getOrDefault(bucket.getKey(), 0).longValue(), bucket.getDocCount()));
assertTrue(AggregationInspectionHelper.hasValue(histogram));
}
);
}
@ -311,6 +340,7 @@ public class AutoDateHistogramAggregatorTests extends AggregatorTestCase {
assertEquals(expectedDocCount.size(), buckets.size());
buckets.forEach(bucket ->
assertEquals(expectedDocCount.getOrDefault(bucket.getKeyAsString(), 0).longValue(), bucket.getDocCount()));
assertTrue(AggregationInspectionHelper.hasValue(histogram));
});
testSearchAndReduceCase(DEFAULT_QUERY, datesForDayInterval,
aggregation -> aggregation.setNumBuckets(5).field(DATE_FIELD).timeZone(DateTimeZone.forOffsetHours(-1)), histogram -> {
@ -323,6 +353,7 @@ public class AutoDateHistogramAggregatorTests extends AggregatorTestCase {
assertEquals(5, buckets.size());
buckets.forEach(bucket ->
assertEquals(expectedDocCount.getOrDefault(bucket.getKeyAsString(), 0).longValue(), bucket.getDocCount()));
assertTrue(AggregationInspectionHelper.hasValue(histogram));
});
}
@ -683,19 +714,19 @@ public class AutoDateHistogramAggregatorTests extends AggregatorTestCase {
private void testSearchCase(final Query query, final List<DateTime> dataset,
final Consumer<AutoDateHistogramAggregationBuilder> configure,
final Consumer<Histogram> verify) throws IOException {
final Consumer<InternalAutoDateHistogram> verify) throws IOException {
executeTestCase(false, query, dataset, configure, verify);
}
private void testSearchAndReduceCase(final Query query, final List<DateTime> dataset,
final Consumer<AutoDateHistogramAggregationBuilder> configure,
final Consumer<Histogram> verify) throws IOException {
final Consumer<InternalAutoDateHistogram> verify) throws IOException {
executeTestCase(true, query, dataset, configure, verify);
}
private void testBothCases(final Query query, final List<DateTime> dataset,
final Consumer<AutoDateHistogramAggregationBuilder> configure,
final Consumer<Histogram> verify) throws IOException {
final Consumer<InternalAutoDateHistogram> verify) throws IOException {
executeTestCase(false, query, dataset, configure, verify);
executeTestCase(true, query, dataset, configure, verify);
}
@ -716,7 +747,7 @@ public class AutoDateHistogramAggregatorTests extends AggregatorTestCase {
private void executeTestCase(final boolean reduced, final Query query, final List<DateTime> dataset,
final Consumer<AutoDateHistogramAggregationBuilder> configure,
final Consumer<Histogram> verify) throws IOException {
final Consumer<InternalAutoDateHistogram> verify) throws IOException {
try (Directory directory = newDirectory()) {
try (RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory)) {
final Document document = new Document();

View File

@ -34,6 +34,7 @@ import org.elasticsearch.index.mapper.DateFieldMapper;
import org.elasticsearch.search.aggregations.AggregationBuilders;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.MultiBucketConsumerService.TooManyBucketsException;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import java.io.IOException;
import java.util.Arrays;
@ -61,7 +62,10 @@ public class DateHistogramAggregatorTests extends AggregatorTestCase {
public void testMatchNoDocs() throws IOException {
testBothCases(new MatchNoDocsQuery(), dataset,
aggregation -> aggregation.dateHistogramInterval(DateHistogramInterval.YEAR).field(DATE_FIELD),
histogram -> assertEquals(0, histogram.getBuckets().size())
histogram -> {
assertEquals(0, histogram.getBuckets().size());
assertFalse(AggregationInspectionHelper.hasValue(histogram));
}
);
}
@ -70,15 +74,24 @@ public class DateHistogramAggregatorTests extends AggregatorTestCase {
testSearchCase(query, dataset,
aggregation -> aggregation.dateHistogramInterval(DateHistogramInterval.YEAR).field(DATE_FIELD),
histogram -> assertEquals(6, histogram.getBuckets().size())
histogram -> {
assertEquals(6, histogram.getBuckets().size());
assertTrue(AggregationInspectionHelper.hasValue(histogram));
}
);
testSearchAndReduceCase(query, dataset,
aggregation -> aggregation.dateHistogramInterval(DateHistogramInterval.YEAR).field(DATE_FIELD),
histogram -> assertEquals(8, histogram.getBuckets().size())
histogram -> {
assertEquals(8, histogram.getBuckets().size());
assertTrue(AggregationInspectionHelper.hasValue(histogram));
}
);
testBothCases(query, dataset,
aggregation -> aggregation.dateHistogramInterval(DateHistogramInterval.YEAR).field(DATE_FIELD).minDocCount(1L),
histogram -> assertEquals(6, histogram.getBuckets().size())
histogram -> {
assertEquals(6, histogram.getBuckets().size());
assertTrue(AggregationInspectionHelper.hasValue(histogram));
}
);
}
@ -89,7 +102,10 @@ public class DateHistogramAggregatorTests extends AggregatorTestCase {
agg.dateHistogramInterval(DateHistogramInterval.YEAR).field(DATE_FIELD);
testSearchCase(query, dates, aggregation,
histogram -> assertEquals(0, histogram.getBuckets().size())
histogram -> {
assertEquals(0, histogram.getBuckets().size());
assertFalse(AggregationInspectionHelper.hasValue(histogram));
}
);
testSearchAndReduceCase(query, dates, aggregation,
histogram -> assertNull(histogram)
@ -99,7 +115,10 @@ public class DateHistogramAggregatorTests extends AggregatorTestCase {
public void testAggregateWrongField() throws IOException {
testBothCases(new MatchAllDocsQuery(), dataset,
aggregation -> aggregation.dateHistogramInterval(DateHistogramInterval.YEAR).field("wrong_field"),
histogram -> assertEquals(0, histogram.getBuckets().size())
histogram -> {
assertEquals(0, histogram.getBuckets().size());
assertFalse(AggregationInspectionHelper.hasValue(histogram));
}
);
}
@ -371,39 +390,39 @@ public class DateHistogramAggregatorTests extends AggregatorTestCase {
private void testSearchCase(Query query, List<String> dataset,
Consumer<DateHistogramAggregationBuilder> configure,
Consumer<Histogram> verify) throws IOException {
Consumer<InternalDateHistogram> verify) throws IOException {
testSearchCase(query, dataset, configure, verify, 10000);
}
private void testSearchCase(Query query, List<String> dataset,
Consumer<DateHistogramAggregationBuilder> configure,
Consumer<Histogram> verify,
Consumer<InternalDateHistogram> verify,
int maxBucket) throws IOException {
executeTestCase(false, query, dataset, configure, verify, maxBucket);
}
private void testSearchAndReduceCase(Query query, List<String> dataset,
Consumer<DateHistogramAggregationBuilder> configure,
Consumer<Histogram> verify) throws IOException {
Consumer<InternalDateHistogram> verify) throws IOException {
testSearchAndReduceCase(query, dataset, configure, verify, 1000);
}
private void testSearchAndReduceCase(Query query, List<String> dataset,
Consumer<DateHistogramAggregationBuilder> configure,
Consumer<Histogram> verify,
Consumer<InternalDateHistogram> verify,
int maxBucket) throws IOException {
executeTestCase(true, query, dataset, configure, verify, maxBucket);
}
private void testBothCases(Query query, List<String> dataset,
Consumer<DateHistogramAggregationBuilder> configure,
Consumer<Histogram> verify) throws IOException {
Consumer<InternalDateHistogram> verify) throws IOException {
testBothCases(query, dataset, configure, verify, 10000);
}
private void testBothCases(Query query, List<String> dataset,
Consumer<DateHistogramAggregationBuilder> configure,
Consumer<Histogram> verify,
Consumer<InternalDateHistogram> verify,
int maxBucket) throws IOException {
testSearchCase(query, dataset, configure, verify, maxBucket);
testSearchAndReduceCase(query, dataset, configure, verify, maxBucket);
@ -411,7 +430,7 @@ public class DateHistogramAggregatorTests extends AggregatorTestCase {
private void executeTestCase(boolean reduced, Query query, List<String> dataset,
Consumer<DateHistogramAggregationBuilder> configure,
Consumer<Histogram> verify,
Consumer<InternalDateHistogram> verify,
int maxBucket) throws IOException {
try (Directory directory = newDirectory()) {

View File

@ -30,6 +30,7 @@ import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.NumberFieldMapper;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
public class HistogramAggregatorTests extends AggregatorTestCase {
@ -49,7 +50,7 @@ public class HistogramAggregatorTests extends AggregatorTestCase {
fieldType.setName("field");
try (IndexReader reader = w.getReader()) {
IndexSearcher searcher = new IndexSearcher(reader);
Histogram histogram = search(searcher, new MatchAllDocsQuery(), aggBuilder, fieldType);
InternalHistogram histogram = search(searcher, new MatchAllDocsQuery(), aggBuilder, fieldType);
assertEquals(4, histogram.getBuckets().size());
assertEquals(-10d, histogram.getBuckets().get(0).getKey());
assertEquals(2, histogram.getBuckets().get(0).getDocCount());
@ -59,6 +60,7 @@ public class HistogramAggregatorTests extends AggregatorTestCase {
assertEquals(2, histogram.getBuckets().get(2).getDocCount());
assertEquals(50d, histogram.getBuckets().get(3).getKey());
assertEquals(1, histogram.getBuckets().get(3).getDocCount());
assertTrue(AggregationInspectionHelper.hasValue(histogram));
}
}
}
@ -79,7 +81,7 @@ public class HistogramAggregatorTests extends AggregatorTestCase {
fieldType.setName("field");
try (IndexReader reader = w.getReader()) {
IndexSearcher searcher = new IndexSearcher(reader);
Histogram histogram = search(searcher, new MatchAllDocsQuery(), aggBuilder, fieldType);
InternalHistogram histogram = search(searcher, new MatchAllDocsQuery(), aggBuilder, fieldType);
assertEquals(4, histogram.getBuckets().size());
assertEquals(-10d, histogram.getBuckets().get(0).getKey());
assertEquals(2, histogram.getBuckets().get(0).getDocCount());
@ -89,6 +91,7 @@ public class HistogramAggregatorTests extends AggregatorTestCase {
assertEquals(2, histogram.getBuckets().get(2).getDocCount());
assertEquals(50d, histogram.getBuckets().get(3).getKey());
assertEquals(1, histogram.getBuckets().get(3).getDocCount());
assertTrue(AggregationInspectionHelper.hasValue(histogram));
}
}
}
@ -109,7 +112,7 @@ public class HistogramAggregatorTests extends AggregatorTestCase {
fieldType.setName("field");
try (IndexReader reader = w.getReader()) {
IndexSearcher searcher = new IndexSearcher(reader);
Histogram histogram = search(searcher, new MatchAllDocsQuery(), aggBuilder, fieldType);
InternalHistogram histogram = search(searcher, new MatchAllDocsQuery(), aggBuilder, fieldType);
assertEquals(4, histogram.getBuckets().size());
assertEquals(-4 * Math.PI, histogram.getBuckets().get(0).getKey());
assertEquals(1, histogram.getBuckets().get(0).getDocCount());
@ -119,6 +122,7 @@ public class HistogramAggregatorTests extends AggregatorTestCase {
assertEquals(2, histogram.getBuckets().get(2).getDocCount());
assertEquals(Math.PI, histogram.getBuckets().get(3).getKey());
assertEquals(1, histogram.getBuckets().get(3).getDocCount());
assertTrue(AggregationInspectionHelper.hasValue(histogram));
}
}
}
@ -140,12 +144,13 @@ public class HistogramAggregatorTests extends AggregatorTestCase {
fieldType.setName("field");
try (IndexReader reader = w.getReader()) {
IndexSearcher searcher = new IndexSearcher(reader);
Histogram histogram = searchAndReduce(searcher, new MatchAllDocsQuery(), aggBuilder, fieldType);
InternalHistogram histogram = searchAndReduce(searcher, new MatchAllDocsQuery(), aggBuilder, fieldType);
assertEquals(2, histogram.getBuckets().size());
assertEquals(-10d, histogram.getBuckets().get(0).getKey());
assertEquals(2, histogram.getBuckets().get(0).getDocCount());
assertEquals(0d, histogram.getBuckets().get(1).getKey());
assertEquals(3, histogram.getBuckets().get(1).getDocCount());
assertTrue(AggregationInspectionHelper.hasValue(histogram));
}
}
}
@ -168,7 +173,7 @@ public class HistogramAggregatorTests extends AggregatorTestCase {
fieldType.setName("field");
try (IndexReader reader = w.getReader()) {
IndexSearcher searcher = new IndexSearcher(reader);
Histogram histogram = search(searcher, new MatchAllDocsQuery(), aggBuilder, fieldType);
InternalHistogram histogram = search(searcher, new MatchAllDocsQuery(), aggBuilder, fieldType);
assertEquals(4, histogram.getBuckets().size());
assertEquals(-10d, histogram.getBuckets().get(0).getKey());
assertEquals(2, histogram.getBuckets().get(0).getDocCount());
@ -178,6 +183,7 @@ public class HistogramAggregatorTests extends AggregatorTestCase {
assertEquals(2, histogram.getBuckets().get(2).getDocCount());
assertEquals(50d, histogram.getBuckets().get(3).getKey());
assertEquals(1, histogram.getBuckets().get(3).getDocCount());
assertTrue(AggregationInspectionHelper.hasValue(histogram));
}
}
}
@ -199,7 +205,7 @@ public class HistogramAggregatorTests extends AggregatorTestCase {
fieldType.setName("field");
try (IndexReader reader = w.getReader()) {
IndexSearcher searcher = new IndexSearcher(reader);
Histogram histogram = search(searcher, new MatchAllDocsQuery(), aggBuilder, fieldType);
InternalHistogram histogram = search(searcher, new MatchAllDocsQuery(), aggBuilder, fieldType);
assertEquals(3, histogram.getBuckets().size());
assertEquals(-10 + Math.PI, histogram.getBuckets().get(0).getKey());
assertEquals(2, histogram.getBuckets().get(0).getDocCount());
@ -207,6 +213,7 @@ public class HistogramAggregatorTests extends AggregatorTestCase {
assertEquals(2, histogram.getBuckets().get(1).getDocCount());
assertEquals(5 + Math.PI, histogram.getBuckets().get(2).getKey());
assertEquals(1, histogram.getBuckets().get(2).getDocCount());
assertTrue(AggregationInspectionHelper.hasValue(histogram));
}
}
}
@ -228,7 +235,7 @@ public class HistogramAggregatorTests extends AggregatorTestCase {
fieldType.setName("field");
try (IndexReader reader = w.getReader()) {
IndexSearcher searcher = new IndexSearcher(reader);
Histogram histogram = searchAndReduce(searcher, new MatchAllDocsQuery(), aggBuilder, fieldType);
InternalHistogram histogram = searchAndReduce(searcher, new MatchAllDocsQuery(), aggBuilder, fieldType);
assertEquals(6, histogram.getBuckets().size());
assertEquals(-15d, histogram.getBuckets().get(0).getKey());
assertEquals(0, histogram.getBuckets().get(0).getDocCount());
@ -242,6 +249,7 @@ public class HistogramAggregatorTests extends AggregatorTestCase {
assertEquals(0, histogram.getBuckets().get(4).getDocCount());
assertEquals(10d, histogram.getBuckets().get(5).getKey());
assertEquals(0, histogram.getBuckets().get(5).getDocCount());
assertTrue(AggregationInspectionHelper.hasValue(histogram));
}
}
}

View File

@ -31,6 +31,7 @@ import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.NumberFieldMapper;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import org.elasticsearch.search.aggregations.support.ValueType;
import java.io.IOException;
@ -45,7 +46,10 @@ public class MissingAggregatorTests extends AggregatorTestCase {
"field",
Queries.newMatchAllQuery(),
doc -> doc.add(new SortedNumericDocValuesField("field", randomLong())),
internalMissing -> assertEquals(internalMissing.getDocCount(), 0));
internalMissing -> {
assertEquals(internalMissing.getDocCount(), 0);
assertFalse(AggregationInspectionHelper.hasValue(internalMissing));
});
}
public void testMatchAllDocs() throws IOException {
@ -54,7 +58,10 @@ public class MissingAggregatorTests extends AggregatorTestCase {
"field",
Queries.newMatchAllQuery(),
doc -> doc.add(new SortedNumericDocValuesField("another_field", randomLong())),
internalMissing -> assertEquals(internalMissing.getDocCount(), numDocs));
internalMissing -> {
assertEquals(internalMissing.getDocCount(), numDocs);
assertTrue(AggregationInspectionHelper.hasValue(internalMissing));
});
}
public void testMatchSparse() throws IOException {
@ -74,6 +81,7 @@ public class MissingAggregatorTests extends AggregatorTestCase {
internalMissing -> {
assertEquals(internalMissing.getDocCount(), count.get());
count.set(0);
assertTrue(AggregationInspectionHelper.hasValue(internalMissing));
});
}
@ -87,6 +95,7 @@ public class MissingAggregatorTests extends AggregatorTestCase {
},
internalMissing -> {
assertEquals(internalMissing.getDocCount(), numDocs);
assertTrue(AggregationInspectionHelper.hasValue(internalMissing));
});
}

View File

@ -63,6 +63,7 @@ import org.elasticsearch.search.aggregations.metrics.Min;
import org.elasticsearch.search.aggregations.metrics.MinAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.InternalSum;
import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.test.VersionUtils;
@ -115,16 +116,16 @@ public class NestedAggregatorTests extends AggregatorTestCase {
NumberFieldMapper.NumberType.LONG);
fieldType.setName(VALUE_FIELD_NAME);
Nested nested = search(newSearcher(indexReader, false, true),
InternalNested nested = search(newSearcher(indexReader, false, true),
new MatchAllDocsQuery(), nestedBuilder, fieldType);
assertEquals(NESTED_AGG, nested.getName());
assertEquals(0, nested.getDocCount());
InternalMax max = (InternalMax)
((InternalAggregation)nested).getProperty(MAX_AGG_NAME);
InternalMax max = (InternalMax) nested.getProperty(MAX_AGG_NAME);
assertEquals(MAX_AGG_NAME, max.getName());
assertEquals(Double.NEGATIVE_INFINITY, max.getValue(), Double.MIN_VALUE);
assertFalse(AggregationInspectionHelper.hasValue(nested));
}
}
}
@ -162,17 +163,18 @@ public class NestedAggregatorTests extends AggregatorTestCase {
NumberFieldMapper.NumberType.LONG);
fieldType.setName(VALUE_FIELD_NAME);
Nested nested = search(newSearcher(indexReader, false, true),
InternalNested nested = search(newSearcher(indexReader, false, true),
new MatchAllDocsQuery(), nestedBuilder, fieldType);
assertEquals(expectedNestedDocs, nested.getDocCount());
assertEquals(NESTED_AGG, nested.getName());
assertEquals(expectedNestedDocs, nested.getDocCount());
InternalMax max = (InternalMax)
((InternalAggregation)nested).getProperty(MAX_AGG_NAME);
InternalMax max = (InternalMax) nested.getProperty(MAX_AGG_NAME);
assertEquals(MAX_AGG_NAME, max.getName());
assertEquals(expectedMaxValue, max.getValue(), Double.MIN_VALUE);
assertTrue(AggregationInspectionHelper.hasValue(nested));
}
}
}
@ -211,17 +213,18 @@ public class NestedAggregatorTests extends AggregatorTestCase {
NumberFieldMapper.NumberType.LONG);
fieldType.setName(VALUE_FIELD_NAME);
Nested nested = search(newSearcher(indexReader, false, true),
InternalNested nested = search(newSearcher(indexReader, false, true),
new MatchAllDocsQuery(), nestedBuilder, fieldType);
assertEquals(expectedNestedDocs, nested.getDocCount());
assertEquals(NESTED_AGG, nested.getName());
assertEquals(expectedNestedDocs, nested.getDocCount());
InternalMax max = (InternalMax)
((InternalAggregation)nested).getProperty(MAX_AGG_NAME);
InternalMax max = (InternalMax) nested.getProperty(MAX_AGG_NAME);
assertEquals(MAX_AGG_NAME, max.getName());
assertEquals(expectedMaxValue, max.getValue(), Double.MIN_VALUE);
assertTrue(AggregationInspectionHelper.hasValue(nested));
}
}
}
@ -263,7 +266,7 @@ public class NestedAggregatorTests extends AggregatorTestCase {
NumberFieldMapper.NumberType.LONG);
fieldType.setName(VALUE_FIELD_NAME);
Nested nested = search(newSearcher(indexReader, false, true),
InternalNested nested = search(newSearcher(indexReader, false, true),
new MatchAllDocsQuery(), nestedBuilder, fieldType);
assertEquals(expectedNestedDocs, nested.getDocCount());
@ -348,13 +351,15 @@ public class NestedAggregatorTests extends AggregatorTestCase {
bq.add(Queries.newNonNestedFilter(VersionUtils.randomVersion(random())), BooleanClause.Occur.MUST);
bq.add(new TermQuery(new Term(IdFieldMapper.NAME, Uid.encodeId("2"))), BooleanClause.Occur.MUST_NOT);
Nested nested = search(newSearcher(indexReader, false, true),
InternalNested nested = search(newSearcher(indexReader, false, true),
new ConstantScoreQuery(bq.build()), nestedBuilder, fieldType);
assertEquals(NESTED_AGG, nested.getName());
// The bug manifests if 6 docs are returned, because currentRootDoc isn't reset the previous child docs from the first
// segment are emitted as hits.
assertEquals(4L, nested.getDocCount());
assertTrue(AggregationInspectionHelper.hasValue(nested));
}
}
}
@ -630,7 +635,7 @@ public class NestedAggregatorTests extends AggregatorTestCase {
assertEquals("filterAgg", filter.getName());
assertEquals(3L, filter.getDocCount());
Nested nested = filter.getAggregations().get(NESTED_AGG);
InternalNested nested = filter.getAggregations().get(NESTED_AGG);
assertEquals(6L, nested.getDocCount());
StringTerms keyAgg = nested.getAggregations().get("key");
@ -687,7 +692,7 @@ public class NestedAggregatorTests extends AggregatorTestCase {
NestedAggregationBuilder aliasAgg = nested(NESTED_AGG, NESTED_OBJECT).subAggregation(
max(MAX_AGG_NAME).field(VALUE_FIELD_NAME + "-alias"));
Nested nested = search(newSearcher(indexReader, false, true),
InternalNested nested = search(newSearcher(indexReader, false, true),
new MatchAllDocsQuery(), agg, fieldType);
Nested aliasNested = search(newSearcher(indexReader, false, true),
new MatchAllDocsQuery(), aliasAgg, fieldType);

View File

@ -39,6 +39,7 @@ import org.elasticsearch.index.mapper.TextFieldMapper.TextFieldType;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.metrics.Min;
import org.elasticsearch.search.aggregations.metrics.MinAggregationBuilder;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import java.io.IOException;
@ -75,10 +76,11 @@ public class SamplerAggregatorTests extends AggregatorTestCase {
try (IndexReader reader = DirectoryReader.open(w)) {
assertEquals("test expects a single segment", 1, reader.leaves().size());
IndexSearcher searcher = new IndexSearcher(reader);
Sampler sampler = searchAndReduce(searcher, new TermQuery(new Term("text", "good")), aggBuilder, textFieldType,
InternalSampler sampler = searchAndReduce(searcher, new TermQuery(new Term("text", "good")), aggBuilder, textFieldType,
numericFieldType);
Min min = sampler.getAggregations().get("min");
assertEquals(5.0, min.getValue(), 0);
assertTrue(AggregationInspectionHelper.hasValue(sampler));
}
}
}

View File

@ -37,8 +37,9 @@ import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.TextFieldMapper.TextFieldType;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.bucket.sampler.Sampler;
import org.elasticsearch.search.aggregations.bucket.sampler.InternalSampler;
import org.elasticsearch.search.aggregations.bucket.sampler.SamplerAggregationBuilder;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import java.io.IOException;
import java.util.Arrays;
@ -88,7 +89,7 @@ public class SignificantTextAggregatorTests extends AggregatorTestCase {
IndexSearcher searcher = new IndexSearcher(reader);
// Search "odd" which should have no duplication
Sampler sampler = searchAndReduce(searcher, new TermQuery(new Term("text", "odd")), aggBuilder, textFieldType);
InternalSampler sampler = searchAndReduce(searcher, new TermQuery(new Term("text", "odd")), aggBuilder, textFieldType);
SignificantTerms terms = sampler.getAggregations().get("sig_text");
assertNull(terms.getBucketByKey("even"));
@ -109,6 +110,7 @@ public class SignificantTextAggregatorTests extends AggregatorTestCase {
assertNotNull(terms.getBucketByKey("even"));
assertTrue(AggregationInspectionHelper.hasValue(sampler));
}
}
}
@ -142,8 +144,9 @@ public class SignificantTextAggregatorTests extends AggregatorTestCase {
SamplerAggregationBuilder samplerAgg = sampler("sampler").subAggregation(agg);
SamplerAggregationBuilder aliasSamplerAgg = sampler("sampler").subAggregation(aliasAgg);
Sampler sampler = searchAndReduce(searcher, new TermQuery(new Term("text", "odd")), samplerAgg, textFieldType);
Sampler aliasSampler = searchAndReduce(searcher, new TermQuery(new Term("text", "odd")), aliasSamplerAgg, textFieldType);
InternalSampler sampler = searchAndReduce(searcher, new TermQuery(new Term("text", "odd")), samplerAgg, textFieldType);
InternalSampler aliasSampler = searchAndReduce(searcher, new TermQuery(new Term("text", "odd")),
aliasSamplerAgg, textFieldType);
SignificantTerms terms = sampler.getAggregations().get("sig_text");
SignificantTerms aliasTerms = aliasSampler.getAggregations().get("sig_text");
@ -157,6 +160,9 @@ public class SignificantTextAggregatorTests extends AggregatorTestCase {
aliasTerms = aliasSampler.getAggregations().get("sig_text");
assertFalse(terms.getBuckets().isEmpty());
assertEquals(terms, aliasTerms);
assertTrue(AggregationInspectionHelper.hasValue(sampler));
assertTrue(AggregationInspectionHelper.hasValue(aliasSampler));
}
}
}

View File

@ -73,6 +73,7 @@ import org.elasticsearch.search.aggregations.bucket.nested.NestedAggregationBuil
import org.elasticsearch.search.aggregations.metrics.InternalTopHits;
import org.elasticsearch.search.aggregations.metrics.TopHitsAggregationBuilder;
import org.elasticsearch.search.aggregations.pipeline.BucketScriptPipelineAggregationBuilder;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.sort.FieldSortBuilder;
import org.elasticsearch.search.sort.ScoreSortBuilder;
@ -199,6 +200,7 @@ public class TermsAggregatorTests extends AggregatorTestCase {
assertEquals(1L, result.getBuckets().get(3).getDocCount());
assertEquals("d", result.getBuckets().get(4).getKeyAsString());
assertEquals(1L, result.getBuckets().get(4).getDocCount());
assertTrue(AggregationInspectionHelper.hasValue((InternalTerms)result));
}
}
}
@ -278,6 +280,7 @@ public class TermsAggregatorTests extends AggregatorTestCase {
assertEquals(1L, result.getBuckets().get(8).getDocCount());
assertEquals("val009", result.getBuckets().get(9).getKeyAsString());
assertEquals(1L, result.getBuckets().get(9).getDocCount());
assertTrue(AggregationInspectionHelper.hasValue((InternalTerms)result));
MappedFieldType fieldType2 = new KeywordFieldMapper.KeywordFieldType();
fieldType2.setName("sv_field");
@ -304,6 +307,7 @@ public class TermsAggregatorTests extends AggregatorTestCase {
assertEquals(1L, result.getBuckets().get(3).getDocCount());
assertEquals("val009", result.getBuckets().get(4).getKeyAsString());
assertEquals(1L, result.getBuckets().get(4).getDocCount());
assertTrue(AggregationInspectionHelper.hasValue((InternalTerms)result));
aggregationBuilder = new TermsAggregationBuilder("_name", ValueType.STRING)
.executionHint(executionHint)
@ -333,6 +337,7 @@ public class TermsAggregatorTests extends AggregatorTestCase {
assertEquals(1L, result.getBuckets().get(6).getDocCount());
assertEquals("val009", result.getBuckets().get(7).getKeyAsString());
assertEquals(1L, result.getBuckets().get(7).getDocCount());
assertTrue(AggregationInspectionHelper.hasValue((InternalTerms)result));
aggregationBuilder = new TermsAggregationBuilder("_name", ValueType.STRING)
.executionHint(executionHint)
@ -349,6 +354,7 @@ public class TermsAggregatorTests extends AggregatorTestCase {
assertEquals(1L, result.getBuckets().get(0).getDocCount());
assertEquals("val011", result.getBuckets().get(1).getKeyAsString());
assertEquals(1L, result.getBuckets().get(1).getDocCount());
assertTrue(AggregationInspectionHelper.hasValue((InternalTerms)result));
aggregationBuilder = new TermsAggregationBuilder("_name", ValueType.STRING)
.executionHint(executionHint)
@ -365,6 +371,7 @@ public class TermsAggregatorTests extends AggregatorTestCase {
assertEquals(1L, result.getBuckets().get(0).getDocCount());
assertEquals("val010", result.getBuckets().get(1).getKeyAsString());
assertEquals(1L, result.getBuckets().get(1).getDocCount());
assertTrue(AggregationInspectionHelper.hasValue((InternalTerms)result));
aggregationBuilder = new TermsAggregationBuilder("_name", ValueType.STRING)
.executionHint(executionHint)
@ -382,6 +389,7 @@ public class TermsAggregatorTests extends AggregatorTestCase {
assertEquals(1L, result.getBuckets().get(0).getDocCount());
assertEquals("val010", result.getBuckets().get(1).getKeyAsString());
assertEquals(1L, result.getBuckets().get(1).getDocCount());
assertTrue(AggregationInspectionHelper.hasValue((InternalTerms)result));
}
}
}
@ -436,6 +444,7 @@ public class TermsAggregatorTests extends AggregatorTestCase {
assertEquals(1L, result.getBuckets().get(0).getDocCount());
assertEquals(5L, result.getBuckets().get(1).getKey());
assertEquals(1L, result.getBuckets().get(1).getDocCount());
assertTrue(AggregationInspectionHelper.hasValue((InternalTerms)result));
aggregationBuilder = new TermsAggregationBuilder("_name", ValueType.LONG)
.executionHint(executionHint)
@ -456,6 +465,7 @@ public class TermsAggregatorTests extends AggregatorTestCase {
assertEquals(1L, result.getBuckets().get(2).getDocCount());
assertEquals(4L, result.getBuckets().get(3).getKey());
assertEquals(1L, result.getBuckets().get(3).getDocCount());
assertTrue(AggregationInspectionHelper.hasValue((InternalTerms)result));
fieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.DOUBLE);
fieldType.setName("double_field");
@ -475,6 +485,7 @@ public class TermsAggregatorTests extends AggregatorTestCase {
assertEquals(1L, result.getBuckets().get(0).getDocCount());
assertEquals(5.0, result.getBuckets().get(1).getKey());
assertEquals(1L, result.getBuckets().get(1).getDocCount());
assertTrue(AggregationInspectionHelper.hasValue((InternalTerms)result));
aggregationBuilder = new TermsAggregationBuilder("_name", ValueType.DOUBLE)
.executionHint(executionHint)
@ -495,6 +506,7 @@ public class TermsAggregatorTests extends AggregatorTestCase {
assertEquals(1L, result.getBuckets().get(2).getDocCount());
assertEquals(4.0, result.getBuckets().get(3).getKey());
assertEquals(1L, result.getBuckets().get(3).getDocCount());
assertTrue(AggregationInspectionHelper.hasValue((InternalTerms)result));
}
}
}

View File

@ -35,9 +35,7 @@ import org.elasticsearch.common.CheckedConsumer;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.NumberFieldMapper;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.metrics.AvgAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.AvgAggregator;
import org.elasticsearch.search.aggregations.metrics.InternalAvg;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import java.io.IOException;
import java.util.Arrays;
@ -52,6 +50,7 @@ public class AvgAggregatorTests extends AggregatorTestCase {
// Intentionally not writing any docs
}, avg -> {
assertEquals(Double.NaN, avg.getValue(), 0);
assertFalse(AggregationInspectionHelper.hasValue(avg));
});
}
@ -61,6 +60,7 @@ public class AvgAggregatorTests extends AggregatorTestCase {
iw.addDocument(singleton(new SortedNumericDocValuesField("wrong_number", 3)));
}, avg -> {
assertEquals(Double.NaN, avg.getValue(), 0);
assertFalse(AggregationInspectionHelper.hasValue(avg));
});
}
@ -71,6 +71,7 @@ public class AvgAggregatorTests extends AggregatorTestCase {
iw.addDocument(singleton(new SortedNumericDocValuesField("number", 3)));
}, avg -> {
assertEquals(4, avg.getValue(), 0);
assertTrue(AggregationInspectionHelper.hasValue(avg));
});
}
@ -81,6 +82,7 @@ public class AvgAggregatorTests extends AggregatorTestCase {
iw.addDocument(singleton(new NumericDocValuesField("number", 3)));
}, avg -> {
assertEquals(4, avg.getValue(), 0);
assertTrue(AggregationInspectionHelper.hasValue(avg));
});
}
@ -91,6 +93,7 @@ public class AvgAggregatorTests extends AggregatorTestCase {
iw.addDocument(Arrays.asList(new IntPoint("number", 3), new SortedNumericDocValuesField("number", 3)));
}, avg -> {
assertEquals(2.5, avg.getValue(), 0);
assertTrue(AggregationInspectionHelper.hasValue(avg));
});
}
@ -101,6 +104,7 @@ public class AvgAggregatorTests extends AggregatorTestCase {
iw.addDocument(Arrays.asList(new IntPoint("number", 3), new SortedNumericDocValuesField("number", 7)));
}, avg -> {
assertEquals(Double.NaN, avg.getValue(), 0);
assertFalse(AggregationInspectionHelper.hasValue(avg));
});
}

View File

@ -34,6 +34,7 @@ import org.elasticsearch.common.CheckedConsumer;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.NumberFieldMapper;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import org.elasticsearch.search.aggregations.support.ValueType;
import java.io.IOException;
@ -48,6 +49,7 @@ public class CardinalityAggregatorTests extends AggregatorTestCase {
// Intentionally not writing any docs
}, card -> {
assertEquals(0.0, card.getValue(), 0);
assertFalse(AggregationInspectionHelper.hasValue(card));
});
}
@ -57,6 +59,7 @@ public class CardinalityAggregatorTests extends AggregatorTestCase {
iw.addDocument(singleton(new SortedNumericDocValuesField("wrong_number", 1)));
}, card -> {
assertEquals(0.0, card.getValue(), 0);
assertFalse(AggregationInspectionHelper.hasValue(card));
});
}
@ -66,6 +69,7 @@ public class CardinalityAggregatorTests extends AggregatorTestCase {
iw.addDocument(singleton(new SortedNumericDocValuesField("number", 1)));
}, card -> {
assertEquals(2, card.getValue(), 0);
assertTrue(AggregationInspectionHelper.hasValue(card));
});
}
@ -75,6 +79,7 @@ public class CardinalityAggregatorTests extends AggregatorTestCase {
iw.addDocument(singleton(new NumericDocValuesField("number", 1)));
}, card -> {
assertEquals(2, card.getValue(), 0);
assertTrue(AggregationInspectionHelper.hasValue(card));
});
}
@ -86,6 +91,7 @@ public class CardinalityAggregatorTests extends AggregatorTestCase {
new SortedNumericDocValuesField("number", 1)));
}, card -> {
assertEquals(1, card.getValue(), 0);
assertTrue(AggregationInspectionHelper.hasValue(card));
});
}
@ -97,6 +103,7 @@ public class CardinalityAggregatorTests extends AggregatorTestCase {
new SortedNumericDocValuesField("number", 1)));
}, card -> {
assertEquals(0.0, card.getValue(), 0);
assertFalse(AggregationInspectionHelper.hasValue(card));
});
}

View File

@ -32,6 +32,7 @@ import org.elasticsearch.common.CheckedConsumer;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.NumberFieldMapper;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import java.io.IOException;
import java.util.function.Consumer;
@ -55,6 +56,7 @@ public class ExtendedStatsAggregatorTests extends AggregatorTestCase {
assertEquals(Double.NaN, stats.getVariance(), 0);
assertEquals(Double.NaN, stats.getStdDeviation(), 0);
assertEquals(0d, stats.getSumOfSquares(), 0);
assertFalse(AggregationInspectionHelper.hasValue(stats));
}
);
}
@ -92,6 +94,7 @@ public class ExtendedStatsAggregatorTests extends AggregatorTestCase {
stats.getStdDeviationBound(ExtendedStats.Bounds.LOWER), TOLERANCE);
assertEquals(expected.stdDevBound(ExtendedStats.Bounds.UPPER, stats.getSigma()),
stats.getStdDeviationBound(ExtendedStats.Bounds.UPPER), TOLERANCE);
assertTrue(AggregationInspectionHelper.hasValue(stats));
}
);
}
@ -128,6 +131,7 @@ public class ExtendedStatsAggregatorTests extends AggregatorTestCase {
stats.getStdDeviationBound(ExtendedStats.Bounds.LOWER), TOLERANCE);
assertEquals(expected.stdDevBound(ExtendedStats.Bounds.UPPER, stats.getSigma()),
stats.getStdDeviationBound(ExtendedStats.Bounds.UPPER), TOLERANCE);
assertTrue(AggregationInspectionHelper.hasValue(stats));
}
);
}

View File

@ -30,6 +30,7 @@ import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.index.mapper.GeoPointFieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import org.elasticsearch.test.geo.RandomGeoGenerator;
import static org.elasticsearch.search.aggregations.metrics.InternalGeoBoundsTests.GEOHASH_TOLERANCE;
@ -55,6 +56,7 @@ public class GeoBoundsAggregatorTests extends AggregatorTestCase {
assertTrue(Double.isInfinite(bounds.posRight));
assertTrue(Double.isInfinite(bounds.negLeft));
assertTrue(Double.isInfinite(bounds.negRight));
assertFalse(AggregationInspectionHelper.hasValue(bounds));
}
}
}
@ -112,6 +114,7 @@ public class GeoBoundsAggregatorTests extends AggregatorTestCase {
assertThat(bounds.posRight, closeTo(posRight, GEOHASH_TOLERANCE));
assertThat(bounds.negRight, closeTo(negRight, GEOHASH_TOLERANCE));
assertThat(bounds.negLeft, closeTo(negLeft, GEOHASH_TOLERANCE));
assertTrue(AggregationInspectionHelper.hasValue(bounds));
}
}
}

View File

@ -31,6 +31,7 @@ import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.metrics.GeoCentroidAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.InternalGeoCentroid;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import org.elasticsearch.test.geo.RandomGeoGenerator;
import java.io.IOException;
@ -52,6 +53,7 @@ public class GeoCentroidAggregatorTests extends AggregatorTestCase {
IndexSearcher searcher = new IndexSearcher(reader);
InternalGeoCentroid result = search(searcher, new MatchAllDocsQuery(), aggBuilder, fieldType);
assertNull(result.centroid());
assertFalse(AggregationInspectionHelper.hasValue(result));
}
}
}
@ -79,6 +81,7 @@ public class GeoCentroidAggregatorTests extends AggregatorTestCase {
fieldType.setName("field");
result = search(searcher, new MatchAllDocsQuery(), aggBuilder, fieldType);
assertNull(result.centroid());
assertFalse(AggregationInspectionHelper.hasValue(result));
}
}
}
@ -149,6 +152,7 @@ public class GeoCentroidAggregatorTests extends AggregatorTestCase {
assertNotNull(centroid);
assertEquals(expectedCentroid.getLat(), centroid.getLat(), GEOHASH_TOLERANCE);
assertEquals(expectedCentroid.getLon(), centroid.getLon(), GEOHASH_TOLERANCE);
assertTrue(AggregationInspectionHelper.hasValue(result));
}
}

View File

@ -35,6 +35,7 @@ import org.elasticsearch.search.aggregations.metrics.Percentile;
import org.elasticsearch.search.aggregations.metrics.PercentileRanks;
import org.elasticsearch.search.aggregations.metrics.PercentileRanksAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.PercentilesMethod;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import org.hamcrest.Matchers;
import java.io.IOException;
@ -55,6 +56,7 @@ public class HDRPercentileRanksAggregatorTests extends AggregatorTestCase {
Percentile rank = ranks.iterator().next();
assertEquals(Double.NaN, rank.getPercent(), 0d);
assertEquals(0.5, rank.getValue(), 0d);
assertFalse(AggregationInspectionHelper.hasValue((InternalHDRPercentileRanks)ranks));
}
}
@ -87,6 +89,7 @@ public class HDRPercentileRanksAggregatorTests extends AggregatorTestCase {
assertEquals(12, rank.getValue(), 0d);
assertThat(rank.getPercent(), Matchers.equalTo(100d));
assertFalse(rankIterator.hasNext());
assertTrue(AggregationInspectionHelper.hasValue((InternalHDRPercentileRanks)ranks));
}
}
}

View File

@ -38,6 +38,7 @@ import org.elasticsearch.search.aggregations.metrics.HDRPercentilesAggregator;
import org.elasticsearch.search.aggregations.metrics.InternalHDRPercentiles;
import org.elasticsearch.search.aggregations.metrics.PercentilesAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.PercentilesMethod;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import java.io.IOException;
import java.util.function.Consumer;
@ -52,6 +53,7 @@ public class HDRPercentilesAggregatorTests extends AggregatorTestCase {
// Intentionally not writing any docs
}, hdr -> {
assertEquals(0L, hdr.state.getTotalCount());
assertFalse(AggregationInspectionHelper.hasValue(hdr));
});
}
@ -61,6 +63,7 @@ public class HDRPercentilesAggregatorTests extends AggregatorTestCase {
iw.addDocument(singleton(new SortedNumericDocValuesField("wrong_number", 1)));
}, hdr -> {
assertEquals(0L, hdr.state.getTotalCount());
assertFalse(AggregationInspectionHelper.hasValue(hdr));
});
}
@ -77,6 +80,7 @@ public class HDRPercentilesAggregatorTests extends AggregatorTestCase {
assertEquals(20.0d, hdr.percentile(50), approximation);
assertEquals(40.0d, hdr.percentile(75), approximation);
assertEquals(60.0d, hdr.percentile(99), approximation);
assertTrue(AggregationInspectionHelper.hasValue(hdr));
});
}
@ -93,6 +97,7 @@ public class HDRPercentilesAggregatorTests extends AggregatorTestCase {
assertEquals(20.0d, hdr.percentile(50), approximation);
assertEquals(40.0d, hdr.percentile(75), approximation);
assertEquals(60.0d, hdr.percentile(99), approximation);
assertTrue(AggregationInspectionHelper.hasValue(hdr));
});
}
@ -107,10 +112,12 @@ public class HDRPercentilesAggregatorTests extends AggregatorTestCase {
testCase(LongPoint.newRangeQuery("row", 0, 2), docs, hdr -> {
assertEquals(2L, hdr.state.getTotalCount());
assertEquals(10.0d, hdr.percentile(randomDoubleBetween(1, 50, true)), 0.05d);
assertTrue(AggregationInspectionHelper.hasValue(hdr));
});
testCase(LongPoint.newRangeQuery("row", 5, 10), docs, hdr -> {
assertEquals(0L, hdr.state.getTotalCount());
assertFalse(AggregationInspectionHelper.hasValue(hdr));
});
}

View File

@ -49,6 +49,7 @@ import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.NumberFieldMapper;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import java.io.IOException;
import java.util.ArrayList;
@ -69,6 +70,7 @@ public class MaxAggregatorTests extends AggregatorTestCase {
// Intentionally not writing any docs
}, max -> {
assertEquals(Double.NEGATIVE_INFINITY, max.getValue(), 0);
assertFalse(AggregationInspectionHelper.hasValue(max));
});
}
@ -78,6 +80,7 @@ public class MaxAggregatorTests extends AggregatorTestCase {
iw.addDocument(singleton(new SortedNumericDocValuesField("wrong_number", 1)));
}, max -> {
assertEquals(Double.NEGATIVE_INFINITY, max.getValue(), 0);
assertFalse(AggregationInspectionHelper.hasValue(max));
});
}
@ -87,6 +90,7 @@ public class MaxAggregatorTests extends AggregatorTestCase {
iw.addDocument(singleton(new SortedNumericDocValuesField("number", 1)));
}, max -> {
assertEquals(7, max.getValue(), 0);
assertTrue(AggregationInspectionHelper.hasValue(max));
});
}
@ -96,6 +100,7 @@ public class MaxAggregatorTests extends AggregatorTestCase {
iw.addDocument(singleton(new NumericDocValuesField("number", 1)));
}, max -> {
assertEquals(7, max.getValue(), 0);
assertTrue(AggregationInspectionHelper.hasValue(max));
});
}
@ -105,6 +110,7 @@ public class MaxAggregatorTests extends AggregatorTestCase {
iw.addDocument(Arrays.asList(new IntPoint("number", 1), new SortedNumericDocValuesField("number", 1)));
}, max -> {
assertEquals(1, max.getValue(), 0);
assertTrue(AggregationInspectionHelper.hasValue(max));
});
}
@ -114,6 +120,7 @@ public class MaxAggregatorTests extends AggregatorTestCase {
iw.addDocument(Arrays.asList(new IntPoint("number", 1), new SortedNumericDocValuesField("number", 1)));
}, max -> {
assertEquals(Double.NEGATIVE_INFINITY, max.getValue(), 0);
assertFalse(AggregationInspectionHelper.hasValue(max));
});
}

View File

@ -35,6 +35,7 @@ import org.elasticsearch.common.CheckedConsumer;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.NumberFieldMapper;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import org.hamcrest.Description;
import org.hamcrest.TypeSafeMatcher;
@ -71,7 +72,10 @@ public class MedianAbsoluteDeviationAggregatorTests extends AggregatorTestCase {
// intentionally not writing any docs
public void testNoDocs() throws IOException {
testCase(new MatchAllDocsQuery(), writer -> {}, agg -> assertThat(agg.getMedianAbsoluteDeviation(), equalTo(Double.NaN)));
testCase(new MatchAllDocsQuery(), writer -> {}, agg -> {
assertThat(agg.getMedianAbsoluteDeviation(), equalTo(Double.NaN));
assertFalse(AggregationInspectionHelper.hasValue(agg));
});
}
public void testNoMatchingField() throws IOException {
@ -81,7 +85,10 @@ public class MedianAbsoluteDeviationAggregatorTests extends AggregatorTestCase {
writer.addDocument(singleton(new SortedNumericDocValuesField("wrong_number", 1)));
writer.addDocument(singleton(new SortedNumericDocValuesField("wrong_number", 2)));
},
agg -> assertThat(agg.getMedianAbsoluteDeviation(), equalTo(Double.NaN))
agg -> {
assertThat(agg.getMedianAbsoluteDeviation(), equalTo(Double.NaN));
assertFalse(AggregationInspectionHelper.hasValue(agg));
}
);
}
@ -94,7 +101,10 @@ public class MedianAbsoluteDeviationAggregatorTests extends AggregatorTestCase {
sample.add(point);
return singleton(new SortedNumericDocValuesField("number", point));
}),
agg -> assertThat(agg.getMedianAbsoluteDeviation(), closeToRelative(calculateMAD(sample)))
agg -> {
assertThat(agg.getMedianAbsoluteDeviation(), closeToRelative(calculateMAD(sample)));
assertTrue(AggregationInspectionHelper.hasValue(agg));
}
);
}
@ -107,7 +117,10 @@ public class MedianAbsoluteDeviationAggregatorTests extends AggregatorTestCase {
sample.add(point);
return singleton(new NumericDocValuesField("number", point));
}),
agg -> assertThat(agg.getMedianAbsoluteDeviation(), closeToRelative(calculateMAD(sample)))
agg -> {
assertThat(agg.getMedianAbsoluteDeviation(), closeToRelative(calculateMAD(sample)));
assertTrue(AggregationInspectionHelper.hasValue(agg));
}
);
}
@ -123,7 +136,10 @@ public class MedianAbsoluteDeviationAggregatorTests extends AggregatorTestCase {
writer.addDocument(Arrays.asList(new IntPoint("number", point), new SortedNumericDocValuesField("number", point)));
}
},
agg -> assertThat(agg.getMedianAbsoluteDeviation(), closeToRelative(calculateMAD(filteredSample)))
agg -> {
assertThat(agg.getMedianAbsoluteDeviation(), closeToRelative(calculateMAD(filteredSample)));
assertTrue(AggregationInspectionHelper.hasValue(agg));
}
);
}
@ -134,7 +150,10 @@ public class MedianAbsoluteDeviationAggregatorTests extends AggregatorTestCase {
writer.addDocument(Arrays.asList(new IntPoint("number", 1), new SortedNumericDocValuesField("number", 1)));
writer.addDocument(Arrays.asList(new IntPoint("number", 2), new SortedNumericDocValuesField("number", 2)));
},
agg -> assertThat(agg.getMedianAbsoluteDeviation(), equalTo(Double.NaN))
agg -> {
assertThat(agg.getMedianAbsoluteDeviation(), equalTo(Double.NaN));
assertFalse(AggregationInspectionHelper.hasValue(agg));
}
);
}

View File

@ -50,6 +50,7 @@ import org.elasticsearch.index.mapper.NumberFieldMapper;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import org.elasticsearch.search.aggregations.support.FieldContext;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
@ -116,6 +117,7 @@ public class MinAggregatorTests extends AggregatorTestCase {
aggregator.postCollection();
InternalMin result = (InternalMin) aggregator.buildAggregation(0L);
assertEquals(-1.0, result.getValue(), 0);
assertTrue(AggregationInspectionHelper.hasValue(result));
indexReader.close();
directory.close();
@ -157,6 +159,7 @@ public class MinAggregatorTests extends AggregatorTestCase {
aggregator.postCollection();
InternalMin result = (InternalMin) aggregator.buildAggregation(0L);
assertEquals(-1.0, result.getValue(), 0);
assertTrue(AggregationInspectionHelper.hasValue(result));
indexReader.close();
directory.close();
@ -189,6 +192,7 @@ public class MinAggregatorTests extends AggregatorTestCase {
aggregator.postCollection();
InternalMin result = (InternalMin) aggregator.buildAggregation(0L);
assertEquals(Double.POSITIVE_INFINITY, result.getValue(), 0);
assertFalse(AggregationInspectionHelper.hasValue(result));
indexReader.close();
directory.close();
@ -212,6 +216,7 @@ public class MinAggregatorTests extends AggregatorTestCase {
aggregator.postCollection();
InternalMin result = (InternalMin) aggregator.buildAggregation(0L);
assertEquals(Double.POSITIVE_INFINITY, result.getValue(), 0);
assertFalse(AggregationInspectionHelper.hasValue(result));
indexReader.close();
directory.close();

View File

@ -31,6 +31,7 @@ import org.elasticsearch.common.CheckedConsumer;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.NumberFieldMapper;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import java.io.IOException;
import java.util.function.Consumer;
@ -51,6 +52,7 @@ public class StatsAggregatorTests extends AggregatorTestCase {
assertEquals(Float.NaN, stats.getAvg(), 0);
assertEquals(Double.POSITIVE_INFINITY, stats.getMin(), 0);
assertEquals(Double.NEGATIVE_INFINITY, stats.getMax(), 0);
assertFalse(AggregationInspectionHelper.hasValue(stats));
}
);
}
@ -81,6 +83,7 @@ public class StatsAggregatorTests extends AggregatorTestCase {
assertEquals(expected.min, stats.getMin(), 0);
assertEquals(expected.max, stats.getMax(), 0);
assertEquals(expected.sum / expected.count, stats.getAvg(), TOLERANCE);
assertTrue(AggregationInspectionHelper.hasValue(stats));
}
);
}
@ -110,6 +113,7 @@ public class StatsAggregatorTests extends AggregatorTestCase {
assertEquals(expected.min, stats.getMin(), 0);
assertEquals(expected.max, stats.getMax(), 0);
assertEquals(expected.sum / expected.count, stats.getAvg(), TOLERANCE);
assertTrue(AggregationInspectionHelper.hasValue(stats));
}
);
}

View File

@ -39,6 +39,7 @@ import org.elasticsearch.common.CheckedConsumer;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.NumberFieldMapper;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import java.io.IOException;
import java.util.Arrays;
@ -53,14 +54,20 @@ public class SumAggregatorTests extends AggregatorTestCase {
public void testNoDocs() throws IOException {
testCase(new MatchAllDocsQuery(), iw -> {
// Intentionally not writing any docs
}, count -> assertEquals(0L, count.getValue(), 0d));
}, count -> {
assertEquals(0L, count.getValue(), 0d);
assertFalse(AggregationInspectionHelper.hasValue(count));
});
}
public void testNoMatchingField() throws IOException {
testCase(new MatchAllDocsQuery(), iw -> {
iw.addDocument(singleton(new NumericDocValuesField("wrong_number", 7)));
iw.addDocument(singleton(new NumericDocValuesField("wrong_number", 1)));
}, count -> assertEquals(0L, count.getValue(), 0d));
}, count -> {
assertEquals(0L, count.getValue(), 0d);
assertFalse(AggregationInspectionHelper.hasValue(count));
});
}
public void testNumericDocValues() throws IOException {
@ -81,7 +88,10 @@ public class SumAggregatorTests extends AggregatorTestCase {
iw.addDocument(singleton(new NumericDocValuesField(FIELD_NAME, 2)));
iw.addDocument(singleton(new NumericDocValuesField(FIELD_NAME, 1)));
iw.addDocument(singleton(new NumericDocValuesField(FIELD_NAME, 2)));
}, count -> assertEquals(24L, count.getValue(), 0d));
}, count -> {
assertEquals(24L, count.getValue(), 0d);
assertTrue(AggregationInspectionHelper.hasValue(count));
});
}
public void testSortedNumericDocValues() throws IOException {
@ -91,7 +101,10 @@ public class SumAggregatorTests extends AggregatorTestCase {
iw.addDocument(Arrays.asList(new SortedNumericDocValuesField(FIELD_NAME, 3),
new SortedNumericDocValuesField(FIELD_NAME, 4)));
iw.addDocument(singleton(new SortedNumericDocValuesField(FIELD_NAME, 1)));
}, count -> assertEquals(15L, count.getValue(), 0d));
}, count -> {
assertEquals(15L, count.getValue(), 0d);
assertTrue(AggregationInspectionHelper.hasValue(count));
});
}
public void testQueryFiltering() throws IOException {
@ -101,14 +114,20 @@ public class SumAggregatorTests extends AggregatorTestCase {
iw.addDocument(Arrays.asList(new StringField("match", "yes", Field.Store.NO), new NumericDocValuesField(FIELD_NAME, 3)));
iw.addDocument(Arrays.asList(new StringField("match", "no", Field.Store.NO), new NumericDocValuesField(FIELD_NAME, 4)));
iw.addDocument(Arrays.asList(new StringField("match", "yes", Field.Store.NO), new NumericDocValuesField(FIELD_NAME, 5)));
}, count -> assertEquals(9L, count.getValue(), 0d));
}, count -> {
assertEquals(9L, count.getValue(), 0d);
assertTrue(AggregationInspectionHelper.hasValue(count));
});
}
public void testStringField() throws IOException {
IllegalStateException e = expectThrows(IllegalStateException.class, () -> {
testCase(new MatchAllDocsQuery(), iw -> {
iw.addDocument(singleton(new SortedDocValuesField(FIELD_NAME, new BytesRef("1"))));
}, count -> assertEquals(0L, count.getValue(), 0d));
}, count -> {
assertEquals(0L, count.getValue(), 0d);
assertFalse(AggregationInspectionHelper.hasValue(count));
});
});
assertEquals("unexpected docvalues type SORTED for field 'field' (expected one of [SORTED_NUMERIC, NUMERIC]). " +
"Re-index with correct docvalues type.", e.getMessage());
@ -159,13 +178,13 @@ public class SumAggregatorTests extends AggregatorTestCase {
private void testCase(Query query,
CheckedConsumer<RandomIndexWriter, IOException> indexer,
Consumer<Sum> verify) throws IOException {
Consumer<InternalSum> verify) throws IOException {
testCase(query, indexer, verify, NumberFieldMapper.NumberType.LONG);
}
private void testCase(Query query,
CheckedConsumer<RandomIndexWriter, IOException> indexer,
Consumer<Sum> verify,
Consumer<InternalSum> verify,
NumberFieldMapper.NumberType fieldNumberType) throws IOException {
try (Directory directory = newDirectory()) {
try (RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory)) {
@ -187,7 +206,7 @@ public class SumAggregatorTests extends AggregatorTestCase {
indexSearcher.search(query, aggregator);
aggregator.postCollection();
verify.accept((Sum) aggregator.buildAggregation(0L));
verify.accept((InternalSum) aggregator.buildAggregation(0L));
}
}
}

View File

@ -35,6 +35,7 @@ import org.elasticsearch.search.aggregations.metrics.Percentile;
import org.elasticsearch.search.aggregations.metrics.PercentileRanks;
import org.elasticsearch.search.aggregations.metrics.PercentileRanksAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.PercentilesMethod;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import org.hamcrest.Matchers;
import java.io.IOException;
@ -55,6 +56,7 @@ public class TDigestPercentileRanksAggregatorTests extends AggregatorTestCase {
Percentile rank = ranks.iterator().next();
assertEquals(Double.NaN, rank.getPercent(), 0d);
assertEquals(0.5, rank.getValue(), 0d);
assertFalse(AggregationInspectionHelper.hasValue(((InternalTDigestPercentileRanks)ranks)));
}
}
@ -91,6 +93,7 @@ public class TDigestPercentileRanksAggregatorTests extends AggregatorTestCase {
// https://github.com/elastic/elasticsearch/issues/14851
// assertThat(rank.getPercent(), Matchers.equalTo(100d));
assertFalse(rankIterator.hasNext());
assertTrue(AggregationInspectionHelper.hasValue(((InternalTDigestPercentileRanks)ranks)));
}
}
}

View File

@ -38,6 +38,7 @@ import org.elasticsearch.search.aggregations.metrics.InternalTDigestPercentiles;
import org.elasticsearch.search.aggregations.metrics.PercentilesAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.PercentilesMethod;
import org.elasticsearch.search.aggregations.metrics.TDigestPercentilesAggregator;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import java.io.IOException;
import java.util.function.Consumer;
@ -52,6 +53,7 @@ public class TDigestPercentilesAggregatorTests extends AggregatorTestCase {
// Intentionally not writing any docs
}, tdigest -> {
assertEquals(0L, tdigest.state.size());
assertFalse(AggregationInspectionHelper.hasValue(tdigest));
});
}
@ -61,6 +63,7 @@ public class TDigestPercentilesAggregatorTests extends AggregatorTestCase {
iw.addDocument(singleton(new SortedNumericDocValuesField("wrong_number", 1)));
}, tdigest -> {
assertEquals(0L, tdigest.state.size());
assertFalse(AggregationInspectionHelper.hasValue(tdigest));
});
}
@ -82,6 +85,7 @@ public class TDigestPercentilesAggregatorTests extends AggregatorTestCase {
assertEquals("2.0", tdigest.percentileAsString(50));
assertEquals(1.0d, tdigest.percentile(22), 0.0d);
assertEquals("1.0", tdigest.percentileAsString(22));
assertTrue(AggregationInspectionHelper.hasValue(tdigest));
});
}
@ -107,6 +111,7 @@ public class TDigestPercentilesAggregatorTests extends AggregatorTestCase {
assertEquals("1.0", tdigest.percentileAsString(25));
assertEquals(0.0d, tdigest.percentile(1), 0.0d);
assertEquals("0.0", tdigest.percentileAsString(1));
assertTrue(AggregationInspectionHelper.hasValue(tdigest));
});
}
@ -127,11 +132,13 @@ public class TDigestPercentilesAggregatorTests extends AggregatorTestCase {
assertEquals(2.0d, tdigest.percentile(100), 0.0d);
assertEquals(1.0d, tdigest.percentile(50), 0.0d);
assertEquals(0.5d, tdigest.percentile(25), 0.0d);
assertTrue(AggregationInspectionHelper.hasValue(tdigest));
});
testCase(LongPoint.newRangeQuery("row", 100, 110), docs, tdigest -> {
assertEquals(0L, tdigest.state.size());
assertEquals(0L, tdigest.state.centroidCount());
assertFalse(AggregationInspectionHelper.hasValue(tdigest));
});
}

View File

@ -50,6 +50,7 @@ import org.elasticsearch.search.aggregations.AggregationBuilders;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
import org.elasticsearch.search.aggregations.metrics.TopHits;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import org.elasticsearch.search.sort.SortOrder;
import java.io.IOException;
@ -74,12 +75,14 @@ public class TopHitsAggregatorTests extends AggregatorTestCase {
assertEquals("type", searchHits.getAt(1).getType());
assertEquals("1", searchHits.getAt(2).getId());
assertEquals("type", searchHits.getAt(2).getType());
assertTrue(AggregationInspectionHelper.hasValue(((InternalTopHits)result)));
}
public void testNoResults() throws Exception {
TopHits result = (TopHits) testCase(new MatchNoDocsQuery(), topHits("_name").sort("string", SortOrder.DESC));
SearchHits searchHits = ((TopHits) result).getHits();
assertEquals(0L, searchHits.getTotalHits().value);
assertFalse(AggregationInspectionHelper.hasValue(((InternalTopHits)result)));
}
/**
@ -106,22 +109,26 @@ public class TopHitsAggregatorTests extends AggregatorTestCase {
assertEquals(2L, searchHits.getTotalHits().value);
assertEquals("2", searchHits.getAt(0).getId());
assertEquals("1", searchHits.getAt(1).getId());
assertTrue(AggregationInspectionHelper.hasValue(((InternalTopHits) terms.getBucketByKey("a").getAggregations().get("top"))));
// The "b" bucket
searchHits = ((TopHits) terms.getBucketByKey("b").getAggregations().get("top")).getHits();
assertEquals(2L, searchHits.getTotalHits().value);
assertEquals("3", searchHits.getAt(0).getId());
assertEquals("1", searchHits.getAt(1).getId());
assertTrue(AggregationInspectionHelper.hasValue(((InternalTopHits) terms.getBucketByKey("b").getAggregations().get("top"))));
// The "c" bucket
searchHits = ((TopHits) terms.getBucketByKey("c").getAggregations().get("top")).getHits();
assertEquals(1L, searchHits.getTotalHits().value);
assertEquals("2", searchHits.getAt(0).getId());
assertTrue(AggregationInspectionHelper.hasValue(((InternalTopHits) terms.getBucketByKey("c").getAggregations().get("top"))));
// The "d" bucket
searchHits = ((TopHits) terms.getBucketByKey("d").getAggregations().get("top")).getHits();
assertEquals(1L, searchHits.getTotalHits().value);
assertEquals("3", searchHits.getAt(0).getId());
assertTrue(AggregationInspectionHelper.hasValue(((InternalTopHits) terms.getBucketByKey("d").getAggregations().get("top"))));
}
private static final MappedFieldType STRING_FIELD_TYPE = new KeywordFieldMapper.KeywordFieldType();

View File

@ -35,15 +35,13 @@ import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.CheckedConsumer;
import org.elasticsearch.index.mapper.BooleanFieldMapper;
import org.elasticsearch.index.mapper.DateFieldMapper;
import org.elasticsearch.index.mapper.GeoPointFieldMapper;
import org.elasticsearch.index.mapper.IpFieldMapper;
import org.elasticsearch.index.mapper.KeywordFieldMapper;
import org.elasticsearch.index.mapper.GeoPointFieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.NumberFieldMapper;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.metrics.ValueCount;
import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.ValueCountAggregator;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import org.elasticsearch.search.aggregations.support.ValueType;
import java.io.IOException;
@ -60,7 +58,10 @@ public class ValueCountAggregatorTests extends AggregatorTestCase {
for (ValueType valueType : ValueType.values()) {
testCase(new MatchAllDocsQuery(), valueType, iw -> {
// Intentionally not writing any docs
}, count -> assertEquals(0L, count.getValue()));
}, count -> {
assertEquals(0L, count.getValue());
assertFalse(AggregationInspectionHelper.hasValue(count));
});
}
}
@ -68,7 +69,10 @@ public class ValueCountAggregatorTests extends AggregatorTestCase {
testCase(new MatchAllDocsQuery(), ValueType.LONG, iw -> {
iw.addDocument(singleton(new SortedNumericDocValuesField("wrong_number", 7)));
iw.addDocument(singleton(new SortedNumericDocValuesField("wrong_number", 1)));
}, count -> assertEquals(0L, count.getValue()));
}, count -> {
assertEquals(0L, count.getValue());
assertFalse(AggregationInspectionHelper.hasValue(count));
});
}
public void testSomeMatchesSortedNumericDocValues() throws IOException {
@ -76,14 +80,20 @@ public class ValueCountAggregatorTests extends AggregatorTestCase {
iw.addDocument(singleton(new SortedNumericDocValuesField("wrong_number", 7)));
iw.addDocument(singleton(new SortedNumericDocValuesField(FIELD_NAME, 7)));
iw.addDocument(singleton(new SortedNumericDocValuesField(FIELD_NAME, 1)));
}, count -> assertEquals(2L, count.getValue()));
}, count -> {
assertEquals(2L, count.getValue());
assertTrue(AggregationInspectionHelper.hasValue(count));
});
}
public void testSomeMatchesNumericDocValues() throws IOException {
testCase(new DocValuesFieldExistsQuery(FIELD_NAME), ValueType.NUMBER, iw -> {
iw.addDocument(singleton(new NumericDocValuesField(FIELD_NAME, 7)));
iw.addDocument(singleton(new NumericDocValuesField(FIELD_NAME, 1)));
}, count -> assertEquals(2L, count.getValue()));
}, count -> {
assertEquals(2L, count.getValue());
assertTrue(AggregationInspectionHelper.hasValue(count));
});
}
public void testQueryFiltering() throws IOException {
@ -93,20 +103,26 @@ public class ValueCountAggregatorTests extends AggregatorTestCase {
iw.addDocument(Arrays.asList(new IntPoint("level", 3), new SortedDocValuesField(FIELD_NAME, new BytesRef("foo"))));
iw.addDocument(Arrays.asList(new IntPoint("level", 5), new SortedDocValuesField(FIELD_NAME, new BytesRef("baz"))));
iw.addDocument(Arrays.asList(new IntPoint("level", 7), new SortedDocValuesField(FIELD_NAME, new BytesRef("baz"))));
}, count -> assertEquals(4L, count.getValue()));
}, count -> {
assertEquals(4L, count.getValue());
assertTrue(AggregationInspectionHelper.hasValue(count));
});
}
public void testQueryFiltersAll() throws IOException {
testCase(IntPoint.newRangeQuery("level", -1, 0), ValueType.STRING, iw -> {
iw.addDocument(Arrays.asList(new IntPoint("level", 3), new SortedDocValuesField(FIELD_NAME, new BytesRef("foo"))));
iw.addDocument(Arrays.asList(new IntPoint("level", 5), new SortedDocValuesField(FIELD_NAME, new BytesRef("baz"))));
}, count -> assertEquals(0L, count.getValue()));
}, count -> {
assertEquals(0L, count.getValue());
assertFalse(AggregationInspectionHelper.hasValue(count));
});
}
private void testCase(Query query,
ValueType valueType,
CheckedConsumer<RandomIndexWriter, IOException> indexer,
Consumer<ValueCount> verify) throws IOException {
Consumer<InternalValueCount> verify) throws IOException {
try (Directory directory = newDirectory()) {
try (RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory)) {
@ -127,7 +143,7 @@ public class ValueCountAggregatorTests extends AggregatorTestCase {
aggregator.preCollection();
indexSearcher.search(query, aggregator);
aggregator.postCollection();
verify.accept((ValueCount) aggregator.buildAggregation(0L));
verify.accept((InternalValueCount) aggregator.buildAggregation(0L));
}
}
}

View File

@ -39,6 +39,7 @@ import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.metrics.InternalWeightedAvg;
import org.elasticsearch.search.aggregations.metrics.WeightedAvgAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.WeightedAvgAggregator;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import org.elasticsearch.search.aggregations.support.MultiValuesSourceFieldConfig;
import org.joda.time.DateTimeZone;
@ -63,6 +64,7 @@ public class WeightedAvgAggregatorTests extends AggregatorTestCase {
// Intentionally not writing any docs
}, avg -> {
assertEquals(Double.NaN, avg.getValue(), 0);
assertFalse(AggregationInspectionHelper.hasValue(avg));
});
}
@ -77,6 +79,7 @@ public class WeightedAvgAggregatorTests extends AggregatorTestCase {
iw.addDocument(singleton(new SortedNumericDocValuesField("wrong_number", 3)));
}, avg -> {
assertEquals(Double.NaN, avg.getValue(), 0);
assertFalse(AggregationInspectionHelper.hasValue(avg));
});
}
@ -95,6 +98,7 @@ public class WeightedAvgAggregatorTests extends AggregatorTestCase {
new SortedNumericDocValuesField("weight_field", 1)));
}, avg -> {
assertEquals(4, avg.getValue(), 0);
assertTrue(AggregationInspectionHelper.hasValue(avg));
});
}
@ -115,6 +119,7 @@ public class WeightedAvgAggregatorTests extends AggregatorTestCase {
}, avg -> {
// (7*2 + 2*3 + 3*3) / (2+3+3) == 3.625
assertEquals(3.625, avg.getValue(), 0);
assertTrue(AggregationInspectionHelper.hasValue(avg));
});
}
@ -133,6 +138,7 @@ public class WeightedAvgAggregatorTests extends AggregatorTestCase {
new SortedNumericDocValuesField("weight_field", 1)));
}, avg -> {
assertEquals(4, avg.getValue(), 0);
assertTrue(AggregationInspectionHelper.hasValue(avg));
});
}
@ -151,6 +157,7 @@ public class WeightedAvgAggregatorTests extends AggregatorTestCase {
new SortedNumericDocValuesField("weight_field", 1)));
}, avg -> {
assertEquals(2.5, avg.getValue(), 0);
assertTrue(AggregationInspectionHelper.hasValue(avg));
});
}
@ -170,6 +177,7 @@ public class WeightedAvgAggregatorTests extends AggregatorTestCase {
}, avg -> {
double value = (2.0*3.0 + 3.0*4.0) / (3.0+4.0);
assertEquals(value, avg.getValue(), 0);
assertTrue(AggregationInspectionHelper.hasValue(avg));
});
}
@ -185,6 +193,7 @@ public class WeightedAvgAggregatorTests extends AggregatorTestCase {
iw.addDocument(Arrays.asList(new IntPoint("value_field", 3), new SortedNumericDocValuesField("value_field", 7)));
}, avg -> {
assertEquals(Double.NaN, avg.getValue(), 0);
assertFalse(AggregationInspectionHelper.hasValue(avg));
});
}
@ -203,6 +212,7 @@ public class WeightedAvgAggregatorTests extends AggregatorTestCase {
new SortedNumericDocValuesField("weight_field", 4)));
}, avg -> {
assertEquals(Double.NaN, avg.getValue(), 0);
assertFalse(AggregationInspectionHelper.hasValue(avg));
});
}
@ -222,6 +232,7 @@ public class WeightedAvgAggregatorTests extends AggregatorTestCase {
}, avg -> {
double value = (2.0*2.0 + 2.0*3.0 + 2.0*4.0) / (2.0+3.0+4.0);
assertEquals(value, avg.getValue(), 0);
assertTrue(AggregationInspectionHelper.hasValue(avg));
});
}
@ -241,6 +252,7 @@ public class WeightedAvgAggregatorTests extends AggregatorTestCase {
}, avg -> {
double value = (2.0*2.0 + 3.0*2.0 + 4.0*2.0) / (2.0+2.0+2.0);
assertEquals(value, avg.getValue(), 0);
assertTrue(AggregationInspectionHelper.hasValue(avg));
});
}
@ -311,6 +323,7 @@ public class WeightedAvgAggregatorTests extends AggregatorTestCase {
}, avg -> {
double value = (((2.0+3.0)/2.0) + ((3.0+4.0)/2.0) + ((4.0+5.0)/2.0)) / (1.0+1.0+1.0);
assertEquals(value, avg.getValue(), 0);
assertTrue(AggregationInspectionHelper.hasValue(avg));
});
}

View File

@ -47,6 +47,7 @@ import org.elasticsearch.search.aggregations.metrics.AvgAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.InternalAvg;
import org.elasticsearch.search.aggregations.metrics.Sum;
import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder;
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
import java.io.IOException;
import java.util.Arrays;
@ -93,6 +94,7 @@ public class CumulativeSumAggregatorTests extends AggregatorTestCase {
for (Histogram.Bucket bucket : buckets) {
sum += ((InternalAvg) (bucket.getAggregations().get("the_avg"))).value();
assertThat(((InternalSimpleValue) (bucket.getAggregations().get("cusum"))).value(), equalTo(sum));
assertTrue(AggregationInspectionHelper.hasValue(((InternalAvg) (bucket.getAggregations().get("the_avg")))));
}
});
}
@ -116,14 +118,37 @@ public class CumulativeSumAggregatorTests extends AggregatorTestCase {
for (int i = 0; i < buckets.size(); i++) {
if (i == 0) {
assertThat(((InternalSimpleValue)(buckets.get(i).getAggregations().get("cusum"))).value(), equalTo(0.0));
assertTrue(AggregationInspectionHelper.hasValue(((InternalSimpleValue) (buckets.get(i)
.getAggregations().get("cusum")))));
} else {
sum += 1.0;
assertThat(((InternalSimpleValue)(buckets.get(i).getAggregations().get("cusum"))).value(), equalTo(sum));
assertTrue(AggregationInspectionHelper.hasValue(((InternalSimpleValue) (buckets.get(i)
.getAggregations().get("cusum")))));
}
}
});
}
public void testCount() throws IOException {
Query query = new MatchAllDocsQuery();
DateHistogramAggregationBuilder aggBuilder = new DateHistogramAggregationBuilder("histo");
aggBuilder.dateHistogramInterval(DateHistogramInterval.DAY).field(HISTO_FIELD);
aggBuilder.subAggregation(new CumulativeSumPipelineAggregationBuilder("cusum", "_count"));
executeTestCase(query, aggBuilder, histogram -> {
assertEquals(10, ((Histogram)histogram).getBuckets().size());
List<? extends Histogram.Bucket> buckets = ((Histogram)histogram).getBuckets();
double sum = 1.0;
for (Histogram.Bucket bucket : buckets) {
assertThat(((InternalSimpleValue) (bucket.getAggregations().get("cusum"))).value(), equalTo(sum));
assertTrue(AggregationInspectionHelper.hasValue(((InternalSimpleValue) (bucket.getAggregations().get("cusum")))));
sum += 1.0;
}
});
}
public void testDocCount() throws IOException {
Query query = new MatchAllDocsQuery();