Browse Source

Aggregations optimisation for memory usage. Added changes to core Aggregator class to support a new mode of deferred collection.
A new "breadth_first" results collection mode allows upper branches of aggregation tree to be calculated and then pruned
to a smaller selection before advancing into executing collection on child branches.

Closes #6128

markharwood 11 years ago
parent
commit
724129e6ce
45 changed files with 1654 additions and 138 deletions
  1. 73 0
      docs/reference/search/aggregations/bucket/terms-aggregation.asciidoc
  2. 172 4
      src/main/java/org/elasticsearch/search/aggregations/Aggregator.java
  3. 19 7
      src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java
  4. 28 2
      src/main/java/org/elasticsearch/search/aggregations/BucketCollector.java
  5. 86 0
      src/main/java/org/elasticsearch/search/aggregations/FilteringBucketCollector.java
  6. 43 0
      src/main/java/org/elasticsearch/search/aggregations/RecordingBucketCollector.java
  7. 167 0
      src/main/java/org/elasticsearch/search/aggregations/RecordingPerReaderBucketCollector.java
  8. 35 12
      src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java
  9. 128 0
      src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java
  10. 1 1
      src/main/java/org/elasticsearch/search/aggregations/bucket/significant/GlobalOrdinalsSignificantTermsAggregator.java
  11. 1 1
      src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java
  12. 1 1
      src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTermsAggregator.java
  13. 2 4
      src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractStringTermsAggregator.java
  14. 13 0
      src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractTermsParametersParser.java
  15. 13 5
      src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsAggregator.java
  16. 19 7
      src/main/java/org/elasticsearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java
  17. 20 7
      src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java
  18. 17 8
      src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java
  19. 19 1
      src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregator.java
  20. 20 18
      src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java
  21. 11 0
      src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsBuilder.java
  22. 2 1
      src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsParser.java
  23. 19 0
      src/main/java/org/elasticsearch/search/aggregations/support/OrderPath.java
  24. 318 0
      src/test/java/org/elasticsearch/benchmark/search/aggregations/SubAggregationSearchCollectModeBenchmark.java
  25. 40 0
      src/test/java/org/elasticsearch/benchmark/search/aggregations/TermsAggregationSearchBenchmark.java
  26. 10 4
      src/test/java/org/elasticsearch/index/fielddata/DisabledFieldDataFormatTests.java
  27. 7 2
      src/test/java/org/elasticsearch/index/mapper/copyto/CopyToMapperIntegrationTests.java
  28. 4 1
      src/test/java/org/elasticsearch/percolator/PercolatorFacetsAndAggregationsTests.java
  29. 7 2
      src/test/java/org/elasticsearch/search/aggregations/CombiTests.java
  30. 9 8
      src/test/java/org/elasticsearch/search/aggregations/RandomTests.java
  31. 82 6
      src/test/java/org/elasticsearch/search/aggregations/bucket/DoubleTermsTests.java
  32. 3 1
      src/test/java/org/elasticsearch/search/aggregations/bucket/GeoDistanceTests.java
  33. 3 1
      src/test/java/org/elasticsearch/search/aggregations/bucket/HistogramTests.java
  34. 80 5
      src/test/java/org/elasticsearch/search/aggregations/bucket/LongTermsTests.java
  35. 3 0
      src/test/java/org/elasticsearch/search/aggregations/bucket/MinDocCountTests.java
  36. 2 1
      src/test/java/org/elasticsearch/search/aggregations/bucket/NaNSortingTests.java
  37. 8 2
      src/test/java/org/elasticsearch/search/aggregations/bucket/NestedTests.java
  38. 3 1
      src/test/java/org/elasticsearch/search/aggregations/bucket/RangeTests.java
  39. 6 0
      src/test/java/org/elasticsearch/search/aggregations/bucket/ReverseNestedTests.java
  40. 4 0
      src/test/java/org/elasticsearch/search/aggregations/bucket/ShardReduceTests.java
  41. 19 9
      src/test/java/org/elasticsearch/search/aggregations/bucket/ShardSizeTermsTests.java
  42. 102 13
      src/test/java/org/elasticsearch/search/aggregations/bucket/StringTermsTests.java
  43. 25 0
      src/test/java/org/elasticsearch/search/aggregations/bucket/TopHitsTests.java
  44. 7 2
      src/test/java/org/elasticsearch/search/aggregations/metrics/CardinalityTests.java
  45. 3 1
      src/test/java/org/elasticsearch/search/suggest/CompletionSuggestSearchTests.java

+ 73 - 0
docs/reference/search/aggregations/bucket/terms-aggregation.asciidoc

@@ -320,6 +320,79 @@ http://docs.oracle.com/javase/7/docs/api/java/util/regex/Pattern.html#UNICODE_CA
 http://docs.oracle.com/javase/7/docs/api/java/util/regex/Pattern.html#UNICODE_CHARACTER_CLASS[`UNICODE_CHARACTER_CLASS`] and
 http://docs.oracle.com/javase/7/docs/api/java/util/regex/Pattern.html#UNIX_LINES[`UNIX_LINES`]
 
+==== Collect mode
+
+coming[1.3.0] Deferring calculation of child aggregations
+
+For fields with many unique terms and a small number of required results it can be more efficient to delay the calculation
+of child aggregations until the top parent-level aggs have been pruned. Ordinarily, all branches of the aggregation tree
+are expanded in one depth-first pass and only then any pruning occurs. In some rare scenarios this can be very wasteful and can hit memory constraints.
+An example problem scenario is querying a movie database for the 10 most popular actors and their 5 most common co-stars: 
+
+[source,js]
+--------------------------------------------------
+{
+    "aggs" : {
+        "actors" : {
+             "terms" : {
+                 "field" : "actors",
+                 "size" : 10
+             },
+            "aggs" : {
+                "costars" : {
+                     "terms" : {
+                         "field" : "actors",
+                         "size" : 5
+                     }
+                 }
+            }
+         }
+    }
+}
+--------------------------------------------------
+
+Even though the number of movies may be comparatively small and we want only 50 result buckets there is a combinatorial explosion of buckets 
+during calculation - a single movie will produce n² buckets where n is the number of actors. The sane option would be to first determine 
+the 10 most popular actors and only then examine the top co-stars for these 10 actors. This alternative strategy is what we call the `breadth_first` collection
+mode as opposed to the default `depth_first` mode:
+ 
+[source,js]
+--------------------------------------------------
+{
+    "aggs" : {
+        "actors" : {
+             "terms" : {
+                 "field" : "actors",
+                 "size" : 10,
+                 "collect_mode" : "breadth_first"
+             },
+            "aggs" : {
+                "costars" : {
+                     "terms" : {
+                         "field" : "actors",
+                         "size" : 5
+                     }
+                 }
+            }
+         }
+    }
+}
+--------------------------------------------------
+
+
+When using `breadth_first` mode the set of documents that fall into the uppermost buckets are
+cached for subsequent replay so there is a memory overhead in doing this which is linear with the number of matching documents. 
+In most requests the volume of buckets generated is smaller than the number of documents that fall into them so the default `depth_first`
+collection mode is normally the best bet but occasionally the `breadth_first` strategy can be significantly more efficient. Currently 
+elasticsearch will always use the `depth_first` collect_mode unless explicitly instructed to use `breadth_first` as in the above example.
+Note that the `order` parameter can still be used to refer to data from a child aggregation when using the `breadth_first` setting - the parent
+aggregation understands that this child aggregation will need to be called first before any of the other child aggregations.
+
+WARNING: It is not possible to nest aggregations such as `top_hits` which require access to match score information under an aggregation that uses
+the `breadth_first` collection mode. This is because this would require a RAM buffer to hold the float score value for every document and
+this would typically be too costly in terms of RAM. 
+
+
 ==== Execution hint
 
 added[1.2.0] The `global_ordinals` execution mode

+ 172 - 4
src/main/java/org/elasticsearch/search/aggregations/Aggregator.java

@@ -20,12 +20,18 @@ package org.elasticsearch.search.aggregations;
 
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.search.Scorer;
+import org.elasticsearch.ElasticsearchParseException;
+import org.elasticsearch.common.ParseField;
 import org.elasticsearch.common.lease.Releasable;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.XContentParser;
+import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
 import org.elasticsearch.search.aggregations.support.AggregationContext;
 import org.elasticsearch.search.internal.SearchContext;
 import org.elasticsearch.search.internal.SearchContext.Lifetime;
+import org.elasticsearch.search.query.QueryPhaseExecutionException;
 
 import java.io.IOException;
 import java.util.*;
@@ -52,6 +58,8 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
         }
     }
 
+    public static final ParseField COLLECT_MODE = new ParseField("collect_mode");
+
     /**
      * Defines the nature of the aggregator's aggregation execution when nested in other aggregators and the buckets they create.
      */
@@ -67,6 +75,83 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
          */
         MULTI_BUCKETS
     }
+    
+    public enum SubAggCollectionMode {
+
+        /**
+         * Creates buckets and delegates to child aggregators in a single pass over
+         * the matching documents
+         */
+        DEPTH_FIRST(new ParseField("depth_first")),
+
+        /**
+         * Creates buckets for all matching docs and then prunes to top-scoring buckets
+         * before a second pass over the data when child aggregators are called
+         * but only for docs from the top-scoring buckets
+         */
+        BREADTH_FIRST(new ParseField("breadth_first"));
+
+        private final ParseField parseField;
+
+        SubAggCollectionMode(ParseField parseField) {
+            this.parseField = parseField;
+        }
+
+        public ParseField parseField() {
+            return parseField;
+        }
+
+        public static SubAggCollectionMode parse(String value) {
+            return parse(value, ParseField.EMPTY_FLAGS);
+        }
+
+        public static SubAggCollectionMode parse(String value, EnumSet<ParseField.Flag> flags) {
+            SubAggCollectionMode[] modes = SubAggCollectionMode.values();
+            for (SubAggCollectionMode mode : modes) {
+                if (mode.parseField.match(value, flags)) {
+                    return mode;
+                }
+            }
+            throw new ElasticsearchParseException("No " + COLLECT_MODE.getPreferredName() + " found for value [" + value + "]");
+        }
+    }
+    
+    // A scorer used for the deferred collection mode to handle any child aggs asking for scores that are not 
+    // recorded.
+    static final Scorer unavailableScorer=new Scorer(null){
+        private final String MSG = "A limitation of the " + SubAggCollectionMode.BREADTH_FIRST.parseField.getPreferredName()
+                + " collection mode is that scores cannot be buffered along with document IDs";
+
+        @Override
+        public float score() throws IOException {
+            throw new ElasticsearchParseException(MSG);
+        }
+
+        @Override
+        public int freq() throws IOException {
+            throw new ElasticsearchParseException(MSG);
+        }
+
+        @Override
+        public int advance(int arg0) throws IOException {
+            throw new ElasticsearchParseException(MSG);
+        }
+
+        @Override
+        public long cost() {
+            throw new ElasticsearchParseException(MSG);
+        }
+
+        @Override
+        public int docID() {
+            throw new ElasticsearchParseException(MSG);
+        }
+
+        @Override
+        public int nextDoc() throws IOException {
+            throw new ElasticsearchParseException(MSG);
+        }};
+    
 
     protected final String name;
     protected final Aggregator parent;
@@ -78,9 +163,10 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
     protected final BucketAggregationMode bucketAggregationMode;
     protected final AggregatorFactories factories;
     protected final Aggregator[] subAggregators;
-    protected final BucketCollector collectableSugAggregators;
+    protected BucketCollector collectableSubAggregators;
 
     private Map<String, Aggregator> subAggregatorbyName;
+    private DeferringBucketCollector recordingWrapper;
 
     /**
      * Constructs a new Aggregator.
@@ -103,8 +189,79 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
         assert factories != null : "sub-factories provided to BucketAggregator must not be null, use AggragatorFactories.EMPTY instead";
         this.factories = factories;
         this.subAggregators = factories.createSubAggregators(this, estimatedBucketsCount);
-        collectableSugAggregators = BucketCollector.wrap(Iterables.filter(Arrays.asList(subAggregators), COLLECTABLE_AGGREGATOR));
         context.searchContext().addReleasable(this, Lifetime.PHASE);
+        // Register a safeguard to highlight any invalid construction logic (call to this constructor without subsequent preCollection call)
+        collectableSubAggregators = new BucketCollector() {
+            void badState(){
+                throw new QueryPhaseExecutionException(Aggregator.this.context.searchContext(),
+                        "preCollection not called on new Aggregator before use", null);                
+            }
+            @Override
+            public void setNextReader(AtomicReaderContext reader) {
+                badState();
+            }
+
+            @Override
+            public void postCollection() throws IOException {
+                badState();
+            }
+
+            @Override
+            public void collect(int docId, long bucketOrdinal) throws IOException {
+                badState();
+            }
+
+            @Override
+            public void gatherAnalysis(BucketAnalysisCollector results, long bucketOrdinal) {
+                badState();
+            }
+        };
+    }
+    protected void preCollection() {
+        Iterable<Aggregator> collectables = Iterables.filter(Arrays.asList(subAggregators), COLLECTABLE_AGGREGATOR);
+        List<BucketCollector> nextPassCollectors = new ArrayList<>();
+        List<BucketCollector> thisPassCollectors = new ArrayList<>();
+        for (Aggregator aggregator : collectables) {
+            if (shouldDefer(aggregator)) {
+                nextPassCollectors.add(aggregator);
+            } else {
+                thisPassCollectors.add(aggregator);
+            }
+        }
+        if (nextPassCollectors.size() > 0) {
+            BucketCollector deferreds = BucketCollector.wrap(nextPassCollectors);
+            recordingWrapper = new DeferringBucketCollector(deferreds, context);
+            // TODO. Without line below we are dependent on subclass aggs
+            // delegating setNextReader calls on to child aggs
+            // which they don't seem to do as a matter of course. Need to move
+            // to a delegation model rather than broadcast
+            context.registerReaderContextAware(recordingWrapper);
+            thisPassCollectors.add(recordingWrapper);            
+        }
+        collectableSubAggregators = BucketCollector.wrap(thisPassCollectors);
+    }
+    
+    /**
+     * This method should be overidden by subclasses that want to defer calculation
+     * of a child aggregation until a first pass is complete and a set of buckets has 
+     * been pruned.
+     * Deferring collection will require the recording of all doc/bucketIds from the first 
+     * pass and then the sub class should call {@link #runDeferredCollections(long...)}  
+     * for the selected set of buckets that survive the pruning.
+     * @param aggregator the child aggregator 
+     * @return true if the aggregator should be deferred
+     * until a first pass at collection has completed
+     */
+    protected boolean shouldDefer(Aggregator aggregator) {
+        return false;
+    }
+    
+    protected void runDeferredCollections(long... bucketOrds){
+        // Being lenient here - ignore calls where there are no deferred collections to playback
+        if (recordingWrapper != null) {
+            context.setScorer(unavailableScorer);
+            recordingWrapper.prepareSelectedBuckets(bucketOrds);
+        } 
     }
 
     /**
@@ -174,14 +331,16 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
      * Called after collection of all document is done.
      */
     public final void postCollection() throws IOException {
-        collectableSugAggregators.postCollection();
+        collectableSubAggregators.postCollection();
         doPostCollection();
     }
 
     /** Called upon release of the aggregator. */
     @Override
     public void close() {
-        doClose();
+        try (Releasable _ = recordingWrapper) {
+            doClose();
+        }
     }
 
     /** Release instance-specific data. */
@@ -197,6 +356,14 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
      * @return  The aggregated & built aggregation
      */
     public abstract InternalAggregation buildAggregation(long owningBucketOrdinal);
+    
+    @Override
+    public void gatherAnalysis(BucketAnalysisCollector results, long bucketOrdinal) {
+        results.add(buildAggregation(bucketOrdinal));
+    }
+    
+    
+    
 
     public abstract InternalAggregation buildEmptyAggregation();
 
@@ -233,4 +400,5 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
         AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException;
 
     }
+
 }

+ 19 - 7
src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java

@@ -20,6 +20,7 @@ package org.elasticsearch.search.aggregations;
 
 import org.apache.lucene.index.AtomicReaderContext;
 import org.elasticsearch.ElasticsearchIllegalArgumentException;
+import org.elasticsearch.ElasticsearchIllegalStateException;
 import org.elasticsearch.common.lease.Releasables;
 import org.elasticsearch.common.util.ObjectArray;
 import org.elasticsearch.search.aggregations.Aggregator.BucketAggregationMode;
@@ -53,6 +54,10 @@ public class AggregatorFactories {
         if (aggregator.shouldCollect()) {
             context.registerReaderContextAware(aggregator);
         }
+        // Once the aggregator is fully constructed perform any initialisation -
+        // can't do everything in constructors if Aggregator base class needs 
+        // to delegate to subclasses as part of construction.
+        aggregator.preCollection();
         return aggregator;
     }
 
@@ -115,13 +120,7 @@ public class AggregatorFactories {
 
                 @Override
                 public InternalAggregation buildAggregation(long owningBucketOrdinal) {
-                    // The bucket ordinal may be out of range in case of eg. a terms/filter/terms where
-                    // the filter matches no document in the highest buckets of the first terms agg
-                    if (owningBucketOrdinal >= aggregators.size() || aggregators.get(owningBucketOrdinal) == null) {
-                        return first.buildEmptyAggregation();
-                    } else {
-                        return aggregators.get(owningBucketOrdinal).buildAggregation(0);
-                    }
+                    throw new ElasticsearchIllegalStateException("Invalid context - aggregation must use addResults() to collect child results");
                 }
 
                 @Override
@@ -133,7 +132,20 @@ public class AggregatorFactories {
                 public void doClose() {
                     Releasables.close(aggregators);
                 }
+
+                @Override
+                public void gatherAnalysis(BucketAnalysisCollector results, long owningBucketOrdinal) {
+                    // The bucket ordinal may be out of range in case of eg. a terms/filter/terms where
+                    // the filter matches no document in the highest buckets of the first terms agg
+                    if (owningBucketOrdinal >= aggregators.size() || aggregators.get(owningBucketOrdinal) == null) {
+                        results.add(first.buildEmptyAggregation());
+                    } else {
+                        aggregators.get(owningBucketOrdinal).gatherAnalysis(results,0);
+                    }                 
+                }
             };
+            
+            aggregators[i].preCollection();
         }
         return aggregators;
     }

+ 28 - 2
src/main/java/org/elasticsearch/search/aggregations/BucketCollector.java

@@ -30,6 +30,17 @@ import java.io.IOException;
  * A Collector that can collect data in separate buckets.
  */
 public abstract class BucketCollector implements ReaderContextAware {
+    
+    /**
+     * Used to gather a summary from a bucket
+     */
+    public interface BucketAnalysisCollector{
+        /**
+         * Used to ask {@link BucketCollector}s for their analysis of the content collected in a bucket
+         * @param analysis an object that represents the summary of a bucket as an {@link Aggregation}
+         */
+        void add(Aggregation aggregation);
+    }
 
     public final static BucketCollector NO_OP_COLLECTOR = new BucketCollector() {
 
@@ -37,16 +48,18 @@ public abstract class BucketCollector implements ReaderContextAware {
         public void collect(int docId, long bucketOrdinal) throws IOException {
             // no-op
         }
-
         @Override
         public void setNextReader(AtomicReaderContext reader) {
             // no-op
         }
-
         @Override
         public void postCollection() throws IOException {
             // no-op
         }
+        @Override
+        public void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal) {
+            // no-op
+        }
     };
 
     /**
@@ -83,6 +96,13 @@ public abstract class BucketCollector implements ReaderContextAware {
                         }
                     }
 
+                    @Override
+                    public void gatherAnalysis(BucketAnalysisCollector results, long bucketOrdinal) {
+                        for (BucketCollector collector : collectors) {
+                            collector.gatherAnalysis(results, bucketOrdinal);
+                        }
+                    }
+
                 };
         }
     }
@@ -105,4 +125,10 @@ public abstract class BucketCollector implements ReaderContextAware {
      */
     public abstract void postCollection() throws IOException;
 
+    /**
+     * Called post-collection to gather the results from surviving buckets.
+     * @param analysisCollector
+     * @param bucketOrdinal
+     */
+    public abstract void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal);
 }

+ 86 - 0
src/main/java/org/elasticsearch/search/aggregations/FilteringBucketCollector.java

@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.AtomicReaderContext;
+import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.ElasticsearchIllegalArgumentException;
+import org.elasticsearch.common.lease.Releasable;
+import org.elasticsearch.common.lease.Releasables;
+import org.elasticsearch.common.util.BigArrays;
+import org.elasticsearch.common.util.LongHash;
+
+import java.io.IOException;
+
+/**
+ * Filters a collection stream of docIds and related buckets using a sorted 
+ * list of valid bucket ordinals.
+ */
+public class FilteringBucketCollector extends BucketCollector implements Releasable {
+    
+    private final LongHash denseMap;
+    private final BucketCollector delegate;
+    
+    /**
+     * 
+     * @param the  valid BucketOrds
+     * @param delegate The collector that will be called for any buckets listed in sortedBucketOrds
+     */
+    public FilteringBucketCollector(long[] validBucketOrds, BucketCollector delegate, BigArrays bigArrays) {
+        denseMap = new LongHash(validBucketOrds.length, bigArrays);
+        for (int i = 0; i < validBucketOrds.length; i++) {
+            denseMap.add(validBucketOrds[i]);
+        }
+        this.delegate = delegate;
+    }
+
+    @Override
+    public final void setNextReader(AtomicReaderContext reader) {
+        delegate.setNextReader(reader);
+    }
+
+    @Override
+    public final void collect(int docId, long bucketOrdinal) throws IOException {
+        long ordinal = denseMap.find(bucketOrdinal);
+        if (ordinal >= 0) {
+            delegate.collect(docId, ordinal);
+        }
+    }
+
+    @Override
+    public final void postCollection() throws IOException {
+        delegate.postCollection();
+    }
+
+    @Override
+    public void close() throws ElasticsearchException {
+        Releasables.close(denseMap);
+    }
+
+    @Override
+    public void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal){        
+        long ordinal = denseMap.find(bucketOrdinal);
+        if (ordinal >= 0) {
+            delegate.gatherAnalysis(analysisCollector, ordinal);
+        } else {
+            throw new ElasticsearchIllegalArgumentException("Aggregation requested on a missing bucket #" + bucketOrdinal);
+        }
+    }
+}

+ 43 - 0
src/main/java/org/elasticsearch/search/aggregations/RecordingBucketCollector.java

@@ -0,0 +1,43 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.search.aggregations;
+
+import org.elasticsearch.ElasticsearchIllegalStateException;
+import org.elasticsearch.common.lease.Releasable;
+
+import java.io.IOException;
+
+/**
+ * Abstraction for implementations that record a "collect" stream for subsequent play-back
+ */
+public abstract class RecordingBucketCollector extends BucketCollector implements Releasable {
+
+    /**
+     * Replay a previously executed set of calls to the {@link #collect(int, long)} method
+     * @param collector the object which will be called to handle the playback
+     * @throws IOException
+     */
+    public abstract void replayCollection(BucketCollector collector) throws IOException;
+
+    @Override
+    public void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal) {
+        throw new ElasticsearchIllegalStateException("gatherAnalysis not supported");
+    }    
+}

+ 167 - 0
src/main/java/org/elasticsearch/search/aggregations/RecordingPerReaderBucketCollector.java

@@ -0,0 +1,167 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
+import org.apache.lucene.util.packed.AppendingPackedLongBuffer;
+import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.search.aggregations.support.AggregationContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Records a "collect" stream for subsequent play-back using a per-segment
+ * object to collect matches. Playback is fast because each collection
+ * contains only the required matches for the current reader.
+ */
+public class RecordingPerReaderBucketCollector extends RecordingBucketCollector  {
+
+    final List<PerSegmentCollects> perSegmentCollections = new ArrayList<>();
+    private PerSegmentCollects currentCollection;
+    private boolean recordingComplete;
+    
+    static class PerSegmentCollects {
+        AtomicReaderContext readerContext;
+        AppendingPackedLongBuffer docs;
+        AppendingPackedLongBuffer buckets;
+        int lastDocId = 0;
+
+        PerSegmentCollects(AtomicReaderContext readerContext) {
+            this.readerContext = readerContext;
+        }
+
+        void collect(int doc, long owningBucketOrdinal) throws IOException {
+            if (docs == null) {
+                // TODO unclear what might be reasonable constructor args to pass to this collection
+                // No way of accurately predicting how many docs will be collected 
+                docs = new AppendingPackedLongBuffer();
+            }
+            // Store as delta-encoded for better compression
+            docs.add(doc - lastDocId);
+            lastDocId = doc;
+            if (buckets == null) {
+                if (owningBucketOrdinal != 0) {
+                    // Store all of the prior bucketOrds (which up until now have
+                    // all been zero based)
+                    buckets = new AppendingPackedLongBuffer();
+                    for (int i = 0; i < docs.size() - 1; i++) {
+                        buckets.add(0);
+                    }
+                    // record the new non-zero bucketID
+                    buckets.add(owningBucketOrdinal);
+                }
+            } else {
+                buckets.add(owningBucketOrdinal);
+            }
+        }
+        void endCollect() {
+            if (docs != null) {
+                docs.freeze();
+            }
+            if (buckets != null) {
+                buckets.freeze();
+            }
+        }
+
+        boolean hasItems() {
+            return docs != null;
+        }
+
+        void replay(BucketCollector collector) throws IOException {
+            lastDocId = 0;
+            collector.setNextReader(readerContext);
+            if (!hasItems()) {
+                return;
+            }
+            if (buckets == null) {
+                final AppendingDeltaPackedLongBuffer.Iterator docsIter = docs.iterator();
+                while (docsIter.hasNext()) {
+                    lastDocId += (int) docsIter.next();
+                    collector.collect(lastDocId, 0);
+                }
+            } else {
+                assert docs.size() == buckets.size();
+                final AppendingDeltaPackedLongBuffer.Iterator docsIter = docs.iterator();
+                final AppendingDeltaPackedLongBuffer.Iterator bucketsIter = buckets.iterator();
+                while (docsIter.hasNext()) {
+                    lastDocId += (int) docsIter.next();
+                    collector.collect(lastDocId, bucketsIter.next());
+                }
+            }
+        }
+    }
+    
+    public RecordingPerReaderBucketCollector(AggregationContext context) {
+        // Call this method to achieve better compression in the recorded arrays of matches
+        context.ensureScoreDocsInOrder();        
+    }
+
+    @Override
+    public void setNextReader(AtomicReaderContext reader) {
+        if(recordingComplete){
+            // The way registration works for listening on reader changes we have the potential to be called > once
+            // TODO fixup the aggs framework so setNextReader calls are delegated to child aggs and not reliant on 
+            // registering a listener.
+            return;
+        }
+        stowLastSegmentCollection();
+        currentCollection = new PerSegmentCollects(reader);
+    }
+
+    private void stowLastSegmentCollection() {
+        if (currentCollection != null) {
+            if (currentCollection.hasItems()) {
+                currentCollection.endCollect();
+                perSegmentCollections.add(currentCollection);
+            }
+            currentCollection = null;
+        }
+    }
+
+    @Override
+    public void collect(int doc, long owningBucketOrdinal) throws IOException {
+        currentCollection.collect(doc, owningBucketOrdinal);
+    }
+
+    /*
+     * Allows clients to replay a stream of collected items. 
+     * 
+     */
+    @Override
+    public void replayCollection(BucketCollector collector) throws IOException{
+        for (PerSegmentCollects collection : perSegmentCollections) {
+            collection.replay(collector);
+        }
+        collector.postCollection();
+    }
+
+    @Override
+    public void postCollection() throws IOException {
+        recordingComplete = true;
+        stowLastSegmentCollection();
+    }
+
+    @Override
+    public void close() throws ElasticsearchException {
+    }    
+}

+ 35 - 12
src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java

@@ -20,13 +20,11 @@ package org.elasticsearch.search.aggregations.bucket;
 
 import org.elasticsearch.common.lease.Releasable;
 import org.elasticsearch.common.util.LongArray;
-import org.elasticsearch.search.aggregations.Aggregator;
-import org.elasticsearch.search.aggregations.AggregatorFactories;
-import org.elasticsearch.search.aggregations.InternalAggregation;
-import org.elasticsearch.search.aggregations.InternalAggregations;
+import org.elasticsearch.search.aggregations.*;
 import org.elasticsearch.search.aggregations.support.AggregationContext;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
 
 /**
@@ -73,7 +71,7 @@ public abstract class BucketsAggregator extends Aggregator {
      * Utility method to collect the given doc in the given bucket but not to update the doc counts of the bucket
      */
     protected final void collectBucketNoCounts(int doc, long bucketOrd) throws IOException {
-        collectableSugAggregators.collect(doc, bucketOrd);
+        collectableSubAggregators.collect(doc, bucketOrd);
     }
 
     /**
@@ -100,18 +98,43 @@ public abstract class BucketsAggregator extends Aggregator {
     }
 
     /**
-     * Utility method to build the aggregations of the given bucket (identified by the bucket ordinal)
+     * Required method to build the child aggregations of the given bucket (identified by the bucket ordinal).
      */
     protected final InternalAggregations bucketAggregations(long bucketOrd) {
-        final InternalAggregation[] aggregations = new InternalAggregation[subAggregators.length];
+        final ArrayList<InternalAggregation> childAggs = new ArrayList<>();
         final long bucketDocCount = bucketDocCount(bucketOrd);
-        for (int i = 0; i < subAggregators.length; i++) {
-            aggregations[i] = bucketDocCount == 0L
-                    ? subAggregators[i].buildEmptyAggregation()
-                    : subAggregators[i].buildAggregation(bucketOrd);
+        if (bucketDocCount == 0L) {
+            // All child aggs marked as empty
+            for (int i = 0; i < subAggregators.length; i++) {
+                childAggs.add(subAggregators[i].buildEmptyAggregation());
+            }
+        } else {
+            BucketAnalysisCollector analysisCollector = new BucketAnalysisCollector() {               
+                @Override
+                public void add(Aggregation analysis) {
+                    childAggs.add((InternalAggregation) analysis);
+                }
+            };
+            // Add the collectable sub aggs by asking the collect tree to gather
+            // results using ordinals that may have undergone transformation as the 
+            // result of the collection process e.g. filtering
+            // to a subset of buckets then rebasing the numbers in the deferred collection
+            collectableSubAggregators.gatherAnalysis(analysisCollector, bucketOrd);
+
+            // Also add the results of any non-collecting sub aggs using the top-level ordinals
+            for (int i = 0; i < subAggregators.length; i++) {
+                if (!subAggregators[i].shouldCollect()) {
+                    // Agg is not part of the collect tree - call directly
+                    childAggs.add(subAggregators[i].buildAggregation(bucketOrd));
+                }
+            }
         }
-        return new InternalAggregations(Arrays.asList(aggregations));
+
+        return new InternalAggregations(childAggs);
+        
     }
+    
+    
 
     /**
      * Utility method to build empty aggregations of the sub aggregators.

+ 128 - 0
src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java

@@ -0,0 +1,128 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.AtomicReaderContext;
+import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.common.lease.Releasable;
+import org.elasticsearch.common.lease.Releasables;
+import org.elasticsearch.search.aggregations.BucketCollector;
+import org.elasticsearch.search.aggregations.FilteringBucketCollector;
+import org.elasticsearch.search.aggregations.RecordingBucketCollector;
+import org.elasticsearch.search.aggregations.RecordingPerReaderBucketCollector;
+import org.elasticsearch.search.aggregations.support.AggregationContext;
+import org.elasticsearch.search.query.QueryPhaseExecutionException;
+
+import java.io.IOException;
+
+/**
+ * Buffers the matches in a collect stream and can replay a subset of the collected buckets
+ * to a deferred set of collectors.
+ * The rationale for not bundling all this logic into {@link RecordingBucketCollector} is to allow
+ * the possibility of alternative recorder impl choices while keeping the logic in here for
+ * setting {@link AggregationContext}'s setNextReader method and preparing the appropriate choice 
+ * of filtering logic for stream replay. These felt like agg-specific functions that should be kept away
+ * from the {@link RecordingBucketCollector} impl which is concentrated on efficient storage of doc and bucket IDs  
+ */
+public class DeferringBucketCollector extends BucketCollector implements Releasable {
+    
+    private final BucketCollector deferred;
+    private final RecordingBucketCollector recording;
+    private final AggregationContext context;
+    private FilteringBucketCollector filteredCollector;
+
+
+    public DeferringBucketCollector (BucketCollector deferred, AggregationContext context) {
+        this.deferred = deferred;
+        this.recording = new RecordingPerReaderBucketCollector(context);
+        this.context = context;
+    }
+
+    @Override
+    public void setNextReader(AtomicReaderContext reader) {
+        recording.setNextReader(reader);
+    }
+
+    @Override
+    public void collect(int docId, long bucketOrdinal) throws IOException {
+        recording.collect(docId, bucketOrdinal);
+    }
+
+    @Override
+    public void postCollection() throws IOException {
+        recording.postCollection();
+    }
+    
+    /**
+     * Plays a selection of the data cached from previous collect calls to the
+     * deferred collector.
+     * 
+     * @param survivingBucketOrds
+     *            the valid bucket ords for which deferred collection should be
+     *            attempted
+     */
+    public void prepareSelectedBuckets(long... survivingBucketOrds) {
+        
+        BucketCollector subs = new BucketCollector() {
+            @Override
+            public void setNextReader(AtomicReaderContext reader) {
+                // Need to set AggregationContext otherwise ValueSources in aggs
+                // don't read any values
+              context.setNextReader(reader);
+              deferred.setNextReader(reader);
+            }
+
+            @Override
+            public void collect(int docId, long bucketOrdinal) throws IOException {
+                deferred.collect(docId, bucketOrdinal);
+            }
+
+            @Override
+            public void postCollection() throws IOException {
+                deferred.postCollection();
+            }
+
+            @Override
+            public void gatherAnalysis(BucketAnalysisCollector results, long bucketOrdinal) {
+                deferred.gatherAnalysis(results, bucketOrdinal);
+            }
+        };
+
+        filteredCollector = new FilteringBucketCollector(survivingBucketOrds, subs, context.bigArrays());
+        try {
+            recording.replayCollection(filteredCollector);
+        } catch (IOException e) {
+            throw new QueryPhaseExecutionException(context.searchContext(), "Failed to replay deferred set of matching docIDs", e);
+        }
+    }
+
+    
+
+    @Override
+    public void close() throws ElasticsearchException {
+        Releasables.close(recording, filteredCollector);
+    }
+
+    @Override
+    public void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal)  {
+        filteredCollector.gatherAnalysis(analysisCollector, bucketOrdinal);
+    }
+
+}

+ 1 - 1
src/main/java/org/elasticsearch/search/aggregations/bucket/significant/GlobalOrdinalsSignificantTermsAggregator.java

@@ -48,7 +48,7 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri
                                                     IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent,
                                                     SignificantTermsAggregatorFactory termsAggFactory) {
 
-        super(name, factories, valuesSource, estimatedBucketCount, maxOrd, null, bucketCountThresholds, includeExclude, aggregationContext, parent);
+        super(name, factories, valuesSource, estimatedBucketCount, maxOrd, null, bucketCountThresholds, includeExclude, aggregationContext, parent, SubAggCollectionMode.DEPTH_FIRST);
         this.termsAggFactory = termsAggFactory;
     }
 

+ 1 - 1
src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java

@@ -42,7 +42,7 @@ public class SignificantLongTermsAggregator extends LongTermsAggregator {
               long estimatedBucketCount, BucketCountThresholds bucketCountThresholds,
               AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggFactory) {
 
-        super(name, factories, valuesSource, format, estimatedBucketCount, null, bucketCountThresholds, aggregationContext, parent);
+        super(name, factories, valuesSource, format, estimatedBucketCount, null, bucketCountThresholds, aggregationContext, parent, SubAggCollectionMode.DEPTH_FIRST);
         this.termsAggFactory = termsAggFactory;
     }
 

+ 1 - 1
src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTermsAggregator.java

@@ -51,7 +51,7 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator {
             IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent,
             SignificantTermsAggregatorFactory termsAggFactory) {
 
-        super(name, factories, valuesSource, estimatedBucketCount, null, bucketCountThresholds, includeExclude, aggregationContext, parent);
+        super(name, factories, valuesSource, estimatedBucketCount, null, bucketCountThresholds, includeExclude, aggregationContext, parent, SubAggCollectionMode.DEPTH_FIRST);
         this.termsAggFactory = termsAggFactory;
     }
 

+ 2 - 4
src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractStringTermsAggregator.java

@@ -28,13 +28,11 @@ import java.util.Collections;
 
 abstract class AbstractStringTermsAggregator extends TermsAggregator {
 
-    protected final InternalOrder order;
 
     public AbstractStringTermsAggregator(String name, AggregatorFactories factories,
             long estimatedBucketsCount, AggregationContext context, Aggregator parent,
-            InternalOrder order, BucketCountThresholds bucketCountThresholds) {
-        super(name, BucketAggregationMode.PER_BUCKET, factories, estimatedBucketsCount, context, parent, bucketCountThresholds);
-        this.order = InternalOrder.validate(order, this);
+            InternalOrder order, BucketCountThresholds bucketCountThresholds, SubAggCollectionMode subAggCollectMode) {
+        super(name, BucketAggregationMode.PER_BUCKET, factories, estimatedBucketsCount, context, parent, bucketCountThresholds, order, subAggCollectMode);
     }
 
     @Override

+ 13 - 0
src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractTermsParametersParser.java

@@ -21,6 +21,8 @@
 package org.elasticsearch.search.aggregations.bucket.terms;
 import org.elasticsearch.common.ParseField;
 import org.elasticsearch.common.xcontent.XContentParser;
+import org.elasticsearch.search.aggregations.Aggregator;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
 import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
 import org.elasticsearch.search.internal.SearchContext;
@@ -34,11 +36,15 @@ public abstract class AbstractTermsParametersParser {
     public static final ParseField MIN_DOC_COUNT_FIELD_NAME = new ParseField("min_doc_count");
     public static final ParseField SHARD_MIN_DOC_COUNT_FIELD_NAME = new ParseField("shard_min_doc_count");
     public static final ParseField REQUIRED_SIZE_FIELD_NAME = new ParseField("size");
+    
 
     //These are the results of the parsing.
     private TermsAggregator.BucketCountThresholds bucketCountThresholds = new TermsAggregator.BucketCountThresholds();
 
     private String executionHint = null;
+    
+    private SubAggCollectionMode collectMode = SubAggCollectionMode.DEPTH_FIRST;
+
 
     IncludeExclude includeExclude;
 
@@ -53,6 +59,11 @@ public abstract class AbstractTermsParametersParser {
     public IncludeExclude getIncludeExclude() {
         return includeExclude;
     }
+    
+    public SubAggCollectionMode getCollectionMode() {
+        return collectMode;
+    }
+
 
     public void parse(String aggregationName, XContentParser parser, SearchContext context, ValuesSourceParser vsParser, IncludeExclude.Parser incExcParser) throws IOException {
         bucketCountThresholds = getDefaultBucketCountThresholds();
@@ -69,6 +80,8 @@ public abstract class AbstractTermsParametersParser {
             } else if (token == XContentParser.Token.VALUE_STRING) {
                 if (EXECUTION_HINT_FIELD_NAME.match(currentFieldName)) {
                     executionHint = parser.text();
+                } else if(Aggregator.COLLECT_MODE.match(currentFieldName)){
+                    collectMode = SubAggCollectionMode.parse(parser.text());
                 } else {
                     parseSpecial(aggregationName, parser, context, token, currentFieldName);
                 }

+ 13 - 5
src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsAggregator.java

@@ -40,18 +40,16 @@ import java.util.Collections;
  */
 public class DoubleTermsAggregator extends TermsAggregator {
 
-    private final InternalOrder order;
     private final ValuesSource.Numeric valuesSource;
     private final ValueFormatter formatter;
     private final LongHash bucketOrds;
     private DoubleValues values;
 
     public DoubleTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format, long estimatedBucketCount,
-                               InternalOrder order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent) {
-        super(name, BucketAggregationMode.PER_BUCKET, factories, estimatedBucketCount, aggregationContext, parent, bucketCountThresholds);
+                               InternalOrder order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode) {
+        super(name, BucketAggregationMode.PER_BUCKET, factories, estimatedBucketCount, aggregationContext, parent, bucketCountThresholds, order, collectionMode);
         this.valuesSource = valuesSource;
         this.formatter = format != null ? format.formatter() : null;
-        this.order = InternalOrder.validate(order, this);
         bucketOrds = new LongHash(estimatedBucketCount, aggregationContext.bigArrays());
     }
 
@@ -117,12 +115,22 @@ public class DoubleTermsAggregator extends TermsAggregator {
             }
         }
 
+        // Get the top buckets
         final InternalTerms.Bucket[] list = new InternalTerms.Bucket[ordered.size()];
+        long survivingBucketOrds[] = new long[ordered.size()];
         for (int i = ordered.size() - 1; i >= 0; --i) {
             final DoubleTerms.Bucket bucket = (DoubleTerms.Bucket) ordered.pop();
-            bucket.aggregations = bucketAggregations(bucket.bucketOrd);
+            survivingBucketOrds[i] = bucket.bucketOrd;
             list[i] = bucket;
         }
+        // replay any deferred collections
+        runDeferredCollections(survivingBucketOrds);    
+        // Now build the aggs
+        for (int i = 0; i < list.length; i++) {
+          list[i].aggregations = bucketAggregations(list[i].bucketOrd);
+        }        
+        
+        
         return new DoubleTerms(name, order, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), Arrays.asList(list));
     }
 

+ 19 - 7
src/main/java/org/elasticsearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java

@@ -34,6 +34,7 @@ import org.elasticsearch.index.fielddata.ordinals.Ordinals;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.InternalAggregation;
+import org.elasticsearch.search.aggregations.bucket.terms.InternalTerms.Bucket;
 import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue;
 import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
 import org.elasticsearch.search.aggregations.support.AggregationContext;
@@ -63,8 +64,8 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
 
     public GlobalOrdinalsStringTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, long estimatedBucketCount,
                                                long maxOrd, InternalOrder order, BucketCountThresholds bucketCountThresholds,
-                                               IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent) {
-        super(name, factories, maxOrd, aggregationContext, parent, order, bucketCountThresholds);
+                                               IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode) {
+        super(name, factories, maxOrd, aggregationContext, parent, order, bucketCountThresholds, collectionMode);
         this.valuesSource = valuesSource;
         this.includeExclude = includeExclude;
     }
@@ -143,12 +144,23 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
             }
         }
 
+        // Get the top buckets
         final InternalTerms.Bucket[] list = new InternalTerms.Bucket[ordered.size()];
+        long survivingBucketOrds[] = new long[ordered.size()];
         for (int i = ordered.size() - 1; i >= 0; --i) {
             final StringTerms.Bucket bucket = (StringTerms.Bucket) ordered.pop();
-            bucket.aggregations = bucket.docCount == 0 ? bucketEmptyAggregations() : bucketAggregations(bucket.bucketOrd);
+            survivingBucketOrds[i] = bucket.bucketOrd;
             list[i] = bucket;
         }
+        //replay any deferred collections
+        runDeferredCollections(survivingBucketOrds);    
+        //Now build the aggs
+        for (int i = 0; i < list.length; i++) {
+          Bucket bucket = list[i];
+          bucket.aggregations = bucket.docCount == 0 ? bucketEmptyAggregations() : bucketAggregations(bucket.bucketOrd);
+        }        
+        
+        
 
         return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), Arrays.asList(list));
     }
@@ -163,9 +175,9 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
 
         public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, long estimatedBucketCount,
                         long maxOrd, InternalOrder order, BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, AggregationContext aggregationContext,
-                        Aggregator parent) {
+                        Aggregator parent, SubAggCollectionMode collectionMode) {
             // Set maxOrd to estimatedBucketCount! To be conservative with memory.
-            super(name, factories, valuesSource, estimatedBucketCount, estimatedBucketCount, order, bucketCountThresholds, includeExclude, aggregationContext, parent);
+            super(name, factories, valuesSource, estimatedBucketCount, estimatedBucketCount, order, bucketCountThresholds, includeExclude, aggregationContext, parent, collectionMode);
             bucketOrds = new LongHash(estimatedBucketCount, aggregationContext.bigArrays());
         }
 
@@ -209,8 +221,8 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
         private LongArray current;
 
         public LowCardinality(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, long estimatedBucketCount,
-                              long maxOrd, InternalOrder order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent) {
-            super(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, null, aggregationContext, parent);
+                              long maxOrd, InternalOrder order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode) {
+            super(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, null, aggregationContext, parent, collectionMode);
             this.segmentDocCounts = bigArrays.newLongArray(maxOrd, true);
         }
 

+ 20 - 7
src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java

@@ -41,20 +41,20 @@ import java.util.Collections;
  */
 public class LongTermsAggregator extends TermsAggregator {
 
-    private final InternalOrder order;
     protected final ValuesSource.Numeric valuesSource;
     protected final @Nullable ValueFormatter formatter;
     protected final LongHash bucketOrds;
     private LongValues values;
 
     public LongTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format, long estimatedBucketCount,
-                               InternalOrder order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent) {
-        super(name, BucketAggregationMode.PER_BUCKET, factories, estimatedBucketCount, aggregationContext, parent, bucketCountThresholds);
+                               InternalOrder order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode) {
+        super(name, BucketAggregationMode.PER_BUCKET, factories, estimatedBucketCount, aggregationContext, parent, bucketCountThresholds, order, subAggCollectMode);
         this.valuesSource = valuesSource;
         this.formatter = format != null ? format.formatter() : null;
-        this.order = InternalOrder.validate(order, this);
         bucketOrds = new LongHash(estimatedBucketCount, aggregationContext.bigArrays());
     }
+    
+    
 
     @Override
     public boolean shouldCollect() {
@@ -77,6 +77,7 @@ public class LongTermsAggregator extends TermsAggregator {
             if (bucketOrdinal < 0) { // already seen
                 bucketOrdinal = - 1 - bucketOrdinal;
                 collectExistingBucket(doc, bucketOrdinal);
+                
             } else {
                 collectBucket(doc, bucketOrdinal);
             }
@@ -116,16 +117,28 @@ public class LongTermsAggregator extends TermsAggregator {
                 spare = (LongTerms.Bucket) ordered.insertWithOverflow(spare);
             }
         }
-
+        
+        
+        
+        // Get the top buckets
         final InternalTerms.Bucket[] list = new InternalTerms.Bucket[ordered.size()];
+        long survivingBucketOrds[] = new long[ordered.size()];
         for (int i = ordered.size() - 1; i >= 0; --i) {
             final LongTerms.Bucket bucket = (LongTerms.Bucket) ordered.pop();
-            bucket.aggregations = bucketAggregations(bucket.bucketOrd);
+            survivingBucketOrds[i] = bucket.bucketOrd;
             list[i] = bucket;
         }
+      
+        runDeferredCollections(survivingBucketOrds);
+
+        //Now build the aggs
+        for (int i = 0; i < list.length; i++) {
+          list[i].aggregations = bucketAggregations(list[i].bucketOrd);
+        }
         return new LongTerms(name, order, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), Arrays.asList(list));
     }
-
+    
+    
     @Override
     public InternalAggregation buildEmptyAggregation() {
         return new LongTerms(name, order, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), Collections.<InternalTerms.Bucket>emptyList());

+ 17 - 8
src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java

@@ -54,9 +54,9 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
 
     public StringTermsAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
                                  InternalOrder order, BucketCountThresholds bucketCountThresholds,
-                                 IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent) {
+                                 IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode) {
 
-        super(name, factories, estimatedBucketCount, aggregationContext, parent, order, bucketCountThresholds);
+        super(name, factories, estimatedBucketCount, aggregationContext, parent, order, bucketCountThresholds, collectionMode);
         this.valuesSource = valuesSource;
         this.includeExclude = includeExclude;
         bucketOrds = new BytesRefHash(estimatedBucketCount, aggregationContext.bigArrays());
@@ -226,15 +226,24 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
             }
         }
 
+        // Get the top buckets
         final InternalTerms.Bucket[] list = new InternalTerms.Bucket[ordered.size()];
+        long survivingBucketOrds[] = new long[ordered.size()];
         for (int i = ordered.size() - 1; i >= 0; --i) {
             final StringTerms.Bucket bucket = (StringTerms.Bucket) ordered.pop();
-            // the terms are owned by the BytesRefHash, we need to pull a copy since the BytesRef hash data may be recycled at some point
-            bucket.termBytes = BytesRef.deepCopyOf(bucket.termBytes);
-            bucket.aggregations = bucketAggregations(bucket.bucketOrd);
+            survivingBucketOrds[i] = bucket.bucketOrd;
             list[i] = bucket;
         }
-
+        // replay any deferred collections
+        runDeferredCollections(survivingBucketOrds);    
+        // Now build the aggs
+        for (int i = 0; i < list.length; i++) {
+          final StringTerms.Bucket bucket = (StringTerms.Bucket)list[i];
+          bucket.termBytes = BytesRef.deepCopyOf(bucket.termBytes);
+          bucket.aggregations = bucketAggregations(bucket.bucketOrd);
+        }        
+        
+        
         return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), Arrays.asList(list));
     }
 
@@ -259,8 +268,8 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
         private LongArray ordinalToBucket;
 
         public WithOrdinals(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals valuesSource, long esitmatedBucketCount,
-                InternalOrder order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent) {
-            super(name, factories, valuesSource, esitmatedBucketCount, order, bucketCountThresholds, null, aggregationContext, parent);
+                InternalOrder order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode) {
+            super(name, factories, valuesSource, esitmatedBucketCount, order, bucketCountThresholds, null, aggregationContext, parent, collectionMode);
             this.valuesSource = valuesSource;
         }
 

+ 19 - 1
src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregator.java

@@ -26,7 +26,9 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
+import org.elasticsearch.search.aggregations.bucket.terms.InternalOrder.Aggregation;
 import org.elasticsearch.search.aggregations.support.AggregationContext;
+import org.elasticsearch.search.aggregations.support.OrderPath;
 
 import java.io.IOException;
 
@@ -125,9 +127,25 @@ public abstract class TermsAggregator extends BucketsAggregator {
     }
 
     protected final BucketCountThresholds bucketCountThresholds;
+    protected InternalOrder order;
+    protected Aggregator aggUsedForSorting;
+    protected SubAggCollectionMode subAggCollectMode;
 
-    public TermsAggregator(String name, BucketAggregationMode bucketAggregationMode, AggregatorFactories factories, long estimatedBucketsCount, AggregationContext context, Aggregator parent, BucketCountThresholds bucketCountThresholds) {
+    public TermsAggregator(String name, BucketAggregationMode bucketAggregationMode, AggregatorFactories factories, long estimatedBucketsCount, AggregationContext context, Aggregator parent, BucketCountThresholds bucketCountThresholds, InternalOrder order, SubAggCollectionMode subAggCollectMode) {
         super(name, bucketAggregationMode, factories, estimatedBucketsCount, context, parent);
         this.bucketCountThresholds = bucketCountThresholds;
+        this.order = InternalOrder.validate(order, this);
+        this.subAggCollectMode = subAggCollectMode;
+        // Don't defer any child agg if we are dependent on it for pruning results
+        if (order instanceof Aggregation){
+            OrderPath path = ((Aggregation) order).path();
+            aggUsedForSorting = path.resolveTopmostAggregator(this, false);
+        }
+    }
+
+    @Override
+    protected boolean shouldDefer(Aggregator aggregator) {
+        return (subAggCollectMode == SubAggCollectionMode.BREADTH_FIRST) && (aggregator != aggUsedForSorting);
     }
+    
 }

+ 20 - 18
src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java

@@ -22,6 +22,7 @@ import org.apache.lucene.search.IndexSearcher;
 import org.elasticsearch.ElasticsearchIllegalArgumentException;
 import org.elasticsearch.common.ParseField;
 import org.elasticsearch.search.aggregations.*;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
 import org.elasticsearch.search.aggregations.support.AggregationContext;
 import org.elasticsearch.search.aggregations.support.ValuesSource;
@@ -40,8 +41,8 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
             @Override
             Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
                               long maxOrd, InternalOrder order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
-                              AggregationContext aggregationContext, Aggregator parent) {
-                return new StringTermsAggregator(name, factories, valuesSource, estimatedBucketCount, order, bucketCountThresholds, includeExclude, aggregationContext, parent);
+                              AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode) {
+                return new StringTermsAggregator(name, factories, valuesSource, estimatedBucketCount, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode);
             }
 
             @Override
@@ -55,11 +56,11 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
             @Override
             Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
                               long maxOrd, InternalOrder order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
-                              AggregationContext aggregationContext, Aggregator parent) {
+                              AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode) {
                 if (includeExclude != null) {
-                    return MAP.create(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent);
+                    return MAP.create(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode);
                 }
-                return new StringTermsAggregator.WithOrdinals(name, factories, (ValuesSource.Bytes.WithOrdinals) valuesSource, estimatedBucketCount, order, bucketCountThresholds, aggregationContext, parent);
+                return new StringTermsAggregator.WithOrdinals(name, factories, (ValuesSource.Bytes.WithOrdinals) valuesSource, estimatedBucketCount, order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode);
             }
 
             @Override
@@ -73,8 +74,8 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
             @Override
             Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
                               long maxOrd, InternalOrder order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
-                              AggregationContext aggregationContext, Aggregator parent) {
-                return new GlobalOrdinalsStringTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent);
+                              AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode) {
+                return new GlobalOrdinalsStringTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode);
             }
 
             @Override
@@ -88,8 +89,8 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
             @Override
             Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
                               long maxOrd, InternalOrder order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
-                              AggregationContext aggregationContext, Aggregator parent) {
-                return new GlobalOrdinalsStringTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent);
+                              AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode) {
+                return new GlobalOrdinalsStringTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode);
             }
 
             @Override
@@ -102,11 +103,11 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
             @Override
             Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
                               long maxOrd, InternalOrder order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
-                              AggregationContext aggregationContext, Aggregator parent) {
+                              AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode) {
                 if (includeExclude != null || factories != null) {
-                    return GLOBAL_ORDINALS.create(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent);
+                    return GLOBAL_ORDINALS.create(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode);
                 }
-                return new GlobalOrdinalsStringTermsAggregator.LowCardinality(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, aggregationContext, parent);
+                return new GlobalOrdinalsStringTermsAggregator.LowCardinality(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode);
             }
 
             @Override
@@ -132,7 +133,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
 
         abstract Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount,
                                    long maxOrd, InternalOrder order, TermsAggregator.BucketCountThresholds bucketCountThresholds,
-                                   IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent);
+                                   IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode);
 
         abstract boolean needsGlobalOrdinals();
 
@@ -145,15 +146,16 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
     private final InternalOrder order;
     private final IncludeExclude includeExclude;
     private final String executionHint;
+    private SubAggCollectionMode subAggCollectMode;
     private final TermsAggregator.BucketCountThresholds bucketCountThresholds;
 
-    public TermsAggregatorFactory(String name, ValuesSourceConfig config, InternalOrder order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, String executionHint) {
-
+    public TermsAggregatorFactory(String name, ValuesSourceConfig config, InternalOrder order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, String executionHint,SubAggCollectionMode executionMode) {
         super(name, StringTerms.TYPE.name(), config);
         this.order = order;
         this.includeExclude = includeExclude;
         this.executionHint = executionHint;
         this.bucketCountThresholds = bucketCountThresholds;
+        this.subAggCollectMode = executionMode;
     }
 
     @Override
@@ -242,7 +244,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
 
             assert execution != null;
             valuesSource.setNeedsGlobalOrdinals(execution.needsGlobalOrdinals());
-            return execution.create(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent);
+            return execution.create(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode);
         }
 
         if (includeExclude != null) {
@@ -252,9 +254,9 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory {
 
         if (valuesSource instanceof ValuesSource.Numeric) {
             if (((ValuesSource.Numeric) valuesSource).isFloatingPoint()) {
-                return new DoubleTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), estimatedBucketCount, order, bucketCountThresholds, aggregationContext, parent);
+                return new DoubleTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), estimatedBucketCount, order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode);
             }
-            return new LongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), estimatedBucketCount, order, bucketCountThresholds, aggregationContext, parent);
+            return new LongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), estimatedBucketCount, order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode);
         }
 
         throw new AggregationExecutionException("terms aggregation cannot be applied to field [" + config.fieldContext().field() +

+ 11 - 0
src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsBuilder.java

@@ -20,6 +20,8 @@
 package org.elasticsearch.search.aggregations.bucket.terms;
 
 import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.search.aggregations.Aggregator;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.ValuesSourceAggregationBuilder;
 
 import java.io.IOException;
@@ -39,6 +41,7 @@ public class TermsBuilder extends ValuesSourceAggregationBuilder<TermsBuilder> {
     private String excludePattern;
     private int excludeFlags;
     private String executionHint;
+    private SubAggCollectionMode collectionMode;
 
     public TermsBuilder(String name) {
         super(name, "terms");
@@ -141,6 +144,11 @@ public class TermsBuilder extends ValuesSourceAggregationBuilder<TermsBuilder> {
         this.executionHint = executionHint;
         return this;
     }
+    
+    public TermsBuilder collectMode(SubAggCollectionMode mode) {
+        this.collectionMode = mode;
+        return this;
+    }
 
     @Override
     protected XContentBuilder doInternalXContent(XContentBuilder builder, Params params) throws IOException {
@@ -157,6 +165,9 @@ public class TermsBuilder extends ValuesSourceAggregationBuilder<TermsBuilder> {
             builder.field("order");
             order.toXContent(builder, params);
         }
+        if (collectionMode != null) {
+            builder.field(Aggregator.COLLECT_MODE.getPreferredName(), collectionMode.parseField().getPreferredName());
+        }
         if (includePattern != null) {
             if (includeFlags == 0) {
                 builder.field("include", includePattern);

+ 2 - 1
src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsParser.java

@@ -32,6 +32,7 @@ import java.io.IOException;
  */
 public class TermsParser implements Aggregator.Parser {
 
+    
     @Override
     public String type() {
         return StringTerms.TYPE.name();
@@ -52,7 +53,7 @@ public class TermsParser implements Aggregator.Parser {
         TermsAggregator.BucketCountThresholds bucketCountThresholds = aggParser.getBucketCountThresholds();
         bucketCountThresholds.ensureValidity();
         InternalOrder order = resolveOrder(aggParser.getOrderKey(), aggParser.isOrderAsc());
-        return new TermsAggregatorFactory(aggregationName, vsParser.config(), order, bucketCountThresholds, aggParser.getIncludeExclude(), aggParser.getExecutionHint());
+        return new TermsAggregatorFactory(aggregationName, vsParser.config(), order, bucketCountThresholds, aggParser.getIncludeExclude(), aggParser.getExecutionHint(), aggParser.getCollectionMode());
     }
 
     static InternalOrder resolveOrder(String key, boolean asc) {

+ 19 - 0
src/main/java/org/elasticsearch/search/aggregations/support/OrderPath.java

@@ -237,6 +237,25 @@ public class OrderPath {
         }
         return aggregator;
     }
+    
+    /**
+     * Resolves the topmost aggregator pointed by this path using the given root as a point of reference.
+     *
+     * @param root      The point of reference of this path
+     * @param validate  Indicates whether the path should be validated first over the given root aggregator
+     * @return          The first child aggregator of the root pointed by this path 
+     */
+    public Aggregator resolveTopmostAggregator(Aggregator root, boolean validate) {
+        if (validate) {
+            validate(root);
+        }
+        
+        OrderPath.Token token = tokens[0];
+        Aggregator aggregator = root.subAggregator(token.name);
+        assert (aggregator instanceof SingleBucketAggregator )
+                || (aggregator instanceof NumericMetricsAggregator) : "this should be picked up before aggregation execution - on validate";
+        return aggregator;
+    }    
 
     /**
      * Validates this path over the given aggregator as a point of reference.

+ 318 - 0
src/test/java/org/elasticsearch/benchmark/search/aggregations/SubAggregationSearchCollectModeBenchmark.java

@@ -0,0 +1,318 @@
+/*
+ * 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.benchmark.search.aggregations;
+
+import com.carrotsearch.hppc.ObjectOpenHashSet;
+import com.carrotsearch.randomizedtesting.generators.RandomStrings;
+import com.google.common.collect.Lists;
+import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
+import org.elasticsearch.action.admin.cluster.stats.ClusterStatsResponse;
+import org.elasticsearch.action.bulk.BulkRequestBuilder;
+import org.elasticsearch.action.bulk.BulkResponse;
+import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.action.search.SearchType;
+import org.elasticsearch.client.Client;
+import org.elasticsearch.client.Requests;
+import org.elasticsearch.common.StopWatch;
+import org.elasticsearch.common.jna.Natives;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.unit.ByteSizeValue;
+import org.elasticsearch.common.unit.SizeValue;
+import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.discovery.Discovery;
+import org.elasticsearch.node.Node;
+import org.elasticsearch.node.internal.InternalNode;
+import org.elasticsearch.search.aggregations.AggregationBuilders;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.elasticsearch.client.Requests.createIndexRequest;
+import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
+import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
+import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
+import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
+import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
+import static org.elasticsearch.node.NodeBuilder.nodeBuilder;
+
+/**
+ *
+ */
+public class SubAggregationSearchCollectModeBenchmark {
+
+    static long COUNT = SizeValue.parseSizeValue("2m").singles();
+    static int BATCH = 1000;
+    static int QUERY_WARMUP = 10;
+    static int QUERY_COUNT = 100;
+    static int NUMBER_OF_TERMS = 200;
+    static int NUMBER_OF_MULTI_VALUE_TERMS = 10;
+    static int STRING_TERM_SIZE = 5;
+
+    static Client client;
+    static InternalNode[] nodes;
+
+    public static void main(String[] args) throws Exception {
+        Natives.tryMlockall();
+        Random random = new Random();
+
+        Settings settings = settingsBuilder()
+                .put("index.refresh_interval", "-1")
+                .put("gateway.type", "local")
+                .put(SETTING_NUMBER_OF_SHARDS, 1)
+                .put(SETTING_NUMBER_OF_REPLICAS, 0)
+                .build();
+
+        String clusterName = SubAggregationSearchCollectModeBenchmark.class.getSimpleName();
+        nodes = new InternalNode[1];
+        for (int i = 0; i < nodes.length; i++) {
+            nodes[i] = (InternalNode) nodeBuilder().clusterName(clusterName)
+                    .settings(settingsBuilder().put(settings).put("name", "node" + i))
+                    .node();
+        }
+
+        Node clientNode = nodeBuilder()
+                .clusterName(clusterName)
+                .settings(settingsBuilder().put(settings).put("name", "client")).client(true).node();
+
+        client = clientNode.client();
+
+        Thread.sleep(10000);
+        try {
+            client.admin().indices().create(createIndexRequest("test").mapping("type1", jsonBuilder()
+              .startObject()
+                .startObject("type1")
+                  .startObject("properties")
+                    .startObject("s_value_dv")
+                      .field("type", "string")
+                      .field("index", "no")
+                      .startObject("fielddata")
+                        .field("format", "doc_values")
+                      .endObject()
+                    .endObject()
+                    .startObject("sm_value_dv")
+                      .field("type", "string")
+                      .field("index", "no")
+                      .startObject("fielddata")
+                        .field("format", "doc_values")
+                      .endObject()
+                    .endObject()
+                    .startObject("l_value_dv")
+                      .field("type", "long")
+                      .field("index", "no")
+                      .startObject("fielddata")
+                        .field("format", "doc_values")
+                      .endObject()
+                    .endObject()
+                    .startObject("lm_value_dv")
+                      .field("type", "long")
+                      .field("index", "no")
+                      .startObject("fielddata")
+                        .field("format", "doc_values")
+                      .endObject()
+                    .endObject()
+                  .endObject()
+                .endObject()
+              .endObject())).actionGet();
+
+            long[] lValues = new long[NUMBER_OF_TERMS];
+            for (int i = 0; i < NUMBER_OF_TERMS; i++) {
+                lValues[i] = ThreadLocalRandom.current().nextLong();
+            }
+            ObjectOpenHashSet<String> uniqueTerms = ObjectOpenHashSet.newInstance();
+            for (int i = 0; i < NUMBER_OF_TERMS; i++) {
+                boolean added;
+                do {
+                    added = uniqueTerms.add(RandomStrings.randomAsciiOfLength(random, STRING_TERM_SIZE));
+                } while (!added);
+            }
+            String[] sValues = uniqueTerms.toArray(String.class);
+            uniqueTerms = null;
+
+            StopWatch stopWatch = new StopWatch().start();
+
+            System.out.println("--> Indexing [" + COUNT + "] ...");
+            long ITERS = COUNT / BATCH;
+            long i = 1;
+            int counter = 0;
+            for (; i <= ITERS; i++) {
+                BulkRequestBuilder request = client.prepareBulk();
+                for (int j = 0; j < BATCH; j++) {
+                    counter++;
+
+                    XContentBuilder builder = jsonBuilder().startObject();
+                    builder.field("id", Integer.toString(counter));
+                    final String sValue = sValues[ThreadLocalRandom.current().nextInt(sValues.length)];
+                    final long lValue = lValues[ThreadLocalRandom.current().nextInt(lValues.length)];
+                    builder.field("s_value", sValue);
+                    builder.field("l_value", lValue);
+                    builder.field("s_value_dv", sValue);
+                    builder.field("l_value_dv", lValue);
+
+                    for (String field : new String[] {"sm_value", "sm_value_dv"}) {
+                        builder.startArray(field);
+                        for (int k = 0; k < NUMBER_OF_MULTI_VALUE_TERMS; k++) {
+                            builder.value(sValues[ThreadLocalRandom.current().nextInt(sValues.length)]);
+                        }
+                        builder.endArray();
+                    }
+
+                    for (String field : new String[] {"lm_value", "lm_value_dv"}) {
+                        builder.startArray(field);
+                        for (int k = 0; k < NUMBER_OF_MULTI_VALUE_TERMS; k++) {
+                            builder.value(lValues[ThreadLocalRandom.current().nextInt(sValues.length)]);
+                        }
+                        builder.endArray();
+                    }
+
+                    builder.endObject();
+
+                    request.add(Requests.indexRequest("test").type("type1").id(Integer.toString(counter))
+                            .source(builder));
+                }
+                BulkResponse response = request.execute().actionGet();
+                if (response.hasFailures()) {
+                    System.err.println("--> failures...");
+                }
+                if (((i * BATCH) % 10000) == 0) {
+                    System.out.println("--> Indexed " + (i * BATCH) + " took " + stopWatch.stop().lastTaskTime());
+                    stopWatch.start();
+                }
+            }
+            System.out.println("--> Indexing took " + stopWatch.totalTime() + ", TPS " + (((double) (COUNT)) / stopWatch.totalTime().secondsFrac()));
+        } catch (Exception e) {
+            System.out.println("--> Index already exists, ignoring indexing phase, waiting for green");
+            ClusterHealthResponse clusterHealthResponse = client.admin().cluster().prepareHealth().setWaitForGreenStatus().setTimeout("10m").execute().actionGet();
+            if (clusterHealthResponse.isTimedOut()) {
+                System.err.println("--> Timed out waiting for cluster health");
+            }
+        }
+        client.admin().indices().prepareRefresh().execute().actionGet();
+        COUNT = client.prepareCount().setQuery(matchAllQuery()).execute().actionGet().getCount();
+        System.out.println("--> Number of docs in index: " + COUNT);
+
+        List<StatsResult> stats = Lists.newArrayList();
+        stats.add(runTest("0000", new SubAggCollectionMode[] {SubAggCollectionMode.DEPTH_FIRST,SubAggCollectionMode.DEPTH_FIRST, SubAggCollectionMode.DEPTH_FIRST, SubAggCollectionMode.DEPTH_FIRST}));
+        stats.add(runTest("0001", new SubAggCollectionMode[] {SubAggCollectionMode.DEPTH_FIRST,SubAggCollectionMode.DEPTH_FIRST, SubAggCollectionMode.DEPTH_FIRST, SubAggCollectionMode.BREADTH_FIRST}));
+        stats.add(runTest("0010", new SubAggCollectionMode[] {SubAggCollectionMode.DEPTH_FIRST,SubAggCollectionMode.DEPTH_FIRST, SubAggCollectionMode.BREADTH_FIRST, SubAggCollectionMode.DEPTH_FIRST}));
+        stats.add(runTest("0011", new SubAggCollectionMode[] {SubAggCollectionMode.DEPTH_FIRST,SubAggCollectionMode.DEPTH_FIRST, SubAggCollectionMode.BREADTH_FIRST, SubAggCollectionMode.BREADTH_FIRST}));
+        stats.add(runTest("0100", new SubAggCollectionMode[] {SubAggCollectionMode.DEPTH_FIRST,SubAggCollectionMode.BREADTH_FIRST, SubAggCollectionMode.DEPTH_FIRST, SubAggCollectionMode.DEPTH_FIRST}));
+        stats.add(runTest("0101", new SubAggCollectionMode[] {SubAggCollectionMode.DEPTH_FIRST,SubAggCollectionMode.BREADTH_FIRST, SubAggCollectionMode.DEPTH_FIRST, SubAggCollectionMode.BREADTH_FIRST}));
+        stats.add(runTest("0110", new SubAggCollectionMode[] {SubAggCollectionMode.DEPTH_FIRST,SubAggCollectionMode.BREADTH_FIRST, SubAggCollectionMode.BREADTH_FIRST, SubAggCollectionMode.DEPTH_FIRST}));
+        stats.add(runTest("0111", new SubAggCollectionMode[] {SubAggCollectionMode.DEPTH_FIRST,SubAggCollectionMode.BREADTH_FIRST, SubAggCollectionMode.BREADTH_FIRST, SubAggCollectionMode.BREADTH_FIRST}));
+        stats.add(runTest("1000", new SubAggCollectionMode[] {SubAggCollectionMode.BREADTH_FIRST,SubAggCollectionMode.DEPTH_FIRST, SubAggCollectionMode.DEPTH_FIRST, SubAggCollectionMode.DEPTH_FIRST}));
+        stats.add(runTest("1001", new SubAggCollectionMode[] {SubAggCollectionMode.BREADTH_FIRST,SubAggCollectionMode.DEPTH_FIRST, SubAggCollectionMode.DEPTH_FIRST, SubAggCollectionMode.BREADTH_FIRST}));
+        stats.add(runTest("1010", new SubAggCollectionMode[] {SubAggCollectionMode.BREADTH_FIRST,SubAggCollectionMode.DEPTH_FIRST, SubAggCollectionMode.BREADTH_FIRST, SubAggCollectionMode.DEPTH_FIRST}));
+        stats.add(runTest("1011", new SubAggCollectionMode[] {SubAggCollectionMode.BREADTH_FIRST,SubAggCollectionMode.DEPTH_FIRST, SubAggCollectionMode.BREADTH_FIRST, SubAggCollectionMode.BREADTH_FIRST}));
+        stats.add(runTest("1100", new SubAggCollectionMode[] {SubAggCollectionMode.BREADTH_FIRST,SubAggCollectionMode.BREADTH_FIRST, SubAggCollectionMode.DEPTH_FIRST, SubAggCollectionMode.DEPTH_FIRST}));
+        stats.add(runTest("1101", new SubAggCollectionMode[] {SubAggCollectionMode.BREADTH_FIRST,SubAggCollectionMode.BREADTH_FIRST, SubAggCollectionMode.DEPTH_FIRST, SubAggCollectionMode.BREADTH_FIRST}));
+        stats.add(runTest("1110", new SubAggCollectionMode[] {SubAggCollectionMode.BREADTH_FIRST,SubAggCollectionMode.BREADTH_FIRST, SubAggCollectionMode.BREADTH_FIRST, SubAggCollectionMode.DEPTH_FIRST}));
+        stats.add(runTest("1111", new SubAggCollectionMode[] {SubAggCollectionMode.BREADTH_FIRST,SubAggCollectionMode.BREADTH_FIRST, SubAggCollectionMode.BREADTH_FIRST, SubAggCollectionMode.BREADTH_FIRST}));
+
+        System.out.println("------------------ SUMMARY ----------------------------------------------");
+        System.out.format(Locale.ENGLISH, "%35s%10s%10s%15s%15s\n", "name", "took", "millis", "fieldata size", "heap used");
+        for (StatsResult stat : stats) {
+            System.out.format(Locale.ENGLISH, "%35s%10s%10d%15s%15s\n", stat.name, TimeValue.timeValueMillis(stat.took), (stat.took / QUERY_COUNT), stat.fieldDataMemoryUsed, stat.heapUsed);
+        }
+        System.out.println("------------------ SUMMARY ----------------------------------------------");
+
+        clientNode.close();
+
+        for (Node node : nodes) {
+            node.close();
+        }
+    }
+
+    public static class StatsResult {
+        final String name;
+        final long took;
+        final ByteSizeValue fieldDataMemoryUsed;
+        final ByteSizeValue heapUsed;
+
+        public StatsResult(String name, long took, ByteSizeValue fieldDataMemoryUsed, ByteSizeValue heapUsed) {
+            this.name = name;
+            this.took = took;
+            this.fieldDataMemoryUsed = fieldDataMemoryUsed;
+            this.heapUsed = heapUsed;
+        }
+    }
+
+    private static StatsResult runTest(String name, SubAggCollectionMode[] collectionModes) {
+        long totalQueryTime;// LM VALUE
+
+        client.admin().indices().prepareClearCache().setFieldDataCache(true).execute().actionGet();
+        System.gc();
+
+        System.out.println("--> Warmup (" + name + ")...");
+        // run just the child query, warm up first
+        for (int j = 0; j < QUERY_WARMUP; j++) {
+            SearchResponse searchResponse = client.prepareSearch("test")
+                    .setSearchType(SearchType.COUNT)
+                    .setQuery(matchAllQuery())
+                    .addAggregation(AggregationBuilders.terms(name + "s_value").field("s_value").collectMode(collectionModes[0])
+                            .subAggregation(AggregationBuilders.terms(name + "l_value").field("l_value").collectMode(collectionModes[1])
+                                    .subAggregation(AggregationBuilders.terms(name + "s_value_dv").field("s_value_dv").collectMode(collectionModes[2])
+                                            .subAggregation(AggregationBuilders.terms(name + "l_value_dv").field("l_value_dv").collectMode(collectionModes[3])))))
+                    .execute().actionGet();
+            if (j == 0) {
+                System.out.println("--> Loading : took: " + searchResponse.getTook());
+            }
+            if (searchResponse.getHits().totalHits() != COUNT) {
+                System.err.println("--> mismatch on hits");
+            }
+        }
+        System.out.println("--> Warmup (" + name + ") DONE");
+
+
+        System.out.println("--> Running (" + name + ")...");
+        totalQueryTime = 0;
+        for (int j = 0; j < QUERY_COUNT; j++) {
+            SearchResponse searchResponse = client.prepareSearch("test")
+                    .setSearchType(SearchType.COUNT)
+                    .setQuery(matchAllQuery())
+                    .addAggregation(AggregationBuilders.terms(name + "s_value").field("s_value").collectMode(collectionModes[0])
+                            .subAggregation(AggregationBuilders.terms(name + "l_value").field("l_value").collectMode(collectionModes[1])
+                                    .subAggregation(AggregationBuilders.terms(name + "s_value_dv").field("s_value_dv").collectMode(collectionModes[2])
+                                            .subAggregation(AggregationBuilders.terms(name + "l_value_dv").field("l_value_dv").collectMode(collectionModes[3])))))
+                    .execute().actionGet();
+            if (searchResponse.getHits().totalHits() != COUNT) {
+                System.err.println("--> mismatch on hits");
+            }
+            totalQueryTime += searchResponse.getTookInMillis();
+        }
+        System.out.println("--> Terms Agg (" + name + "): " + (totalQueryTime / QUERY_COUNT) + "ms");
+
+        String[] nodeIds = new String[nodes.length];
+        for (int i = 0; i < nodeIds.length; i++) {
+            nodeIds[i] = nodes[i].injector().getInstance(Discovery.class).localNode().getId();
+        }
+
+        ClusterStatsResponse clusterStateResponse = client.admin().cluster().prepareClusterStats().setNodesIds(nodeIds).get();
+        ByteSizeValue heapUsed = clusterStateResponse.getNodesStats().getJvm().getHeapUsed();
+        System.out.println("--> Heap used: " + heapUsed);
+        ByteSizeValue fieldDataMemoryUsed = clusterStateResponse.getIndicesStats().getFieldData().getMemorySize();
+        System.out.println("--> Fielddata memory size: " + fieldDataMemoryUsed);
+
+        return new StatsResult(name, totalQueryTime, fieldDataMemoryUsed, heapUsed);
+    }
+
+}

+ 40 - 0
src/test/java/org/elasticsearch/benchmark/search/aggregations/TermsAggregationSearchBenchmark.java

@@ -41,6 +41,7 @@ import org.elasticsearch.discovery.Discovery;
 import org.elasticsearch.node.Node;
 import org.elasticsearch.node.internal.InternalNode;
 import org.elasticsearch.search.aggregations.AggregationBuilders;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 
 import java.util.List;
 import java.util.Locale;
@@ -95,6 +96,17 @@ public class TermsAggregationSearchBenchmark {
             SearchRequestBuilder addTermsStatsAgg(SearchRequestBuilder builder, String name, String keyField, String valueField) {
                 return builder.addAggregation(AggregationBuilders.terms(name).field(keyField).subAggregation(AggregationBuilders.stats("stats").field(valueField)));
             }
+        },
+        AGGREGATION_DEFERRED {
+            @Override
+            SearchRequestBuilder addTermsAgg(SearchRequestBuilder builder, String name, String field, String executionHint) {
+                return builder.addAggregation(AggregationBuilders.terms(name).executionHint(executionHint).field(field).collectMode(SubAggCollectionMode.BREADTH_FIRST));
+            }
+
+            @Override
+            SearchRequestBuilder addTermsStatsAgg(SearchRequestBuilder builder, String name, String keyField, String valueField) {
+                return builder.addAggregation(AggregationBuilders.terms(name).field(keyField).collectMode(SubAggCollectionMode.BREADTH_FIRST).subAggregation(AggregationBuilders.stats("stats").field(valueField)));
+            }
         };
         abstract SearchRequestBuilder addTermsAgg(SearchRequestBuilder builder, String name, String field, String executionHint);
         abstract SearchRequestBuilder addTermsStatsAgg(SearchRequestBuilder builder, String name, String keyField, String valueField);
@@ -251,10 +263,18 @@ public class TermsAggregationSearchBenchmark {
         stats.add(terms("terms_agg_s_dv_local_ordinals", Method.AGGREGATION, "s_value_dv", "ordinals"));
         stats.add(terms("terms_agg_map_s", Method.AGGREGATION, "s_value", "map"));
         stats.add(terms("terms_agg_map_s_dv", Method.AGGREGATION, "s_value_dv", "map"));
+        stats.add(terms("terms_agg_def_s", Method.AGGREGATION_DEFERRED, "s_value", null));
+        stats.add(terms("terms_agg_def_s_local_ordinals", Method.AGGREGATION_DEFERRED, "s_value", "ordinals"));
+        stats.add(terms("terms_agg_def_s_dv", Method.AGGREGATION_DEFERRED, "s_value_dv", null));
+        stats.add(terms("terms_agg_def_s_dv_local_ordinals", Method.AGGREGATION_DEFERRED, "s_value_dv", "ordinals"));
+        stats.add(terms("terms_agg_def_map_s", Method.AGGREGATION_DEFERRED, "s_value", "map"));
+        stats.add(terms("terms_agg_def_map_s_dv", Method.AGGREGATION_DEFERRED, "s_value_dv", "map"));
         stats.add(terms("terms_facet_l", Method.FACET, "l_value", null));
         stats.add(terms("terms_facet_l_dv", Method.FACET, "l_value_dv", null));
         stats.add(terms("terms_agg_l", Method.AGGREGATION, "l_value", null));
         stats.add(terms("terms_agg_l_dv", Method.AGGREGATION, "l_value_dv", null));
+        stats.add(terms("terms_agg_def_l", Method.AGGREGATION_DEFERRED, "l_value", null));
+        stats.add(terms("terms_agg_def_l_dv", Method.AGGREGATION_DEFERRED, "l_value_dv", null));
         stats.add(terms("terms_facet_sm", Method.FACET, "sm_value", null));
         stats.add(terms("terms_facet_sm_dv", Method.FACET, "sm_value_dv", null));
         stats.add(terms("terms_facet_map_sm", Method.FACET, "sm_value", "map"));
@@ -265,36 +285,56 @@ public class TermsAggregationSearchBenchmark {
         stats.add(terms("terms_agg_sm_dv_local_ordinals", Method.AGGREGATION, "sm_value_dv", "ordinals"));
         stats.add(terms("terms_agg_map_sm", Method.AGGREGATION, "sm_value", "map"));
         stats.add(terms("terms_agg_map_sm_dv", Method.AGGREGATION, "sm_value_dv", "map"));
+        stats.add(terms("terms_agg_def_sm", Method.AGGREGATION_DEFERRED, "sm_value", null));
+        stats.add(terms("terms_agg_def_sm_local_ordinals", Method.AGGREGATION_DEFERRED, "sm_value", "ordinals"));
+        stats.add(terms("terms_agg_def_sm_dv", Method.AGGREGATION_DEFERRED, "sm_value_dv", null));
+        stats.add(terms("terms_agg_def_sm_dv_local_ordinals", Method.AGGREGATION_DEFERRED, "sm_value_dv", "ordinals"));
+        stats.add(terms("terms_agg_def_map_sm", Method.AGGREGATION_DEFERRED, "sm_value", "map"));
+        stats.add(terms("terms_agg_def_map_sm_dv", Method.AGGREGATION_DEFERRED, "sm_value_dv", "map"));
         stats.add(terms("terms_facet_lm", Method.FACET, "lm_value", null));
         stats.add(terms("terms_facet_lm_dv", Method.FACET, "lm_value_dv", null));
         stats.add(terms("terms_agg_lm", Method.AGGREGATION, "lm_value", null));
         stats.add(terms("terms_agg_lm_dv", Method.AGGREGATION, "lm_value_dv", null));
+        stats.add(terms("terms_agg_def_lm", Method.AGGREGATION_DEFERRED, "lm_value", null));
+        stats.add(terms("terms_agg_def_lm_dv", Method.AGGREGATION_DEFERRED, "lm_value_dv", null));
 
         stats.add(termsStats("terms_stats_facet_s_l", Method.FACET, "s_value", "l_value", null));
         stats.add(termsStats("terms_stats_facet_s_l_dv", Method.FACET, "s_value_dv", "l_value_dv", null));
         stats.add(termsStats("terms_stats_agg_s_l", Method.AGGREGATION, "s_value", "l_value", null));
         stats.add(termsStats("terms_stats_agg_s_l_dv", Method.AGGREGATION, "s_value_dv", "l_value_dv", null));
+        stats.add(termsStats("terms_stats_agg_def_s_l", Method.AGGREGATION_DEFERRED, "s_value", "l_value", null));
+        stats.add(termsStats("terms_stats_agg_def_s_l_dv", Method.AGGREGATION_DEFERRED, "s_value_dv", "l_value_dv", null));
         stats.add(termsStats("terms_stats_facet_s_lm", Method.FACET, "s_value", "lm_value", null));
         stats.add(termsStats("terms_stats_facet_s_lm_dv", Method.FACET, "s_value_dv", "lm_value_dv", null));
         stats.add(termsStats("terms_stats_agg_s_lm", Method.AGGREGATION, "s_value", "lm_value", null));
         stats.add(termsStats("terms_stats_agg_s_lm_dv", Method.AGGREGATION, "s_value_dv", "lm_value_dv", null));
+        stats.add(termsStats("terms_stats_agg_def_s_lm", Method.AGGREGATION_DEFERRED, "s_value", "lm_value", null));
+        stats.add(termsStats("terms_stats_agg_def_s_lm_dv", Method.AGGREGATION_DEFERRED, "s_value_dv", "lm_value_dv", null));
         stats.add(termsStats("terms_stats_facet_sm_l", Method.FACET, "sm_value", "l_value", null));
         stats.add(termsStats("terms_stats_facet_sm_l_dv", Method.FACET, "sm_value_dv", "l_value_dv", null));
         stats.add(termsStats("terms_stats_agg_sm_l", Method.AGGREGATION, "sm_value", "l_value", null));
         stats.add(termsStats("terms_stats_agg_sm_l_dv", Method.AGGREGATION, "sm_value_dv", "l_value_dv", null));
+        stats.add(termsStats("terms_stats_agg_def_sm_l", Method.AGGREGATION_DEFERRED, "sm_value", "l_value", null));
+        stats.add(termsStats("terms_stats_agg_def_sm_l_dv", Method.AGGREGATION_DEFERRED, "sm_value_dv", "l_value_dv", null));
 
         stats.add(termsStats("terms_stats_facet_s_l", Method.FACET, "s_value", "l_value", null));
         stats.add(termsStats("terms_stats_facet_s_l_dv", Method.FACET, "s_value_dv", "l_value_dv", null));
         stats.add(termsStats("terms_stats_agg_s_l", Method.AGGREGATION, "s_value", "l_value", null));
         stats.add(termsStats("terms_stats_agg_s_l_dv", Method.AGGREGATION, "s_value_dv", "l_value_dv", null));
+        stats.add(termsStats("terms_stats_agg_def_s_l", Method.AGGREGATION_DEFERRED, "s_value", "l_value", null));
+        stats.add(termsStats("terms_stats_agg_def_s_l_dv", Method.AGGREGATION_DEFERRED, "s_value_dv", "l_value_dv", null));
         stats.add(termsStats("terms_stats_facet_s_lm", Method.FACET, "s_value", "lm_value", null));
         stats.add(termsStats("terms_stats_facet_s_lm_dv", Method.FACET, "s_value_dv", "lm_value_dv", null));
         stats.add(termsStats("terms_stats_agg_s_lm", Method.AGGREGATION, "s_value", "lm_value", null));
         stats.add(termsStats("terms_stats_agg_s_lm_dv", Method.AGGREGATION, "s_value_dv", "lm_value_dv", null));
+        stats.add(termsStats("terms_stats_agg_def_s_lm", Method.AGGREGATION_DEFERRED, "s_value", "lm_value", null));
+        stats.add(termsStats("terms_stats_agg_def_s_lm_dv", Method.AGGREGATION_DEFERRED, "s_value_dv", "lm_value_dv", null));
         stats.add(termsStats("terms_stats_facet_sm_l", Method.FACET, "sm_value", "l_value", null));
         stats.add(termsStats("terms_stats_facet_sm_l_dv", Method.FACET, "sm_value_dv", "l_value_dv", null));
         stats.add(termsStats("terms_stats_agg_sm_l", Method.AGGREGATION, "sm_value", "l_value", null));
         stats.add(termsStats("terms_stats_agg_sm_l_dv", Method.AGGREGATION, "sm_value_dv", "l_value_dv", null));
+        stats.add(termsStats("terms_stats_agg_def_sm_l", Method.AGGREGATION_DEFERRED, "sm_value", "l_value", null));
+        stats.add(termsStats("terms_stats_agg_def_sm_l_dv", Method.AGGREGATION_DEFERRED, "sm_value_dv", "l_value_dv", null));
         
         System.out.println("------------------ SUMMARY ----------------------------------------------");
         System.out.format(Locale.ENGLISH, "%35s%10s%10s%15s\n", "name", "took", "millis", "fieldata size");

+ 10 - 4
src/test/java/org/elasticsearch/index/fielddata/DisabledFieldDataFormatTests.java

@@ -23,6 +23,7 @@ import org.elasticsearch.action.search.SearchPhaseExecutionException;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.search.aggregations.AggregationBuilders;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope;
 
@@ -49,10 +50,12 @@ public class DisabledFieldDataFormatTests extends ElasticsearchIntegrationTest {
         // disable field data
         updateFormat("disabled");
 
+        SubAggCollectionMode aggCollectionMode = randomFrom(SubAggCollectionMode.values());
         SearchResponse resp = null;
         // try to run something that relies on field data and make sure that it fails
         try {
-            resp = client().prepareSearch("test").addAggregation(AggregationBuilders.terms("t").field("s")).execute().actionGet();
+            resp = client().prepareSearch("test").addAggregation(AggregationBuilders.terms("t").field("s")
+                    .collectMode(aggCollectionMode)).execute().actionGet();
             assertFailures(resp);
         } catch (SearchPhaseExecutionException e) {
             // expected
@@ -62,21 +65,24 @@ public class DisabledFieldDataFormatTests extends ElasticsearchIntegrationTest {
         updateFormat("paged_bytes");
 
         // try to run something that relies on field data and make sure that it works
-        resp = client().prepareSearch("test").addAggregation(AggregationBuilders.terms("t").field("s")).execute().actionGet();
+        resp = client().prepareSearch("test").addAggregation(AggregationBuilders.terms("t").field("s")
+                .collectMode(aggCollectionMode)).execute().actionGet();
         assertNoFailures(resp);
 
         // disable it again
         updateFormat("disabled");
 
         // this time, it should work because segments are already loaded
-        resp = client().prepareSearch("test").addAggregation(AggregationBuilders.terms("t").field("s")).execute().actionGet();
+        resp = client().prepareSearch("test").addAggregation(AggregationBuilders.terms("t").field("s")
+                .collectMode(aggCollectionMode)).execute().actionGet();
         assertNoFailures(resp);
 
         // but add more docs and the new segment won't be loaded
         client().prepareIndex("test", "type", "-1").setSource("s", "value").execute().actionGet();
         refresh();
         try {
-            resp = client().prepareSearch("test").addAggregation(AggregationBuilders.terms("t").field("s")).execute().actionGet();
+            resp = client().prepareSearch("test").addAggregation(AggregationBuilders.terms("t").field("s")
+                    .collectMode(aggCollectionMode)).execute().actionGet();
             assertFailures(resp);
         } catch (SearchPhaseExecutionException e) {
             // expected

+ 7 - 2
src/test/java/org/elasticsearch/index/mapper/copyto/CopyToMapperIntegrationTests.java

@@ -24,6 +24,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.index.query.QueryBuilders;
 import org.elasticsearch.search.aggregations.AggregationBuilders;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.junit.Test;
@@ -54,10 +55,14 @@ public class CopyToMapperIntegrationTests extends ElasticsearchIntegrationTest {
         }
         client().admin().indices().prepareRefresh("test-idx").execute().actionGet();
 
+        SubAggCollectionMode aggCollectionMode = randomFrom(SubAggCollectionMode.values());
+        
         SearchResponse response = client().prepareSearch("test-idx")
                 .setQuery(QueryBuilders.termQuery("even", true))
-                .addAggregation(AggregationBuilders.terms("test").field("test_field").size(recordCount * 2))
-                .addAggregation(AggregationBuilders.terms("test_raw").field("test_field_raw").size(recordCount * 2))
+                .addAggregation(AggregationBuilders.terms("test").field("test_field").size(recordCount * 2)
+                        .collectMode(aggCollectionMode))
+                .addAggregation(AggregationBuilders.terms("test_raw").field("test_field_raw").size(recordCount * 2)
+                        .collectMode(aggCollectionMode))
                 .execute().actionGet();
 
         assertThat(response.getHits().totalHits(), equalTo((long) recordCount));

+ 4 - 1
src/test/java/org/elasticsearch/percolator/PercolatorFacetsAndAggregationsTests.java

@@ -23,6 +23,7 @@ import org.elasticsearch.action.percolate.PercolateResponse;
 import org.elasticsearch.index.query.QueryBuilder;
 import org.elasticsearch.search.aggregations.Aggregation;
 import org.elasticsearch.search.aggregations.AggregationBuilders;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
 import org.elasticsearch.search.facet.FacetBuilders;
 import org.elasticsearch.search.facet.terms.TermsFacet;
@@ -79,7 +80,9 @@ public class PercolatorFacetsAndAggregationsTests extends ElasticsearchIntegrati
 
             boolean useAggs = randomBoolean();
             if (useAggs) {
-                percolateRequestBuilder.addAggregation(AggregationBuilders.terms("a").field("field2"));
+                SubAggCollectionMode aggCollectionMode = randomFrom(SubAggCollectionMode.values());
+                percolateRequestBuilder.addAggregation(AggregationBuilders.terms("a").field("field2")
+                        .collectMode(aggCollectionMode ));
             } else {
                 percolateRequestBuilder.addFacet(FacetBuilders.termsFacet("a").field("field2"));
 

+ 7 - 2
src/test/java/org/elasticsearch/search/aggregations/CombiTests.java

@@ -23,6 +23,7 @@ import com.carrotsearch.hppc.IntIntMap;
 import com.carrotsearch.hppc.IntIntOpenHashMap;
 import org.elasticsearch.action.index.IndexRequestBuilder;
 import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.missing.Missing;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
@@ -80,9 +81,11 @@ public class CombiTests extends ElasticsearchIntegrationTest {
         ensureSearchable();
 
 
+        SubAggCollectionMode aggCollectionMode = randomFrom(SubAggCollectionMode.values());
         SearchResponse response = client().prepareSearch("idx")
                 .addAggregation(missing("missing_values").field("value"))
-                .addAggregation(terms("values").field("value"))
+                .addAggregation(terms("values").field("value")
+                        .collectMode(aggCollectionMode ))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -122,9 +125,11 @@ public class CombiTests extends ElasticsearchIntegrationTest {
 
         ensureSearchable("idx");
 
+        SubAggCollectionMode aggCollectionMode = randomFrom(SubAggCollectionMode.values());
         SearchResponse searchResponse = client().prepareSearch("idx")
                 .addAggregation(histogram("values").field("value1").interval(1)
-                        .subAggregation(terms("names").field("name")))
+                        .subAggregation(terms("names").field("name")
+                                .collectMode(aggCollectionMode )))
                 .execute().actionGet();
 
         assertThat(searchResponse.getHits().getTotalHits(), Matchers.equalTo(0l));

+ 9 - 8
src/test/java/org/elasticsearch/search/aggregations/RandomTests.java

@@ -31,6 +31,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.index.fielddata.ordinals.InternalGlobalOrdinalsBuilder;
 import org.elasticsearch.index.query.FilterBuilders;
 import org.elasticsearch.index.query.RangeFilterBuilder;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.filter.Filter;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.range.Range;
@@ -214,12 +215,12 @@ public class RandomTests extends ElasticsearchIntegrationTest {
         };
 
         SearchResponse resp = client().prepareSearch("idx")
-                .addAggregation(terms("long").field("long_values").size(maxNumTerms).subAggregation(min("min").field("num")))
-                .addAggregation(terms("double").field("double_values").size(maxNumTerms).subAggregation(max("max").field("num")))
-                .addAggregation(terms("string_map").field("string_values").executionHint(TermsAggregatorFactory.ExecutionMode.MAP.toString()).size(maxNumTerms).subAggregation(stats("stats").field("num")))
-                .addAggregation(terms("string_ordinals").field("string_values").executionHint(TermsAggregatorFactory.ExecutionMode.ORDINALS.toString()).size(maxNumTerms).subAggregation(extendedStats("stats").field("num")))
-                .addAggregation(terms("string_global_ordinals").field("string_values").executionHint(globalOrdinalModes[randomInt(globalOrdinalModes.length - 1)].toString()).size(maxNumTerms).subAggregation(extendedStats("stats").field("num")))
-                .addAggregation(terms("string_global_ordinals_doc_values").field("string_values.doc_values").executionHint(globalOrdinalModes[randomInt(globalOrdinalModes.length - 1)].toString()).size(maxNumTerms).subAggregation(extendedStats("stats").field("num")))
+                .addAggregation(terms("long").field("long_values").size(maxNumTerms).collectMode(randomFrom(SubAggCollectionMode.values())).subAggregation(min("min").field("num")))
+                .addAggregation(terms("double").field("double_values").size(maxNumTerms).collectMode(randomFrom(SubAggCollectionMode.values())).subAggregation(max("max").field("num")))
+                .addAggregation(terms("string_map").field("string_values").collectMode(randomFrom(SubAggCollectionMode.values())).executionHint(TermsAggregatorFactory.ExecutionMode.MAP.toString()).size(maxNumTerms).subAggregation(stats("stats").field("num")))
+                .addAggregation(terms("string_ordinals").field("string_values").collectMode(randomFrom(SubAggCollectionMode.values())).executionHint(TermsAggregatorFactory.ExecutionMode.ORDINALS.toString()).size(maxNumTerms).subAggregation(extendedStats("stats").field("num")))
+                .addAggregation(terms("string_global_ordinals").field("string_values").collectMode(randomFrom(SubAggCollectionMode.values())).executionHint(globalOrdinalModes[randomInt(globalOrdinalModes.length - 1)].toString()).size(maxNumTerms).subAggregation(extendedStats("stats").field("num")))
+                .addAggregation(terms("string_global_ordinals_doc_values").field("string_values.doc_values").collectMode(randomFrom(SubAggCollectionMode.values())).executionHint(globalOrdinalModes[randomInt(globalOrdinalModes.length - 1)].toString()).size(maxNumTerms).subAggregation(extendedStats("stats").field("num")))
                 .execute().actionGet();
         assertAllSuccessful(resp);
         assertEquals(numDocs, resp.getHits().getTotalHits());
@@ -274,7 +275,7 @@ public class RandomTests extends ElasticsearchIntegrationTest {
         assertNoFailures(client().admin().indices().prepareRefresh("idx").setIndicesOptions(IndicesOptions.lenientExpandOpen()).execute().get());
 
         SearchResponse resp = client().prepareSearch("idx")
-                .addAggregation(terms("terms").field("values").script("floor(_value / interval)").param("interval", interval).size(maxNumTerms))
+                .addAggregation(terms("terms").field("values").collectMode(randomFrom(SubAggCollectionMode.values())).script("floor(_value / interval)").param("interval", interval).size(maxNumTerms))
                 .addAggregation(histogram("histo").field("values").interval(interval))
                 .execute().actionGet();
 
@@ -304,7 +305,7 @@ public class RandomTests extends ElasticsearchIntegrationTest {
         }
         indexRandom(true, indexingRequests);
 
-        SearchResponse response = client().prepareSearch("idx").addAggregation(terms("terms").field("double_value").subAggregation(percentiles("pcts").field("double_value"))).execute().actionGet();
+        SearchResponse response = client().prepareSearch("idx").addAggregation(terms("terms").field("double_value").collectMode(randomFrom(SubAggCollectionMode.values())).subAggregation(percentiles("pcts").field("double_value"))).execute().actionGet();
         assertAllSuccessful(response);
         assertEquals(numDocs, response.getHits().getTotalHits());
     }

+ 82 - 6
src/test/java/org/elasticsearch/search/aggregations/bucket/DoubleTermsTests.java

@@ -23,6 +23,7 @@ import org.elasticsearch.action.index.IndexRequestBuilder;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.index.query.FilterBuilders;
 import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.filter.Filter;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
@@ -102,7 +103,8 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
                         .minDocCount(randomInt(1))
-                        .size(0))
+                        .size(0)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -117,7 +119,8 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
     public void singleValueField() throws Exception {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
-                        .field(SINGLE_VALUED_FIELD_NAME))
+                        .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -143,6 +146,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
                         .size(20)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.term(true))) // we need to sort by terms cause we're checking the first 20 values
                 .execute().actionGet();
 
@@ -168,6 +172,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.term(true)))
                 .execute().actionGet();
 
@@ -194,6 +199,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.term(false)))
                 .execute().actionGet();
 
@@ -220,6 +226,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(sum("sum").field(MULTI_VALUED_FIELD_NAME)))
                 .execute().actionGet();
 
@@ -248,6 +255,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(sum("sum")))
                 .execute().actionGet();
 
@@ -276,6 +284,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .script("_value + 1"))
                 .execute().actionGet();
 
@@ -300,7 +309,8 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
     public void multiValuedField() throws Exception {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
-                        .field(MULTI_VALUED_FIELD_NAME))
+                        .field(MULTI_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -329,6 +339,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(MULTI_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .script("_value + 1"))
                 .execute().actionGet();
 
@@ -358,6 +369,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(MULTI_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .script("(long) _value / 1000 + 1"))
                 .execute().actionGet();
 
@@ -398,6 +410,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(MULTI_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .script("_value + 1")
                         .subAggregation(sum("sum")))
                 .execute().actionGet();
@@ -434,6 +447,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
     public void script_SingleValue() throws Exception {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .script("doc['" + MULTI_VALUED_FIELD_NAME + "'].value"))
                 .execute().actionGet();
 
@@ -459,6 +473,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(sum("sum")))
                 .execute().actionGet();
 
@@ -486,6 +501,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
     public void script_MultiValued() throws Exception {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .script("doc['" + MULTI_VALUED_FIELD_NAME + "'].values"))
                 .execute().actionGet();
 
@@ -520,6 +536,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
 
             SearchResponse response = client().prepareSearch("idx").setTypes("type")
                     .addAggregation(terms("terms")
+                            .collectMode(randomFrom(SubAggCollectionMode.values()))
                             .script("doc['" + MULTI_VALUED_FIELD_NAME + "'].values")
                             .subAggregation(sum("sum")))
                     .execute().actionGet();
@@ -537,6 +554,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
     public void script_MultiValued_WithAggregatorInherited_WithExplicitType() throws Exception {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .script("doc['" + MULTI_VALUED_FIELD_NAME + "'].values")
                         .valueType(Terms.ValueType.DOUBLE)
                         .subAggregation(sum("sum")))
@@ -575,7 +593,8 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx_unmapped").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
-                        .size(randomInt(5)))
+                        .size(randomInt(5))
+                        .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -591,7 +610,8 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
     public void partiallyUnmapped() throws Exception {
         SearchResponse response = client().prepareSearch("idx_unmapped", "idx").setTypes("type")
                 .addAggregation(terms("terms")
-                        .field(SINGLE_VALUED_FIELD_NAME))
+                        .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -637,6 +657,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.aggregation("avg_i", asc))
                         .subAggregation(avg("avg_i").field(SINGLE_VALUED_FIELD_NAME))
                 ).execute().actionGet();
@@ -660,12 +681,56 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
         }
     }
 
+    @Test
+    public void singleValuedField_OrderedBySingleValueSubAggregationAscWithSubTermsAgg() throws Exception {
+        boolean asc = true;
+        SearchResponse response = client().prepareSearch("idx").setTypes("type")
+                .addAggregation(terms("terms")
+                        .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
+                        .order(Terms.Order.aggregation("avg_i", asc))
+                        .subAggregation(avg("avg_i").field(SINGLE_VALUED_FIELD_NAME)).subAggregation(terms("subTerms").field(MULTI_VALUED_FIELD_NAME)
+                                .collectMode(randomFrom(SubAggCollectionMode.values())))
+                ).execute().actionGet();
+
+
+        assertSearchResponse(response);
+
+        Terms terms = response.getAggregations().get("terms");
+        assertThat(terms, notNullValue());
+        assertThat(terms.getName(), equalTo("terms"));
+        assertThat(terms.getBuckets().size(), equalTo(5));
+
+        for (int i = 0; i < 5; i++) {
+            Terms.Bucket bucket = terms.getBucketByKey("" + (double) i);
+            assertThat(bucket, notNullValue());
+            assertThat(key(bucket), equalTo("" + (double)i));
+            assertThat(bucket.getDocCount(), equalTo(1l));
+            
+            Avg avg = bucket.getAggregations().get("avg_i");
+            assertThat(avg, notNullValue());
+            assertThat(avg.getValue(), equalTo((double) i));
+            
+            Terms subTermsAgg = bucket.getAggregations().get("subTerms");
+            assertThat(subTermsAgg, notNullValue());
+            assertThat(subTermsAgg.getBuckets().size(), equalTo(2));
+            double j = i;
+            for (Terms.Bucket subBucket : subTermsAgg.getBuckets()) {
+                assertThat(subBucket, notNullValue());
+                assertThat(key(subBucket), equalTo(String.valueOf(j)));
+                assertThat(subBucket.getDocCount(), equalTo(1l));
+                j++;
+            }
+        }
+    }
+
     @Test
     public void singleValuedField_OrderedBySingleBucketSubAggregationAsc() throws Exception {
         boolean asc = randomBoolean();
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("num_tags")
                         .field("num_tag")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.aggregation("filter", asc))
                         .subAggregation(filter("filter").filter(FilterBuilders.matchAllFilter()))
                 ).execute().actionGet();
@@ -703,6 +768,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("tags")
                         .field("num_tag")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.aggregation("filter1>filter2>max", asc))
                         .subAggregation(filter("filter1").filter(FilterBuilders.matchAllFilter())
                                 .subAggregation(filter("filter2").filter(FilterBuilders.matchAllFilter())
@@ -758,6 +824,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
             client().prepareSearch("idx").setTypes("type")
                     .addAggregation(terms("terms")
                             .field(SINGLE_VALUED_FIELD_NAME)
+                            .collectMode(randomFrom(SubAggCollectionMode.values()))
                             .order(Terms.Order.aggregation("avg_i", true))
                     ).execute().actionGet();
 
@@ -775,8 +842,10 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
             client().prepareSearch("idx").setTypes("type")
                     .addAggregation(terms("terms")
                             .field(SINGLE_VALUED_FIELD_NAME)
+                            .collectMode(randomFrom(SubAggCollectionMode.values()))
                             .order(Terms.Order.aggregation("num_tags", true))
-                            .subAggregation(terms("num_tags").field("num_tags"))
+                            .subAggregation(terms("num_tags").field("num_tags")
+                                    .collectMode(randomFrom(SubAggCollectionMode.values())))
                     ).execute().actionGet();
 
             fail("Expected search to fail when trying to sort terms aggregation by sug-aggregation which is not of a metrics type");
@@ -793,6 +862,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
             client().prepareSearch("idx").setTypes("type")
                     .addAggregation(terms("terms")
                             .field(SINGLE_VALUED_FIELD_NAME)
+                            .collectMode(randomFrom(SubAggCollectionMode.values()))
                             .order(Terms.Order.aggregation("stats.foo", true))
                             .subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))
                     ).execute().actionGet();
@@ -812,6 +882,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
             client().prepareSearch("idx").setTypes("type")
                     .addAggregation(terms("terms")
                             .field(SINGLE_VALUED_FIELD_NAME)
+                            .collectMode(randomFrom(SubAggCollectionMode.values()))
                             .order(Terms.Order.aggregation("stats", true))
                             .subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))
                     ).execute().actionGet();
@@ -830,6 +901,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.aggregation("avg_i", asc))
                         .subAggregation(avg("avg_i").field(SINGLE_VALUED_FIELD_NAME))
                 ).execute().actionGet();
@@ -862,6 +934,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.aggregation("stats.avg", asc))
                         .subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))
                 ).execute().actionGet();
@@ -892,6 +965,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.aggregation("stats.avg", asc))
                         .subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))
                 ).execute().actionGet();
@@ -922,6 +996,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.aggregation("stats.variance", asc))
                         .subAggregation(extendedStats("stats").field(SINGLE_VALUED_FIELD_NAME))
                 ).execute().actionGet();
@@ -951,6 +1026,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .setQuery(functionScoreQuery(matchAllQuery()).add(ScoreFunctionBuilders.scriptFunction("doc['" + SINGLE_VALUED_FIELD_NAME + "'].value")))
                 .addAggregation(terms("terms")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .script("ceil(_doc.score/3)")
                 ).execute().actionGet();
 

+ 3 - 1
src/test/java/org/elasticsearch/search/aggregations/bucket/GeoDistanceTests.java

@@ -23,6 +23,7 @@ import org.elasticsearch.action.index.IndexRequestBuilder;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.common.unit.DistanceUnit;
 import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.range.geodistance.GeoDistance;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
@@ -297,7 +298,8 @@ public class GeoDistanceTests extends ElasticsearchIntegrationTest {
                         .addUnboundedTo(500)
                         .addRange(500, 1000)
                         .addUnboundedFrom(1000)
-                        .subAggregation(terms("cities").field("city")))
+                        .subAggregation(terms("cities").field("city")
+                                .collectMode(randomFrom(SubAggCollectionMode.values()))))
                 .execute().actionGet();
 
         assertSearchResponse(response);

+ 3 - 1
src/test/java/org/elasticsearch/search/aggregations/bucket/HistogramTests.java

@@ -21,6 +21,7 @@ package org.elasticsearch.search.aggregations.bucket;
 import com.carrotsearch.hppc.LongOpenHashSet;
 import org.elasticsearch.action.index.IndexRequestBuilder;
 import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.filter.Filter;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
@@ -608,7 +609,8 @@ 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)
+                                .collectMode(randomFrom(SubAggCollectionMode.values())).order(Terms.Order.term(true))))
                 .execute().actionGet();
 
         assertSearchResponse(response);

+ 80 - 5
src/test/java/org/elasticsearch/search/aggregations/bucket/LongTermsTests.java

@@ -22,6 +22,7 @@ import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.action.index.IndexRequestBuilder;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.index.query.FilterBuilders;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.filter.Filter;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
@@ -104,6 +105,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("high_card_type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .minDocCount(randomInt(1))
                         .size(0))
                 .execute().actionGet();
@@ -120,7 +122,8 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
     public void singleValueField() throws Exception {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
-                        .field(SINGLE_VALUED_FIELD_NAME))
+                        .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -146,6 +149,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
                         .size(20)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.term(true))) // we need to sort by terms cause we're checking the first 20 values
                 .execute().actionGet();
 
@@ -171,6 +175,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.term(true)))
                 .execute().actionGet();
         assertSearchResponse(response);
@@ -195,6 +200,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.term(false)))
                 .execute().actionGet();
 
@@ -221,6 +227,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(sum("sum").field(MULTI_VALUED_FIELD_NAME)))
                 .execute().actionGet();
 
@@ -249,6 +256,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(sum("sum")))
                 .execute().actionGet();
 
@@ -277,6 +285,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .script("_value + 1"))
                 .execute().actionGet();
 
@@ -301,7 +310,8 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
     public void multiValuedField() throws Exception {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
-                        .field(MULTI_VALUED_FIELD_NAME))
+                        .field(MULTI_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -330,6 +340,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(MULTI_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .script("_value - 1"))
                 .execute().actionGet();
 
@@ -359,6 +370,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(MULTI_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .script("floor(_value / 1000 + 1)"))
                 .execute().actionGet();
 
@@ -399,6 +411,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(MULTI_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .script("_value + 1")
                         .subAggregation(sum("sum")))
                 .execute().actionGet();
@@ -435,6 +448,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
     public void script_SingleValue() throws Exception {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .script("doc['" + SINGLE_VALUED_FIELD_NAME + "'].value"))
                 .execute().actionGet();
 
@@ -460,6 +474,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(sum("sum")))
                 .execute().actionGet();
 
@@ -487,6 +502,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
     public void script_MultiValued() throws Exception {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .script("doc['" + MULTI_VALUED_FIELD_NAME + "'].values"))
                 .execute().actionGet();
 
@@ -521,6 +537,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
 
             SearchResponse response = client().prepareSearch("idx").setTypes("type")
                     .addAggregation(terms("terms")
+                            .collectMode(randomFrom(SubAggCollectionMode.values()))
                             .script("doc['" + MULTI_VALUED_FIELD_NAME + "'].values")
                             .subAggregation(sum("sum")))
                     .execute().actionGet();
@@ -536,6 +553,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
     public void script_MultiValued_WithAggregatorInherited_WithExplicitType() throws Exception {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .script("doc['" + MULTI_VALUED_FIELD_NAME + "'].values")
                         .valueType(Terms.ValueType.LONG)
                         .subAggregation(sum("sum")))
@@ -574,7 +592,8 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx_unmapped").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
-                        .size(randomInt(5)))
+                        .size(randomInt(5))
+                        .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -590,7 +609,8 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
     public void partiallyUnmapped() throws Exception {
         SearchResponse response = client().prepareSearch("idx_unmapped", "idx").setTypes("type")
                 .addAggregation(terms("terms")
-                        .field(SINGLE_VALUED_FIELD_NAME))
+                        .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -636,6 +656,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.aggregation("avg_i", asc))
                         .subAggregation(avg("avg_i").field(SINGLE_VALUED_FIELD_NAME))
                 ).execute().actionGet();
@@ -659,12 +680,56 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
         }
     }
 
+    @Test
+    public void singleValuedField_OrderedBySingleValueSubAggregationAscWithTermsSubAgg() throws Exception {
+        boolean asc = true;
+        SearchResponse response = client().prepareSearch("idx").setTypes("type")
+                .addAggregation(terms("terms")
+                        .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
+                        .order(Terms.Order.aggregation("avg_i", asc))
+                        .subAggregation(avg("avg_i").field(SINGLE_VALUED_FIELD_NAME)).subAggregation(terms("subTerms").field(MULTI_VALUED_FIELD_NAME)
+                                .collectMode(randomFrom(SubAggCollectionMode.values())))
+                ).execute().actionGet();
+
+
+        assertSearchResponse(response);
+
+        Terms terms = response.getAggregations().get("terms");
+        assertThat(terms, notNullValue());
+        assertThat(terms.getName(), equalTo("terms"));
+        assertThat(terms.getBuckets().size(), equalTo(5));
+
+        for (int i = 0; i < 5; i++) {
+            Terms.Bucket bucket = terms.getBucketByKey("" + i);
+            assertThat(bucket, notNullValue());
+            assertThat(key(bucket), equalTo("" + i));
+            assertThat(bucket.getDocCount(), equalTo(1l));
+            
+            Avg avg = bucket.getAggregations().get("avg_i");
+            assertThat(avg, notNullValue());
+            assertThat(avg.getValue(), equalTo((double) i));
+            
+            Terms subTermsAgg = bucket.getAggregations().get("subTerms");
+            assertThat(subTermsAgg, notNullValue());
+            assertThat(subTermsAgg.getBuckets().size(), equalTo(2));
+            int j = i;
+            for (Terms.Bucket subBucket : subTermsAgg.getBuckets()) {
+                assertThat(subBucket, notNullValue());
+                assertThat(key(subBucket), equalTo(String.valueOf(j)));
+                assertThat(subBucket.getDocCount(), equalTo(1l));
+                j++;
+            }
+        }
+    }
+
     @Test
     public void singleValuedField_OrderedBySingleBucketSubAggregationAsc() throws Exception {
         boolean asc = randomBoolean();
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("num_tags")
                         .field("num_tag")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.aggregation("filter", asc))
                         .subAggregation(filter("filter").filter(FilterBuilders.matchAllFilter()))
                 ).execute().actionGet();
@@ -702,6 +767,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("tags")
                         .field("num_tag")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.aggregation("filter1>filter2>max", asc))
                         .subAggregation(filter("filter1").filter(FilterBuilders.matchAllFilter())
                                 .subAggregation(filter("filter2").filter(FilterBuilders.matchAllFilter())
@@ -757,6 +823,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
             client().prepareSearch("idx").setTypes("type")
                     .addAggregation(terms("terms")
                             .field(SINGLE_VALUED_FIELD_NAME)
+                            .collectMode(randomFrom(SubAggCollectionMode.values()))
                             .order(Terms.Order.aggregation("avg_i", true))
                     ).execute().actionGet();
 
@@ -774,8 +841,10 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
             client().prepareSearch("idx").setTypes("type")
                     .addAggregation(terms("terms")
                             .field(SINGLE_VALUED_FIELD_NAME)
+                            .collectMode(randomFrom(SubAggCollectionMode.values()))
                             .order(Terms.Order.aggregation("num_tags", true))
-                            .subAggregation(terms("num_tags").field("num_tags"))
+                            .subAggregation(terms("num_tags").field("num_tags")
+                                    .collectMode(randomFrom(SubAggCollectionMode.values())))
                     ).execute().actionGet();
 
             fail("Expected search to fail when trying to sort terms aggregation by sug-aggregation which is not of a metrics type");
@@ -792,6 +861,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
             client().prepareSearch("idx").setTypes("type")
                     .addAggregation(terms("terms")
                             .field(SINGLE_VALUED_FIELD_NAME)
+                            .collectMode(randomFrom(SubAggCollectionMode.values()))
                             .order(Terms.Order.aggregation("stats.foo", true))
                             .subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))
                     ).execute().actionGet();
@@ -811,6 +881,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
             client().prepareSearch("idx").setTypes("type")
                     .addAggregation(terms("terms")
                             .field(SINGLE_VALUED_FIELD_NAME)
+                            .collectMode(randomFrom(SubAggCollectionMode.values()))
                             .order(Terms.Order.aggregation("stats", true))
                             .subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))
                     ).execute().actionGet();
@@ -829,6 +900,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.aggregation("avg_i", asc))
                         .subAggregation(avg("avg_i").field(SINGLE_VALUED_FIELD_NAME))
                 ).execute().actionGet();
@@ -861,6 +933,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.aggregation("stats.avg", asc))
                         .subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))
                 ).execute().actionGet();
@@ -891,6 +964,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.aggregation("stats.avg", asc))
                         .subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME))
                 ).execute().actionGet();
@@ -921,6 +995,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.aggregation("stats.variance", asc))
                         .subAggregation(extendedStats("stats").field(SINGLE_VALUED_FIELD_NAME))
                 ).execute().actionGet();

+ 3 - 0
src/test/java/org/elasticsearch/search/aggregations/bucket/MinDocCountTests.java

@@ -28,6 +28,7 @@ import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.action.search.SearchType;
 import org.elasticsearch.index.query.QueryBuilder;
 import org.elasticsearch.index.query.QueryBuilders;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogram;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
@@ -273,6 +274,7 @@ public class MinDocCountTests extends ElasticsearchIntegrationTest {
                 .setSearchType(SearchType.COUNT)
                 .setQuery(QUERY)
                 .addAggregation(script.apply(terms("terms"), field)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .executionHint(StringTermsTests.randomExecutionHint())
                         .order(order)
                         .size(cardinality + randomInt(10))
@@ -289,6 +291,7 @@ public class MinDocCountTests extends ElasticsearchIntegrationTest {
                     .setSearchType(SearchType.COUNT)
                     .setQuery(QUERY)
                     .addAggregation(script.apply(terms("terms"), field)
+                            .collectMode(randomFrom(SubAggCollectionMode.values()))
                             .executionHint(StringTermsTests.randomExecutionHint())
                             .order(order)
                             .size(size)

+ 2 - 1
src/test/java/org/elasticsearch/search/aggregations/bucket/NaNSortingTests.java

@@ -23,6 +23,7 @@ import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.common.util.Comparators;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.search.aggregations.Aggregation;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
 import org.elasticsearch.search.aggregations.metrics.MetricsAggregationBuilder;
@@ -136,7 +137,7 @@ public class NaNSortingTests extends ElasticsearchIntegrationTest {
         final boolean asc = randomBoolean();
         SubAggregation agg = randomFrom(SubAggregation.values());
         SearchResponse response = client().prepareSearch("idx")
-                .addAggregation(terms("terms").field(fieldName).subAggregation(agg.builder()).order(Terms.Order.aggregation(agg.sortKey(), asc)))
+                .addAggregation(terms("terms").field(fieldName).collectMode(randomFrom(SubAggCollectionMode.values())).subAggregation(agg.builder()).order(Terms.Order.aggregation(agg.sortKey(), asc)))
                 .execute().actionGet();
 
         assertSearchResponse(response);

+ 8 - 2
src/test/java/org/elasticsearch/search/aggregations/bucket/NestedTests.java

@@ -22,6 +22,7 @@ import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.action.index.IndexRequestBuilder;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.nested.Nested;
 import org.elasticsearch.search.aggregations.bucket.terms.LongTerms;
@@ -55,6 +56,7 @@ public class NestedTests extends ElasticsearchIntegrationTest {
 
     static int numParents;
     static int[] numChildren;
+    static SubAggCollectionMode aggCollectionMode;
 
     @Override
     public void setupSuiteScopeCluster() throws Exception {
@@ -66,6 +68,8 @@ public class NestedTests extends ElasticsearchIntegrationTest {
 
         numParents = randomIntBetween(3, 10);
         numChildren = new int[numParents];
+        aggCollectionMode = randomFrom(SubAggCollectionMode.values());
+        System.out.println("AGG COLLECTION MODE: " + aggCollectionMode);
         int totalChildren = 0;
         for (int i = 0; i < numParents; ++i) {
             if (i == numParents - 1 && totalChildren == 0) {
@@ -140,7 +144,6 @@ public class NestedTests extends ElasticsearchIntegrationTest {
                                 .endArray()
                             .endObject())
         );
-
         indexRandom(true, builders);
         ensureSearchable();
     }
@@ -202,7 +205,8 @@ public class NestedTests extends ElasticsearchIntegrationTest {
     public void nestedWithSubTermsAgg() throws Exception {
         SearchResponse response = client().prepareSearch("idx")
                 .addAggregation(nested("nested").path("nested")
-                        .subAggregation(terms("values").field("nested.value").size(100)))
+                        .subAggregation(terms("values").field("nested.value").size(100)
+                                .collectMode(aggCollectionMode)))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -251,6 +255,7 @@ public class NestedTests extends ElasticsearchIntegrationTest {
     public void nestedAsSubAggregation() throws Exception {
         SearchResponse response = client().prepareSearch("idx")
                 .addAggregation(terms("top_values").field("value").size(100)
+                        .collectMode(aggCollectionMode)
                         .subAggregation(nested("nested").path("nested")
                                 .subAggregation(max("max_value").field("nested.value"))))
                 .execute().actionGet();
@@ -280,6 +285,7 @@ public class NestedTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx_nested_nested_aggs")
                 .addAggregation(nested("level1").path("nested1")
                         .subAggregation(terms("a").field("nested1.a")
+                                .collectMode(aggCollectionMode)
                                 .subAggregation(nested("level2").path("nested1.nested2")
                                         .subAggregation(sum("sum").field("nested1.nested2.b")))))
                 .get();

+ 3 - 1
src/test/java/org/elasticsearch/search/aggregations/bucket/RangeTests.java

@@ -20,6 +20,7 @@ package org.elasticsearch.search.aggregations.bucket;
 
 import org.elasticsearch.action.index.IndexRequestBuilder;
 import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.range.Range;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
@@ -78,7 +79,8 @@ public class RangeTests extends ElasticsearchIntegrationTest {
     @Test
     public void rangeAsSubAggregation() throws Exception {
         SearchResponse response = client().prepareSearch("idx")
-                .addAggregation(terms("terms").field(MULTI_VALUED_FIELD_NAME).size(100).subAggregation(
+                .addAggregation(terms("terms").field(MULTI_VALUED_FIELD_NAME).size(100)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).subAggregation(
                         range("range").field(SINGLE_VALUED_FIELD_NAME)
                             .addUnboundedTo(3)
                             .addRange(3, 6)

+ 6 - 0
src/test/java/org/elasticsearch/search/aggregations/bucket/ReverseNestedTests.java

@@ -21,6 +21,7 @@ package org.elasticsearch.search.aggregations.bucket;
 import org.elasticsearch.action.search.SearchPhaseExecutionException;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.nested.Nested;
 import org.elasticsearch.search.aggregations.bucket.nested.ReverseNested;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
@@ -119,6 +120,7 @@ public class ReverseNestedTests extends ElasticsearchIntegrationTest {
                                                 reverseNested("nested1_to_field1")
                                                         .subAggregation(
                                                                 terms("field1").field("field1")
+                                                                .collectMode(randomFrom(SubAggCollectionMode.values()))
                                                         )
                                         )
                         )
@@ -133,10 +135,12 @@ public class ReverseNestedTests extends ElasticsearchIntegrationTest {
                 .addAggregation(nested("nested1").path("nested1.nested2")
                         .subAggregation(
                                 terms("field2").field("nested1.nested2.field2")
+                                .collectMode(randomFrom(SubAggCollectionMode.values()))
                                         .subAggregation(
                                                 reverseNested("nested1_to_field1").path("nested1")
                                                         .subAggregation(
                                                                 terms("field1").field("nested1.field1")
+                                                                .collectMode(randomFrom(SubAggCollectionMode.values()))
                                                         )
                                         )
                         )
@@ -148,10 +152,12 @@ public class ReverseNestedTests extends ElasticsearchIntegrationTest {
     public void testReverseNestedAggWithoutNestedAgg() throws Exception {
         client().prepareSearch("idx")
                 .addAggregation(terms("field2").field("nested1.nested2.field2")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                                 .subAggregation(
                                         reverseNested("nested1_to_field1")
                                                 .subAggregation(
                                                         terms("field1").field("nested1.field1")
+                                                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                                                 )
                                 )
                 ).get();

+ 4 - 0
src/test/java/org/elasticsearch/search/aggregations/bucket/ShardReduceTests.java

@@ -23,6 +23,7 @@ import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.common.geo.GeoHashUtils;
 import org.elasticsearch.index.query.FilterBuilders;
 import org.elasticsearch.index.query.QueryBuilders;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.filter.Filter;
 import org.elasticsearch.search.aggregations.bucket.geogrid.GeoHashGrid;
 import org.elasticsearch.search.aggregations.bucket.global.Global;
@@ -170,6 +171,7 @@ public class ShardReduceTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx")
                 .setQuery(QueryBuilders.matchAllQuery())
                 .addAggregation(terms("terms").field("term-s")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(dateHistogram("histo").field("date").interval(DateHistogram.Interval.DAY).minDocCount(0)))
                 .execute().actionGet();
 
@@ -186,6 +188,7 @@ public class ShardReduceTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx")
                 .setQuery(QueryBuilders.matchAllQuery())
                 .addAggregation(terms("terms").field("term-l")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(dateHistogram("histo").field("date").interval(DateHistogram.Interval.DAY).minDocCount(0)))
                 .execute().actionGet();
 
@@ -202,6 +205,7 @@ public class ShardReduceTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx")
                 .setQuery(QueryBuilders.matchAllQuery())
                 .addAggregation(terms("terms").field("term-d")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(dateHistogram("histo").field("date").interval(DateHistogram.Interval.DAY).minDocCount(0)))
                 .execute().actionGet();
 

+ 19 - 9
src/test/java/org/elasticsearch/search/aggregations/bucket/ShardSizeTermsTests.java

@@ -20,6 +20,7 @@ package org.elasticsearch.search.aggregations.bucket;
 
 import com.google.common.collect.ImmutableMap;
 import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
 import org.junit.Test;
 
@@ -40,7 +41,8 @@ public class ShardSizeTermsTests extends ShardSizeTests {
 
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .setQuery(matchAllQuery())
-                .addAggregation(terms("keys").field("key").size(3).order(Terms.Order.count(false)))
+                .addAggregation(terms("keys").field("key").size(3)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).order(Terms.Order.count(false)))
                 .execute().actionGet();
 
         Terms  terms = response.getAggregations().get("keys");
@@ -65,7 +67,8 @@ public class ShardSizeTermsTests extends ShardSizeTests {
 
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .setQuery(matchAllQuery())
-                .addAggregation(terms("keys").field("key").size(3).shardSize(5).order(Terms.Order.count(false)))
+                .addAggregation(terms("keys").field("key").size(3)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).shardSize(5).order(Terms.Order.count(false)))
                 .execute().actionGet();
 
         Terms terms = response.getAggregations().get("keys");
@@ -90,7 +93,8 @@ public class ShardSizeTermsTests extends ShardSizeTests {
 
         SearchResponse response = client().prepareSearch("idx").setTypes("type").setRouting("1")
                 .setQuery(matchAllQuery())
-                .addAggregation(terms("keys").field("key").size(3).shardSize(5).order(Terms.Order.count(false)))
+                .addAggregation(terms("keys").field("key").size(3)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).shardSize(5).order(Terms.Order.count(false)))
                 .execute().actionGet();
 
         Terms terms = response.getAggregations().get("keys");
@@ -115,7 +119,8 @@ public class ShardSizeTermsTests extends ShardSizeTests {
 
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .setQuery(matchAllQuery())
-                .addAggregation(terms("keys").field("key").size(3).order(Terms.Order.count(false)))
+                .addAggregation(terms("keys").field("key").size(3)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).order(Terms.Order.count(false)))
                 .execute().actionGet();
 
         Terms terms = response.getAggregations().get("keys");
@@ -140,7 +145,8 @@ public class ShardSizeTermsTests extends ShardSizeTests {
 
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .setQuery(matchAllQuery())
-                .addAggregation(terms("keys").field("key").size(3).shardSize(5).order(Terms.Order.count(false)))
+                .addAggregation(terms("keys").field("key").size(3)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).shardSize(5).order(Terms.Order.count(false)))
                 .execute().actionGet();
 
         Terms terms = response.getAggregations().get("keys");
@@ -165,7 +171,8 @@ public class ShardSizeTermsTests extends ShardSizeTests {
 
         SearchResponse response = client().prepareSearch("idx").setTypes("type").setRouting("1")
                 .setQuery(matchAllQuery())
-                .addAggregation(terms("keys").field("key").size(3).shardSize(5).order(Terms.Order.count(false)))
+                .addAggregation(terms("keys").field("key").size(3)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).shardSize(5).order(Terms.Order.count(false)))
                 .execute().actionGet();
 
         Terms terms = response.getAggregations().get("keys");
@@ -190,7 +197,8 @@ public class ShardSizeTermsTests extends ShardSizeTests {
 
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .setQuery(matchAllQuery())
-                .addAggregation(terms("keys").field("key").size(3).order(Terms.Order.count(false)))
+                .addAggregation(terms("keys").field("key").size(3)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).order(Terms.Order.count(false)))
                 .execute().actionGet();
 
         Terms terms = response.getAggregations().get("keys");
@@ -215,7 +223,8 @@ public class ShardSizeTermsTests extends ShardSizeTests {
 
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .setQuery(matchAllQuery())
-                .addAggregation(terms("keys").field("key").size(3).shardSize(5).order(Terms.Order.count(false)))
+                .addAggregation(terms("keys").field("key").size(3)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).shardSize(5).order(Terms.Order.count(false)))
                 .execute().actionGet();
 
         Terms terms = response.getAggregations().get("keys");
@@ -240,7 +249,8 @@ public class ShardSizeTermsTests extends ShardSizeTests {
 
         SearchResponse response = client().prepareSearch("idx").setTypes("type").setRouting("1")
                 .setQuery(matchAllQuery())
-                .addAggregation(terms("keys").field("key").size(3).shardSize(5).order(Terms.Order.count(false)))
+                .addAggregation(terms("keys").field("key").size(3)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).shardSize(5).order(Terms.Order.count(false)))
                 .execute().actionGet();
 
         Terms terms = response.getAggregations().get("keys");

+ 102 - 13
src/test/java/org/elasticsearch/search/aggregations/bucket/StringTermsTests.java

@@ -24,6 +24,7 @@ import org.elasticsearch.action.index.IndexRequestBuilder;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.index.mapper.internal.IndexFieldMapper;
 import org.elasticsearch.index.query.FilterBuilders;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.filter.Filter;
 import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
@@ -109,6 +110,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .minDocCount(minDocCount)
                         .size(0))
                 .execute().actionGet();
@@ -126,7 +128,8 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
-                        .field(SINGLE_VALUED_FIELD_NAME))
+                        .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -157,7 +160,8 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
             SearchResponse response = client().prepareSearch("idx").setTypes("type")
                     .addAggregation(terms("terms")
                             .executionHint(executionMode == null ? null : executionMode.toString())
-                            .field(SINGLE_VALUED_FIELD_NAME))
+                            .field(SINGLE_VALUED_FIELD_NAME)
+                            .collectMode(randomFrom(SubAggCollectionMode.values())))
                     .execute().actionGet();
             assertSearchResponse(response);
 
@@ -183,7 +187,9 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("high_card_type")
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
-                        .field(SINGLE_VALUED_FIELD_NAME).include("val00.+"))
+                        .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
+                        .include("val00.+"))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -206,7 +212,9 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
         response = client().prepareSearch("idx").setTypes("high_card_type")
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
-                        .field(SINGLE_VALUED_FIELD_NAME).include("val00.+").exclude("(val000|val001)"))
+                        .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
+                        .include("val00.+").exclude("(val000|val001)"))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -229,7 +237,9 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
         response = client().prepareSearch("idx").setTypes("high_card_type")
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
-                        .field(SINGLE_VALUED_FIELD_NAME).exclude("val0[1-9]+.+"))
+                        .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
+                        .exclude("val0[1-9]+.+"))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -257,7 +267,8 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("high_card_type")
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
-                        .field(SINGLE_VALUED_FIELD_NAME).include("VAL00.+", Pattern.CASE_INSENSITIVE))
+                        .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).include("VAL00.+", Pattern.CASE_INSENSITIVE))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -281,7 +292,8 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
         response = client().prepareSearch("idx").setTypes("high_card_type")
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
-                        .field(SINGLE_VALUED_FIELD_NAME).include("val00.+").exclude("( val000 | VAL001 )#this is a comment", Pattern.CASE_INSENSITIVE | Pattern.COMMENTS))
+                        .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).include("val00.+").exclude("( val000 | VAL001 )#this is a comment", Pattern.CASE_INSENSITIVE | Pattern.COMMENTS))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -305,7 +317,8 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
         response = client().prepareSearch("idx").setTypes("high_card_type")
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
-                        .field(SINGLE_VALUED_FIELD_NAME).exclude("val0[1-9]+.+", 0))
+                        .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())).exclude("val0[1-9]+.+", 0))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -331,6 +344,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                         .executionHint(randomExecutionHint())
                         .field(SINGLE_VALUED_FIELD_NAME)
                         .size(20)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.term(true))) // we need to sort by terms cause we're checking the first 20 values
                 .execute().actionGet();
 
@@ -355,6 +369,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.term(true)))
                 .execute().actionGet();
 
@@ -380,6 +395,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.term(false)))
                 .execute().actionGet();
 
@@ -405,6 +421,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(count("count").field(MULTI_VALUED_FIELD_NAME)))
                 .execute().actionGet();
 
@@ -432,6 +449,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .subAggregation(count("count")))
                 .execute().actionGet();
 
@@ -459,6 +477,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .script("'foo_' + _value"))
                 .execute().actionGet();
 
@@ -483,6 +502,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
                         .field(MULTI_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .script("_value.substring(0,3)"))
                 .execute().actionGet();
 
@@ -504,7 +524,8 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
-                        .field(MULTI_VALUED_FIELD_NAME))
+                        .field(MULTI_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -532,6 +553,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
                         .field(MULTI_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .script("'foo_' + _value"))
                 .execute().actionGet();
 
@@ -578,6 +600,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
                         .field(MULTI_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .script("'foo_' + _value")
                         .subAggregation(count("count")))
                 .execute().actionGet();
@@ -611,6 +634,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
     public void script_SingleValue() throws Exception {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .executionHint(randomExecutionHint())
                         .script("doc['" + SINGLE_VALUED_FIELD_NAME + "'].value"))
                 .execute().actionGet();
@@ -634,6 +658,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
     public void script_SingleValue_ExplicitSingleValue() throws Exception {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .executionHint(randomExecutionHint())
                         .script("doc['" + SINGLE_VALUED_FIELD_NAME + "'].value"))
                 .execute().actionGet();
@@ -657,6 +682,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
     public void script_SingleValue_WithSubAggregator_Inherited() throws Exception {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .executionHint(randomExecutionHint())
                         .script("doc['" + SINGLE_VALUED_FIELD_NAME + "'].value")
                         .subAggregation(count("count")))
@@ -684,6 +710,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
     public void script_MultiValued() throws Exception {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .executionHint(randomExecutionHint())
                         .script("doc['" + MULTI_VALUED_FIELD_NAME + "'].values"))
                 .execute().actionGet();
@@ -711,6 +738,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
     public void script_MultiValued_WithAggregatorInherited() throws Exception {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
                 .addAggregation(terms("terms")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .executionHint(randomExecutionHint())
                         .script("doc['" + MULTI_VALUED_FIELD_NAME + "'].values")
                         .subAggregation(count("count")))
@@ -747,7 +775,8 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
                         .size(randomInt(5))
-                        .field(SINGLE_VALUED_FIELD_NAME))
+                        .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -763,7 +792,8 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
         SearchResponse response = client().prepareSearch("idx", "idx_unmapped").setTypes("type")
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
-                        .field(SINGLE_VALUED_FIELD_NAME))
+                        .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values())))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -785,7 +815,8 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
     public void stringTermsNestedIntoPerBucketAggregator() throws Exception {
         // no execution hint so that the logic that decides whether or not to use ordinals is executed
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
-                .addAggregation(filter("filter").filter(termFilter(MULTI_VALUED_FIELD_NAME, "val3")).subAggregation(terms("terms").field(MULTI_VALUED_FIELD_NAME)))
+                .addAggregation(filter("filter").filter(termFilter(MULTI_VALUED_FIELD_NAME, "val3")).subAggregation(terms("terms").field(MULTI_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))))
                 .execute().actionGet();
 
         assertThat(response.getFailedShards(), equalTo(0));
@@ -832,6 +863,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.aggregation("avg_i", asc))
                         .subAggregation(avg("avg_i").field("i"))
                 ).execute().actionGet();
@@ -863,6 +895,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                 .addAggregation(terms("tags")
                         .executionHint(randomExecutionHint())
                         .field("tag")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.aggregation("filter", asc))
                         .subAggregation(filter("filter").filter(FilterBuilders.matchAllFilter()))
                 ).execute().actionGet();
@@ -901,6 +934,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                 .addAggregation(terms("tags")
                         .executionHint(randomExecutionHint())
                         .field("tag")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.aggregation("filter1>filter2>stats.max", asc))
                         .subAggregation(filter("filter1").filter(FilterBuilders.matchAllFilter())
                                 .subAggregation(filter("filter2").filter(FilterBuilders.matchAllFilter())
@@ -958,6 +992,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                     .addAggregation(terms("terms")
                             .executionHint(randomExecutionHint())
                             .field(SINGLE_VALUED_FIELD_NAME)
+                            .collectMode(randomFrom(SubAggCollectionMode.values()))
                             .order(Terms.Order.aggregation("avg_i", true))
                     ).execute().actionGet();
 
@@ -976,8 +1011,10 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                     .addAggregation(terms("terms")
                             .executionHint(randomExecutionHint())
                             .field(SINGLE_VALUED_FIELD_NAME)
+                            .collectMode(randomFrom(SubAggCollectionMode.values()))
                             .order(Terms.Order.aggregation("values", true))
-                            .subAggregation(terms("values").field("i"))
+                            .subAggregation(terms("values").field("i")
+                                    .collectMode(randomFrom(SubAggCollectionMode.values())))
                     ).execute().actionGet();
 
             fail("Expected search to fail when trying to sort terms aggregation by sug-aggregation which is not of a metrics or single-bucket type");
@@ -994,6 +1031,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                     .addAggregation(terms("terms")
                             .executionHint(randomExecutionHint())
                             .field(SINGLE_VALUED_FIELD_NAME)
+                            .collectMode(randomFrom(SubAggCollectionMode.values()))
                             .order(Terms.Order.aggregation("stats.foo", true))
                             .subAggregation(stats("stats").field("i"))
                     ).execute().actionGet();
@@ -1014,6 +1052,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                     .addAggregation(terms("terms")
                             .executionHint(randomExecutionHint())
                             .field(SINGLE_VALUED_FIELD_NAME)
+                            .collectMode(randomFrom(SubAggCollectionMode.values()))
                             .order(Terms.Order.aggregation("stats", true))
                             .subAggregation(stats("stats").field("i"))
                     ).execute().actionGet();
@@ -1033,6 +1072,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.aggregation("avg_i", asc))
                         .subAggregation(avg("avg_i").field("i"))
                 ).execute().actionGet();
@@ -1066,6 +1106,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.aggregation("stats.avg", asc))
                         .subAggregation(stats("stats").field("i"))
                 ).execute().actionGet();
@@ -1098,6 +1139,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.aggregation("stats.avg", asc))
                         .subAggregation(stats("stats").field("i"))
                 ).execute().actionGet();
@@ -1130,6 +1172,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
                 .addAggregation(terms("terms")
                         .executionHint(randomExecutionHint())
                         .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .order(Terms.Order.aggregation("stats.sum_of_squares", asc))
                         .subAggregation(extendedStats("stats").field("i"))
                 ).execute().actionGet();
@@ -1154,11 +1197,57 @@ public class StringTermsTests extends ElasticsearchIntegrationTest {
         }
 
     }
+    
+    @Test
+    public void singleValuedField_OrderedByStatsAggAscWithTermsSubAgg() throws Exception {
+        boolean asc = true;
+        SearchResponse response = client().prepareSearch("idx").setTypes("type")
+                .addAggregation(terms("terms")
+                        .executionHint(randomExecutionHint())
+                        .field(SINGLE_VALUED_FIELD_NAME)
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
+                        .order(Terms.Order.aggregation("stats.sum_of_squares", asc))
+                        .subAggregation(extendedStats("stats").field("i")).subAggregation(terms("subTerms").field("s_values")
+                                .collectMode(randomFrom(SubAggCollectionMode.values())))
+                ).execute().actionGet();
+
+        assertSearchResponse(response);
+
+        Terms terms = response.getAggregations().get("terms");
+        assertThat(terms, notNullValue());
+        assertThat(terms.getName(), equalTo("terms"));
+        assertThat(terms.getBuckets().size(), equalTo(5));
+
+        int i = 0;
+        for (Terms.Bucket bucket : terms.getBuckets()) {
+            assertThat(bucket, notNullValue());
+            assertThat(key(bucket), equalTo("val" + i));
+            assertThat(bucket.getDocCount(), equalTo(1l));
+
+            ExtendedStats stats = bucket.getAggregations().get("stats");
+            assertThat(stats, notNullValue());
+            assertThat(stats.getMax(), equalTo((double) i));
+            
+            Terms subTermsAgg = bucket.getAggregations().get("subTerms");
+            assertThat(subTermsAgg, notNullValue());
+            assertThat(subTermsAgg.getBuckets().size(), equalTo(2));
+            int j = i;
+            for (Terms.Bucket subBucket : subTermsAgg.getBuckets()) {
+                assertThat(subBucket, notNullValue());
+                assertThat(key(subBucket), equalTo("val" + j));
+                assertThat(subBucket.getDocCount(), equalTo(1l));
+                j++;
+            }
+            i++;
+        }
+
+    }
 
     @Test
     public void indexMetaField() throws Exception {
         SearchResponse response = client().prepareSearch("idx", "empty_bucket_idx").setTypes("type")
                 .addAggregation(terms("terms")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
                         .executionHint(randomExecutionHint())
                         .field(IndexFieldMapper.NAME)
                 ).execute().actionGet();

+ 25 - 0
src/test/java/org/elasticsearch/search/aggregations/bucket/TopHitsTests.java

@@ -27,6 +27,7 @@ import org.elasticsearch.index.query.FilterBuilders;
 import org.elasticsearch.search.SearchHit;
 import org.elasticsearch.search.SearchHitField;
 import org.elasticsearch.search.SearchHits;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
 import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregatorFactory.ExecutionMode;
 import org.elasticsearch.search.aggregations.bucket.tophits.TopHits;
@@ -409,6 +410,30 @@ public class TopHitsTests extends ElasticsearchIntegrationTest {
             assertThat(e.getMessage(), containsString("Aggregator [top_tags_hits] of type [top_hits] cannot accept sub-aggregations"));
         }
     }
+    
+    @Test
+    public void testFailDeferred() throws Exception {
+        
+        try {
+            client().prepareSearch("idx")
+                    .setTypes("type")
+                    .addAggregation(
+                            terms("terms").executionHint(randomExecutionHint()).field(TERMS_AGGS_FIELD)
+                                    .collectMode(SubAggCollectionMode.BREADTH_FIRST)
+                                    .subAggregation(topHits("hits").addSort(SortBuilders.fieldSort(SORT_FIELD).order(SortOrder.DESC))))
+                    .get();
+            fail();
+        } catch (Exception e) {
+            // It is considered a parse failure if the search request asks for top_hits
+            // under an aggregation with collect_mode set to breadth_first as this would
+            // require the buffering of scores alongside each document ID and that is a
+            // a RAM cost we are not willing to pay 
+            assertThat(e.getMessage(), containsString("ElasticsearchParseException"));
+        }
+    }
+    
+    
+    
 
     @Test
     public void testEmptyIndex() throws Exception {

+ 7 - 2
src/test/java/org/elasticsearch/search/aggregations/metrics/CardinalityTests.java

@@ -23,6 +23,7 @@ import org.elasticsearch.action.index.IndexRequestBuilder;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.common.settings.ImmutableSettings;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.bucket.terms.Terms;
 import org.elasticsearch.search.aggregations.metrics.cardinality.Cardinality;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
@@ -397,7 +398,9 @@ public class CardinalityTests extends ElasticsearchIntegrationTest {
     @Test
     public void asSubAgg() throws Exception {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
-                .addAggregation(terms("terms").field("str_value").subAggregation(cardinality("cardinality").precisionThreshold(precisionThreshold).field("str_values")))
+                .addAggregation(terms("terms").field("str_value")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
+                        .subAggregation(cardinality("cardinality").precisionThreshold(precisionThreshold).field("str_values")))
                 .execute().actionGet();
 
         assertSearchResponse(response);
@@ -414,7 +417,9 @@ public class CardinalityTests extends ElasticsearchIntegrationTest {
     @Test
     public void asSubAggHashed() throws Exception {
         SearchResponse response = client().prepareSearch("idx").setTypes("type")
-                .addAggregation(terms("terms").field("str_value").subAggregation(cardinality("cardinality").precisionThreshold(precisionThreshold).field("str_values.hash")))
+                .addAggregation(terms("terms").field("str_value")
+                        .collectMode(randomFrom(SubAggCollectionMode.values()))
+                        .subAggregation(cardinality("cardinality").precisionThreshold(precisionThreshold).field("str_values.hash")))
                 .execute().actionGet();
 
         assertSearchResponse(response);

+ 3 - 1
src/test/java/org/elasticsearch/search/suggest/CompletionSuggestSearchTests.java

@@ -40,6 +40,7 @@ import org.elasticsearch.index.mapper.MapperParsingException;
 import org.elasticsearch.index.mapper.core.CompletionFieldMapper;
 import org.elasticsearch.percolator.PercolatorService;
 import org.elasticsearch.search.aggregations.AggregationBuilders;
+import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.sort.FieldSortBuilder;
 import org.elasticsearch.search.suggest.completion.CompletionStats;
 import org.elasticsearch.search.suggest.completion.CompletionSuggestion;
@@ -1030,7 +1031,8 @@ public class CompletionSuggestSearchTests extends ElasticsearchIntegrationTest {
         ).setRefresh(true).get();
 
         try {
-            client().prepareSearch(INDEX).addAggregation(AggregationBuilders.terms("suggest_agg").field(FIELD)).execute().actionGet();
+            client().prepareSearch(INDEX).addAggregation(AggregationBuilders.terms("suggest_agg").field(FIELD)
+                    .collectMode(randomFrom(SubAggCollectionMode.values()))).execute().actionGet();
             // Exception must be thrown
             assertFalse(true);
         } catch (SearchPhaseExecutionException e) {