Преглед изворни кода

Added extended_bounds support for date_/histogram aggs

By default the date_/histogram returns all the buckets within the range of the data itself, that is, the documents with the smallest values (on which with histogram) will determine the min bucket (the bucket with the smallest key) and the documents with the highest values will determine the max bucket (the bucket with the highest key). Often, when when requesting empty buckets (min_doc_count : 0), this causes a confusion, specifically, when the data is also filtered.

To understand why, let's look at an example:

Lets say the you're filtering your request to get all docs from the last month, and in the date_histogram aggs you'd like to slice the data per day. You also specify min_doc_count:0 so that you'd still get empty buckets for those days to which no document belongs. By default, if the first document that fall in this last month also happen to fall on the first day of the **second week** of the month, the date_histogram will **not** return empty buckets for all those days prior to that second week. The reason for that is that by default the histogram aggregations only start building buckets when they encounter documents (hence, missing on all the days of the first week in our example).

With extended_bounds, you now can "force" the histogram aggregations to start building buckets on a specific min values and also keep on building buckets up to a max value (even if there are no documents anymore). Using extended_bounds only makes sense when min_doc_count is 0 (the empty buckets will never be returned if the min_doc_count is greater than 0).

Note that (as the name suggest) extended_bounds is **not** filtering buckets. Meaning, if the min bounds is higher than the values extracted from the documents, the documents will still dictate what the min bucket will be (and the same goes to the extended_bounds.max and the max bucket). For filtering buckets, one should nest the histogram agg under a range filter agg with the appropriate min/max.

Closes #5224
uboness пре 11 година
родитељ
комит
7d6ad8d91c

+ 4 - 2
docs/reference/search/aggregations/bucket/datehistogram-aggregation.asciidoc

@@ -131,6 +131,8 @@ Response:
 --------------------------------------------------
 
 Like with the normal <<search-aggregations-bucket-histogram-aggregation,histogram>>, both document level scripts and
-value level scripts are supported. It is also possilbe to control the order of the returned buckets using the `order`
+value level scripts are supported. It is also possible to control the order of the returned buckets using the `order`
 settings and filter the returned buckets based on a `min_doc_count` setting (by defaults to all buckets with
-`min_doc_count > 1` will be returned).
+`min_doc_count > 0` will be returned). This histogram also supports the `extended_bounds` settings, that enables extending
+the bounds of the histogram beyond the data itself (to read more on why you'd want to do that please refer to the
+explanation <<search-aggregations-bucket-histogram-aggregation-extended-bounds,here>>.

+ 48 - 0
docs/reference/search/aggregations/bucket/histogram-aggregation.asciidoc

@@ -112,6 +112,54 @@ Response:
 
 <1> No documents were found that belong in this bucket, yet it is still returned with zero `doc_count`.
 
+[[search-aggregations-bucket-histogram-aggregation-extended-bounds]]
+By default the date_/histogram returns all the buckets within the range of the data itself, that is, the documents with
+the smallest values (on which with histogram) will determine the min bucket (the bucket with the smallest key) and the
+documents with the highest values will determine the max bucket (the bucket with the highest key). Often, when when
+requesting empty buckets (`"min_doc_count" : 0`), this causes a confusion, specifically, when the data is also filtered.
+
+To understand why, let's look at an example:
+
+Lets say the you're filtering your request to get all docs with values between `0` and `500`, in addition you'd like
+to slice the data per price using a histogram with an interval of `50`. You also specify `"min_doc_count" : 0` as you'd
+like to get all buckets even the empty ones. If it happens that all products (documents) have prices higher than `100`,
+the first bucket you'll get will be the one with `100` as its key. This is confusing, as many times, you'd also like
+to get those buckets between `0 - 100`.
+
+With `extended_bounds` setting, you now can "force" the histogram aggregation to start building buckets on a specific
+`min` values and also keep on building buckets up to a `max` value (even if there are no documents anymore). Using
+`extended_bounds` only makes sense when `min_doc_count` is 0 (the empty buckets will never be returned if `min_doc_count`
+is greater than 0).
+
+Note that (as the name suggest) `extended_bounds` is **not** filtering buckets. Meaning, if the `extended_bounds.min` is higher
+than the values extracted from the documents, the documents will still dictate what the first bucket will be (and the
+same goes for the `extended_bounds.max` and the last bucket). For filtering buckets, one should nest the histogram aggregation
+under a range `filter` aggregation with the appropriate `from`/`to` settings.
+
+Example:
+
+[source,js]
+--------------------------------------------------
+{
+    "query" : {
+        "filtered" : { "range" : { "price" : { "to" : "500" } } }
+    },
+    "aggs" : {
+        "prices" : {
+            "histogram" : {
+                "field" : "price",
+                "interval" : 50,
+                "min_doc_count" : 0,
+                "extended_bounds" : {
+                    "min" : 0,
+                    "max" : 500
+                }
+            }
+        }
+    }
+}
+--------------------------------------------------
+
 ==== Order
 
 By default the returned buckets are sorted by their `key` ascending, though the order behaviour can be controled

+ 32 - 0
src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramBuilder.java

@@ -23,6 +23,7 @@ import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.search.aggregations.ValuesSourceAggregationBuilder;
 import org.elasticsearch.search.builder.SearchSourceBuilderException;
+import org.joda.time.DateTime;
 
 import java.io.IOException;
 
@@ -34,6 +35,8 @@ public class DateHistogramBuilder extends ValuesSourceAggregationBuilder<DateHis
     private Object interval;
     private Histogram.Order order;
     private Long minDocCount;
+    private Object extendedBoundsMin;
+    private Object extendedBoundsMax;
     private String preZone;
     private String postZone;
     private boolean preZoneAdjustLargeInterval;
@@ -101,6 +104,24 @@ public class DateHistogramBuilder extends ValuesSourceAggregationBuilder<DateHis
         return this;
     }
 
+    public DateHistogramBuilder extendedBounds(Long min, Long max) {
+        extendedBoundsMin = min;
+        extendedBoundsMax = max;
+        return this;
+    }
+
+    public DateHistogramBuilder extendedBounds(String min, String max) {
+        extendedBoundsMin = min;
+        extendedBoundsMax = max;
+        return this;
+    }
+
+    public DateHistogramBuilder extendedBounds(DateTime min, DateTime max) {
+        extendedBoundsMin = min;
+        extendedBoundsMax = max;
+        return this;
+    }
+
     @Override
     protected XContentBuilder doInternalXContent(XContentBuilder builder, Params params) throws IOException {
         if (interval == null) {
@@ -148,6 +169,17 @@ public class DateHistogramBuilder extends ValuesSourceAggregationBuilder<DateHis
             builder.field("format", format);
         }
 
+        if (extendedBoundsMin != null || extendedBoundsMax != null) {
+            builder.startObject(DateHistogramParser.EXTENDED_BOUNDS.getPreferredName());
+            if (extendedBoundsMin != null) {
+                builder.field("min", extendedBoundsMin);
+            }
+            if (extendedBoundsMax != null) {
+                builder.field("max", extendedBoundsMax);
+            }
+            builder.endObject();
+        }
+
         return builder;
     }
 

+ 42 - 4
src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramParser.java

@@ -19,6 +19,7 @@
 package org.elasticsearch.search.aggregations.bucket.histogram;
 
 import com.google.common.collect.ImmutableMap;
+import org.elasticsearch.common.ParseField;
 import org.elasticsearch.common.collect.MapBuilder;
 import org.elasticsearch.common.joda.DateMathParser;
 import org.elasticsearch.common.rounding.DateTimeUnit;
@@ -48,6 +49,8 @@ import java.util.Map;
  */
 public class DateHistogramParser implements Aggregator.Parser {
 
+    static final ParseField EXTENDED_BOUNDS = new ParseField("extended_bounds");
+
     private final ImmutableMap<String, DateTimeUnit> dateFieldUnits;
 
     public DateHistogramParser() {
@@ -87,6 +90,7 @@ public class DateHistogramParser implements Aggregator.Parser {
         Map<String, Object> scriptParams = null;
         boolean keyed = false;
         long minDocCount = 1;
+        ExtendedBounds extendedBounds = null;
         InternalOrder order = (InternalOrder) Histogram.Order.KEY_ASC;
         String interval = null;
         boolean preZoneAdjustLargeInterval = false;
@@ -162,6 +166,32 @@ public class DateHistogramParser implements Aggregator.Parser {
                             //TODO should we throw an error if the value is not "asc" or "desc"???
                         }
                     }
+                } else if (EXTENDED_BOUNDS.match(currentFieldName)) {
+                    extendedBounds = new ExtendedBounds();
+                    while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
+                        if (token == XContentParser.Token.FIELD_NAME) {
+                            currentFieldName = parser.currentName();
+                        } else if (token == XContentParser.Token.VALUE_STRING) {
+                            if ("min".equals(currentFieldName)) {
+                                extendedBounds.minAsStr = parser.text();
+                            } else if ("max".equals(currentFieldName)) {
+                                extendedBounds.maxAsStr = parser.text();
+                            } else {
+                                throw new SearchParseException(context, "Unknown extended_bounds key for a " + token + " in aggregation [" + aggregationName + "]: [" + currentFieldName + "].");
+                            }
+                        } else if (token == XContentParser.Token.VALUE_NUMBER) {
+                            if ("min".equals(currentFieldName)) {
+                                extendedBounds.min = parser.longValue();
+                            } else if ("max".equals(currentFieldName)) {
+                                extendedBounds.max = parser.longValue();
+                            } else {
+                                throw new SearchParseException(context, "Unknown extended_bounds key for a " + token + " in aggregation [" + aggregationName + "]: [" + currentFieldName + "].");
+                            }
+                        } else {
+                            throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
+                        }
+                    }
+
                 } else {
                     throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].");
                 }
@@ -209,17 +239,21 @@ public class DateHistogramParser implements Aggregator.Parser {
             if (searchScript != null) {
                 ValueParser valueParser = new ValueParser.DateMath(new DateMathParser(DateFieldMapper.Defaults.DATE_TIME_FORMATTER, DateFieldMapper.Defaults.TIME_UNIT));
                 config.parser(valueParser);
-                return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, minDocCount, InternalDateHistogram.FACTORY);
+                return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, minDocCount, extendedBounds, InternalDateHistogram.FACTORY);
             }
 
             // falling back on the get field data context
-            return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, minDocCount, InternalDateHistogram.FACTORY);
+            return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, minDocCount, extendedBounds, InternalDateHistogram.FACTORY);
         }
 
         FieldMapper<?> mapper = context.smartNameFieldMapper(field);
         if (mapper == null) {
             config.unmapped(true);
-            return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, minDocCount, InternalDateHistogram.FACTORY);
+            if (format == null) {
+                config.formatter(new ValueFormatter.DateTime(DateFieldMapper.Defaults.DATE_TIME_FORMATTER));
+            }
+            config.parser(new ValueParser.DateMath(new DateMathParser(DateFieldMapper.Defaults.DATE_TIME_FORMATTER, DateFieldMapper.Defaults.TIME_UNIT)));
+            return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, minDocCount, extendedBounds, InternalDateHistogram.FACTORY);
         }
 
         if (!(mapper instanceof DateFieldMapper)) {
@@ -228,7 +262,11 @@ public class DateHistogramParser implements Aggregator.Parser {
 
         IndexFieldData<?> indexFieldData = context.fieldData().getForField(mapper);
         config.fieldContext(new FieldContext(field, indexFieldData));
-        return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, minDocCount, InternalDateHistogram.FACTORY);
+        if (format == null) {
+            config.formatter(new ValueFormatter.DateTime(((DateFieldMapper) mapper).dateTimeFormatter()));
+        }
+        config.parser(new ValueParser.DateMath(new DateMathParser(((DateFieldMapper) mapper).dateTimeFormatter(), DateFieldMapper.Defaults.TIME_UNIT)));
+        return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, minDocCount, extendedBounds, InternalDateHistogram.FACTORY);
     }
 
     private static InternalOrder resolveOrder(String key, boolean asc) {

+ 91 - 0
src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/ExtendedBounds.java

@@ -0,0 +1,91 @@
+/*
+ * 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.bucket.histogram;
+
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.rounding.Rounding;
+import org.elasticsearch.search.SearchParseException;
+import org.elasticsearch.search.aggregations.support.numeric.ValueParser;
+import org.elasticsearch.search.internal.SearchContext;
+
+import java.io.IOException;
+
+/**
+ *
+ */
+public class ExtendedBounds {
+
+    Long min;
+    Long max;
+
+    String minAsStr;
+    String maxAsStr;
+
+    ExtendedBounds() {} //for serialization
+
+    ExtendedBounds(Long min, Long max) {
+        this.min = min;
+        this.max = max;
+    }
+
+    void processAndValidate(String aggName, SearchContext context, ValueParser parser) {
+        if (minAsStr != null) {
+            min = parser != null ? parser.parseLong(minAsStr, context) : Long.parseLong(minAsStr);
+        }
+        if (maxAsStr != null) {
+            max = parser != null ? parser.parseLong(maxAsStr, context) : Long.parseLong(maxAsStr);
+        }
+        if (min != null && max != null && min.compareTo(max) > 0) {
+            throw new SearchParseException(context, "[extended_bounds.min][" + min + "] cannot be greater than " +
+                    "[extended_bounds.max][" + max + "] for histogram aggregation [" + aggName + "]");
+        }
+    }
+
+    ExtendedBounds round(Rounding rounding) {
+        return new ExtendedBounds(min != null ? rounding.round(min) : null, max != null ? rounding.round(max) : null);
+    }
+
+    void writeTo(StreamOutput out) throws IOException {
+        if (min != null) {
+            out.writeBoolean(true);
+            out.writeLong(min);
+        } else {
+            out.writeBoolean(false);
+        }
+        if (max != null) {
+            out.writeBoolean(true);
+            out.writeLong(max);
+        } else {
+            out.writeBoolean(false);
+        }
+    }
+
+    static ExtendedBounds readFrom(StreamInput in) throws IOException {
+        ExtendedBounds bounds = new ExtendedBounds();
+        if (in.readBoolean()) {
+            bounds.min = in.readLong();
+        }
+        if (in.readBoolean()) {
+            bounds.max = in.readLong();
+        }
+        return bounds;
+    }
+}

+ 26 - 17
src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregator.java

@@ -47,29 +47,24 @@ public class HistogramAggregator extends BucketsAggregator {
     private final InternalOrder order;
     private final boolean keyed;
     private final long minDocCount;
+    private final ExtendedBounds extendedBounds;
     private final InternalHistogram.Factory histogramFactory;
 
     private final LongHash bucketOrds;
     private LongValues values;
 
-    public HistogramAggregator(String name,
-                               AggregatorFactories factories,
-                               Rounding rounding,
-                               InternalOrder order,
-                               boolean keyed,
-                               long minDocCount,
-                               @Nullable NumericValuesSource valuesSource,
-                               long initialCapacity,
-                               InternalHistogram.Factory<?> histogramFactory,
-                               AggregationContext aggregationContext,
-                               Aggregator parent) {
+    public HistogramAggregator(String name, AggregatorFactories factories, Rounding rounding, InternalOrder order,
+                               boolean keyed, long minDocCount, @Nullable ExtendedBounds extendedBounds,
+                               @Nullable NumericValuesSource valuesSource, long initialCapacity, InternalHistogram.Factory<?> histogramFactory,
+                               AggregationContext aggregationContext, Aggregator parent) {
 
         super(name, BucketAggregationMode.PER_BUCKET, factories, initialCapacity, aggregationContext, parent);
-        this.valuesSource = valuesSource;
         this.rounding = rounding;
         this.order = order;
         this.keyed = keyed;
         this.minDocCount = minDocCount;
+        this.extendedBounds = extendedBounds;
+        this.valuesSource = valuesSource;
         this.histogramFactory = histogramFactory;
 
         bucketOrds = new LongHash(initialCapacity, aggregationContext.bigArrays());
@@ -123,14 +118,14 @@ public class HistogramAggregator extends BucketsAggregator {
 
         // value source will be null for unmapped fields
         ValueFormatter formatter = valuesSource != null ? valuesSource.formatter() : null;
-        InternalHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 ? new InternalHistogram.EmptyBucketInfo(rounding, buildEmptySubAggregations()) : null;
+        InternalHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 ? new InternalHistogram.EmptyBucketInfo(rounding, buildEmptySubAggregations(), extendedBounds) : null;
         return histogramFactory.create(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed);
     }
 
     @Override
     public InternalAggregation buildEmptyAggregation() {
         ValueFormatter formatter = valuesSource != null ? valuesSource.formatter() : null;
-        InternalHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 ? new InternalHistogram.EmptyBucketInfo(rounding, buildEmptySubAggregations()) : null;
+        InternalHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 ? new InternalHistogram.EmptyBucketInfo(rounding, buildEmptySubAggregations(), extendedBounds) : null;
         return histogramFactory.create(name, Collections.emptyList(), order, minDocCount, emptyBucketInfo, formatter, keyed);
     }
 
@@ -145,27 +140,41 @@ public class HistogramAggregator extends BucketsAggregator {
         private final InternalOrder order;
         private final boolean keyed;
         private final long minDocCount;
+        private final ExtendedBounds extendedBounds;
         private final InternalHistogram.Factory<?> histogramFactory;
 
         public Factory(String name, ValuesSourceConfig<NumericValuesSource> valueSourceConfig,
-                       Rounding rounding, InternalOrder order, boolean keyed, long minDocCount, InternalHistogram.Factory<?> histogramFactory) {
+                       Rounding rounding, InternalOrder order, boolean keyed, long minDocCount,
+                       ExtendedBounds extendedBounds, InternalHistogram.Factory<?> histogramFactory) {
+
             super(name, histogramFactory.type(), valueSourceConfig);
             this.rounding = rounding;
             this.order = order;
             this.keyed = keyed;
             this.minDocCount = minDocCount;
+            this.extendedBounds = extendedBounds;
             this.histogramFactory = histogramFactory;
         }
 
         @Override
         protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent) {
-            return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, null, 0, histogramFactory, aggregationContext, parent);
+            return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, null, null, 0, histogramFactory, aggregationContext, parent);
         }
 
         @Override
         protected Aggregator create(NumericValuesSource valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent) {
             // todo if we'll keep track of min/max values in IndexFieldData, we could use the max here to come up with a better estimation for the buckets count
-            return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, valuesSource, 50, histogramFactory, aggregationContext, parent);
+
+            // we need to round the bounds given by the user and we have to do it for every aggregator we crate
+            // as the rounding is not necessarily an idempotent operation.
+            // todo we need to think of a better structure to the factory/agtor code so we won't need to do that
+            ExtendedBounds roundedBounds = null;
+            if (extendedBounds != null) {
+                // we need to process & validate here using the parser
+                extendedBounds.processAndValidate(name, aggregationContext.searchContext(), valuesSource != null ? valuesSource.parser() : null);
+                roundedBounds = extendedBounds.round(rounding);
+            }
+            return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, roundedBounds, valuesSource, 50, histogramFactory, aggregationContext, parent);
         }
 
     }

+ 18 - 0
src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramBuilder.java

@@ -33,6 +33,8 @@ public class HistogramBuilder extends ValuesSourceAggregationBuilder<HistogramBu
     private Long interval;
     private Histogram.Order order;
     private Long minDocCount;
+    private Long extendedBoundsMin;
+    private Long extendedBoundsMax;
 
     /**
      * Constructs a new histogram aggregation builder.
@@ -76,6 +78,12 @@ public class HistogramBuilder extends ValuesSourceAggregationBuilder<HistogramBu
         return this;
     }
 
+    public HistogramBuilder extendedBounds(Long min, Long max) {
+        extendedBoundsMin = min;
+        extendedBoundsMax = max;
+        return this;
+    }
+
     @Override
     protected XContentBuilder doInternalXContent(XContentBuilder builder, Params params) throws IOException {
         if (interval == null) {
@@ -92,6 +100,16 @@ public class HistogramBuilder extends ValuesSourceAggregationBuilder<HistogramBu
             builder.field("min_doc_count", minDocCount);
         }
 
+        if (extendedBoundsMin != null || extendedBoundsMax != null) {
+            builder.startObject(HistogramParser.EXTENDED_BOUNDS.getPreferredName());
+            if (extendedBoundsMin != null) {
+                builder.field("min", extendedBoundsMin);
+            }
+            if (extendedBoundsMax != null) {
+                builder.field("max", extendedBoundsMax);
+            }
+            builder.endObject();
+        }
         return builder;
     }
 

+ 28 - 3
src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramParser.java

@@ -18,6 +18,7 @@
  */
 package org.elasticsearch.search.aggregations.bucket.histogram;
 
+import org.elasticsearch.common.ParseField;
 import org.elasticsearch.common.rounding.Rounding;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.fielddata.IndexFieldData;
@@ -39,6 +40,8 @@ import java.util.Map;
  */
 public class HistogramParser implements Aggregator.Parser {
 
+    static final ParseField EXTENDED_BOUNDS = new ParseField("extended_bounds");
+
     @Override
     public String type() {
         return InternalHistogram.TYPE.name();
@@ -59,6 +62,7 @@ public class HistogramParser implements Aggregator.Parser {
         long interval = -1;
         boolean assumeSorted = false;
         String format = null;
+        ExtendedBounds extendedBounds = null;
 
         XContentParser.Token token;
         String currentFieldName = null;
@@ -109,6 +113,22 @@ public class HistogramParser implements Aggregator.Parser {
                             order = resolveOrder(currentFieldName, asc);
                         }
                     }
+                } else if (EXTENDED_BOUNDS.match(currentFieldName)) {
+                    extendedBounds = new ExtendedBounds();
+                    while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
+                        if (token == XContentParser.Token.FIELD_NAME) {
+                            currentFieldName = parser.currentName();
+                        } else if (token.isValue()) {
+                            if ("min".equals(currentFieldName)) {
+                                extendedBounds.min = parser.longValue(true);
+                            } else if ("max".equals(currentFieldName)) {
+                                extendedBounds.max = parser.longValue(true);
+                            } else {
+                                throw new SearchParseException(context, "Unknown extended_bounds key for a " + token + " in aggregation [" + aggregationName + "]: [" + currentFieldName + "].");
+                            }
+                        }
+                    }
+
                 } else {
                     throw new SearchParseException(context, "Unknown key for a " + token + " in aggregation [" + aggregationName + "]: [" + currentFieldName + "].");
                 }
@@ -122,6 +142,11 @@ public class HistogramParser implements Aggregator.Parser {
         }
         Rounding rounding = new Rounding.Interval(interval);
 
+        if (extendedBounds != null) {
+            // with numeric histogram, we can process here and fail fast if the bounds are invalid
+            extendedBounds.processAndValidate(aggregationName, context, null);
+        }
+
         if (script != null) {
             config.script(context.scriptService().search(context.lookup(), scriptLang, script, scriptParams));
         }
@@ -132,13 +157,13 @@ public class HistogramParser implements Aggregator.Parser {
         }
 
         if (field == null) {
-            return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, minDocCount, InternalHistogram.FACTORY);
+            return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, minDocCount, extendedBounds, InternalHistogram.FACTORY);
         }
 
         FieldMapper<?> mapper = context.smartNameFieldMapper(field);
         if (mapper == null) {
             config.unmapped(true);
-            return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, minDocCount, InternalHistogram.FACTORY);
+            return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, minDocCount, extendedBounds, InternalHistogram.FACTORY);
         }
 
         IndexFieldData<?> indexFieldData = context.fieldData().getForField(mapper);
@@ -148,7 +173,7 @@ public class HistogramParser implements Aggregator.Parser {
             config.formatter(new ValueFormatter.Number.Pattern(format));
         }
 
-        return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, minDocCount, InternalHistogram.FACTORY);
+        return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, minDocCount, extendedBounds, InternalHistogram.FACTORY);
 
     }
 

+ 5 - 0
src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java

@@ -69,6 +69,11 @@ public class InternalDateHistogram extends InternalHistogram<InternalDateHistogr
         public DateTime getKeyAsDate() {
             return new DateTime(key, DateTimeZone.UTC);
         }
+
+        @Override
+        public String toString() {
+            return getKey();
+        }
     }
 
     static class Factory extends InternalHistogram.Factory<InternalDateHistogram.Bucket> {

+ 110 - 11
src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogram.java

@@ -21,6 +21,7 @@ package org.elasticsearch.search.aggregations.bucket.histogram;
 import com.carrotsearch.hppc.LongObjectOpenHashMap;
 import com.google.common.collect.Lists;
 import org.apache.lucene.util.CollectionUtil;
+import org.elasticsearch.Version;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.rounding.Rounding;
@@ -123,22 +124,43 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
     }
 
     static class EmptyBucketInfo {
+
         final Rounding rounding;
         final InternalAggregations subAggregations;
+        final ExtendedBounds bounds;
 
         EmptyBucketInfo(Rounding rounding, InternalAggregations subAggregations) {
+            this(rounding, subAggregations, null);
+        }
+
+        EmptyBucketInfo(Rounding rounding, InternalAggregations subAggregations, ExtendedBounds bounds) {
             this.rounding = rounding;
             this.subAggregations = subAggregations;
+            this.bounds = bounds;
         }
 
         public static EmptyBucketInfo readFrom(StreamInput in) throws IOException {
-            return new EmptyBucketInfo(Rounding.Streams.read(in), InternalAggregations.readAggregations(in));
+            Rounding rounding = Rounding.Streams.read(in);
+            InternalAggregations aggs = InternalAggregations.readAggregations(in);
+            if (in.getVersion().onOrAfter(Version.V_1_1_0)) {
+                if (in.readBoolean()) {
+                    return new EmptyBucketInfo(rounding, aggs, ExtendedBounds.readFrom(in));
+                }
+            }
+            return new EmptyBucketInfo(rounding, aggs);
         }
 
         public static void writeTo(EmptyBucketInfo info, StreamOutput out) throws IOException {
             Rounding.Streams.write(info.rounding, out);
             info.subAggregations.writeTo(out);
+            if (out.getVersion().onOrAfter(Version.V_1_1_0)) {
+                out.writeBoolean(info.bounds != null);
+                if (info.bounds != null) {
+                    info.bounds.writeTo(out);
+                }
+            }
         }
+
     }
 
     static class Factory<B extends InternalHistogram.Bucket> {
@@ -222,27 +244,65 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
                 return histo;
             }
 
-
             CollectionUtil.introSort(histo.buckets, order.asc ? InternalOrder.KEY_ASC.comparator() : InternalOrder.KEY_DESC.comparator());
             List<B> list = order.asc ? histo.buckets : Lists.reverse(histo.buckets);
-            B prevBucket = null;
+            B lastBucket = null;
             ListIterator<B> iter = list.listIterator();
+
+            // we need to fill the gaps with empty buckets
             if (minDocCount == 0) {
-                // we need to fill the gaps with empty buckets
+                ExtendedBounds bounds = emptyBucketInfo.bounds;
+
+                // first adding all the empty buckets *before* the actual data (based on th extended_bounds.min the user requested)
+                if (bounds != null) {
+                    B firstBucket = iter.hasNext() ? list.get(iter.nextIndex()) : null;
+                    if (firstBucket == null) {
+                        if (bounds.min != null && bounds.max != null) {
+                            long key = bounds.min;
+                            long max = bounds.max;
+                            while (key <= max) {
+                                iter.add(createBucket(key, 0, emptyBucketInfo.subAggregations, formatter));
+                                key = emptyBucketInfo.rounding.nextRoundingValue(key);
+                            }
+                        }
+                    } else {
+                        if (bounds.min != null) {
+                            long key = bounds.min;
+                            while (key < firstBucket.key) {
+                                iter.add(createBucket(key, 0, emptyBucketInfo.subAggregations, formatter));
+                                key = emptyBucketInfo.rounding.nextRoundingValue(key);
+                            }
+                        }
+                    }
+                }
+
+                // now adding the empty buckets within the actual data,
+                // e.g. if the data series is [1,2,3,7] there are 3 empty buckets that will be created for 4,5,6
                 while (iter.hasNext()) {
                     // look ahead on the next bucket without advancing the iter
                     // so we'll be able to insert elements at the right position
                     B nextBucket = list.get(iter.nextIndex());
                     nextBucket.aggregations.reduce(reduceContext.bigArrays());
-                    if (prevBucket != null) {
-                        long key = emptyBucketInfo.rounding.nextRoundingValue(prevBucket.key);
+                    if (lastBucket != null) {
+                        long key = emptyBucketInfo.rounding.nextRoundingValue(lastBucket.key);
                         while (key != nextBucket.key) {
                             iter.add(createBucket(key, 0, emptyBucketInfo.subAggregations, formatter));
                             key = emptyBucketInfo.rounding.nextRoundingValue(key);
                         }
                     }
-                    prevBucket = iter.next();
+                    lastBucket = iter.next();
+                }
+
+                // finally, adding the empty buckets *after* the actual data (based on the extended_bounds.max requested by the user)
+                if (bounds != null && lastBucket != null && bounds.max != null && bounds.max > lastBucket.key) {
+                    long key = emptyBucketInfo.rounding.nextRoundingValue(lastBucket.key);
+                    long max = bounds.max;
+                    while (key <= max) {
+                        iter.add(createBucket(key, 0, emptyBucketInfo.subAggregations, formatter));
+                        key = emptyBucketInfo.rounding.nextRoundingValue(key);
+                    }
                 }
+
             } else {
                 while (iter.hasNext()) {
                     InternalHistogram.Bucket bucket = iter.next();
@@ -290,18 +350,57 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
         if (minDocCount == 0) {
             CollectionUtil.introSort(reducedBuckets, order.asc ? InternalOrder.KEY_ASC.comparator() : InternalOrder.KEY_DESC.comparator());
             List<B> list = order.asc ? reducedBuckets : Lists.reverse(reducedBuckets);
-            B prevBucket = null;
+            B lastBucket = null;
+            ExtendedBounds bounds = emptyBucketInfo.bounds;
             ListIterator<B> iter = list.listIterator();
+
+            // first adding all the empty buckets *before* the actual data (based on th extended_bounds.min the user requested)
+            if (bounds != null) {
+                B firstBucket = iter.hasNext() ? list.get(iter.nextIndex()) : null;
+                if (firstBucket == null) {
+                    if (bounds.min != null && bounds.max != null) {
+                        long key = bounds.min;
+                        long max = bounds.max;
+                        while (key <= max) {
+                            iter.add(createBucket(key, 0, emptyBucketInfo.subAggregations, formatter));
+                            key = emptyBucketInfo.rounding.nextRoundingValue(key);
+                        }
+                    }
+                } else {
+                    if (bounds.min != null) {
+                        long key = bounds.min;
+                        if (key < firstBucket.key) {
+                            while (key < firstBucket.key) {
+                                iter.add(createBucket(key, 0, emptyBucketInfo.subAggregations, formatter));
+                                key = emptyBucketInfo.rounding.nextRoundingValue(key);
+                            }
+                        }
+                    }
+                }
+            }
+
+            // now adding the empty buckets within the actual data,
+            // e.g. if the data series is [1,2,3,7] there're 3 empty buckets that will be created for 4,5,6
             while (iter.hasNext()) {
                 B nextBucket = list.get(iter.nextIndex());
-                if (prevBucket != null) {
-                    long key = emptyBucketInfo.rounding.nextRoundingValue(prevBucket.key);
+                if (lastBucket != null) {
+                    long key = emptyBucketInfo.rounding.nextRoundingValue(lastBucket.key);
                     while (key != nextBucket.key) {
                         iter.add(createBucket(key, 0, emptyBucketInfo.subAggregations, formatter));
                         key = emptyBucketInfo.rounding.nextRoundingValue(key);
                     }
                 }
-                prevBucket = iter.next();
+                lastBucket = iter.next();
+            }
+
+            // finally, adding the empty buckets *after* the actual data (based on the extended_bounds.max requested by the user)
+            if (bounds != null && lastBucket != null && bounds.max != null && bounds.max > lastBucket.key) {
+                long key = emptyBucketInfo.rounding.nextRoundingValue(lastBucket.key);
+                long max = bounds.max;
+                while (key <= max) {
+                    iter.add(createBucket(key, 0, emptyBucketInfo.subAggregations, formatter));
+                    key = emptyBucketInfo.rounding.nextRoundingValue(key);
+                }
             }
 
             if (order != InternalOrder.KEY_ASC && order != InternalOrder.KEY_DESC) {

+ 129 - 0
src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramTests.java

@@ -21,6 +21,8 @@ package org.elasticsearch.search.aggregations.bucket;
 import org.elasticsearch.action.index.IndexRequestBuilder;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.common.joda.Joda;
+import org.elasticsearch.common.settings.ImmutableSettings;
+import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.index.mapper.core.DateFieldMapper;
 import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
@@ -32,6 +34,7 @@ import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.hamcrest.Matchers;
 import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;
+import org.joda.time.format.DateTimeFormat;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -61,6 +64,19 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
         return DateFieldMapper.Defaults.DATE_TIME_FORMATTER.parser().parseDateTime(date);
     }
 
+    private static String format(DateTime date, String pattern) {
+        return DateTimeFormat.forPattern(pattern).print(date);
+    }
+
+    private IndexRequestBuilder indexDoc(String idx, DateTime date, int value) throws Exception {
+        return client().prepareIndex(idx, "type").setSource(jsonBuilder()
+                .startObject()
+                .field("date", date)
+                .field("value", value)
+                .startArray("dates").value(date).value(date.plusMonths(1).plusDays(1)).endArray()
+                .endObject());
+    }
+
     private IndexRequestBuilder indexDoc(int month, int day, int value) throws Exception {
         return client().prepareIndex("idx", "type").setSource(jsonBuilder()
                 .startObject()
@@ -1064,4 +1080,117 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
         assertThat(bucket, Matchers.notNullValue());
         assertThat(bucket.getDocCount(), equalTo(3l));
     }
+
+    @Override
+    public Settings indexSettings() {
+        ImmutableSettings.Builder builder = ImmutableSettings.builder();
+        builder.put("index.number_of_shards", 1).put("index.number_of_replicas", 0);
+        return builder.build();
+    }
+
+    @Test
+    public void singleValueField_WithExtendedBounds() throws Exception {
+
+        String pattern = "yyyy-MM-dd";
+        // we're testing on days, so the base must be rounded to a day
+        int interval = randomIntBetween(1, 2); // in days
+        long intervalMillis = interval * 24 * 60 * 60 * 1000;
+        DateTime base = new DateTime(DateTimeZone.UTC).dayOfMonth().roundFloorCopy();
+        DateTime baseKey = new DateTime(intervalMillis * (base.getMillis() / intervalMillis), DateTimeZone.UTC);
+
+        createIndex("idx2");
+        int numOfBuckets = randomIntBetween(3, 6);
+        int emptyBucketIndex = randomIntBetween(1, numOfBuckets - 2); // should be in the middle
+
+        long[] docCounts = new long[numOfBuckets];
+        List<IndexRequestBuilder> builders = new ArrayList<IndexRequestBuilder>();
+        for (int i = 0; i < numOfBuckets; i++) {
+            if (i == emptyBucketIndex) {
+                docCounts[i] = 0;
+            } else {
+                int docCount = randomIntBetween(1, 3);
+                for (int j = 0; j < docCount; j++) {
+                    DateTime date = baseKey.plusDays(i * interval + randomIntBetween(0, interval - 1));
+                    builders.add(indexDoc("idx2", date, j));
+                }
+                docCounts[i] = docCount;
+            }
+        }
+        indexRandom(true, builders);
+        ensureSearchable("idx2");
+
+        DateTime lastDataBucketKey = baseKey.plusDays((numOfBuckets - 1) * interval);
+
+        // randomizing the number of buckets on the min bound
+        // (can sometimes fall within the data range, but more frequently will fall before the data range)
+        int addedBucketsLeft = randomIntBetween(0, numOfBuckets);
+        DateTime boundsMinKey;
+        if (frequently()) {
+            boundsMinKey = baseKey.minusDays(addedBucketsLeft * interval);
+        } else {
+            boundsMinKey = baseKey.plus(addedBucketsLeft * interval);
+            addedBucketsLeft = 0;
+        }
+        DateTime boundsMin = boundsMinKey.plusDays(randomIntBetween(0, interval - 1));
+
+        // randomizing the number of buckets on the max bound
+        // (can sometimes fall within the data range, but more frequently will fall after the data range)
+        int addedBucketsRight = randomIntBetween(0, numOfBuckets);
+        int boundsMaxKeyDelta = addedBucketsRight * interval;
+        if (rarely()) {
+            addedBucketsRight = 0;
+            boundsMaxKeyDelta = -boundsMaxKeyDelta;
+        }
+        DateTime boundsMaxKey = lastDataBucketKey.plusDays(boundsMaxKeyDelta);
+        DateTime boundsMax = boundsMaxKey.plusDays(randomIntBetween(0, interval - 1));
+
+        // it could be that the random bounds.min we chose ended up greater than bounds.max - this should
+        // trigger an error
+        boolean invalidBoundsError = boundsMin.isAfter(boundsMax);
+
+        // constructing the newly expected bucket list
+        int bucketsCount = numOfBuckets + addedBucketsLeft + addedBucketsRight;
+        long[] extendedValueCounts = new long[bucketsCount];
+        System.arraycopy(docCounts, 0, extendedValueCounts, addedBucketsLeft, docCounts.length);
+
+        SearchResponse response = null;
+        try {
+            response = client().prepareSearch("idx2")
+                    .addAggregation(dateHistogram("histo")
+                            .field("date")
+                            .interval(DateHistogram.Interval.days(interval))
+                            .minDocCount(0)
+                            .extendedBounds(boundsMin, boundsMax)
+                            .format(pattern))
+                    .execute().actionGet();
+
+            if (invalidBoundsError) {
+                fail("Expected an exception to be thrown when bounds.min is greater than bounds.max");
+                return;
+            }
+
+        } catch (Exception e) {
+            if (invalidBoundsError) {
+                // expected
+                return;
+            } else {
+                throw e;
+            }
+        }
+        assertSearchResponse(response);
+
+        DateHistogram histo = response.getAggregations().get("histo");
+        assertThat(histo, notNullValue());
+        assertThat(histo.getName(), equalTo("histo"));
+        assertThat(histo.getBuckets().size(), equalTo(bucketsCount));
+
+        DateTime key = baseKey.isBefore(boundsMinKey) ? baseKey : boundsMinKey;
+        for (int i = 0; i < bucketsCount; i++) {
+            DateHistogram.Bucket bucket = histo.getBucketByKey(format(key, pattern));
+            assertThat(bucket, notNullValue());
+            assertThat(bucket.getKeyAsDate(), equalTo(key));
+            assertThat(bucket.getDocCount(), equalTo(extendedValueCounts[i]));
+            key = key.plusDays(interval);
+        }
+    }
 }

+ 91 - 15
src/test/java/org/elasticsearch/search/aggregations/bucket/HistogramTests.java

@@ -67,19 +67,19 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
 
         numValueBuckets = numDocs / interval + 1;
         valueCounts = new long[numValueBuckets];
-        for (int i = 0; i < numDocs; ++i) {
+        for (int i = 0; i < numDocs; i++) {
             final int bucket = (i + 1) / interval;
-            ++valueCounts[bucket];
+            valueCounts[bucket]++;
         }
 
         numValuesBuckets = (numDocs + 1) / interval + 1;
         valuesCounts = new long[numValuesBuckets];
-        for (int i = 0; i < numDocs; ++i) {
+        for (int i = 0; i < numDocs; i++) {
             final int bucket1 = (i + 1) / interval;
             final int bucket2 = (i + 2) / interval;
-            ++valuesCounts[bucket1];
+            valuesCounts[bucket1]++;
             if (bucket1 != bucket2) {
-                ++valuesCounts[bucket2];
+                valuesCounts[bucket2]++;
             }
         }
 
@@ -158,7 +158,7 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
 
         List<Histogram.Bucket> buckets = new ArrayList<Histogram.Bucket>(histo.getBuckets());
         for (int i = 0; i < numValueBuckets; ++i) {
-            Histogram.Bucket bucket = buckets.get(numValueBuckets -i - 1);
+            Histogram.Bucket bucket = buckets.get(numValueBuckets - i - 1);
             assertThat(bucket, notNullValue());
             assertThat(bucket.getKeyAsNumber().longValue(), equalTo((long) i * interval));
             assertThat(bucket.getDocCount(), equalTo(valueCounts[i]));
@@ -227,7 +227,7 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
     public void singleValuedField_WithSubAggregation() throws Exception {
         SearchResponse response = client().prepareSearch("idx")
                 .addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval)
-                    .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME)))
+                        .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME)))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -469,7 +469,7 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
         assertThat(histo.getBuckets().size(), equalTo(numValueBuckets));
 
         LongOpenHashSet visited = new LongOpenHashSet();
-        double prevMax = asc? Double.NEGATIVE_INFINITY : Double.POSITIVE_INFINITY;
+        double prevMax = asc ? Double.NEGATIVE_INFINITY : Double.POSITIVE_INFINITY;
         List<Histogram.Bucket> buckets = new ArrayList<Histogram.Bucket>(histo.getBuckets());
         for (int i = 0; i < numValueBuckets; ++i) {
             Histogram.Bucket bucket = buckets.get(i);
@@ -500,7 +500,7 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
 
         final int numBuckets = (numDocs + 1) / interval - 2 / interval + 1;
         final long[] counts = new long[(numDocs + 1) / interval + 1];
-        for (int i = 0; i < numDocs ; ++i) {
+        for (int i = 0; i < numDocs; ++i) {
             ++counts[(i + 2) / interval];
         }
 
@@ -555,7 +555,7 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
 
         List<Histogram.Bucket> buckets = new ArrayList<Histogram.Bucket>(histo.getBuckets());
         for (int i = 0; i < numValuesBuckets; ++i) {
-            Histogram.Bucket bucket = buckets.get(numValuesBuckets -i - 1);
+            Histogram.Bucket bucket = buckets.get(numValuesBuckets - i - 1);
             assertThat(bucket, notNullValue());
             assertThat(bucket.getKeyAsNumber().longValue(), equalTo((long) i * interval));
             assertThat(bucket.getDocCount(), equalTo(valuesCounts[i]));
@@ -573,7 +573,7 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
 
         final int numBuckets = (numDocs + 2) / interval - 2 / interval + 1;
         final long[] counts = new long[(numDocs + 2) / interval + 1];
-        for (int i = 0; i < numDocs ; ++i) {
+        for (int i = 0; i < numDocs; ++i) {
             final int bucket1 = (i + 2) / interval;
             final int bucket2 = (i + 3) / interval;
             ++counts[bucket1];
@@ -599,7 +599,7 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
     public void multiValuedField_WithValueScript_WithInheritedSubAggregator() throws Exception {
         SearchResponse response = client().prepareSearch("idx")
                 .addAggregation(histogram("histo").field(MULTI_VALUED_FIELD_NAME).script("_value + 1").interval(interval)
-                    .subAggregation(terms(MULTI_VALUED_FIELD_NAME).order(Terms.Order.term(true))))
+                        .subAggregation(terms(MULTI_VALUED_FIELD_NAME).order(Terms.Order.term(true))))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -607,7 +607,7 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
 
         final int numBuckets = (numDocs + 2) / interval - 2 / interval + 1;
         final long[] counts = new long[(numDocs + 2) / interval + 1];
-        for (int i = 0; i < numDocs ; ++i) {
+        for (int i = 0; i < numDocs; ++i) {
             final int bucket1 = (i + 2) / interval;
             final int bucket2 = (i + 3) / interval;
             ++counts[bucket1];
@@ -665,7 +665,7 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
     public void script_SingleValue_WithSubAggregator_Inherited() throws Exception {
         SearchResponse response = client().prepareSearch("idx")
                 .addAggregation(histogram("histo").script("doc['" + SINGLE_VALUED_FIELD_NAME + "'].value").interval(interval)
-                    .subAggregation(sum("sum")))
+                        .subAggregation(sum("sum")))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -721,7 +721,7 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
     public void script_MultiValued_WithAggregatorInherited() throws Exception {
         SearchResponse response = client().prepareSearch("idx")
                 .addAggregation(histogram("histo").script("doc['" + MULTI_VALUED_FIELD_NAME + "'].values").interval(interval)
-                    .subAggregation(sum("sum")))
+                        .subAggregation(sum("sum")))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -818,4 +818,80 @@ public class HistogramTests extends ElasticsearchIntegrationTest {
         assertThat(histo.getBuckets().isEmpty(), is(true));
     }
 
+    @Test
+    public void singleValuedField_WithExtendedBounds() throws Exception {
+        int lastDataBucketKey = (numValueBuckets - 1) * interval;
+
+        // randomizing the number of buckets on the min bound
+        // (can sometimes fall within the data range, but more frequently will fall before the data range)
+        int addedBucketsLeft = randomIntBetween(0, numValueBuckets);
+        long boundsMinKey = addedBucketsLeft * interval;
+        if (frequently()) {
+            boundsMinKey = -boundsMinKey;
+        } else {
+            addedBucketsLeft = 0;
+        }
+        long boundsMin = boundsMinKey + randomIntBetween(0, interval - 1);
+
+        // randomizing the number of buckets on the max bound
+        // (can sometimes fall within the data range, but more frequently will fall after the data range)
+        int addedBucketsRight = randomIntBetween(0, numValueBuckets);
+        long boundsMaxKeyDelta = addedBucketsRight * interval;
+        if (rarely()) {
+            addedBucketsRight = 0;
+            boundsMaxKeyDelta = -boundsMaxKeyDelta;
+        }
+        long boundsMaxKey = lastDataBucketKey + boundsMaxKeyDelta;
+        long boundsMax = boundsMaxKey + randomIntBetween(0, interval - 1);
+
+
+        // it could be that the random bounds.min we chose ended up greater than bounds.max - this should cause an
+        // error
+        boolean invalidBoundsError = boundsMin > boundsMax;
+
+        // constructing the newly expected bucket list
+        int bucketsCount = numValueBuckets + addedBucketsLeft + addedBucketsRight;
+        long[] extendedValueCounts = new long[bucketsCount];
+        System.arraycopy(valueCounts, 0, extendedValueCounts, addedBucketsLeft, valueCounts.length);
+
+        SearchResponse response = null;
+        try {
+            response = client().prepareSearch("idx")
+                    .addAggregation(histogram("histo")
+                            .field(SINGLE_VALUED_FIELD_NAME)
+                            .interval(interval)
+                            .minDocCount(0)
+                            .extendedBounds(boundsMin, boundsMax))
+                    .execute().actionGet();
+
+            if (invalidBoundsError) {
+                fail("Expected an exception to be thrown when bounds.min is greater than bounds.max");
+                return;
+            }
+
+        } catch (Exception e) {
+            if (invalidBoundsError) {
+                // expected
+                return;
+            } else {
+                throw e;
+            }
+        }
+        assertSearchResponse(response);
+
+        Histogram histo = response.getAggregations().get("histo");
+        assertThat(histo, notNullValue());
+        assertThat(histo.getName(), equalTo("histo"));
+        assertThat(histo.getBuckets().size(), equalTo(bucketsCount));
+
+        long key = Math.min(boundsMinKey, 0);
+        for (int i = 0; i < bucketsCount; i++) {
+            Histogram.Bucket bucket = histo.getBucketByKey(key);
+            assertThat(bucket, notNullValue());
+            assertThat(bucket.getKeyAsNumber().longValue(), equalTo(key));
+            assertThat(bucket.getDocCount(), equalTo(extendedValueCounts[i]));
+            key += interval;
+        }
+    }
+
 }