median absolute deviation agg (#34482)

This commit adds a new single value metric aggregation that calculates
the statistic called median absolute deviation, which is a measure of
variability that works on more types of data than standard deviation

Our calculation of MAD is approximated using t-digests. In the collect
phase, we collect each value visited into a t-digest. In the reduce
phase, we merge all value t-digests, then create a t-digest of
deviations using the first t-digest's median and centroids
This commit is contained in:
Andy Bristol 2018-10-30 07:22:52 -07:00 committed by GitHub
parent 7bd113d284
commit b8280ea7cc
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
19 changed files with 2016 additions and 0 deletions

View File

@ -156,6 +156,8 @@ import org.elasticsearch.search.aggregations.metrics.StatsAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.TopHitsAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.MedianAbsoluteDeviationAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.ParsedMedianAbsoluteDeviation;
import org.elasticsearch.search.aggregations.pipeline.InternalSimpleValue;
import org.elasticsearch.search.aggregations.pipeline.ParsedSimpleValue;
import org.elasticsearch.search.aggregations.pipeline.InternalBucketMetricValue;
@ -1537,6 +1539,7 @@ public class RestHighLevelClient implements Closeable {
map.put(InternalTDigestPercentiles.NAME, (p, c) -> ParsedTDigestPercentiles.fromXContent(p, (String) c));
map.put(InternalTDigestPercentileRanks.NAME, (p, c) -> ParsedTDigestPercentileRanks.fromXContent(p, (String) c));
map.put(PercentilesBucketPipelineAggregationBuilder.NAME, (p, c) -> ParsedPercentilesBucket.fromXContent(p, (String) c));
map.put(MedianAbsoluteDeviationAggregationBuilder.NAME, (p, c) -> ParsedMedianAbsoluteDeviation.fromXContent(p, (String) c));
map.put(MinAggregationBuilder.NAME, (p, c) -> ParsedMin.fromXContent(p, (String) c));
map.put(MaxAggregationBuilder.NAME, (p, c) -> ParsedMax.fromXContent(p, (String) c));
map.put(SumAggregationBuilder.NAME, (p, c) -> ParsedSum.fromXContent(p, (String) c));

View File

@ -1082,6 +1082,34 @@ buildRestTests.setups['calendar_outages_addevent'] = buildRestTests.setups['cale
]}
'''
// used by median absolute deviation aggregation
buildRestTests.setups['reviews'] = '''
- do:
indices.create:
index: reviews
body:
settings:
number_of_shards: 1
number_of_replicas: 0
mappings:
_doc:
properties:
product:
type: keyword
rating:
type: long
- do:
bulk:
index: reviews
type: _doc
refresh: true
body: |
{"index": {"_id": "1"}}
{"product": "widget-foo", "rating": 1}
{"index": {"_id": "2"}}
{"product": "widget-foo", "rating": 5}
'''
buildRestTests.setups['remote_cluster'] = buildRestTests.setups['host'] + '''
- do:
cluster.put_settings:

View File

@ -41,6 +41,8 @@ include::metrics/tophits-aggregation.asciidoc[]
include::metrics/valuecount-aggregation.asciidoc[]
include::metrics/median-absolute-deviation-aggregation.asciidoc[]

View File

@ -0,0 +1,189 @@
[[search-aggregations-metrics-median-absolute-deviation-aggregation]]
=== Median Absolute Deviation Aggregation
This `single-value` aggregation approximates the https://en.wikipedia.org/wiki/Median_absolute_deviation[median absolute deviation]
of its search results.
Median absolute deviation is a measure of variability. It is a robust
statistic, meaning that it is useful for describing data that may have
outliers, or may not be normally distributed. For such data it can be more
descriptive than standard deviation.
It is calculated as the median of each data point's deviation from the median
of the entire sample. That is, for a random variable X, the median absolute
deviation is median(|median(X) - X~i~|).
==== Example
Assume our data represents product reviews on a one to five star scale.
Such reviews are usually summarized as a mean, which is easily understandable
but doesn't describe the reviews' variability. Estimating the median absolute
deviation can provide insight into how much reviews vary from one another.
In this example we have a product which has an average rating of
3 stars. Let's look at its ratings' median absolute deviation to determine
how much they vary
[source,js]
---------------------------------------------------------
GET reviews/_search
{
"size": 0,
"aggs": {
"review_average": {
"avg": {
"field": "rating"
}
},
"review_variability": {
"median_absolute_deviation": {
"field": "rating" <1>
}
}
}
}
---------------------------------------------------------
// CONSOLE
// TEST[setup:reviews]
<1> `rating` must be a numeric field
The resulting median absolute deviation of `2` tells us that there is a fair
amount of variability in the ratings. Reviewers must have diverse opinions about
this product.
[source,js]
---------------------------------------------------------
{
...
"aggregations": {
"review_average": {
"value": 3.0
},
"review_variability": {
"value": 2.0
}
}
}
---------------------------------------------------------
// TESTRESPONSE[s/\.\.\./"took": $body.took,"timed_out": false,"_shards": $body._shards,"hits": $body.hits,/]
==== Approximation
The naive implementation of calculating median absolute deviation stores the
entire sample in memory, so this aggregation instead calculates an
approximation. It uses the https://github.com/tdunning/t-digest[TDigest data structure]
to approximate the sample median and the median of deviations from the sample
median. For more about the approximation characteristics of TDigests, see
<<search-aggregations-metrics-percentile-aggregation-approximation>>.
The tradeoff between resource usage and accuracy of a TDigest's quantile
approximation, and therefore the accuracy of this aggregation's approximation
of median absolute deviation, is controlled by the `compression` parameter. A
higher `compression` setting provides a more accurate approximation at the
cost of higher memory usage. For more about the characteristics of the TDigest
`compression` parameter see
<<search-aggregations-metrics-percentile-aggregation-compression>>.
[source,js]
---------------------------------------------------------
GET reviews/_search
{
"size": 0,
"aggs": {
"review_variability": {
"median_absolute_deviation": {
"field": "rating",
"compression": 100
}
}
}
}
---------------------------------------------------------
// CONSOLE
// TEST[setup:reviews]
The default `compression` value for this aggregation is `1000`. At this
compression level this aggregation is usually within 5% of the exact result,
but observed performance will depend on the sample data.
==== Script
This metric aggregation supports scripting. In our example above, product
reviews are on a scale of one to five. If we wanted to modify them to a scale
of one to ten, we can using scripting.
To provide an inline script:
[source,js]
---------------------------------------------------------
GET reviews/_search
{
"size": 0,
"aggs": {
"review_variability": {
"median_absolute_deviation": {
"script": {
"lang": "painless",
"source": "doc['rating'].value * params.scaleFactor",
"params": {
"scaleFactor": 2
}
}
}
}
}
}
---------------------------------------------------------
// CONSOLE
// TEST[setup:reviews]
To provide a stored script:
[source,js]
---------------------------------------------------------
GET reviews/_search
{
"size": 0,
"aggs": {
"review_variability": {
"median_absolute_deviation": {
"script": {
"id": "my_script",
"params": {
"field": "rating"
}
}
}
}
}
}
---------------------------------------------------------
// CONSOLE
// TEST[setup:reviews,stored_example_script]
==== Missing value
The `missing` parameter defines how documents that are missing a value should be
treated. By default they will be ignored but it is also possible to treat them
as if they had a value.
Let's be optimistic and assume some reviewers loved the product so much that
they forgot to give it a rating. We'll assign them five stars
[source,js]
---------------------------------------------------------
GET reviews/_search
{
"size": 0,
"aggs": {
"review_variability": {
"median_absolute_deviation": {
"field": "rating",
"missing": 5
}
}
}
}
---------------------------------------------------------
// CONSOLE
// TEST[setup:reviews]

View File

@ -0,0 +1,143 @@
setup:
- skip:
version: " - 6.6.0"
reason: "added in 6.6.0"
- do:
indices.create:
index: test
body:
settings:
number_of_replicas: 0
mappings:
_doc:
properties:
int_field:
type: integer
double_field:
type: double
incomplete_field:
type: integer
- do:
bulk:
refresh: true
body:
- index:
_index: test
_type: _doc
- int_field: 100
double_field: 100.0
incomplete_field: 1000
- index:
_index: test
_type: _doc
- int_field: 200
double_field: 200.0
incomplete_field: 2000
- index:
_index: test
_type: _doc
- int_field: 300
double_field: 300.0
---
"basic test":
- do:
search:
body:
aggs:
mad_int:
median_absolute_deviation:
field: int_field
mad_double:
median_absolute_deviation:
field: double_field
- match: { hits.total: 3 }
- length: { hits.hits: 3 }
- match: { aggregations.mad_int.value: 100 }
- match: { aggregations.mad_double.value: 100 }
---
"with setting compression":
- do:
search:
body:
aggs:
mad_int:
median_absolute_deviation:
field: int_field
compression: 500
mad_double:
median_absolute_deviation:
field: double_field
compression: 500
- match: { hits.total: 3 }
- length: { hits.hits: 3 }
- match: { aggregations.mad_int.value: 100 }
- match: { aggregations.mad_double.value: 100 }
---
"no documents":
- do:
search:
body:
query:
bool:
filter:
term:
non_existent_field: non_existent_value
aggs:
mad_no_docs:
median_absolute_deviation:
field: non_existent_field
- match: { hits.total: 0 }
- length: { hits.hits: 0 }
- match: { aggregations.mad_no_docs.value: null }
---
"missing value":
- do:
search:
body:
aggs:
mad_missing:
median_absolute_deviation:
field: incomplete_field
missing: 3000
- match: { hits.total: 3 }
- length: { hits.hits: 3 }
- match: { aggregations.mad_missing.value: 1000 }
---
"bad arguments":
- do:
catch: /\[compression\] must be greater than 0. Found \[0.0\] in \[mad\]/
search:
body:
aggs:
mad:
median_absolute_deviation:
field: int_field
compression: 0
- do:
catch: /\[compression\] must be greater than 0. Found \[-1.0\] in \[mad\]/
search:
body:
aggs:
mad:
median_absolute_deviation:
field: int_field
compression: -1

View File

@ -183,6 +183,8 @@ import org.elasticsearch.search.aggregations.metrics.InternalValueCount;
import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.InternalWeightedAvg;
import org.elasticsearch.search.aggregations.metrics.WeightedAvgAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.InternalMedianAbsoluteDeviation;
import org.elasticsearch.search.aggregations.metrics.MedianAbsoluteDeviationAggregationBuilder;
import org.elasticsearch.search.aggregations.pipeline.InternalSimpleValue;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.InternalBucketMetricValue;
@ -364,6 +366,9 @@ public class SearchModule {
PercentileRanksAggregationBuilder::parse)
.addResultReader(InternalTDigestPercentileRanks.NAME, InternalTDigestPercentileRanks::new)
.addResultReader(InternalHDRPercentileRanks.NAME, InternalHDRPercentileRanks::new));
registerAggregation(new AggregationSpec(MedianAbsoluteDeviationAggregationBuilder.NAME,
MedianAbsoluteDeviationAggregationBuilder::new, MedianAbsoluteDeviationAggregationBuilder::parse)
.addResultReader(InternalMedianAbsoluteDeviation::new));
registerAggregation(new AggregationSpec(CardinalityAggregationBuilder.NAME, CardinalityAggregationBuilder::new,
CardinalityAggregationBuilder::parse).addResultReader(InternalCardinality::new));
registerAggregation(new AggregationSpec(GlobalAggregationBuilder.NAME, GlobalAggregationBuilder::new,

View File

@ -83,6 +83,8 @@ import org.elasticsearch.search.aggregations.metrics.TopHitsAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.ValueCount;
import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.WeightedAvgAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.MedianAbsoluteDeviationAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.MedianAbsoluteDeviation;
import java.util.Map;
@ -316,6 +318,13 @@ public class AggregationBuilders {
return new PercentileRanksAggregationBuilder(name, values);
}
/**
* Create a new {@link MedianAbsoluteDeviation} aggregation with the given name
*/
public static MedianAbsoluteDeviationAggregationBuilder medianAbsoluteDeviation(String name) {
return new MedianAbsoluteDeviationAggregationBuilder(name);
}
/**
* Create a new {@link Cardinality} aggregation with the given name.
*/

View File

@ -0,0 +1,137 @@
/*
* 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.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Objects;
public class InternalMedianAbsoluteDeviation extends InternalNumericMetricsAggregation.SingleValue implements MedianAbsoluteDeviation {
static double computeMedianAbsoluteDeviation(TDigestState valuesSketch) {
if (valuesSketch.size() == 0) {
return Double.NaN;
} else {
final double approximateMedian = valuesSketch.quantile(0.5);
final TDigestState approximatedDeviationsSketch = new TDigestState(valuesSketch.compression());
valuesSketch.centroids().forEach(centroid -> {
final double deviation = Math.abs(approximateMedian - centroid.mean());
approximatedDeviationsSketch.add(deviation, centroid.count());
});
return approximatedDeviationsSketch.quantile(0.5);
}
}
private final TDigestState valuesSketch;
private final double medianAbsoluteDeviation;
InternalMedianAbsoluteDeviation(String name,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData,
DocValueFormat format,
TDigestState valuesSketch) {
super(name, pipelineAggregators, metaData);
this.format = Objects.requireNonNull(format);
this.valuesSketch = Objects.requireNonNull(valuesSketch);
this.medianAbsoluteDeviation = computeMedianAbsoluteDeviation(this.valuesSketch);
}
public InternalMedianAbsoluteDeviation(StreamInput in) throws IOException {
super(in);
format = in.readNamedWriteable(DocValueFormat.class);
valuesSketch = TDigestState.read(in);
medianAbsoluteDeviation = in.readDouble();
}
@Override
protected void doWriteTo(StreamOutput out) throws IOException {
out.writeNamedWriteable(format);
TDigestState.write(valuesSketch, out);
out.writeDouble(medianAbsoluteDeviation);
}
@Override
public InternalAggregation doReduce(List<InternalAggregation> aggregations, ReduceContext reduceContext) {
final TDigestState valueMerged = new TDigestState(valuesSketch.compression());
for (InternalAggregation aggregation : aggregations) {
final InternalMedianAbsoluteDeviation madAggregation = (InternalMedianAbsoluteDeviation) aggregation;
valueMerged.add(madAggregation.valuesSketch);
}
return new InternalMedianAbsoluteDeviation(name, pipelineAggregators(), metaData, format, valueMerged);
}
@Override
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
final boolean anyResults = valuesSketch.size() > 0;
final Double mad = anyResults
? getMedianAbsoluteDeviation()
: null;
builder.field(CommonFields.VALUE.getPreferredName(), mad);
if (format != DocValueFormat.RAW && anyResults) {
builder.field(CommonFields.VALUE_AS_STRING.getPreferredName(), format.format(mad).toString());
}
return builder;
}
@Override
protected int doHashCode() {
return Objects.hash(valuesSketch);
}
@Override
protected boolean doEquals(Object obj) {
InternalMedianAbsoluteDeviation other = (InternalMedianAbsoluteDeviation) obj;
return Objects.equals(valuesSketch, other.valuesSketch);
}
@Override
public String getWriteableName() {
return MedianAbsoluteDeviationAggregationBuilder.NAME;
}
public TDigestState getValuesSketch() {
return valuesSketch;
}
@Override
public double value() {
return getMedianAbsoluteDeviation();
}
@Override
public double getMedianAbsoluteDeviation() {
return medianAbsoluteDeviation;
}
}

View File

@ -0,0 +1,33 @@
/*
* 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;
/**
* An aggregation that approximates the median absolute deviation of a numeric field
*
* @see <a href="https://en.wikipedia.org/wiki/Median_absolute_deviation">https://en.wikipedia.org/wiki/Median_absolute_deviation</a>
*/
public interface MedianAbsoluteDeviation extends NumericMetricsAggregation.SingleValue {
/**
* Returns the median absolute deviation statistic computed for this aggregation
*/
double getMedianAbsoluteDeviation();
}

View File

@ -0,0 +1,140 @@
/*
* 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.common.ParseField;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder.LeafOnly;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Map;
import java.util.Objects;
public class MedianAbsoluteDeviationAggregationBuilder extends LeafOnly<ValuesSource.Numeric, MedianAbsoluteDeviationAggregationBuilder> {
public static final String NAME = "median_absolute_deviation";
private static final ParseField COMPRESSION_FIELD = new ParseField("compression");
private static final ObjectParser<MedianAbsoluteDeviationAggregationBuilder, Void> PARSER;
static {
PARSER = new ObjectParser<>(NAME);
ValuesSourceParserHelper.declareNumericFields(PARSER, true, true, false);
PARSER.declareDouble(MedianAbsoluteDeviationAggregationBuilder::compression, COMPRESSION_FIELD);
}
public static MedianAbsoluteDeviationAggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
return PARSER.parse(parser, new MedianAbsoluteDeviationAggregationBuilder(aggregationName), null);
}
private double compression = 1000d;
public MedianAbsoluteDeviationAggregationBuilder(String name) {
super(name, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
}
public MedianAbsoluteDeviationAggregationBuilder(StreamInput in) throws IOException {
super(in, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
compression = in.readDouble();
}
protected MedianAbsoluteDeviationAggregationBuilder(MedianAbsoluteDeviationAggregationBuilder clone,
AggregatorFactories.Builder factoriesBuilder,
Map<String, Object> metaData) {
super(clone, factoriesBuilder, metaData);
this.compression = clone.compression;
}
/**
* Returns the compression factor of the t-digest sketches used
*/
public double compression() {
return compression;
}
/**
* Set the compression factor of the t-digest sketches used
*/
public MedianAbsoluteDeviationAggregationBuilder compression(double compression) {
if (compression <= 0d) {
throw new IllegalArgumentException(
"[" + COMPRESSION_FIELD.getPreferredName() + "] must be greater than 0. Found [" + compression + "] in [" + name + "]");
}
this.compression = compression;
return this;
}
@Override
protected AggregationBuilder shallowCopy(AggregatorFactories.Builder factoriesBuilder, Map<String, Object> metaData) {
return new MedianAbsoluteDeviationAggregationBuilder(this, factoriesBuilder, metaData);
}
@Override
protected void innerWriteTo(StreamOutput out) throws IOException {
out.writeDouble(compression);
}
@Override
protected ValuesSourceAggregatorFactory<ValuesSource.Numeric, ?> innerBuild(SearchContext context,
ValuesSourceConfig<ValuesSource.Numeric> config,
AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactoriesBuilder)
throws IOException {
return new MedianAbsoluteDeviationAggregatorFactory(name, config, context, parent, subFactoriesBuilder, metaData, compression);
}
@Override
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
builder.field(COMPRESSION_FIELD.getPreferredName(), compression);
return builder;
}
@Override
protected int innerHashCode() {
return Objects.hash(compression);
}
@Override
protected boolean innerEquals(Object obj) {
MedianAbsoluteDeviationAggregationBuilder other = (MedianAbsoluteDeviationAggregationBuilder) obj;
return Objects.equals(compression, other.compression);
}
@Override
public String getType() {
return NAME;
}
}

View File

@ -0,0 +1,145 @@
/*
* 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.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.ScoreMode;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.ObjectArray;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import static org.elasticsearch.search.aggregations.metrics.InternalMedianAbsoluteDeviation.computeMedianAbsoluteDeviation;
public class MedianAbsoluteDeviationAggregator extends NumericMetricsAggregator.SingleValue {
private final ValuesSource.Numeric valuesSource;
private final DocValueFormat format;
private final double compression;
private ObjectArray<TDigestState> valueSketches;
MedianAbsoluteDeviationAggregator(String name,
SearchContext context,
Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData,
@Nullable ValuesSource.Numeric valuesSource,
DocValueFormat format,
double compression) throws IOException {
super(name, context, parent, pipelineAggregators, metaData);
this.valuesSource = valuesSource;
this.format = Objects.requireNonNull(format);
this.compression = compression;
this.valueSketches = context.bigArrays().newObjectArray(1);
}
private boolean hasDataForBucket(long bucketOrd) {
return bucketOrd < valueSketches.size() && valueSketches.get(bucketOrd) != null;
}
@Override
public double metric(long owningBucketOrd) {
if (hasDataForBucket(owningBucketOrd)) {
return computeMedianAbsoluteDeviation(valueSketches.get(owningBucketOrd));
} else {
return Double.NaN;
}
}
@Override
public ScoreMode scoreMode() {
if (valuesSource != null && valuesSource.needsScores()) {
return ScoreMode.COMPLETE;
} else {
return ScoreMode.COMPLETE_NO_SCORES;
}
}
@Override
protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
if (valuesSource == null) {
return LeafBucketCollector.NO_OP_COLLECTOR;
}
final BigArrays bigArrays = context.bigArrays();
final SortedNumericDoubleValues values = valuesSource.doubleValues(ctx);
return new LeafBucketCollectorBase(sub, values) {
@Override
public void collect(int doc, long bucket) throws IOException {
valueSketches = bigArrays.grow(valueSketches, bucket + 1);
TDigestState valueSketch = valueSketches.get(bucket);
if (valueSketch == null) {
valueSketch = new TDigestState(compression);
valueSketches.set(bucket, valueSketch);
}
if (values.advanceExact(doc)) {
final int valueCount = values.docValueCount();
for (int i = 0; i < valueCount; i++) {
final double value = values.nextValue();
valueSketch.add(value);
}
}
}
};
}
@Override
public InternalAggregation buildAggregation(long bucket) throws IOException {
if (hasDataForBucket(bucket)) {
final TDigestState valueSketch = valueSketches.get(bucket);
return new InternalMedianAbsoluteDeviation(name, pipelineAggregators(), metaData(), format, valueSketch);
} else {
return buildEmptyAggregation();
}
}
@Override
public InternalAggregation buildEmptyAggregation() {
return new InternalMedianAbsoluteDeviation(name, pipelineAggregators(), metaData(), format, new TDigestState(compression));
}
@Override
public void doClose() {
Releasables.close(valueSketches);
}
}

View File

@ -0,0 +1,87 @@
/*
* 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.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class MedianAbsoluteDeviationAggregatorFactory extends
ValuesSourceAggregatorFactory<ValuesSource.Numeric, MedianAbsoluteDeviationAggregatorFactory> {
private final double compression;
MedianAbsoluteDeviationAggregatorFactory(String name,
ValuesSourceConfig<ValuesSource.Numeric> config,
SearchContext context,
AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData,
double compression) throws IOException {
super(name, config, context, parent, subFactoriesBuilder, metaData);
this.compression = compression;
}
@Override
protected Aggregator createUnmapped(Aggregator parent,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new MedianAbsoluteDeviationAggregator(
name,
context,
parent,
pipelineAggregators,
metaData,
null,
config.format(),
compression
);
}
@Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource,
Aggregator parent,
boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException {
return new MedianAbsoluteDeviationAggregator(
name,
context,
parent,
pipelineAggregators,
metaData,
valuesSource,
config.format(),
compression
);
}
}

View File

@ -0,0 +1,65 @@
/*
* 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.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException;
public class ParsedMedianAbsoluteDeviation extends ParsedSingleValueNumericMetricsAggregation implements MedianAbsoluteDeviation {
private static final ObjectParser<ParsedMedianAbsoluteDeviation, Void> PARSER = new ObjectParser<>(
ParsedMedianAbsoluteDeviation.class.getSimpleName(),
true,
ParsedMedianAbsoluteDeviation::new
);
static {
declareSingleValueFields(PARSER, Double.NaN);
}
public static ParsedMedianAbsoluteDeviation fromXContent(XContentParser parser, String name) {
ParsedMedianAbsoluteDeviation parsedMedianAbsoluteDeviation = PARSER.apply(parser, null);
parsedMedianAbsoluteDeviation.setName(name);
return parsedMedianAbsoluteDeviation;
}
@Override
protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
final boolean hasValue = Double.isFinite(getMedianAbsoluteDeviation());
builder.field(CommonFields.VALUE.getPreferredName(), hasValue ? getMedianAbsoluteDeviation() : null);
if (hasValue && valueAsString != null) {
builder.field(CommonFields.VALUE_AS_STRING.getPreferredName(), valueAsString);
}
return builder;
}
@Override
public double getMedianAbsoluteDeviation() {
return value();
}
@Override
public String getType() {
return MedianAbsoluteDeviationAggregationBuilder.NAME;
}
}

View File

@ -55,6 +55,7 @@ import org.elasticsearch.search.aggregations.bucket.terms.LongTermsTests;
import org.elasticsearch.search.aggregations.bucket.terms.StringTermsTests;
import org.elasticsearch.search.aggregations.metrics.InternalExtendedStatsTests;
import org.elasticsearch.search.aggregations.metrics.InternalMaxTests;
import org.elasticsearch.search.aggregations.metrics.InternalMedianAbsoluteDeviationTests;
import org.elasticsearch.search.aggregations.metrics.InternalMinTests;
import org.elasticsearch.search.aggregations.metrics.InternalStatsBucketTests;
import org.elasticsearch.search.aggregations.metrics.InternalStatsTests;
@ -148,6 +149,7 @@ public class AggregationsTests extends ESTestCase {
aggsTests.add(new InternalBinaryRangeTests());
aggsTests.add(new InternalTopHitsTests());
aggsTests.add(new InternalCompositeTests());
aggsTests.add(new InternalMedianAbsoluteDeviationTests());
return Collections.unmodifiableList(aggsTests);
}

View File

@ -0,0 +1,110 @@
/*
* 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.common.io.stream.Writeable;
import org.elasticsearch.search.aggregations.ParsedAggregation;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.test.InternalAggregationTestCase;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
public class InternalMedianAbsoluteDeviationTests extends InternalAggregationTestCase<InternalMedianAbsoluteDeviation> {
@Override
protected InternalMedianAbsoluteDeviation createTestInstance(String name,
List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) {
final TDigestState valuesSketch = new TDigestState(randomDoubleBetween(20, 1000, true));
final int numberOfValues = frequently()
? randomIntBetween(0, 1000)
: 0;
for (int i = 0; i < numberOfValues; i++) {
valuesSketch.add(randomDouble());
}
return new InternalMedianAbsoluteDeviation(name, pipelineAggregators, metaData, randomNumericDocValueFormat(), valuesSketch);
}
@Override
protected void assertReduced(InternalMedianAbsoluteDeviation reduced, List<InternalMedianAbsoluteDeviation> inputs) {
final TDigestState expectedValuesSketch = new TDigestState(reduced.getValuesSketch().compression());
long totalCount = 0;
for (InternalMedianAbsoluteDeviation input : inputs) {
expectedValuesSketch.add(input.getValuesSketch());
totalCount += input.getValuesSketch().size();
}
assertEquals(totalCount, reduced.getValuesSketch().size());
if (totalCount > 0) {
assertEquals(expectedValuesSketch.quantile(0), reduced.getValuesSketch().quantile(0), 0d);
assertEquals(expectedValuesSketch.quantile(1), reduced.getValuesSketch().quantile(1), 0d);
}
}
@Override
protected void assertFromXContent(InternalMedianAbsoluteDeviation internalMAD, ParsedAggregation parsedAggregation) throws IOException {
assertTrue(parsedAggregation instanceof ParsedMedianAbsoluteDeviation);
ParsedMedianAbsoluteDeviation parsedMAD = (ParsedMedianAbsoluteDeviation) parsedAggregation;
// Double.compare handles NaN, which we use for no result
assertEquals(internalMAD.getMedianAbsoluteDeviation(), parsedMAD.getMedianAbsoluteDeviation(), 0);
}
@Override
protected Writeable.Reader<InternalMedianAbsoluteDeviation> instanceReader() {
return InternalMedianAbsoluteDeviation::new;
}
@Override
protected InternalMedianAbsoluteDeviation mutateInstance(InternalMedianAbsoluteDeviation instance) throws IOException {
String name = instance.getName();
TDigestState valuesSketch = instance.getValuesSketch();
Map<String, Object> metaData = instance.getMetaData();
switch (between(0, 2)) {
case 0:
name += randomAlphaOfLengthBetween(2, 10);
break;
case 1:
final TDigestState newValuesSketch = new TDigestState(instance.getValuesSketch().compression());
final int numberOfValues = between(10, 100);
for (int i = 0; i < numberOfValues; i++) {
newValuesSketch.add(randomDouble());
}
valuesSketch = newValuesSketch;
break;
case 2:
if (metaData == null) {
metaData = new HashMap<>(1);
} else {
metaData = new HashMap<>(metaData);
}
metaData.put(randomAlphaOfLengthBetween(2, 10), randomInt());
break;
}
return new InternalMedianAbsoluteDeviation(name, instance.pipelineAggregators(), metaData, instance.format, valuesSketch);
}
}

View File

@ -0,0 +1,251 @@
/*
* 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.apache.lucene.document.IntPoint;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.search.DocValuesFieldExistsQuery;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.store.Directory;
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.hamcrest.Description;
import org.hamcrest.TypeSafeMatcher;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.stream.IntStream;
import static java.util.Collections.singleton;
import static org.elasticsearch.search.aggregations.metrics.MedianAbsoluteDeviationAggregatorTests.ExactMedianAbsoluteDeviation.calculateMAD;
import static org.elasticsearch.search.aggregations.metrics.MedianAbsoluteDeviationAggregatorTests.IsCloseToRelative.closeToRelative;
import static org.hamcrest.Matchers.equalTo;
public class MedianAbsoluteDeviationAggregatorTests extends AggregatorTestCase {
private static final int SAMPLE_MIN = -1000000;
private static final int SAMPLE_MAX = 1000000;
private static <T extends IndexableField> CheckedConsumer<RandomIndexWriter, IOException> randomSample(
int size,
Function<Long, Iterable<T>> field) {
return writer -> {
for (int i = 0; i < size; i++) {
final long point = randomLongBetween(SAMPLE_MIN, SAMPLE_MAX);
Iterable<T> document = field.apply(point);
writer.addDocument(document);
}
};
}
// intentionally not writing any docs
public void testNoDocs() throws IOException {
testCase(new MatchAllDocsQuery(), writer -> {}, agg -> assertThat(agg.getMedianAbsoluteDeviation(), equalTo(Double.NaN)));
}
public void testNoMatchingField() throws IOException {
testCase(
new MatchAllDocsQuery(),
writer -> {
writer.addDocument(singleton(new SortedNumericDocValuesField("wrong_number", 1)));
writer.addDocument(singleton(new SortedNumericDocValuesField("wrong_number", 2)));
},
agg -> assertThat(agg.getMedianAbsoluteDeviation(), equalTo(Double.NaN))
);
}
public void testSomeMatchesSortedNumericDocValues() throws IOException {
final int size = randomIntBetween(100, 1000);
final List<Long> sample = new ArrayList<>(size);
testCase(
new DocValuesFieldExistsQuery("number"),
randomSample(size, point -> {
sample.add(point);
return singleton(new SortedNumericDocValuesField("number", point));
}),
agg -> assertThat(agg.getMedianAbsoluteDeviation(), closeToRelative(calculateMAD(sample)))
);
}
public void testSomeMatchesNumericDocValues() throws IOException {
final int size = randomIntBetween(100, 1000);
final List<Long> sample = new ArrayList<>(size);
testCase(
new DocValuesFieldExistsQuery("number"),
randomSample(size, point -> {
sample.add(point);
return singleton(new NumericDocValuesField("number", point));
}),
agg -> assertThat(agg.getMedianAbsoluteDeviation(), closeToRelative(calculateMAD(sample)))
);
}
public void testQueryFiltering() throws IOException {
final int lowerRange = 1;
final int upperRange = 500;
final int[] sample = IntStream.rangeClosed(1, 1000).toArray();
final int[] filteredSample = Arrays.stream(sample).filter(point -> point >= lowerRange && point <= upperRange).toArray();
testCase(
IntPoint.newRangeQuery("number", lowerRange, upperRange),
writer -> {
for (int point : sample) {
writer.addDocument(Arrays.asList(new IntPoint("number", point), new SortedNumericDocValuesField("number", point)));
}
},
agg -> assertThat(agg.getMedianAbsoluteDeviation(), closeToRelative(calculateMAD(filteredSample)))
);
}
public void testQueryFiltersAll() throws IOException {
testCase(
IntPoint.newRangeQuery("number", -1, 0),
writer -> {
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))
);
}
private void testCase(Query query,
CheckedConsumer<RandomIndexWriter, IOException> buildIndex,
Consumer<InternalMedianAbsoluteDeviation> verify) throws IOException {
try (Directory directory = newDirectory()) {
try (RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory)) {
buildIndex.accept(indexWriter);
}
try (IndexReader indexReader = DirectoryReader.open(directory)) {
IndexSearcher indexSearcher = newSearcher(indexReader, true, true);
MedianAbsoluteDeviationAggregationBuilder builder = new MedianAbsoluteDeviationAggregationBuilder("mad")
.field("number")
.compression(randomDoubleBetween(20, 1000, true));
MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG);
fieldType.setName("number");
MedianAbsoluteDeviationAggregator aggregator = createAggregator(builder, indexSearcher, fieldType);
aggregator.preCollection();
indexSearcher.search(query, aggregator);
aggregator.postCollection();
verify.accept((InternalMedianAbsoluteDeviation) aggregator.buildAggregation(0L));
}
}
}
public static class IsCloseToRelative extends TypeSafeMatcher<Double> {
private final double expected;
private final double error;
public IsCloseToRelative(double expected, double error) {
this.expected = expected;
this.error = error;
}
@Override
protected boolean matchesSafely(Double actual) {
final double deviation = Math.abs(actual - expected);
final double observedError = deviation / Math.abs(expected);
return observedError <= error;
}
@Override
public void describeTo(Description description) {
description
.appendText("within ")
.appendValue(error * 100)
.appendText(" percent of ")
.appendValue(expected);
}
public static IsCloseToRelative closeToRelative(double expected, double error) {
return new IsCloseToRelative(expected, error);
}
public static IsCloseToRelative closeToRelative(double expected) {
return closeToRelative(expected, 0.05);
}
}
/**
* This class is an implementation of median absolute deviation that computes an exact value, rather than the approximation used in the
* aggregation. It's used to verify that the aggregation's approximate results are close enough to the exact result
*/
public static class ExactMedianAbsoluteDeviation {
public static double calculateMAD(int[] sample) {
return calculateMAD(Arrays.stream(sample)
.mapToDouble(point -> (double) point)
.toArray());
}
public static double calculateMAD(long[] sample) {
return calculateMAD(Arrays.stream(sample)
.mapToDouble(point -> (double) point)
.toArray());
}
public static double calculateMAD(List<Long> sample) {
return calculateMAD(sample.stream()
.mapToDouble(Long::doubleValue)
.toArray());
}
public static double calculateMAD(double[] sample) {
final double median = calculateMedian(sample);
final double[] deviations = Arrays.stream(sample)
.map(point -> Math.abs(median - point))
.toArray();
final double mad = calculateMedian(deviations);
return mad;
}
private static double calculateMedian(double[] sample) {
final double[] sorted = Arrays.copyOf(sample, sample.length);
Arrays.sort(sorted);
final int halfway = (int) Math.ceil(sorted.length / 2d);
final double median = (sorted[halfway - 1] + sorted[halfway]) / 2d;
return median;
}
}
}

View File

@ -0,0 +1,621 @@
/*
* 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.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptType;
import org.elasticsearch.search.aggregations.AggregationTestScriptsPlugin;
import org.elasticsearch.search.aggregations.BucketOrder;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.bucket.filter.Filter;
import org.elasticsearch.search.aggregations.bucket.global.Global;
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
import org.elasticsearch.search.aggregations.bucket.range.Range;
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.function.Supplier;
import java.util.stream.LongStream;
import static java.util.Collections.emptyMap;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.search.aggregations.AggregationBuilders.filter;
import static org.elasticsearch.search.aggregations.AggregationBuilders.global;
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
import static org.elasticsearch.search.aggregations.AggregationBuilders.range;
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
import static org.elasticsearch.search.aggregations.metrics.MedianAbsoluteDeviationAggregatorTests.IsCloseToRelative.closeToRelative;
import static org.elasticsearch.search.aggregations.metrics.MedianAbsoluteDeviationAggregatorTests.ExactMedianAbsoluteDeviation.calculateMAD;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasSize;
import static org.hamcrest.Matchers.sameInstance;
import static org.hamcrest.core.Is.is;
import static org.hamcrest.core.IsNull.notNullValue;
public class MedianAbsoluteDeviationIT extends AbstractNumericTestCase {
private static final int MIN_SAMPLE_VALUE = -1000000;
private static final int MAX_SAMPLE_VALUE = 1000000;
private static final int NUMBER_OF_DOCS = 1000;
private static final Supplier<Long> sampleSupplier = () -> randomLongBetween(MIN_SAMPLE_VALUE, MAX_SAMPLE_VALUE);
private static long[] singleValueSample;
private static long[] multiValueSample;
private static double singleValueExactMAD;
private static double multiValueExactMAD;
@Override
public void setupSuiteScopeCluster() throws Exception {
final Settings settings = Settings.builder()
.put(SETTING_NUMBER_OF_SHARDS, 1)
.put(SETTING_NUMBER_OF_REPLICAS, 0)
.build();
createIndex("idx", settings);
createIndex("idx_unmapped", settings);
minValue = MIN_SAMPLE_VALUE;
minValues = MIN_SAMPLE_VALUE;
maxValue = MAX_SAMPLE_VALUE;
maxValues = MAX_SAMPLE_VALUE;
singleValueSample = new long[NUMBER_OF_DOCS];
multiValueSample = new long[NUMBER_OF_DOCS * 2];
List<IndexRequestBuilder> builders = new ArrayList<>();
for (int i = 0; i < NUMBER_OF_DOCS; i++) {
final long singleValueDatapoint = sampleSupplier.get();
final long firstMultiValueDatapoint = sampleSupplier.get();
final long secondMultiValueDatapoint = sampleSupplier.get();
singleValueSample[i] = singleValueDatapoint;
multiValueSample[i * 2] = firstMultiValueDatapoint;
multiValueSample[(i * 2) + 1] = secondMultiValueDatapoint;
IndexRequestBuilder builder = client().prepareIndex("idx", "_doc", String.valueOf(i))
.setSource(jsonBuilder()
.startObject()
.field("value", singleValueDatapoint)
.startArray("values")
.value(firstMultiValueDatapoint)
.value(secondMultiValueDatapoint)
.endArray()
.endObject());
builders.add(builder);
}
singleValueExactMAD = calculateMAD(singleValueSample);
multiValueExactMAD = calculateMAD(multiValueSample);
indexRandom(true, builders);
prepareCreate("empty_bucket_idx")
.addMapping("type", "value", "type=integer")
.execute()
.actionGet();
builders = new ArrayList<>();
for (int i = 0; i < 2; i++) {
builders.add(client().prepareIndex("empty_bucket_idx", "type", String.valueOf(i)).setSource(jsonBuilder()
.startObject()
.field("value", i*2)
.endObject()));
}
indexRandom(true, builders);
ensureSearchable();
}
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return Collections.singleton(AggregationTestScriptsPlugin.class);
}
private static MedianAbsoluteDeviationAggregationBuilder randomBuilder() {
final MedianAbsoluteDeviationAggregationBuilder builder = new MedianAbsoluteDeviationAggregationBuilder("mad");
if (randomBoolean()) {
builder.compression(randomDoubleBetween(20, 1000, false));
}
return builder;
}
@Override
public void testEmptyAggregation() throws Exception {
final SearchResponse response = client()
.prepareSearch("empty_bucket_idx")
.addAggregation(
histogram("histogram")
.field("value")
.interval(1)
.minDocCount(0)
.subAggregation(
randomBuilder()
.field("value")))
.execute()
.actionGet();
assertHitCount(response, 2);
final Histogram histogram = response.getAggregations().get("histogram");
assertThat(histogram, notNullValue());
final Histogram.Bucket bucket = histogram.getBuckets().get(1);
assertThat(bucket, notNullValue());
final MedianAbsoluteDeviation mad = bucket.getAggregations().get("mad");
assertThat(mad, notNullValue());
assertThat(mad.getName(), is("mad"));
assertThat(mad.getMedianAbsoluteDeviation(), is(Double.NaN));
}
@Override
public void testUnmapped() throws Exception {
final SearchResponse response = client()
.prepareSearch("idx_unmapped")
.setQuery(matchAllQuery())
.addAggregation(
randomBuilder()
.field("value"))
.execute()
.actionGet();
assertHitCount(response, 0);
final MedianAbsoluteDeviation mad = response.getAggregations().get("mad");
assertThat(mad, notNullValue());
assertThat(mad.getName(), is("mad"));
assertThat(mad.getMedianAbsoluteDeviation(), is(Double.NaN));
}
@Override
public void testSingleValuedField() throws Exception {
final SearchResponse response = client()
.prepareSearch("idx")
.setQuery(matchAllQuery())
.addAggregation(
randomBuilder()
.field("value"))
.execute()
.actionGet();
assertHitCount(response, NUMBER_OF_DOCS);
final MedianAbsoluteDeviation mad = response.getAggregations().get("mad");
assertThat(mad, notNullValue());
assertThat(mad.getName(), is("mad"));
assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(singleValueExactMAD));
}
@Override
public void testSingleValuedFieldGetProperty() throws Exception {
final SearchResponse response = client()
.prepareSearch("idx")
.setQuery(matchAllQuery())
.addAggregation(
global("global")
.subAggregation(
randomBuilder()
.field("value")))
.execute()
.actionGet();
assertHitCount(response, NUMBER_OF_DOCS);
final Global global = response.getAggregations().get("global");
assertThat(global, notNullValue());
assertThat(global.getName(), is("global"));
assertThat(global.getDocCount(), is((long) NUMBER_OF_DOCS));
assertThat(global.getAggregations(), notNullValue());
assertThat(global.getAggregations().asMap().entrySet(), hasSize(1));
final MedianAbsoluteDeviation mad = global.getAggregations().get("mad");
assertThat(mad, notNullValue());
assertThat(mad.getName(), is("mad"));
assertThat(((InternalAggregation) global).getProperty("mad"), sameInstance(mad));
}
@Override
public void testSingleValuedFieldPartiallyUnmapped() throws Exception {
final SearchResponse response = client()
.prepareSearch("idx", "idx_unmapped")
.setQuery(matchAllQuery())
.addAggregation(
randomBuilder()
.field("value"))
.execute()
.actionGet();
assertHitCount(response, NUMBER_OF_DOCS);
final MedianAbsoluteDeviation mad = response.getAggregations().get("mad");
assertThat(mad, notNullValue());
assertThat(mad.getName(), is("mad"));
assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(singleValueExactMAD));
}
@Override
public void testSingleValuedFieldWithValueScript() throws Exception {
final SearchResponse response = client()
.prepareSearch("idx")
.setQuery(matchAllQuery())
.addAggregation(
randomBuilder()
.field("value")
.script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value + 1", Collections.emptyMap())))
.execute()
.actionGet();
assertHitCount(response, NUMBER_OF_DOCS);
final MedianAbsoluteDeviation mad = response.getAggregations().get("mad");
assertThat(mad, notNullValue());
assertThat(mad.getName(), is("mad"));
final double fromIncrementedSampleMAD = calculateMAD(Arrays.stream(singleValueSample)
.map(point -> point + 1)
.toArray());
assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(fromIncrementedSampleMAD));
}
@Override
public void testSingleValuedFieldWithValueScriptWithParams() throws Exception {
final Map<String, Object> params = new HashMap<>();
params.put("inc", 1);
final SearchResponse response = client()
.prepareSearch("idx")
.setQuery(matchAllQuery())
.addAggregation(
randomBuilder()
.field("value")
.script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value + inc", params)))
.execute()
.actionGet();
assertHitCount(response, NUMBER_OF_DOCS);
final MedianAbsoluteDeviation mad = response.getAggregations().get("mad");
assertThat(mad, notNullValue());
assertThat(mad.getName(), is("mad"));
final double fromIncrementedSampleMAD = calculateMAD(Arrays.stream(singleValueSample)
.map(point -> point + 1)
.toArray());
assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(fromIncrementedSampleMAD));
}
@Override
public void testMultiValuedField() throws Exception {
final SearchResponse response = client()
.prepareSearch("idx")
.setQuery(matchAllQuery())
.addAggregation(
randomBuilder()
.field("values"))
.execute()
.actionGet();
assertHitCount(response, NUMBER_OF_DOCS);
final MedianAbsoluteDeviation mad = response.getAggregations().get("mad");
assertThat(mad, notNullValue());
assertThat(mad.getName(), is("mad"));
assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(multiValueExactMAD));
}
@Override
public void testMultiValuedFieldWithValueScript() throws Exception {
final SearchResponse response = client()
.prepareSearch("idx")
.setQuery(matchAllQuery())
.addAggregation(
randomBuilder()
.field("values")
.script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value + 1", Collections.emptyMap())))
.execute()
.actionGet();
assertHitCount(response, NUMBER_OF_DOCS);
final MedianAbsoluteDeviation mad = response.getAggregations().get("mad");
assertThat(mad, notNullValue());
final double fromIncrementedSampleMAD = calculateMAD(Arrays.stream(multiValueSample)
.map(point -> point + 1)
.toArray());
assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(fromIncrementedSampleMAD));
}
@Override
public void testMultiValuedFieldWithValueScriptWithParams() throws Exception {
final Map<String, Object> params = new HashMap<>();
params.put("inc", 1);
final SearchResponse response = client()
.prepareSearch("idx")
.setQuery(matchAllQuery())
.addAggregation(
randomBuilder()
.field("values")
.script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value + inc", params)))
.execute()
.actionGet();
assertHitCount(response, NUMBER_OF_DOCS);
final MedianAbsoluteDeviation mad = response.getAggregations().get("mad");
assertThat(mad, notNullValue());
final double fromIncrementedSampleMAD = calculateMAD(Arrays.stream(multiValueSample)
.map(point -> point + 1)
.toArray());
assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(fromIncrementedSampleMAD));
}
@Override
public void testScriptSingleValued() throws Exception {
final SearchResponse response = client()
.prepareSearch("idx")
.setQuery(matchAllQuery())
.addAggregation(
randomBuilder()
.script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "doc['value'].value", Collections.emptyMap())))
.execute()
.actionGet();
assertHitCount(response, NUMBER_OF_DOCS);
final MedianAbsoluteDeviation mad = response.getAggregations().get("mad");
assertThat(mad, notNullValue());
assertThat(mad.getName(), is("mad"));
assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(singleValueExactMAD));
}
@Override
public void testScriptSingleValuedWithParams() throws Exception {
final Map<String, Object> params = new HashMap<>();
params.put("inc", 1);
final SearchResponse response = client()
.prepareSearch("idx")
.setQuery(matchAllQuery())
.addAggregation(
randomBuilder()
.script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "doc['value'].value + inc", params)))
.execute()
.actionGet();
assertHitCount(response, NUMBER_OF_DOCS);
final MedianAbsoluteDeviation mad = response.getAggregations().get("mad");
assertThat(mad, notNullValue());
assertThat(mad.getName(), is("mad"));
final double fromIncrementedSampleMAD = calculateMAD(Arrays.stream(singleValueSample)
.map(point -> point + 1)
.toArray());
assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(fromIncrementedSampleMAD));
}
@Override
public void testScriptMultiValued() throws Exception {
final SearchResponse response = client()
.prepareSearch("idx")
.setQuery(matchAllQuery())
.addAggregation(
randomBuilder()
.script(new Script(
ScriptType.INLINE,
AggregationTestScriptsPlugin.NAME,
"doc['values'].values",
Collections.emptyMap())))
.execute()
.actionGet();
assertHitCount(response, NUMBER_OF_DOCS);
final MedianAbsoluteDeviation mad = response.getAggregations().get("mad");
assertThat(mad, notNullValue());
assertThat(mad.getName(), is("mad"));
assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(multiValueExactMAD));
}
@Override
public void testScriptMultiValuedWithParams() throws Exception {
final Map<String, Object> params = new HashMap<>();
params.put("inc", 1);
final SearchResponse response = client()
.prepareSearch("idx")
.setQuery(matchAllQuery())
.addAggregation(
randomBuilder()
.script(new Script(
ScriptType.INLINE,
AggregationTestScriptsPlugin.NAME,
"[ doc['value'].value, doc['value'].value + inc ]",
params)))
.execute()
.actionGet();
assertHitCount(response, NUMBER_OF_DOCS);
final MedianAbsoluteDeviation mad = response.getAggregations().get("mad");
assertThat(mad, notNullValue());
assertThat(mad.getName(), is("mad"));
final double fromIncrementedSampleMAD = calculateMAD(Arrays.stream(singleValueSample)
.flatMap(point -> LongStream.of(point, point + 1))
.toArray());
assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(fromIncrementedSampleMAD));
}
public void testAsSubAggregation() throws Exception {
final int rangeBoundary = (MAX_SAMPLE_VALUE + MIN_SAMPLE_VALUE) / 2;
final SearchResponse response = client()
.prepareSearch("idx")
.setQuery(matchAllQuery())
.addAggregation(
range("range")
.field("value")
.addRange(MIN_SAMPLE_VALUE, rangeBoundary)
.addRange(rangeBoundary, MAX_SAMPLE_VALUE)
.subAggregation(
randomBuilder()
.field("value")))
.execute()
.actionGet();
assertHitCount(response, NUMBER_OF_DOCS);
final long[] lowerBucketSample = Arrays.stream(singleValueSample)
.filter(point -> point >= MIN_SAMPLE_VALUE && point < rangeBoundary)
.toArray();
final long[] upperBucketSample = Arrays.stream(singleValueSample)
.filter(point -> point >= rangeBoundary && point < MAX_SAMPLE_VALUE)
.toArray();
final Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
List<? extends Range.Bucket> buckets = range.getBuckets();
assertThat(buckets, notNullValue());
assertThat(buckets, hasSize(2));
final Range.Bucket lowerBucket = buckets.get(0);
assertThat(lowerBucket, notNullValue());
final MedianAbsoluteDeviation lowerBucketMAD = lowerBucket.getAggregations().get("mad");
assertThat(lowerBucketMAD, notNullValue());
assertThat(lowerBucketMAD.getMedianAbsoluteDeviation(), closeToRelative(calculateMAD(lowerBucketSample)));
final Range.Bucket upperBucket = buckets.get(1);
assertThat(upperBucket, notNullValue());
final MedianAbsoluteDeviation upperBucketMAD = upperBucket.getAggregations().get("mad");
assertThat(upperBucketMAD, notNullValue());
assertThat(upperBucketMAD.getMedianAbsoluteDeviation(), closeToRelative(calculateMAD(upperBucketSample)));
}
@Override
public void testOrderByEmptyAggregation() throws Exception {
final int numberOfBuckets = 10;
final SearchResponse response = client()
.prepareSearch("idx")
.setQuery(matchAllQuery())
.addAggregation(
terms("terms")
.field("value")
.size(numberOfBuckets)
.order(BucketOrder.compound(BucketOrder.aggregation("filter>mad", true)))
.subAggregation(
filter("filter", termQuery("value", MAX_SAMPLE_VALUE + 1))
.subAggregation(
randomBuilder()
.field("value"))))
.execute()
.actionGet();
assertHitCount(response, NUMBER_OF_DOCS);
final Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
List<? extends Terms.Bucket> buckets = terms.getBuckets();
assertThat(buckets, notNullValue());
assertThat(buckets, hasSize(numberOfBuckets));
for (int i = 0; i < numberOfBuckets; i++) {
Terms.Bucket bucket = buckets.get(i);
assertThat(bucket, notNullValue());
Filter filter = bucket.getAggregations().get("filter");
assertThat(filter, notNullValue());
assertThat(filter.getDocCount(), equalTo(0L));
MedianAbsoluteDeviation mad = filter.getAggregations().get("mad");
assertThat(mad, notNullValue());
assertThat(mad.getMedianAbsoluteDeviation(), equalTo(Double.NaN));
}
}
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(
prepareCreate("cache_test_idx")
.addMapping("type", "d", "type=long")
.setSettings(Settings.builder()
.put("requests.cache.enable", true)
.put("number_of_shards", 1)
.put("number_of_replicas", 1))
.get());
indexRandom(true,
client().prepareIndex("cache_test_idx", "type", "1").setSource("s", 1),
client().prepareIndex("cache_test_idx", "type", "2").setSource("s", 2));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0)
.addAggregation(randomBuilder()
.field("d")
.script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - 1", emptyMap()))).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(randomBuilder().field("d")).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
}

View File

@ -0,0 +1,43 @@
/*
* 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;
public class MedianAbsoluteDeviationTests extends AbstractNumericMetricTestCase<MedianAbsoluteDeviationAggregationBuilder> {
@Override
protected MedianAbsoluteDeviationAggregationBuilder doCreateTestAggregatorFactory() {
MedianAbsoluteDeviationAggregationBuilder builder =
new MedianAbsoluteDeviationAggregationBuilder(randomAlphaOfLengthBetween(1, 20));
if (randomBoolean()) {
builder.compression(randomDoubleBetween(0, 1000.0, false));
}
if (randomBoolean()) {
builder.missing("MISSING");
}
if (randomBoolean()) {
builder.format("###.00");
}
return builder;
}
}

View File

@ -95,6 +95,7 @@ import org.elasticsearch.search.aggregations.metrics.InternalHDRPercentiles;
import org.elasticsearch.search.aggregations.metrics.InternalTDigestPercentileRanks;
import org.elasticsearch.search.aggregations.metrics.InternalTDigestPercentiles;
import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.MedianAbsoluteDeviationAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.MinAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.ParsedAvg;
import org.elasticsearch.search.aggregations.metrics.ParsedCardinality;
@ -104,6 +105,7 @@ import org.elasticsearch.search.aggregations.metrics.ParsedGeoCentroid;
import org.elasticsearch.search.aggregations.metrics.ParsedHDRPercentileRanks;
import org.elasticsearch.search.aggregations.metrics.ParsedHDRPercentiles;
import org.elasticsearch.search.aggregations.metrics.ParsedMax;
import org.elasticsearch.search.aggregations.metrics.ParsedMedianAbsoluteDeviation;
import org.elasticsearch.search.aggregations.metrics.ParsedMin;
import org.elasticsearch.search.aggregations.metrics.ParsedScriptedMetric;
import org.elasticsearch.search.aggregations.metrics.ParsedStats;
@ -178,6 +180,7 @@ public abstract class InternalAggregationTestCase<T extends InternalAggregation>
map.put(InternalTDigestPercentiles.NAME, (p, c) -> ParsedTDigestPercentiles.fromXContent(p, (String) c));
map.put(InternalTDigestPercentileRanks.NAME, (p, c) -> ParsedTDigestPercentileRanks.fromXContent(p, (String) c));
map.put(PercentilesBucketPipelineAggregationBuilder.NAME, (p, c) -> ParsedPercentilesBucket.fromXContent(p, (String) c));
map.put(MedianAbsoluteDeviationAggregationBuilder.NAME, (p, c) -> ParsedMedianAbsoluteDeviation.fromXContent(p, (String) c));
map.put(MinAggregationBuilder.NAME, (p, c) -> ParsedMin.fromXContent(p, (String) c));
map.put(MaxAggregationBuilder.NAME, (p, c) -> ParsedMax.fromXContent(p, (String) c));
map.put(SumAggregationBuilder.NAME, (p, c) -> ParsedSum.fromXContent(p, (String) c));