1
0
Эх сурвалжийг харах

Add support for range aggregations on histogram mapped fields (#74146)

This adds support for the range aggregation over `histogram` mapped fields.

Decisions made for implementation:

 - Sub-aggregations are not allowed. This is to simplify implementation and follows the prior art set by the `histogram` aggregation
 - Nothing fancy is done with the ranges. No filter translations as we cannot easily do a `range` filter query against histogram fields. This may be an optimization in the future.
 - Ranges check the histogram value ONLY. No interpolation of values is done. If we have better statistics around the histogram this MAY be possible.
Benjamin Trent 4 жил өмнө
parent
commit
07b336f1b0

+ 100 - 0
docs/reference/aggregations/bucket/range-aggregation.asciidoc

@@ -324,3 +324,103 @@ Response:
 }
 --------------------------------------------------
 // TESTRESPONSE[s/\.\.\.//]
+[[search-aggregations-bucket-range-aggregation-histogram-fields]]
+==== Histogram fields
+
+Running a range aggregation over histogram fields computes the total number of counts for each configured range.
+
+This is done without interpolating between the histogram field values. Consequently, it is possible to have a range
+that is "in-between" two histogram values. The resulting range bucket would have a zero doc count.
+
+Here is an example, executing a range 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&filter_path=aggregations
+{
+  "aggs": {
+    "latency_ranges": {
+      "range": {
+        "field": "latency_histo",
+        "ranges": [
+          {"to": 2},
+          {"from": 2, "to": 3},
+          {"from": 3, "to": 10},
+          {"from": 10}
+        ]
+      }
+    }
+  }
+}
+--------------------------------------------------
+
+
+The `range` aggregation will sum the counts of each range computed based on the `values` and
+return the following output:
+
+[source,console-result]
+--------------------------------------------------
+{
+  "aggregations": {
+    "latency_ranges": {
+      "buckets": [
+        {
+          "key": "*-2.0",
+          "to": 2,
+          "doc_count": 11
+        },
+        {
+          "key": "2.0-3.0",
+          "from": 2,
+          "to": 3,
+          "doc_count": 0
+        },
+        {
+          "key": "3.0-10.0",
+          "from": 3,
+          "to": 10,
+          "doc_count": 55
+        },
+        {
+          "key": "10.0-*",
+          "from": 10,
+          "doc_count": 31
+        }
+      ]
+    }
+  }
+}
+--------------------------------------------------
+// TESTRESPONSE[skip:test not setup]
+
+[IMPORTANT]
+========
+Range 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 range 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 range aggregation over a histogram field, no sub-aggregations are allowed.**
+========

+ 1 - 0
docs/reference/mapping/types/histogram.asciidoc

@@ -44,6 +44,7 @@ following aggregations and queries:
 * <<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
+* <<search-aggregations-bucket-range-aggregation-histogram-fields,range>> aggregation
 * <<query-dsl-exists-query,exists>> query
 
 [[mapping-types-histogram-building-histogram]]

+ 62 - 28
server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregator.java

@@ -164,7 +164,7 @@ public abstract class RangeAggregator extends BucketsAggregator {
             return this.key;
         }
 
-        boolean matches(double value) {
+        public boolean matches(double value) {
             return value >= from && value < to;
         }
 
@@ -430,17 +430,17 @@ public abstract class RangeAggregator extends BucketsAggregator {
         );
     }
 
-    private final ValuesSource.Numeric valuesSource;
+    protected final ValuesSource valuesSource;
     private final DocValueFormat format;
     protected final Range[] ranges;
     private final boolean keyed;
     private final InternalRange.Factory rangeFactory;
     private final double averageDocsPerRange;
 
-    private RangeAggregator(
+    public RangeAggregator(
         String name,
         AggregatorFactories factories,
-        ValuesSource.Numeric valuesSource,
+        ValuesSource valuesSource,
         DocValueFormat format,
         InternalRange.Factory rangeFactory,
         Range[] ranges,
@@ -469,23 +469,6 @@ public abstract class RangeAggregator extends BucketsAggregator {
         return super.scoreMode();
     }
 
-    @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
-        final SortedNumericDoubleValues values = valuesSource.doubleValues(ctx);
-        return new LeafBucketCollectorBase(sub, values) {
-            @Override
-            public void collect(int doc, long bucket) throws IOException {
-                if (values.advanceExact(doc)) {
-                    final int valuesCount = values.docValueCount();
-                    for (int i = 0, lo = 0; i < valuesCount; ++i) {
-                        final double value = values.nextValue();
-                        lo = RangeAggregator.this.collect(sub, doc, value, bucket, lo);
-                    }
-                }
-            }
-        };
-    }
-
     protected long subBucketOrdinal(long owningBucketOrdinal, int rangeOrd) {
         return owningBucketOrdinal * ranges.length + rangeOrd;
     }
@@ -556,10 +539,61 @@ public abstract class RangeAggregator extends BucketsAggregator {
         }
     }
 
-    protected abstract int collect(LeafBucketCollector sub, int doc, double value, long owningBucketOrdinal, int lowBound)
-        throws IOException;
+    private abstract static class NumericRangeAggregator extends RangeAggregator {
+
+        NumericRangeAggregator(
+            String name,
+            AggregatorFactories factories,
+            ValuesSource.Numeric valuesSource,
+            DocValueFormat format,
+            Factory<?, ?> rangeFactory,
+            Range[] ranges,
+            double averageDocsPerRange,
+            boolean keyed,
+            AggregationContext context,
+            Aggregator parent,
+            CardinalityUpperBound cardinality,
+            Map<String, Object> metadata
+        ) throws IOException {
+            super(
+                name,
+                factories,
+                valuesSource,
+                format,
+                rangeFactory,
+                ranges,
+                averageDocsPerRange,
+                keyed,
+                context,
+                parent,
+                cardinality,
+                metadata
+            );
+        }
+
+        @Override
+        public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+            final SortedNumericDoubleValues values = ((ValuesSource.Numeric)this.valuesSource).doubleValues(ctx);
+            return new LeafBucketCollectorBase(sub, values) {
+                @Override
+                public void collect(int doc, long bucket) throws IOException {
+                    if (values.advanceExact(doc)) {
+                        final int valuesCount = values.docValueCount();
+                        for (int i = 0, lo = 0; i < valuesCount; ++i) {
+                            final double value = values.nextValue();
+                            lo = NumericRangeAggregator.this.collect(sub, doc, value, bucket, lo);
+                        }
+                    }
+                }
+            };
+        }
+
+        protected abstract int collect(LeafBucketCollector sub, int doc, double value, long owningBucketOrdinal, int lowBound)
+            throws IOException;
+    }
+
+    static class NoOverlap extends NumericRangeAggregator {
 
-    static class NoOverlap extends RangeAggregator {
         NoOverlap(
             String name,
             AggregatorFactories factories,
@@ -609,13 +643,13 @@ public abstract class RangeAggregator extends BucketsAggregator {
         }
     }
 
-    private static class Overlap extends RangeAggregator {
+    private static class Overlap extends NumericRangeAggregator {
         Overlap(
             String name,
             AggregatorFactories factories,
             Numeric valuesSource,
             DocValueFormat format,
-            Factory rangeFactory,
+            Factory<?, ?> rangeFactory,
             Range[] ranges,
             double averageDocsPerRange,
             boolean keyed,
@@ -690,7 +724,7 @@ public abstract class RangeAggregator extends BucketsAggregator {
 
             for (int i = startLo; i <= endHi; ++i) {
                 if (ranges[i].matches(value)) {
-                            collectBucket(sub, doc, subBucketOrdinal(owningBucketOrdinal, i));
+                    collectBucket(sub, doc, subBucketOrdinal(owningBucketOrdinal, i));
                 }
             }
 
@@ -759,7 +793,7 @@ public abstract class RangeAggregator extends BucketsAggregator {
         }
     }
 
-    private static boolean hasOverlap(Range[] ranges) {
+    public static boolean hasOverlap(Range[] ranges) {
         double lastEnd = ranges[0].to;
         for (int i = 1; i < ranges.length; ++i) {
             if (ranges[i].from < lastEnd) {

+ 2 - 1
x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/AnalyticsPlugin.java

@@ -161,7 +161,8 @@ public class AnalyticsPlugin extends Plugin implements SearchPlugin, ActionPlugi
             AnalyticsAggregatorFactory::registerHistoBackedAverageAggregator,
             AnalyticsAggregatorFactory::registerHistoBackedHistogramAggregator,
             AnalyticsAggregatorFactory::registerHistoBackedMinggregator,
-            AnalyticsAggregatorFactory::registerHistoBackedMaxggregator
+            AnalyticsAggregatorFactory::registerHistoBackedMaxggregator,
+            AnalyticsAggregatorFactory::registerHistoBackedRangeAggregator
         );
     }
 

+ 11 - 0
x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/AnalyticsAggregatorFactory.java

@@ -7,6 +7,7 @@
 package org.elasticsearch.xpack.analytics.aggregations;
 
 import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregationBuilder;
+import org.elasticsearch.search.aggregations.bucket.range.RangeAggregationBuilder;
 import org.elasticsearch.search.aggregations.metrics.AvgAggregationBuilder;
 import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder;
 import org.elasticsearch.search.aggregations.metrics.MinAggregationBuilder;
@@ -18,6 +19,7 @@ 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.bucket.range.HistoBackedRangeAggregator;
 import org.elasticsearch.xpack.analytics.aggregations.metrics.HistoBackedAvgAggregator;
 import org.elasticsearch.xpack.analytics.aggregations.metrics.HistoBackedHDRPercentileRanksAggregator;
 import org.elasticsearch.xpack.analytics.aggregations.metrics.HistoBackedHDRPercentilesAggregator;
@@ -101,4 +103,13 @@ public class AnalyticsAggregatorFactory {
         builder.register(MaxAggregationBuilder.REGISTRY_KEY, AnalyticsValuesSourceType.HISTOGRAM, HistoBackedMaxAggregator::new, true);
     }
 
+    public static void registerHistoBackedRangeAggregator(ValuesSourceRegistry.Builder builder) {
+        builder.register(
+            RangeAggregationBuilder.REGISTRY_KEY,
+            AnalyticsValuesSourceType.HISTOGRAM,
+            HistoBackedRangeAggregator::build,
+            true
+        );
+    }
+
 }

+ 313 - 0
x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/bucket/range/HistoBackedRangeAggregator.java

@@ -0,0 +1,313 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.analytics.aggregations.bucket.range;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.elasticsearch.index.fielddata.HistogramValue;
+import org.elasticsearch.index.fielddata.HistogramValues;
+import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.Aggregator;
+import org.elasticsearch.search.aggregations.AggregatorFactories;
+import org.elasticsearch.search.aggregations.CardinalityUpperBound;
+import org.elasticsearch.search.aggregations.LeafBucketCollector;
+import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
+import org.elasticsearch.search.aggregations.bucket.range.InternalRange;
+import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
+import org.elasticsearch.search.aggregations.support.AggregationContext;
+import org.elasticsearch.search.aggregations.support.ValuesSource;
+import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
+import org.elasticsearch.xpack.analytics.aggregations.support.HistogramValuesSource;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * Class for supporting range aggregation on histogram mapped fields
+ */
+public abstract class HistoBackedRangeAggregator extends RangeAggregator {
+
+    // TODO it would be good one day to possibly interpolate between ranges in the histogram fields
+    // If we knew the underlying data structure that created the histogram value, we could provide more accurate
+    // data counts for the ranges
+    public static HistoBackedRangeAggregator build(
+        String name,
+        AggregatorFactories factories,
+        ValuesSourceConfig valuesSourceConfig,
+        InternalRange.Factory<?, ?> rangeFactory,
+        RangeAggregator.Range[] ranges,
+        boolean keyed,
+        AggregationContext context,
+        Aggregator parent,
+        CardinalityUpperBound cardinality,
+        Map<String, Object> metadata
+    ) throws IOException {
+        final double avgRange = ((double) context.searcher().getIndexReader().maxDoc()) / ranges.length;
+        if (hasOverlap(ranges)) {
+            return new Overlap(
+                name,
+                factories,
+                valuesSourceConfig.getValuesSource(),
+                valuesSourceConfig.format(),
+                rangeFactory,
+                ranges,
+                avgRange,
+                keyed,
+                context,
+                parent,
+                cardinality,
+                metadata
+            );
+        }
+        return new NoOverlap(
+            name,
+            factories,
+            valuesSourceConfig.getValuesSource(),
+            valuesSourceConfig.format(),
+            rangeFactory,
+            ranges,
+            avgRange,
+            keyed,
+            context,
+            parent,
+            cardinality,
+            metadata
+        );
+    }
+
+    public HistoBackedRangeAggregator(
+        String name,
+        AggregatorFactories factories,
+        ValuesSource valuesSource,
+        DocValueFormat format,
+        InternalRange.Factory<?, ?> rangeFactory,
+        Range[] ranges,
+        double averageDocsPerRange,
+        boolean keyed,
+        AggregationContext context,
+        Aggregator parent,
+        CardinalityUpperBound cardinality,
+        Map<String, Object> metadata
+    ) throws IOException {
+        super(
+            name,
+            factories,
+            valuesSource,
+            format,
+            rangeFactory,
+            ranges,
+            averageDocsPerRange,
+            keyed,
+            context,
+            parent,
+            cardinality,
+            metadata
+        );
+        if (subAggregators().length > 0) {
+            throw new IllegalArgumentException("Range aggregation on histogram fields does not support sub-aggregations");
+        }
+    }
+
+    @Override
+    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+        if ((valuesSource instanceof HistogramValuesSource.Histogram) == false) {
+            return LeafBucketCollector.NO_OP_COLLECTOR;
+        }
+        final HistogramValuesSource.Histogram valuesSource = (HistogramValuesSource.Histogram)this.valuesSource;
+        final HistogramValues values = valuesSource.getHistogramValues(ctx);
+        return new LeafBucketCollectorBase(sub, values) {
+            @Override
+            public void collect(int doc, long bucket) throws IOException {
+                if (values.advanceExact(doc)) {
+                    final HistogramValue sketch = values.histogram();
+                    double previousValue = Double.NEGATIVE_INFINITY;
+                    int lo = 0;
+                    // Histogram values are expected to be in ascending order.
+                    while (sketch.next()) {
+                        final double value = sketch.value();
+                        assert previousValue <= value : "histogram field [" + name + "] unexpectedly out of order";
+                        previousValue = value;
+                        // Collecting the bucket automatically increments the count by the docCountProvider,
+                        // account for that here
+                        final int count = sketch.count() - docCountProvider.getDocCount(doc);
+                        lo = HistoBackedRangeAggregator.this.collect(sub, doc, value, bucket, lo, count);
+                    }
+                }
+            }
+        };
+    }
+
+    abstract int collect(
+        LeafBucketCollector sub,
+        int doc,
+        double value,
+        long owningBucketOrdinal,
+        int lowBound,
+        int count
+    ) throws IOException;
+
+    private static class NoOverlap extends HistoBackedRangeAggregator {
+
+        private NoOverlap(
+            String name,
+            AggregatorFactories factories,
+            ValuesSource valuesSource,
+            DocValueFormat format,
+            InternalRange.Factory<?, ?> rangeFactory,
+            Range[] ranges,
+            double averageDocsPerRange,
+            boolean keyed,
+            AggregationContext context,
+            Aggregator parent,
+            CardinalityUpperBound cardinality,
+            Map<String, Object> metadata
+        ) throws IOException {
+            super(
+                name,
+                factories,
+                valuesSource,
+                format,
+                rangeFactory,
+                ranges,
+                averageDocsPerRange,
+                keyed,
+                context,
+                parent,
+                cardinality,
+                metadata
+            );
+        }
+
+        @Override
+        public int collect(
+            LeafBucketCollector sub,
+            int doc,
+            double value,
+            long owningBucketOrdinal,
+            int lowBound,
+            int count
+        ) throws IOException {
+            int lo = lowBound, hi = ranges.length - 1;
+            while (lo <= hi) {
+                final int mid = (lo + hi) >>> 1;
+                if (value < ranges[mid].getFrom()) {
+                    hi = mid - 1;
+                } else if (value >= ranges[mid].getTo()) {
+                    lo = mid + 1;
+                } else {
+                    long bucketOrd = subBucketOrdinal(owningBucketOrdinal, mid);
+                    collectBucket(sub, doc, bucketOrd);
+                    incrementBucketDocCount(bucketOrd, count);
+                    // It could be that multiple histogram values fall in the same range
+                    // So, don't increment the final mid here to catch those values
+                    return mid;
+                }
+            }
+            return lo;
+        }
+    }
+
+    private static class Overlap extends HistoBackedRangeAggregator {
+
+        private final double[] maxTo;
+        Overlap(
+            String name,
+            AggregatorFactories factories,
+            ValuesSource valuesSource,
+            DocValueFormat format,
+            InternalRange.Factory<?, ?> rangeFactory,
+            Range[] ranges,
+            double averageDocsPerRange,
+            boolean keyed,
+            AggregationContext context,
+            Aggregator parent,
+            CardinalityUpperBound cardinality,
+            Map<String, Object> metadata
+        ) throws IOException {
+            super(
+                name,
+                factories,
+                valuesSource,
+                format,
+                rangeFactory,
+                ranges,
+                averageDocsPerRange,
+                keyed,
+                context,
+                parent,
+                cardinality,
+                metadata
+            );
+            maxTo = new double[ranges.length];
+            maxTo[0] = ranges[0].getTo();
+            for (int i = 1; i < ranges.length; ++i) {
+                maxTo[i] = Math.max(ranges[i].getTo(), maxTo[i - 1]);
+            }
+        }
+
+        @Override
+        public int collect(
+            LeafBucketCollector sub,
+            int doc,
+            double value,
+            long owningBucketOrdinal,
+            int lowBound,
+            int count
+        ) throws IOException {
+            int lo = lowBound, hi = ranges.length - 1; // all candidates are between these indexes
+            int mid = (lo + hi) >>> 1;
+            while (lo <= hi) {
+                if (value < ranges[mid].getFrom()) {
+                    hi = mid - 1;
+                } else if (value >= maxTo[mid]) {
+                    lo = mid + 1;
+                } else {
+                    break;
+                }
+                mid = (lo + hi) >>> 1;
+            }
+            // No candidate range found, return current lo
+            if (lo > hi) return lo;
+
+            // binary search the lower bound
+            int startLo = lo, startHi = mid;
+            while (startLo <= startHi) {
+                final int startMid = (startLo + startHi) >>> 1;
+                if (value >= maxTo[startMid]) {
+                    startLo = startMid + 1;
+                } else {
+                    startHi = startMid - 1;
+                }
+            }
+
+            // binary search the upper bound
+            int endLo = mid, endHi = hi;
+            while (endLo <= endHi) {
+                final int endMid = (endLo + endHi) >>> 1;
+                if (value < ranges[endMid].getFrom()) {
+                    endHi = endMid - 1;
+                } else {
+                    endLo = endMid + 1;
+                }
+            }
+
+            assert startLo == lowBound || value >= maxTo[startLo - 1];
+            assert endHi == ranges.length - 1 || value < ranges[endHi + 1].getFrom();
+
+            for (int i = startLo; i <= endHi; ++i) {
+                if (ranges[i].matches(value)) {
+                    long bucketOrd = subBucketOrdinal(owningBucketOrdinal, i);
+                    collectBucket(sub, doc, bucketOrd);
+                    incrementBucketDocCount(bucketOrd, count);
+                }
+            }
+            // It could be that multiple histogram values fall in the same range
+            // So, return the bottom part of the search
+            return startLo;
+        }
+    }
+}

+ 26 - 0
x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/AnalyticsTestsUtils.java

@@ -10,6 +10,8 @@ package org.elasticsearch.xpack.analytics;
 
 import java.io.IOException;
 
+import org.HdrHistogram.DoubleHistogram;
+import org.HdrHistogram.DoubleHistogramIterationValue;
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.elasticsearch.common.io.stream.BytesStreamOutput;
 import org.elasticsearch.search.aggregations.metrics.TDigestState;
@@ -36,4 +38,28 @@ public final class AnalyticsTestsUtils {
         return new BinaryDocValuesField(fieldName, streamOutput.bytes().toBytesRef());
     }
 
+    public static BinaryDocValuesField hdrHistogramFieldDocValues(String fieldName, double[] values) throws IOException {
+        DoubleHistogram histogram = new DoubleHistogram(3);
+        histogram.setAutoResize(true);
+        for (double value: values) {
+            histogram.recordValue(value);
+        }
+        BytesStreamOutput streamOutput = new BytesStreamOutput();
+        for(DoubleHistogramIterationValue value : histogram.recordedValues()) {
+            streamOutput.writeVInt((int)value.getCountAtValueIteratedTo());
+            streamOutput.writeDouble(value.getValueIteratedTo());
+        }
+        return new BinaryDocValuesField(fieldName, streamOutput.bytes().toBytesRef());
+    }
+
+    public static BinaryDocValuesField histogramFieldDocValues(String fieldName, double[] values, int[] counts) throws IOException {
+        assert values.length == counts.length;
+        BytesStreamOutput streamOutput = new BytesStreamOutput();
+        for (int i = 0; i < values.length; i++) {
+            streamOutput.writeVInt(counts[i]);
+            streamOutput.writeDouble(values[i]);
+        }
+        return new BinaryDocValuesField(fieldName, streamOutput.bytes().toBytesRef());
+    }
+
 }

+ 382 - 0
x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/aggregations/bucket/range/HistoBackedRangeAggregatorTests.java

@@ -0,0 +1,382 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.analytics.aggregations.bucket.range;
+
+import org.apache.lucene.document.DoubleDocValuesField;
+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.index.mapper.CustomTermFreqField;
+import org.elasticsearch.index.mapper.MappedFieldType;
+import org.elasticsearch.index.mapper.NumberFieldMapper;
+import org.elasticsearch.plugins.SearchPlugin;
+import org.elasticsearch.search.aggregations.AggregationBuilder;
+import org.elasticsearch.search.aggregations.AggregatorTestCase;
+import org.elasticsearch.search.aggregations.bucket.range.InternalRange;
+import org.elasticsearch.search.aggregations.bucket.range.RangeAggregationBuilder;
+import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
+import org.elasticsearch.search.aggregations.metrics.Percentiles;
+import org.elasticsearch.search.aggregations.metrics.PercentilesAggregationBuilder;
+import org.elasticsearch.search.aggregations.metrics.PercentilesConfig;
+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.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.IntStream;
+
+import static java.util.Collections.singleton;
+import static org.elasticsearch.xpack.analytics.AnalyticsTestsUtils.hdrHistogramFieldDocValues;
+import static org.elasticsearch.xpack.analytics.AnalyticsTestsUtils.histogramFieldDocValues;
+import static org.hamcrest.Matchers.lessThan;
+
+public class HistoBackedRangeAggregatorTests extends AggregatorTestCase {
+
+    private static final String HISTO_FIELD_NAME = "histo_field";
+    private static final String RAW_FIELD_NAME = "raw_field";
+
+    @SuppressWarnings("rawtypes")
+    public void testPercentilesAccuracy() throws Exception {
+        long absError = 0L;
+        long docCount = 0L;
+        for (int k = 0; k < 10; k++) {
+            try (Directory dir = newDirectory();
+                 RandomIndexWriter w = new RandomIndexWriter(random(), dir)) {
+                docCount += generateDocs(w);
+                double[] steps = IntStream.range(2, 99)
+                    .filter(i -> i % 2 == 0)
+                    .mapToDouble(Double::valueOf)
+                    .toArray();
+
+                PercentilesAggregationBuilder rawPercentiles = new PercentilesAggregationBuilder("my_agg")
+                    .field(RAW_FIELD_NAME)
+                    .percentilesConfig(new PercentilesConfig.Hdr())
+                    .percentiles(steps);
+
+                PercentilesAggregationBuilder aggregatedPercentiles = new PercentilesAggregationBuilder("my_agg")
+                    .field(HISTO_FIELD_NAME)
+                    .percentilesConfig(new PercentilesConfig.Hdr())
+                    .percentiles(steps);
+
+                try (IndexReader reader = w.getReader()) {
+                    IndexSearcher searcher = new IndexSearcher(reader);
+                    RangeAggregationBuilder aggBuilder = new RangeAggregationBuilder("my_agg")
+                        .field(HISTO_FIELD_NAME);
+
+                    RangeAggregationBuilder rawFieldAgg = new RangeAggregationBuilder("my_agg")
+                        .field(RAW_FIELD_NAME);
+                    Percentiles rawPercentileResults = searchAndReduce(
+                        searcher,
+                        new MatchAllDocsQuery(),
+                        rawPercentiles,
+                        defaultFieldType(RAW_FIELD_NAME)
+                    );
+                    Percentiles aggregatedPercentileResults = searchAndReduce(
+                        searcher,
+                        new MatchAllDocsQuery(),
+                        aggregatedPercentiles,
+                        defaultFieldType(HISTO_FIELD_NAME)
+                    );
+                    aggBuilder.addUnboundedTo(aggregatedPercentileResults.percentile(steps[0]));
+                    rawFieldAgg.addUnboundedTo(rawPercentileResults.percentile(steps[0]));
+
+                    for (int i = 1; i < steps.length; i++) {
+                        aggBuilder.addRange(
+                            aggregatedPercentileResults.percentile(steps[i - 1]),
+                            aggregatedPercentileResults.percentile(steps[i]));
+                        rawFieldAgg.addRange(rawPercentileResults.percentile(steps[i - 1]), rawPercentileResults.percentile(steps[i]));
+                    }
+                    aggBuilder.addUnboundedFrom(aggregatedPercentileResults.percentile(steps[steps.length - 1]));
+                    rawFieldAgg.addUnboundedFrom(rawPercentileResults.percentile(steps[steps.length - 1]));
+
+                    InternalRange<? extends InternalRange.Bucket, ? extends InternalRange> range = searchAndReduce(
+                        searcher,
+                        new MatchAllDocsQuery(),
+                        aggBuilder,
+                        defaultFieldType(HISTO_FIELD_NAME)
+                    );
+                    InternalRange<? extends InternalRange.Bucket, ? extends InternalRange> rawRange = searchAndReduce(
+                        searcher,
+                        new MatchAllDocsQuery(),
+                        rawFieldAgg,
+                        defaultFieldType(RAW_FIELD_NAME)
+                    );
+                    for (int j = 0; j < rawRange.getBuckets().size(); j++) {
+                        absError += Math.abs(range.getBuckets().get(j).getDocCount() - rawRange.getBuckets().get(j).getDocCount());
+                    }
+                }
+            }
+        }
+        assertThat((double)absError/docCount, lessThan(0.1));
+    }
+
+    @SuppressWarnings("rawtypes")
+    public void testMediumRangesAccuracy() throws Exception {
+        List<RangeAggregator.Range> ranges = Arrays.asList(
+        new RangeAggregator.Range(null, null, 2.0),
+        new RangeAggregator.Range(null, 2.0, 4.0),
+        new RangeAggregator.Range(null, 4.0, 6.0),
+        new RangeAggregator.Range(null, 6.0, 8.0),
+        new RangeAggregator.Range(null, 8.0, 9.0),
+        new RangeAggregator.Range(null, 8.0, 11.0),
+        new RangeAggregator.Range(null, 11.0, 12.0),
+        new RangeAggregator.Range(null, 12.0, null));
+        testRanges(ranges, "manual_medium_ranges");
+    }
+
+    public void testLargerRangesAccuracy() throws Exception {
+        List<RangeAggregator.Range> ranges = Arrays.asList(
+            new RangeAggregator.Range(null, null, 8.0),
+            new RangeAggregator.Range(null, 8.0, 12.0),
+            new RangeAggregator.Range(null, 12.0, null));
+        testRanges(ranges, "manual_big_ranges");
+    }
+
+    public void testSmallerRangesAccuracy() throws Exception {
+        List<RangeAggregator.Range> ranges = Arrays.asList(
+            new RangeAggregator.Range(null, null, 1.0),
+            new RangeAggregator.Range(null, 1.0, 1.5),
+            new RangeAggregator.Range(null, 1.5, 2.0),
+            new RangeAggregator.Range(null, 2.0, 2.5),
+            new RangeAggregator.Range(null, 2.5, 3.0),
+            new RangeAggregator.Range(null, 3.0, 3.5),
+            new RangeAggregator.Range(null, 3.5, 4.0),
+            new RangeAggregator.Range(null, 4.0, 4.5),
+            new RangeAggregator.Range(null, 4.5, 5.0),
+            new RangeAggregator.Range(null, 5.0, 5.5),
+            new RangeAggregator.Range(null, 5.5, 6.0),
+            new RangeAggregator.Range(null, 6.0, 6.5),
+            new RangeAggregator.Range(null, 6.5, 7.0),
+            new RangeAggregator.Range(null, 7.0, 7.5),
+            new RangeAggregator.Range(null, 7.5, 8.0),
+            new RangeAggregator.Range(null, 8.0, 8.5),
+            new RangeAggregator.Range(null, 8.5, 9.0),
+            new RangeAggregator.Range(null, 9.0, 9.5),
+            new RangeAggregator.Range(null, 9.5, 10.0),
+            new RangeAggregator.Range(null, 10.0, 10.5),
+            new RangeAggregator.Range(null, 10.5, 11.0),
+            new RangeAggregator.Range(null, 11.0, 11.5),
+            new RangeAggregator.Range(null, 11.5, 12.0),
+            new RangeAggregator.Range(null, 12.0, null)
+            );
+        testRanges(ranges, "manual_small_ranges");
+    }
+
+    @SuppressWarnings("rawtypes")
+    private void testRanges(List<RangeAggregator.Range> ranges, String name) throws Exception {
+        long absError = 0L;
+        long docCount = 0L;
+        for (int k = 0; k < 10; k++) {
+            try (Directory dir = newDirectory();
+                 RandomIndexWriter w = new RandomIndexWriter(random(), dir)) {
+                docCount += generateDocs(w);
+
+                try (IndexReader reader = w.getReader()) {
+                    IndexSearcher searcher = new IndexSearcher(reader);
+                    RangeAggregationBuilder aggBuilder = new RangeAggregationBuilder("my_agg")
+                        .field(HISTO_FIELD_NAME);
+                    RangeAggregationBuilder rawFieldAgg = new RangeAggregationBuilder("my_agg")
+                        .field(RAW_FIELD_NAME);
+                    ranges.forEach(r -> {
+                        aggBuilder.addRange(r);
+                        rawFieldAgg.addRange(r);
+                    });
+
+                    InternalRange<? extends InternalRange.Bucket, ? extends InternalRange> range = searchAndReduce(
+                        searcher,
+                        new MatchAllDocsQuery(),
+                        aggBuilder,
+                        defaultFieldType(HISTO_FIELD_NAME)
+                    );
+                    InternalRange<? extends InternalRange.Bucket, ? extends InternalRange> rawRange = searchAndReduce(
+                        searcher,
+                        new MatchAllDocsQuery(),
+                        rawFieldAgg,
+                        defaultFieldType(RAW_FIELD_NAME)
+                    );
+                    for (int j = 0; j < rawRange.getBuckets().size(); j++) {
+                        absError += Math.abs(range.getBuckets().get(j).getDocCount() - rawRange.getBuckets().get(j).getDocCount());
+                    }
+                }
+            }
+        }
+        assertThat("test " + name, (double)absError/docCount, lessThan(0.1));
+    }
+
+    @SuppressWarnings("rawtypes")
+    public void testOverlapping() throws Exception {
+        try (Directory dir = newDirectory();
+             RandomIndexWriter w = new RandomIndexWriter(random(), dir)) {
+            w.addDocument(Arrays.asList(
+                histogramFieldDocValues(HISTO_FIELD_NAME, new double[] {0, 1.2, 10, 12, 24}, new int[] {3, 1, 2, 4, 6}),
+                new CustomTermFreqField("_doc_count", "_doc_count", 16))
+            );
+            w.addDocument(Arrays.asList(
+                histogramFieldDocValues(HISTO_FIELD_NAME, new double[] {5.3, 6, 6, 20}, new int[] {1, 3, 4, 5}),
+                new CustomTermFreqField("_doc_count", "_doc_count", 13))
+            );
+            w.addDocument(Arrays.asList(
+                histogramFieldDocValues(HISTO_FIELD_NAME, new double[] {-10, 0.01, 10, 10, 30}, new int[] {10, 2, 4, 14, 11}),
+                new CustomTermFreqField("_doc_count", "_doc_count", 41))
+            );
+
+            RangeAggregationBuilder aggBuilder = new RangeAggregationBuilder("my_agg")
+                .field(HISTO_FIELD_NAME)
+                .addUnboundedTo(0)
+                .addRange(5, 10)
+                .addRange(7, 10)
+                .addRange(0, 20)
+                .addRange(0, 10)
+                .addRange(10, 20)
+                .addUnboundedFrom(20);
+            try (IndexReader reader = w.getReader()) {
+                IndexSearcher searcher = new IndexSearcher(reader);
+                InternalRange<? extends InternalRange.Bucket, ? extends InternalRange> range = searchAndReduce(
+                    searcher,
+                    new MatchAllDocsQuery(),
+                    aggBuilder,
+                    defaultFieldType(HISTO_FIELD_NAME)
+                );
+                assertTrue(AggregationInspectionHelper.hasValue(range));
+                assertEquals(7, range.getBuckets().size());
+
+                assertEquals(10, range.getBuckets().get(0).getDocCount());
+                assertEquals("*-0.0", range.getBuckets().get(0).getKey());
+
+                assertEquals(14, range.getBuckets().get(1).getDocCount());
+                assertEquals("0.0-10.0", range.getBuckets().get(1).getKey());
+
+                assertEquals(38, range.getBuckets().get(2).getDocCount());
+                assertEquals("0.0-20.0", range.getBuckets().get(2).getKey());
+
+                assertEquals(8, range.getBuckets().get(3).getDocCount());
+                assertEquals("5.0-10.0", range.getBuckets().get(3).getKey());
+
+                assertEquals(0, range.getBuckets().get(4).getDocCount());
+                assertEquals("7.0-10.0", range.getBuckets().get(4).getKey());
+
+                assertEquals(24, range.getBuckets().get(5).getDocCount());
+                assertEquals("10.0-20.0", range.getBuckets().get(5).getKey());
+
+                assertEquals(22, range.getBuckets().get(6).getDocCount());
+                assertEquals("20.0-*", range.getBuckets().get(6).getKey());
+            }
+        }
+    }
+
+    @SuppressWarnings("rawtypes")
+    public void testNonOverlapping() throws Exception {
+        try (Directory dir = newDirectory();
+             RandomIndexWriter w = new RandomIndexWriter(random(), dir)) {
+            w.addDocument(Arrays.asList(
+                histogramFieldDocValues(HISTO_FIELD_NAME, new double[] {0, 1.2, 10, 12, 24}, new int[] {3, 1, 2, 4, 6}),
+                new CustomTermFreqField("_doc_count", "_doc_count", 16))
+            );
+            w.addDocument(Arrays.asList(
+                histogramFieldDocValues(HISTO_FIELD_NAME, new double[] {5.3, 6, 6, 20}, new int[] {1, 3, 4, 5}),
+                new CustomTermFreqField("_doc_count", "_doc_count", 13))
+            );
+            w.addDocument(Arrays.asList(
+                histogramFieldDocValues(HISTO_FIELD_NAME, new double[] {-10, 0.01, 10, 10, 30}, new int[] {10, 2, 4, 14, 11}),
+                new CustomTermFreqField("_doc_count", "_doc_count", 41))
+            );
+
+            RangeAggregationBuilder aggBuilder = new RangeAggregationBuilder("my_agg")
+                .field(HISTO_FIELD_NAME)
+                .addUnboundedTo(0)
+                .addRange(0, 10)
+                .addRange(10, 20)
+                .addUnboundedFrom(20);
+            try (IndexReader reader = w.getReader()) {
+                IndexSearcher searcher = new IndexSearcher(reader);
+                InternalRange<? extends InternalRange.Bucket, ? extends InternalRange> range = searchAndReduce(
+                    searcher,
+                    new MatchAllDocsQuery(),
+                    aggBuilder,
+                    defaultFieldType(HISTO_FIELD_NAME)
+                );
+                assertTrue(AggregationInspectionHelper.hasValue(range));
+                assertEquals(4, range.getBuckets().size());
+
+                assertEquals(10, range.getBuckets().get(0).getDocCount());
+                assertEquals("*-0.0", range.getBuckets().get(0).getKey());
+
+                assertEquals(14, range.getBuckets().get(1).getDocCount());
+                assertEquals("0.0-10.0", range.getBuckets().get(1).getKey());
+
+                assertEquals(24, range.getBuckets().get(2).getDocCount());
+                assertEquals("10.0-20.0", range.getBuckets().get(2).getKey());
+
+                assertEquals(22, range.getBuckets().get(3).getDocCount());
+                assertEquals("20.0-*", range.getBuckets().get(3).getKey());
+            }
+        }
+    }
+
+    public void testSubAggs() throws Exception {
+        try (Directory dir = newDirectory();
+             RandomIndexWriter w = new RandomIndexWriter(random(), dir)) {
+
+            w.addDocument(singleton(histogramFieldDocValues(HISTO_FIELD_NAME, new double[] {-4.5, 4.3})));
+            w.addDocument(singleton(histogramFieldDocValues(HISTO_FIELD_NAME, new double[] {-5, 3.2 })));
+
+            RangeAggregationBuilder aggBuilder = new RangeAggregationBuilder("my_agg")
+                .field(HISTO_FIELD_NAME)
+                .addRange(-1.0, 3.0)
+                .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(HISTO_FIELD_NAME))
+                );
+                assertEquals("Range aggregation on histogram fields does not support sub-aggregations", e.getMessage());
+            }
+        }
+    }
+
+    private long generateDocs(RandomIndexWriter w) throws Exception {
+        double[] lows = new double[50];
+        double[] mids = new double[50];
+        double[] highs = new double[50];
+        for (int j = 0; j < 50; j++) {
+            lows[j] = randomDoubleBetween(0.0, 5.0, true);
+            mids[j] = randomDoubleBetween(7.0, 9.0, false);
+            highs[j] = randomDoubleBetween(10.0, 13.0, false);
+            w.addDocument(singleton(new DoubleDocValuesField(RAW_FIELD_NAME, lows[j])));
+            w.addDocument(singleton(new DoubleDocValuesField(RAW_FIELD_NAME, mids[j])));
+            w.addDocument(singleton(new DoubleDocValuesField(RAW_FIELD_NAME, highs[j])));
+        }
+        w.addDocument(singleton(hdrHistogramFieldDocValues(HISTO_FIELD_NAME, lows)));
+        w.addDocument(singleton(hdrHistogramFieldDocValues(HISTO_FIELD_NAME, mids)));
+        w.addDocument(singleton(hdrHistogramFieldDocValues(HISTO_FIELD_NAME, highs)));
+        return 150;
+    }
+
+    @Override
+    protected List<SearchPlugin> getSearchPlugins() {
+        return List.of(new AnalyticsPlugin());
+    }
+
+    @Override
+    protected AggregationBuilder createAggBuilderForTypeTest(MappedFieldType fieldType, String fieldName) {
+        return new RangeAggregationBuilder("_name").field(fieldName);
+    }
+
+    private MappedFieldType defaultFieldType(String fieldName) {
+        if (fieldName.equals(HISTO_FIELD_NAME)) {
+            return new HistogramFieldMapper.HistogramFieldType(fieldName, Collections.emptyMap());
+        } else {
+            return new NumberFieldMapper.NumberFieldType(fieldName, NumberFieldMapper.NumberType.DOUBLE);
+        }
+    }
+}

+ 1 - 0
x-pack/plugin/build.gradle

@@ -123,6 +123,7 @@ tasks.named("yamlRestCompatTest").configure {
 
   //TODO: blacklist specific to REST API compatibility
   restTestBlacklist.addAll([
+    'analytics/histogram/Ranges over histogram',
     'indices.freeze/10_basic/Basic',
     'indices.freeze/10_basic/Test index options',
     'indices.freeze/20_stats/Translog stats on frozen indices',

+ 75 - 21
x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/analytics/histogram.yml

@@ -20,6 +20,25 @@ setup:
           - '{"latency": {"values" : [0.1, 0.2, 0.3, 0.4, 0.5], "counts" : [3, 7, 23, 12, 6]}}'
           - '{"index": {}}'
           - '{"latency": {"values" : [0, 0.1, 0.2, 0.3, 0.4, 0.5], "counts" : [3, 2, 5, 10, 1, 8]}}'
+  - do:
+      indices.create:
+        index: "histo_with_doc_count"
+        body:
+          mappings:
+            properties:
+              latency:
+                type: "histogram"
+  - do:
+      headers:
+        Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
+      bulk:
+        index: "histo_with_doc_count"
+        refresh: true
+        body:
+          - '{"index": {}}'
+          - '{"_doc_count": 50, "latency": {"values" : [0.1, 0.2, 0.3, 0.4, 0.5], "counts" : [3, 7, 23, 12, 5]}}'
+          - '{"index": {}}'
+          - '{"_doc_count": 10, "latency": {"values" : [0.1, 0.2, 0.3, 0.4, 0.5], "counts" : [1, 1, 1, 1, 6]}}'
 ---
 "Histogram requires values in increasing order":
   - do:
@@ -59,7 +78,6 @@ setup:
   - match: { aggregations.histo_min.value:  0}
   - match: { aggregations.histo_max.value:  0.5}
 
-
 ---
 "Histogram over histograms":
 
@@ -80,28 +98,8 @@ setup:
   - match: { aggregations.histo.buckets.0.doc_count: 20 }
   - match: { aggregations.histo.buckets.1.key: 0.3 }
   - match: { aggregations.histo.buckets.1.doc_count: 60 }
-
 ---
 "Histogram with _doc_count":
-  - do:
-      indices.create:
-        index: "histo_with_doc_count"
-        body:
-          mappings:
-            properties:
-              latency:
-                type: "histogram"
-  - do:
-      headers:
-        Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
-      bulk:
-        index: "histo_with_doc_count"
-        refresh: true
-        body:
-          - '{"index": {}}'
-          - '{"_doc_count": 50, "latency": {"values" : [0.1, 0.2, 0.3, 0.4, 0.5], "counts" : [3, 7, 23, 12, 5]}}'
-          - '{"index": {}}'
-          - '{"_doc_count": 10, "latency": {"values" : [0.1, 0.2, 0.3, 0.4, 0.5], "counts" : [1, 1, 1, 1, 6]}}'
   - do:
       search:
         index: "histo_with_doc_count"
@@ -117,3 +115,59 @@ setup:
   - length: { aggregations.histo.buckets: 1 }
   - match: { aggregations.histo.buckets.0.key: 0.0 }
   - match: { aggregations.histo.buckets.0.doc_count: 60 }
+
+---
+"Ranges over histogram":
+  - do:
+      search:
+        index: "test"
+        body:
+          size: 0
+          aggs:
+            ranges:
+              range:
+                field: "latency"
+                ranges:
+                  - {to: 0}
+                  - {from: 0, to: 0.3}
+                  - {from: 0.3, to: 0.5}
+                  - {from: 0.5}
+
+  - match: { hits.total.value: 2 }
+  - length: { aggregations.ranges.buckets: 4 }
+  - match: { aggregations.ranges.buckets.0.key: "*-0.0" }
+  - match: { aggregations.ranges.buckets.0.doc_count: 0 }
+  - match: { aggregations.ranges.buckets.1.key: "0.0-0.3" }
+  - match: { aggregations.ranges.buckets.1.doc_count: 20 }
+  - match: { aggregations.ranges.buckets.2.key: "0.3-0.5" }
+  - match: { aggregations.ranges.buckets.2.doc_count: 46 }
+  - match: { aggregations.ranges.buckets.3.key: "0.5-*" }
+  - match: { aggregations.ranges.buckets.3.doc_count: 14 }
+
+---
+"Ranges over histogram with doc_count":
+  - do:
+      search:
+        index: "histo_with_doc_count"
+        body:
+          size: 0
+          aggs:
+            ranges:
+              range:
+                field: "latency"
+                ranges:
+                  - {to: 0}
+                  - {from: 0, to: 0.3}
+                  - {from: 0.3, to: 0.5}
+                  - {from: 0.5}
+
+  - match: { hits.total.value: 2 }
+  - length: { aggregations.ranges.buckets: 4 }
+  - match: { aggregations.ranges.buckets.0.key: "*-0.0" }
+  - match: { aggregations.ranges.buckets.0.doc_count: 0 }
+  - match: { aggregations.ranges.buckets.1.key: "0.0-0.3" }
+  - match: { aggregations.ranges.buckets.1.doc_count: 12 }
+  - match: { aggregations.ranges.buckets.2.key: "0.3-0.5" }
+  - match: { aggregations.ranges.buckets.2.doc_count: 37 }
+  - match: { aggregations.ranges.buckets.3.key: "0.5-*" }
+  - match: { aggregations.ranges.buckets.3.doc_count: 11 }