[7.x] Histogram integration on Histogram field type (#59431)
Backports #58930 to 7.x Implements histogram aggregation over histogram fields as requested in #53285.
This commit is contained in:
parent
a7895ff458
commit
3868bcc7b8
|
@ -286,3 +286,93 @@ POST /sales/_search?size=0
|
|||
// TEST[setup:sales]
|
||||
|
||||
<1> Documents without a value in the `quantity` field will fall into the same bucket as documents that have the value `0`.
|
||||
|
||||
[[search-aggregations-bucket-histogram-aggregation-histogram-fields]]
|
||||
==== Histogram fields
|
||||
|
||||
Running a histogram aggregation over histogram fields computes the total number of counts for each interval.
|
||||
|
||||
For example, executing a histogram aggregation against the following index that stores pre-aggregated histograms
|
||||
with latency metrics (in milliseconds) for different networks:
|
||||
|
||||
[source,console]
|
||||
--------------------------------------------------
|
||||
PUT metrics_index/_doc/1
|
||||
{
|
||||
"network.name" : "net-1",
|
||||
"latency_histo" : {
|
||||
"values" : [1, 3, 8, 12, 15],
|
||||
"counts" : [3, 7, 23, 12, 6]
|
||||
}
|
||||
}
|
||||
|
||||
PUT metrics_index/_doc/2
|
||||
{
|
||||
"network.name" : "net-2",
|
||||
"latency_histo" : {
|
||||
"values" : [1, 6, 8, 12, 14],
|
||||
"counts" : [8, 17, 8, 7, 6]
|
||||
}
|
||||
}
|
||||
|
||||
POST /metrics_index/_search?size=0
|
||||
{
|
||||
"aggs" : {
|
||||
"latency_buckets" : {
|
||||
"histogram" : {
|
||||
"field" : "latency_histo",
|
||||
"interval" : 5
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
--------------------------------------------------
|
||||
|
||||
|
||||
The `histogram` aggregation will sum the counts of each interval computed based on the `values` and
|
||||
return the following output:
|
||||
|
||||
[source,console-result]
|
||||
--------------------------------------------------
|
||||
{
|
||||
...
|
||||
"aggregations": {
|
||||
"prices" : {
|
||||
"buckets": [
|
||||
{
|
||||
"key": 0.0,
|
||||
"doc_count": 18
|
||||
},
|
||||
{
|
||||
"key": 5.0,
|
||||
"doc_count": 48
|
||||
},
|
||||
{
|
||||
"key": 10.0,
|
||||
"doc_count": 25
|
||||
},
|
||||
{
|
||||
"key": 15.0,
|
||||
"doc_count": 6
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
}
|
||||
--------------------------------------------------
|
||||
// TESTRESPONSE[skip:test not setup]
|
||||
|
||||
[IMPORTANT]
|
||||
========
|
||||
Histogram aggregation is a bucket aggregation, which partitions documents into buckets rather than calculating metrics over fields like
|
||||
metrics aggregations do. Each bucket represents a collection of documents which sub-aggregations can run on.
|
||||
On the other hand, a histogram field is a pre-aggregated field representing multiple values inside a single field:
|
||||
buckets of numerical data and a count of items/documents for each bucket. This mismatch between the histogram aggregations expected input
|
||||
(expecting raw documents) and the histogram field (that provides summary information) limits the outcome of the aggregation
|
||||
to only the doc counts for each bucket.
|
||||
|
||||
|
||||
**Consequently, when executing a histogram aggregation over a histogram field, no sub-aggregations are allowed.**
|
||||
========
|
||||
|
||||
Also, when running histogram aggregation over histogram field the `missing` parameter is not supported.
|
||||
|
|
|
@ -41,6 +41,7 @@ following aggregations and queries:
|
|||
* <<search-aggregations-metrics-percentile-aggregation,percentiles>> aggregation
|
||||
* <<search-aggregations-metrics-percentile-rank-aggregation,percentile ranks>> aggregation
|
||||
* <<search-aggregations-metrics-boxplot-aggregation,boxplot>> aggregation
|
||||
* <<search-aggregations-bucket-histogram-aggregation-histogram-fields,histogram>> aggregation
|
||||
* <<query-dsl-exists-query,exists>> query
|
||||
|
||||
[[mapping-types-histogram-building-histogram]]
|
||||
|
|
|
@ -29,7 +29,6 @@ import org.elasticsearch.search.aggregations.CardinalityUpperBound;
|
|||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregatorSupplier;
|
||||
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
|
||||
import org.elasticsearch.search.aggregations.support.HistogramAggregatorSupplier;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
|
||||
|
|
|
@ -7,7 +7,7 @@
|
|||
* 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
|
||||
* 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
|
||||
|
@ -16,7 +16,7 @@
|
|||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.elasticsearch.search.aggregations.support;
|
||||
package org.elasticsearch.search.aggregations.bucket.histogram;
|
||||
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
|
@ -152,12 +152,12 @@ public final class InternalHistogram extends InternalMultiBucketAggregation<Inte
|
|||
}
|
||||
}
|
||||
|
||||
static class EmptyBucketInfo {
|
||||
public static class EmptyBucketInfo {
|
||||
|
||||
final double interval, offset, minBound, maxBound;
|
||||
final InternalAggregations subAggregations;
|
||||
|
||||
EmptyBucketInfo(double interval, double offset, double minBound, double maxBound, InternalAggregations subAggregations) {
|
||||
public EmptyBucketInfo(double interval, double offset, double minBound, double maxBound, InternalAggregations subAggregations) {
|
||||
this.interval = interval;
|
||||
this.offset = offset;
|
||||
this.minBound = minBound;
|
||||
|
@ -203,8 +203,15 @@ public final class InternalHistogram extends InternalMultiBucketAggregation<Inte
|
|||
private final long minDocCount;
|
||||
final EmptyBucketInfo emptyBucketInfo;
|
||||
|
||||
InternalHistogram(String name, List<Bucket> buckets, BucketOrder order, long minDocCount, EmptyBucketInfo emptyBucketInfo,
|
||||
DocValueFormat formatter, boolean keyed, Map<String, Object> metadata) {
|
||||
public InternalHistogram(
|
||||
String name,
|
||||
List<Bucket> buckets,
|
||||
BucketOrder order,
|
||||
long minDocCount,
|
||||
EmptyBucketInfo emptyBucketInfo,
|
||||
DocValueFormat formatter,
|
||||
boolean keyed,
|
||||
Map<String, Object> metadata) {
|
||||
super(name, metadata);
|
||||
this.buckets = buckets;
|
||||
this.order = order;
|
||||
|
|
|
@ -31,7 +31,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
|
|||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.watcher.ResourceWatcherService;
|
||||
import org.elasticsearch.xpack.analytics.action.TransportAnalyticsStatsAction;
|
||||
import org.elasticsearch.xpack.analytics.aggregations.metrics.AnalyticsAggregatorFactory;
|
||||
import org.elasticsearch.xpack.analytics.aggregations.AnalyticsAggregatorFactory;
|
||||
import org.elasticsearch.xpack.analytics.normalize.NormalizePipelineAggregationBuilder;
|
||||
import org.elasticsearch.xpack.analytics.boxplot.BoxplotAggregationBuilder;
|
||||
import org.elasticsearch.xpack.analytics.boxplot.InternalBoxplot;
|
||||
|
@ -157,7 +157,8 @@ public class AnalyticsPlugin extends Plugin implements SearchPlugin, ActionPlugi
|
|||
AnalyticsAggregatorFactory::registerPercentileRanksAggregator,
|
||||
AnalyticsAggregatorFactory::registerHistoBackedSumAggregator,
|
||||
AnalyticsAggregatorFactory::registerHistoBackedValueCountAggregator,
|
||||
AnalyticsAggregatorFactory::registerHistoBackedAverageAggregator
|
||||
AnalyticsAggregatorFactory::registerHistoBackedAverageAggregator,
|
||||
AnalyticsAggregatorFactory::registerHistoBackedHistogramAggregator
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -3,8 +3,10 @@
|
|||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.analytics.aggregations.metrics;
|
||||
package org.elasticsearch.xpack.analytics.aggregations;
|
||||
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregationBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregatorSupplier;
|
||||
import org.elasticsearch.search.aggregations.metrics.AvgAggregationBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.MetricAggregatorSupplier;
|
||||
import org.elasticsearch.search.aggregations.metrics.PercentileRanksAggregationBuilder;
|
||||
|
@ -15,6 +17,14 @@ import org.elasticsearch.search.aggregations.metrics.PercentilesMethod;
|
|||
import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder;
|
||||
import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
|
||||
import org.elasticsearch.xpack.analytics.aggregations.bucket.histogram.HistoBackedHistogramAggregator;
|
||||
import org.elasticsearch.xpack.analytics.aggregations.metrics.HistoBackedAvgAggregator;
|
||||
import org.elasticsearch.xpack.analytics.aggregations.metrics.HistoBackedHDRPercentileRanksAggregator;
|
||||
import org.elasticsearch.xpack.analytics.aggregations.metrics.HistoBackedHDRPercentilesAggregator;
|
||||
import org.elasticsearch.xpack.analytics.aggregations.metrics.HistoBackedSumAggregator;
|
||||
import org.elasticsearch.xpack.analytics.aggregations.metrics.HistoBackedTDigestPercentileRanksAggregator;
|
||||
import org.elasticsearch.xpack.analytics.aggregations.metrics.HistoBackedTDigestPercentilesAggregator;
|
||||
import org.elasticsearch.xpack.analytics.aggregations.metrics.HistoBackedValueCountAggregator;
|
||||
import org.elasticsearch.xpack.analytics.aggregations.support.AnalyticsValuesSourceType;
|
||||
|
||||
public class AnalyticsAggregatorFactory {
|
||||
|
@ -83,4 +93,11 @@ public class AnalyticsAggregatorFactory {
|
|||
(MetricAggregatorSupplier) HistoBackedAvgAggregator::new
|
||||
);
|
||||
}
|
||||
|
||||
public static void registerHistoBackedHistogramAggregator(ValuesSourceRegistry.Builder builder) {
|
||||
builder.register(HistogramAggregationBuilder.NAME,
|
||||
AnalyticsValuesSourceType.HISTOGRAM,
|
||||
(HistogramAggregatorSupplier) HistoBackedHistogramAggregator::new
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,94 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.xpack.analytics.aggregations.bucket.histogram;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.elasticsearch.index.fielddata.HistogramValue;
|
||||
import org.elasticsearch.index.fielddata.HistogramValues;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.BucketOrder;
|
||||
import org.elasticsearch.search.aggregations.CardinalityUpperBound;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollector;
|
||||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.AbstractHistogramAggregator;
|
||||
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.xpack.analytics.aggregations.support.HistogramValuesSource;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
public class HistoBackedHistogramAggregator extends AbstractHistogramAggregator {
|
||||
|
||||
private final HistogramValuesSource.Histogram valuesSource;
|
||||
|
||||
public HistoBackedHistogramAggregator(
|
||||
String name,
|
||||
AggregatorFactories factories,
|
||||
double interval,
|
||||
double offset,
|
||||
BucketOrder order,
|
||||
boolean keyed,
|
||||
long minDocCount,
|
||||
double minBound,
|
||||
double maxBound,
|
||||
ValuesSourceConfig valuesSourceConfig,
|
||||
SearchContext context,
|
||||
Aggregator parent,
|
||||
CardinalityUpperBound cardinalityUpperBound,
|
||||
Map<String, Object> metadata) throws IOException {
|
||||
super(name, factories, interval, offset, order, keyed, minDocCount, minBound, maxBound,
|
||||
valuesSourceConfig.format(), context, parent, cardinalityUpperBound, metadata);
|
||||
|
||||
// TODO: Stop using null here
|
||||
this.valuesSource = valuesSourceConfig.hasValues() ? (HistogramValuesSource.Histogram) valuesSourceConfig.getValuesSource() : null;
|
||||
|
||||
// Sub aggregations are not allowed when running histogram agg over histograms
|
||||
if (subAggregators().length > 0) {
|
||||
throw new IllegalArgumentException("Histogram aggregation on histogram fields does not support sub-aggregations");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
|
||||
final LeafBucketCollector sub) throws IOException {
|
||||
if (valuesSource == null) {
|
||||
return LeafBucketCollector.NO_OP_COLLECTOR;
|
||||
}
|
||||
|
||||
final HistogramValues values = valuesSource.getHistogramValues(ctx);
|
||||
return new LeafBucketCollectorBase(sub, values) {
|
||||
@Override
|
||||
public void collect(int doc, long owningBucketOrd) throws IOException {
|
||||
if (values.advanceExact(doc)) {
|
||||
final HistogramValue sketch = values.histogram();
|
||||
|
||||
double previousKey = Double.NEGATIVE_INFINITY;
|
||||
while (sketch.next()) {
|
||||
final double value = sketch.value();
|
||||
final int count = sketch.count();
|
||||
|
||||
double key = Math.floor((value - offset) / interval);
|
||||
assert key >= previousKey;
|
||||
long bucketOrd = bucketOrds.add(owningBucketOrd, Double.doubleToLongBits(key));
|
||||
if (bucketOrd < 0) { // already seen
|
||||
bucketOrd = -1 - bucketOrd;
|
||||
collectExistingBucket(sub, doc, bucketOrd);
|
||||
} else {
|
||||
collectBucket(sub, doc, bucketOrd);
|
||||
}
|
||||
// We have added the document already. We should increment doc_count by count - 1
|
||||
// so that we have added it count times.
|
||||
incrementBucketDocCount(bucketOrd, count - 1);
|
||||
previousKey = key;
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -32,7 +32,7 @@ import java.util.Map;
|
|||
* Average aggregator operating over histogram datatypes {@link HistogramValuesSource}
|
||||
* The aggregation computes weighted average by taking counts into consideration for each value
|
||||
*/
|
||||
class HistoBackedAvgAggregator extends NumericMetricsAggregator.SingleValue {
|
||||
public class HistoBackedAvgAggregator extends NumericMetricsAggregator.SingleValue {
|
||||
|
||||
private final HistogramValuesSource.Histogram valuesSource;
|
||||
|
||||
|
@ -41,7 +41,7 @@ class HistoBackedAvgAggregator extends NumericMetricsAggregator.SingleValue {
|
|||
DoubleArray compensations;
|
||||
DocValueFormat format;
|
||||
|
||||
HistoBackedAvgAggregator(
|
||||
public HistoBackedAvgAggregator(
|
||||
String name,
|
||||
ValuesSourceConfig valuesSourceConfig,
|
||||
SearchContext context,
|
||||
|
|
|
@ -17,9 +17,9 @@ import org.elasticsearch.search.internal.SearchContext;
|
|||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
class HistoBackedHDRPercentileRanksAggregator extends AbstractHistoBackedHDRPercentilesAggregator {
|
||||
public class HistoBackedHDRPercentileRanksAggregator extends AbstractHistoBackedHDRPercentilesAggregator {
|
||||
|
||||
HistoBackedHDRPercentileRanksAggregator(String name, ValuesSource valuesSource, SearchContext context, Aggregator parent,
|
||||
public HistoBackedHDRPercentileRanksAggregator(String name, ValuesSource valuesSource, SearchContext context, Aggregator parent,
|
||||
double[] percents, int numberOfSignificantValueDigits, boolean keyed, DocValueFormat format,
|
||||
Map<String, Object> metadata) throws IOException {
|
||||
super(name, valuesSource, context, parent, percents, numberOfSignificantValueDigits, keyed, format, metadata);
|
||||
|
|
|
@ -19,8 +19,8 @@ import java.util.Map;
|
|||
|
||||
public class HistoBackedHDRPercentilesAggregator extends AbstractHistoBackedHDRPercentilesAggregator {
|
||||
|
||||
HistoBackedHDRPercentilesAggregator(String name, ValuesSource valuesSource, SearchContext context, Aggregator parent, double[] percents,
|
||||
int numberOfSignificantValueDigits, boolean keyed, DocValueFormat formatter,
|
||||
public HistoBackedHDRPercentilesAggregator(String name, ValuesSource valuesSource, SearchContext context, Aggregator parent,
|
||||
double[] percents, int numberOfSignificantValueDigits, boolean keyed, DocValueFormat formatter,
|
||||
Map<String, Object> metadata) throws IOException {
|
||||
super(name, valuesSource, context, parent, percents, numberOfSignificantValueDigits, keyed, formatter,
|
||||
metadata);
|
||||
|
|
|
@ -33,7 +33,7 @@ import java.util.Map;
|
|||
* The aggregator sums each histogram value multiplied by its count.
|
||||
* Eg for a histogram of response times, this is an approximate "total time spent".
|
||||
*/
|
||||
class HistoBackedSumAggregator extends NumericMetricsAggregator.SingleValue {
|
||||
public class HistoBackedSumAggregator extends NumericMetricsAggregator.SingleValue {
|
||||
|
||||
private final HistogramValuesSource.Histogram valuesSource;
|
||||
private final DocValueFormat format;
|
||||
|
@ -41,7 +41,7 @@ class HistoBackedSumAggregator extends NumericMetricsAggregator.SingleValue {
|
|||
private DoubleArray sums;
|
||||
private DoubleArray compensations;
|
||||
|
||||
HistoBackedSumAggregator(
|
||||
public HistoBackedSumAggregator(
|
||||
String name,
|
||||
ValuesSourceConfig valuesSourceConfig,
|
||||
SearchContext context,
|
||||
|
|
|
@ -17,9 +17,9 @@ import org.elasticsearch.search.internal.SearchContext;
|
|||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
class HistoBackedTDigestPercentileRanksAggregator extends AbstractHistoBackedTDigestPercentilesAggregator {
|
||||
public class HistoBackedTDigestPercentileRanksAggregator extends AbstractHistoBackedTDigestPercentilesAggregator {
|
||||
|
||||
HistoBackedTDigestPercentileRanksAggregator(String name,
|
||||
public HistoBackedTDigestPercentileRanksAggregator(String name,
|
||||
ValuesSource valuesSource,
|
||||
SearchContext context,
|
||||
Aggregator parent,
|
||||
|
|
|
@ -17,9 +17,9 @@ import org.elasticsearch.search.internal.SearchContext;
|
|||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
class HistoBackedTDigestPercentilesAggregator extends AbstractHistoBackedTDigestPercentilesAggregator {
|
||||
public class HistoBackedTDigestPercentilesAggregator extends AbstractHistoBackedTDigestPercentilesAggregator {
|
||||
|
||||
HistoBackedTDigestPercentilesAggregator(String name,
|
||||
public HistoBackedTDigestPercentilesAggregator(String name,
|
||||
ValuesSource valuesSource,
|
||||
SearchContext context,
|
||||
Aggregator parent,
|
||||
|
|
|
@ -0,0 +1,232 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.xpack.analytics.aggregations.bucket.histogram;
|
||||
|
||||
import com.tdunning.math.stats.Centroid;
|
||||
import com.tdunning.math.stats.TDigest;
|
||||
import org.apache.lucene.document.BinaryDocValuesField;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.mapper.MappedFieldType;
|
||||
import org.elasticsearch.plugins.SearchPlugin;
|
||||
import org.elasticsearch.search.aggregations.AggregationBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorTestCase;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregationBuilder;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram;
|
||||
import org.elasticsearch.search.aggregations.metrics.TDigestState;
|
||||
import org.elasticsearch.search.aggregations.metrics.TopHitsAggregationBuilder;
|
||||
import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper;
|
||||
import org.elasticsearch.xpack.analytics.AnalyticsPlugin;
|
||||
import org.elasticsearch.xpack.analytics.mapper.HistogramFieldMapper;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
import static java.util.Collections.singleton;
|
||||
|
||||
public class HistoBackedHistogramAggregatorTests extends AggregatorTestCase {
|
||||
|
||||
private static final String FIELD_NAME = "field";
|
||||
|
||||
public void testHistograms() throws Exception {
|
||||
try (Directory dir = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir)) {
|
||||
w.addDocument(singleton(getDocValue(FIELD_NAME, new double[] {0, 1.2, 10, 12, 24})));
|
||||
w.addDocument(singleton(getDocValue(FIELD_NAME, new double[] {5.3, 6, 6, 20})));
|
||||
w.addDocument(singleton(getDocValue(FIELD_NAME, new double[] {-10, 0.01, 10, 10, 30})));
|
||||
|
||||
HistogramAggregationBuilder aggBuilder = new HistogramAggregationBuilder("my_agg")
|
||||
.field(FIELD_NAME)
|
||||
.interval(5);
|
||||
try (IndexReader reader = w.getReader()) {
|
||||
IndexSearcher searcher = new IndexSearcher(reader);
|
||||
InternalHistogram histogram = searchAndReduce(searcher, new MatchAllDocsQuery(), aggBuilder, defaultFieldType(FIELD_NAME));
|
||||
assertEquals(9, histogram.getBuckets().size());
|
||||
assertEquals(-10d, histogram.getBuckets().get(0).getKey());
|
||||
assertEquals(1, histogram.getBuckets().get(0).getDocCount());
|
||||
assertEquals(-5d, histogram.getBuckets().get(1).getKey());
|
||||
assertEquals(0, histogram.getBuckets().get(1).getDocCount());
|
||||
assertEquals(0d, histogram.getBuckets().get(2).getKey());
|
||||
assertEquals(3, histogram.getBuckets().get(2).getDocCount());
|
||||
assertEquals(5d, histogram.getBuckets().get(3).getKey());
|
||||
assertEquals(3, histogram.getBuckets().get(3).getDocCount());
|
||||
assertEquals(10d, histogram.getBuckets().get(4).getKey());
|
||||
assertEquals(4, histogram.getBuckets().get(4).getDocCount());
|
||||
assertEquals(15d, histogram.getBuckets().get(5).getKey());
|
||||
assertEquals(0, histogram.getBuckets().get(5).getDocCount());
|
||||
assertEquals(20d, histogram.getBuckets().get(6).getKey());
|
||||
assertEquals(2, histogram.getBuckets().get(6).getDocCount());
|
||||
assertEquals(25d, histogram.getBuckets().get(7).getKey());
|
||||
assertEquals(0, histogram.getBuckets().get(7).getDocCount());
|
||||
assertEquals(30d, histogram.getBuckets().get(8).getKey());
|
||||
assertEquals(1, histogram.getBuckets().get(8).getDocCount());
|
||||
assertTrue(AggregationInspectionHelper.hasValue(histogram));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testMinDocCount() throws Exception {
|
||||
try (Directory dir = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir)) {
|
||||
w.addDocument(singleton(getDocValue(FIELD_NAME, new double[] {0, 1.2, 10, 12, 24})));
|
||||
w.addDocument(singleton(getDocValue(FIELD_NAME, new double[] {5.3, 6, 6, 20})));
|
||||
w.addDocument(singleton(getDocValue(FIELD_NAME, new double[] {-10, 0.01, 10, 10, 30, 90})));
|
||||
|
||||
HistogramAggregationBuilder aggBuilder = new HistogramAggregationBuilder("my_agg")
|
||||
.field(FIELD_NAME)
|
||||
.interval(5)
|
||||
.minDocCount(2);
|
||||
try (IndexReader reader = w.getReader()) {
|
||||
IndexSearcher searcher = new IndexSearcher(reader);
|
||||
InternalHistogram histogram = searchAndReduce(searcher, new MatchAllDocsQuery(), aggBuilder, defaultFieldType(FIELD_NAME));
|
||||
assertEquals(4, histogram.getBuckets().size());
|
||||
assertEquals(0d, histogram.getBuckets().get(0).getKey());
|
||||
assertEquals(3, histogram.getBuckets().get(0).getDocCount());
|
||||
assertEquals(5d, histogram.getBuckets().get(1).getKey());
|
||||
assertEquals(3, histogram.getBuckets().get(1).getDocCount());
|
||||
assertEquals(10d, histogram.getBuckets().get(2).getKey());
|
||||
assertEquals(4, histogram.getBuckets().get(2).getDocCount());
|
||||
assertEquals(20d, histogram.getBuckets().get(3).getKey());
|
||||
assertEquals(2, histogram.getBuckets().get(3).getDocCount());
|
||||
assertTrue(AggregationInspectionHelper.hasValue(histogram));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testRandomOffset() throws Exception {
|
||||
try (Directory dir = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir)) {
|
||||
// Note, these values are carefully chosen to ensure that no matter what offset we pick, no two can end up in the same bucket
|
||||
w.addDocument(singleton(getDocValue(FIELD_NAME, new double[] {3.2, 9.3})));
|
||||
w.addDocument(singleton(getDocValue(FIELD_NAME, new double[] {-5, 3.2 })));
|
||||
|
||||
final double offset = randomDouble();
|
||||
final double interval = 5;
|
||||
final double expectedOffset = offset % interval;
|
||||
HistogramAggregationBuilder aggBuilder = new HistogramAggregationBuilder("my_agg")
|
||||
.field(FIELD_NAME)
|
||||
.interval(interval)
|
||||
.offset(offset)
|
||||
.minDocCount(1);
|
||||
try (IndexReader reader = w.getReader()) {
|
||||
IndexSearcher searcher = new IndexSearcher(reader);
|
||||
InternalHistogram histogram = searchAndReduce(searcher, new MatchAllDocsQuery(), aggBuilder, defaultFieldType(FIELD_NAME));
|
||||
|
||||
assertEquals(3, histogram.getBuckets().size());
|
||||
assertEquals(-10 + expectedOffset, histogram.getBuckets().get(0).getKey());
|
||||
assertEquals(1, histogram.getBuckets().get(0).getDocCount());
|
||||
|
||||
assertEquals(expectedOffset, histogram.getBuckets().get(1).getKey());
|
||||
assertEquals(2, histogram.getBuckets().get(1).getDocCount());
|
||||
|
||||
assertEquals(5 + expectedOffset, histogram.getBuckets().get(2).getKey());
|
||||
assertEquals(1, histogram.getBuckets().get(2).getDocCount());
|
||||
|
||||
assertTrue(AggregationInspectionHelper.hasValue(histogram));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testExtendedBounds() throws Exception {
|
||||
try (Directory dir = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir)) {
|
||||
|
||||
w.addDocument(singleton(getDocValue(FIELD_NAME, new double[] {-4.5, 4.3})));
|
||||
w.addDocument(singleton(getDocValue(FIELD_NAME, new double[] {-5, 3.2 })));
|
||||
|
||||
HistogramAggregationBuilder aggBuilder = new HistogramAggregationBuilder("my_agg")
|
||||
.field(FIELD_NAME)
|
||||
.interval(5)
|
||||
.extendedBounds(-12, 13);
|
||||
try (IndexReader reader = w.getReader()) {
|
||||
IndexSearcher searcher = new IndexSearcher(reader);
|
||||
InternalHistogram histogram = searchAndReduce(searcher, new MatchAllDocsQuery(), aggBuilder, defaultFieldType(FIELD_NAME));
|
||||
assertEquals(6, histogram.getBuckets().size());
|
||||
assertEquals(-15d, histogram.getBuckets().get(0).getKey());
|
||||
assertEquals(0, histogram.getBuckets().get(0).getDocCount());
|
||||
assertEquals(-10d, histogram.getBuckets().get(1).getKey());
|
||||
assertEquals(0, histogram.getBuckets().get(1).getDocCount());
|
||||
assertEquals(-5d, histogram.getBuckets().get(2).getKey());
|
||||
assertEquals(2, histogram.getBuckets().get(2).getDocCount());
|
||||
assertEquals(0d, histogram.getBuckets().get(3).getKey());
|
||||
assertEquals(2, histogram.getBuckets().get(3).getDocCount());
|
||||
assertEquals(5d, histogram.getBuckets().get(4).getKey());
|
||||
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));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that sub-aggregations are not supported
|
||||
*/
|
||||
public void testSubAggs() throws Exception {
|
||||
try (Directory dir = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir)) {
|
||||
|
||||
w.addDocument(singleton(getDocValue(FIELD_NAME, new double[] {-4.5, 4.3})));
|
||||
w.addDocument(singleton(getDocValue(FIELD_NAME, new double[] {-5, 3.2 })));
|
||||
|
||||
HistogramAggregationBuilder aggBuilder = new HistogramAggregationBuilder("my_agg")
|
||||
.field(FIELD_NAME)
|
||||
.interval(5)
|
||||
.extendedBounds(-12, 13)
|
||||
.subAggregation(new TopHitsAggregationBuilder("top_hits"));
|
||||
try (IndexReader reader = w.getReader()) {
|
||||
IndexSearcher searcher = new IndexSearcher(reader);
|
||||
|
||||
IllegalArgumentException e = expectThrows(IllegalArgumentException.class,
|
||||
() -> searchAndReduce(searcher, new MatchAllDocsQuery(), aggBuilder, defaultFieldType(FIELD_NAME))
|
||||
);
|
||||
|
||||
assertEquals("Histogram aggregation on histogram fields does not support sub-aggregations", e.getMessage());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private BinaryDocValuesField getDocValue(String fieldName, double[] values) throws IOException {
|
||||
TDigest histogram = new TDigestState(100.0); //default
|
||||
for (double value : values) {
|
||||
histogram.add(value);
|
||||
}
|
||||
BytesStreamOutput streamOutput = new BytesStreamOutput();
|
||||
histogram.compress();
|
||||
Collection<Centroid> centroids = histogram.centroids();
|
||||
Iterator<Centroid> iterator = centroids.iterator();
|
||||
while ( iterator.hasNext()) {
|
||||
Centroid centroid = iterator.next();
|
||||
streamOutput.writeVInt(centroid.count());
|
||||
streamOutput.writeDouble(centroid.mean());
|
||||
}
|
||||
return new BinaryDocValuesField(fieldName, streamOutput.bytes().toBytesRef());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<SearchPlugin> getSearchPlugins() {
|
||||
return org.elasticsearch.common.collect.List.of(new AnalyticsPlugin(Settings.EMPTY));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AggregationBuilder createAggBuilderForTypeTest(MappedFieldType fieldType, String fieldName) {
|
||||
return new HistogramAggregationBuilder("_name").field(fieldName);
|
||||
}
|
||||
|
||||
private MappedFieldType defaultFieldType(String fieldName) {
|
||||
return new HistogramFieldMapper.HistogramFieldType(fieldName, true, Collections.emptyMap());
|
||||
}
|
||||
|
||||
}
|
|
@ -46,4 +46,24 @@ setup:
|
|||
- match: { aggregations.histo_avg.value: 0.3125}
|
||||
|
||||
|
||||
---
|
||||
"Histogram over histograms":
|
||||
|
||||
- do:
|
||||
search:
|
||||
index: "test"
|
||||
body:
|
||||
size: 0
|
||||
aggs:
|
||||
histo:
|
||||
histogram:
|
||||
field: "latency"
|
||||
interval: 0.3
|
||||
|
||||
|
||||
- match: { hits.total.value: 2 }
|
||||
- length: { aggregations.histo.buckets: 2 }
|
||||
- match: { aggregations.histo.buckets.0.key: 0.0 }
|
||||
- match: { aggregations.histo.buckets.0.doc_count: 20 }
|
||||
- match: { aggregations.histo.buckets.1.key: 0.3 }
|
||||
- match: { aggregations.histo.buckets.1.doc_count: 60 }
|
||||
|
|
Loading…
Reference in New Issue