Sfoglia il codice sorgente

Remove tech debt on Aggregations#getLeafCollector (#88230)

Remove #getLeafCollector(LeafReaderContext, LeafBucketCollector) in favour of 
#getLeafCollector(ggregationExecutionContext, LeafBucketCollector)
Ignacio Vera 3 anni fa
parent
commit
d1ba276587
79 ha cambiato i file con 277 aggiunte e 264 eliminazioni
  1. 3 3
      modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/stats/MatrixStatsAggregator.java
  2. 8 3
      modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentJoinAggregator.java
  3. 7 13
      server/src/main/java/org/elasticsearch/search/aggregations/AggregatorBase.java
  4. 1 2
      server/src/main/java/org/elasticsearch/search/aggregations/NonCollectingAggregator.java
  5. 6 3
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregator.java
  6. 10 9
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java
  7. 6 3
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregator.java
  8. 5 4
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterByFilterAggregator.java
  9. 3 2
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FiltersAggregator.java
  10. 4 3
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregator.java
  11. 4 4
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregator.java
  12. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java
  13. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java
  14. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateRangeHistogramAggregator.java
  15. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/NumericHistogramAggregator.java
  16. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/RangeHistogramAggregator.java
  17. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/VariableWidthHistogramAggregator.java
  18. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregator.java
  19. 6 4
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregator.java
  20. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregator.java
  21. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/prefix/IpPrefixAggregator.java
  22. 4 4
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/BinaryRangeAggregator.java
  23. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregator.java
  24. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregator.java
  25. 5 5
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/random/RandomSamplerAggregator.java
  26. 6 6
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java
  27. 3 2
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongRareTermsAggregator.java
  28. 3 2
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/MapStringTermsAggregator.java
  29. 3 2
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/NumericTermsAggregator.java
  30. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringRareTermsAggregator.java
  31. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/metrics/AbstractHDRPercentilesAggregator.java
  32. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/metrics/AbstractTDigestPercentilesAggregator.java
  33. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/metrics/AvgAggregator.java
  34. 3 2
      server/src/main/java/org/elasticsearch/search/aggregations/metrics/CardinalityAggregator.java
  35. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/metrics/ExtendedStatsAggregator.java
  36. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsAggregator.java
  37. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregator.java
  38. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/metrics/GlobalOrdCardinalityAggregator.java
  39. 4 4
      server/src/main/java/org/elasticsearch/search/aggregations/metrics/MaxAggregator.java
  40. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregator.java
  41. 4 4
      server/src/main/java/org/elasticsearch/search/aggregations/metrics/MinAggregator.java
  42. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricAggregator.java
  43. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/metrics/StatsAggregator.java
  44. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/metrics/SumAggregator.java
  45. 3 3
      server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregator.java
  46. 5 5
      server/src/main/java/org/elasticsearch/search/aggregations/metrics/ValueCountAggregator.java
  47. 4 4
      server/src/main/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregator.java
  48. 1 8
      server/src/main/java/org/elasticsearch/search/aggregations/timeseries/TimeSeriesAggregator.java
  49. 1 2
      server/src/test/java/org/elasticsearch/search/aggregations/AdaptingAggregatorTests.java
  50. 1 2
      server/src/test/java/org/elasticsearch/search/aggregations/AggregatorBaseTests.java
  51. 2 2
      server/src/test/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregatorTests.java
  52. 1 1
      test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java
  53. 3 3
      x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/bucket/histogram/HistoBackedHistogramAggregator.java
  54. 3 3
      x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/bucket/range/HistoBackedRangeAggregator.java
  55. 3 3
      x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/metrics/AbstractHistoBackedHDRPercentilesAggregator.java
  56. 3 3
      x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/metrics/AbstractHistoBackedTDigestPercentilesAggregator.java
  57. 3 3
      x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/metrics/HistoBackedAvgAggregator.java
  58. 3 3
      x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/metrics/HistoBackedMaxAggregator.java
  59. 3 3
      x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/metrics/HistoBackedMinAggregator.java
  60. 3 3
      x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/metrics/HistoBackedSumAggregator.java
  61. 3 3
      x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/metrics/HistoBackedValueCountAggregator.java
  62. 6 4
      x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/boxplot/BoxplotAggregator.java
  63. 3 2
      x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/multiterms/MultiTermsAggregator.java
  64. 3 3
      x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/rate/HistogramRateAggregator.java
  65. 4 4
      x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/rate/NumericRateAggregator.java
  66. 2 2
      x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/rate/RateAggregatorFactory.java
  67. 3 3
      x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/stringstats/StringStatsAggregator.java
  68. 3 2
      x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/topmetrics/TopMetricsAggregator.java
  69. 4 4
      x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/ttest/PairedTTestAggregator.java
  70. 6 5
      x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/ttest/UnpairedTTestAggregator.java
  71. 7 4
      x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricBackedAvgAggregator.java
  72. 3 3
      x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricBackedMaxAggregator.java
  73. 3 3
      x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricBackedMinAggregator.java
  74. 3 3
      x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricBackedSumAggregator.java
  75. 3 3
      x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricBackedValueCountAggregator.java
  76. 3 3
      x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/aggs/categorization/CategorizeTextAggregator.java
  77. 3 3
      x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
  78. 3 3
      x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/metrics/GeoShapeBoundsAggregator.java
  79. 3 3
      x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/metrics/GeoShapeCentroidAggregator.java

+ 3 - 3
modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/stats/MatrixStatsAggregator.java

@@ -7,12 +7,12 @@
  */
 package org.elasticsearch.search.aggregations.matrix.stats;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.ObjectArray;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.NumericDoubleValues;
 import org.elasticsearch.search.MultiValueMode;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -58,13 +58,13 @@ final class MatrixStatsAggregator extends MetricsAggregator {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSources == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
         final NumericDoubleValues[] values = new NumericDoubleValues[valuesSources.fieldNames().length];
         for (int i = 0; i < values.length; ++i) {
-            values[i] = valuesSources.getField(i, ctx);
+            values[i] = valuesSources.getField(i, aggCtx.getLeafReaderContext());
         }
 
         return new LeafBucketCollectorBase(sub, values) {

+ 8 - 3
modules/parent-join/src/main/java/org/elasticsearch/join/aggregations/ParentJoinAggregator.java

@@ -22,6 +22,7 @@ import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.BitArray;
 import org.elasticsearch.core.Releasable;
 import org.elasticsearch.core.Releasables;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.CardinalityUpperBound;
@@ -85,12 +86,16 @@ public abstract class ParentJoinAggregator extends BucketsAggregator implements
     }
 
     @Override
-    public final LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public final LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub)
+        throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        final SortedSetDocValues globalOrdinals = valuesSource.globalOrdinalsValues(ctx);
-        final Bits parentDocs = Lucene.asSequentialAccessBits(ctx.reader().maxDoc(), inFilter.scorerSupplier(ctx));
+        final SortedSetDocValues globalOrdinals = valuesSource.globalOrdinalsValues(aggCtx.getLeafReaderContext());
+        final Bits parentDocs = Lucene.asSequentialAccessBits(
+            aggCtx.getLeafReaderContext().reader().maxDoc(),
+            inFilter.scorerSupplier(aggCtx.getLeafReaderContext())
+        );
         return new LeafBucketCollector() {
             @Override
             public void collect(int docId, long owningBucketOrd) throws IOException {

+ 7 - 13
server/src/main/java/org/elasticsearch/search/aggregations/AggregatorBase.java

@@ -178,7 +178,7 @@ public abstract class AggregatorBase extends Aggregator {
      * {@link LeafBucketCollector#collect(int, long) collect} for every hit. So any
      * {@link Aggregator} that returns a customer {@linkplain LeafBucketCollector}
      * from this method runs at best {@code O(hits)} time. See the
-     * {@link SumAggregator#getLeafCollector(LeafReaderContext, LeafBucketCollector) sum}
+     * {@link SumAggregator#getLeafCollector(AggregationExecutionContext, LeafBucketCollector) sum}
      * {@linkplain Aggregator} for a fairly strait forward example of this.
      * <p>
      * Some {@linkplain Aggregator}s are able to correctly collect results on
@@ -188,7 +188,7 @@ public abstract class AggregatorBase extends Aggregator {
      * return {@link LeafBucketCollector#NO_OP_COLLECTOR} to signal that they've
      * done their own collection. These aggregations can do better than
      * {@code O(hits)}. See the
-     * {@link MinAggregator#getLeafCollector(LeafReaderContext, LeafBucketCollector) min}
+     * {@link MinAggregator#getLeafCollector(AggregationExecutionContext, LeafBucketCollector) min}
      * {@linkplain Aggregator} for an example of an aggregation that does this. It
      * happens to run in constant time in some cases.
      * <p>
@@ -203,27 +203,21 @@ public abstract class AggregatorBase extends Aggregator {
      * path before building the {@linkplain Aggregator} rather than on each
      * leaf. Either is fine.
      */
-    protected abstract LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException;
-
-    // TODO: Remove this method in refactoring
-    protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub, AggregationExecutionContext aggCtx)
-        throws IOException {
-        return getLeafCollector(ctx, sub);
-    }
+    protected abstract LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException;
 
     /**
      * Collect results for this leaf.
      * <p>
      * Implemented by the {@linkplain Aggregator} base class to correctly set
      * up sub {@linkplain Aggregator}s. See the
-     * {@link #getLeafCollector(LeafReaderContext, LeafBucketCollector) abstract delegate}
+     * {@link #getLeafCollector(AggregationExecutionContext, LeafBucketCollector) abstract delegate}
      * for more details on what this does.
      */
     @Override
     public final LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx) throws IOException {
         preGetSubLeafCollectors(aggCtx.getLeafReaderContext());
         final LeafBucketCollector sub = collectableSubAggregators.getLeafCollector(aggCtx);
-        return getLeafCollector(aggCtx.getLeafReaderContext(), sub, aggCtx);
+        return getLeafCollector(aggCtx, sub);
     }
 
     /**
@@ -335,7 +329,7 @@ public abstract class AggregatorBase extends Aggregator {
      * The "top level" query that will filter the results sent to this
      * {@linkplain Aggregator}. Used by all {@linkplain Aggregator}s that
      * perform extra collection phases in addition to the one done in
-     * {@link #getLeafCollector(LeafReaderContext, LeafBucketCollector)}.
+     * {@link #getLeafCollector(AggregationExecutionContext, LeafBucketCollector)}.
      */
     protected final Query topLevelQuery() {
         return context.query();
@@ -345,7 +339,7 @@ public abstract class AggregatorBase extends Aggregator {
      * The searcher for the shard this {@linkplain Aggregator} is running
      * against. Used by all {@linkplain Aggregator}s that perform extra
      * collection phases in addition to the one done in
-     * {@link #getLeafCollector(LeafReaderContext, LeafBucketCollector)}
+     * {@link #getLeafCollector(AggregationExecutionContext, LeafBucketCollector)}
      * and by to look up extra "background" information about contents of
      * the shard itself.
      */

+ 1 - 2
server/src/main/java/org/elasticsearch/search/aggregations/NonCollectingAggregator.java

@@ -8,7 +8,6 @@
 
 package org.elasticsearch.search.aggregations;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.elasticsearch.search.aggregations.support.AggregationContext;
 
 import java.io.IOException;
@@ -33,7 +32,7 @@ public abstract class NonCollectingAggregator extends AggregatorBase {
     }
 
     @Override
-    public final LeafBucketCollector getLeafCollector(LeafReaderContext reader, LeafBucketCollector sub) {
+    public final LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) {
         // the framework will automatically eliminate it
         return LeafBucketCollector.NO_OP_COLLECTOR;
     }

+ 6 - 3
server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregator.java

@@ -8,7 +8,6 @@
 
 package org.elasticsearch.search.aggregations.bucket.adjacency;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.Bits;
 import org.elasticsearch.common.io.stream.StreamInput;
@@ -16,6 +15,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
 import org.elasticsearch.common.lucene.Lucene;
 import org.elasticsearch.index.query.QueryBuilder;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.CardinalityUpperBound;
@@ -138,11 +138,14 @@ public class AdjacencyMatrixAggregator extends BucketsAggregator {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         // no need to provide deleted docs to the filter
         final Bits[] bits = new Bits[filters.length];
         for (int i = 0; i < filters.length; ++i) {
-            bits[i] = Lucene.asSequentialAccessBits(ctx.reader().maxDoc(), filters[i].scorerSupplier(ctx));
+            bits[i] = Lucene.asSequentialAccessBits(
+                aggCtx.getLeafReaderContext().reader().maxDoc(),
+                filters[i].scorerSupplier(aggCtx.getLeafReaderContext())
+            );
         }
         return new LeafBucketCollectorBase(sub, null) {
             @Override

+ 10 - 9
server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java

@@ -37,6 +37,7 @@ import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.IndexSortConfig;
 import org.elasticsearch.lucene.queries.SearchAfterSortedDocQuery;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.AggregationExecutionException;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
@@ -436,24 +437,24 @@ public final class CompositeAggregator extends BucketsAggregator implements Size
     }
 
     @Override
-    protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+    protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
         finishLeaf();
 
         boolean fillDocIdSet = deferredCollectors != NO_OP_COLLECTOR;
 
-        Sort indexSortPrefix = buildIndexSortPrefix(ctx);
+        Sort indexSortPrefix = buildIndexSortPrefix(aggCtx.getLeafReaderContext());
         int sortPrefixLen = computeSortPrefixLen(indexSortPrefix);
 
         SortedDocsProducer sortedDocsProducer = (sortPrefixLen == 0 && parent == null)
-            ? sources[0].createSortedDocsProducerOrNull(ctx.reader(), topLevelQuery())
+            ? sources[0].createSortedDocsProducerOrNull(aggCtx.getLeafReaderContext().reader(), topLevelQuery())
             : null;
         if (sortedDocsProducer != null) {
             // Visit documents sorted by the leading source of the composite definition and terminates
             // when the leading source value is guaranteed to be greater than the lowest composite bucket
             // in the queue.
-            DocIdSet docIdSet = sortedDocsProducer.processLeaf(topLevelQuery(), queue, ctx, fillDocIdSet);
+            DocIdSet docIdSet = sortedDocsProducer.processLeaf(topLevelQuery(), queue, aggCtx.getLeafReaderContext(), fillDocIdSet);
             if (fillDocIdSet) {
-                entries.add(new Entry(ctx, docIdSet));
+                entries.add(new Entry(aggCtx.getLeafReaderContext(), docIdSet));
             }
             // We can bypass search entirely for this segment, the processing is done in the previous call.
             // Throwing this exception will terminate the execution of the search for this root aggregation,
@@ -462,15 +463,15 @@ public final class CompositeAggregator extends BucketsAggregator implements Size
             return LeafBucketCollector.NO_OP_COLLECTOR;
         } else {
             if (fillDocIdSet) {
-                currentLeaf = ctx;
-                docIdSetBuilder = new RoaringDocIdSet.Builder(ctx.reader().maxDoc());
+                currentLeaf = aggCtx.getLeafReaderContext();
+                docIdSetBuilder = new RoaringDocIdSet.Builder(aggCtx.getLeafReaderContext().reader().maxDoc());
             }
             if (rawAfterKey != null && sortPrefixLen > 0) {
                 // We have an after key and index sort is applicable so we jump directly to the doc
                 // that is after the index sort prefix using the rawAfterKey and we start collecting
                 // document from there.
                 try {
-                    processLeafFromQuery(ctx, indexSortPrefix);
+                    processLeafFromQuery(aggCtx.getLeafReaderContext(), indexSortPrefix);
                 } catch (CollectionTerminatedException e) {
                     /*
                      * Signal that there isn't anything to collect. We're going
@@ -481,7 +482,7 @@ public final class CompositeAggregator extends BucketsAggregator implements Size
             } else {
                 final LeafBucketCollector inner;
                 try {
-                    inner = queue.getLeafCollector(ctx, getFirstPassCollector(docIdSetBuilder, sortPrefixLen));
+                    inner = queue.getLeafCollector(aggCtx.getLeafReaderContext(), getFirstPassCollector(docIdSetBuilder, sortPrefixLen));
                 } catch (CollectionTerminatedException e) {
                     return LeafBucketCollector.NO_OP_COLLECTOR;
                 }

+ 6 - 3
server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregator.java

@@ -7,10 +7,10 @@
  */
 package org.elasticsearch.search.aggregations.bucket.filter;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.Bits;
 import org.elasticsearch.common.lucene.Lucene;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.CardinalityUpperBound;
@@ -46,9 +46,12 @@ public class FilterAggregator extends BucketsAggregator implements SingleBucketA
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         // no need to provide deleted docs to the filter
-        final Bits bits = Lucene.asSequentialAccessBits(ctx.reader().maxDoc(), filter.get().scorerSupplier(ctx));
+        final Bits bits = Lucene.asSequentialAccessBits(
+            aggCtx.getLeafReaderContext().reader().maxDoc(),
+            filter.get().scorerSupplier(aggCtx.getLeafReaderContext())
+        );
         return new LeafBucketCollectorBase(sub, null) {
             @Override
             public void collect(int doc, long bucket) throws IOException {

+ 5 - 4
server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterByFilterAggregator.java

@@ -16,6 +16,7 @@ import org.apache.lucene.util.Bits;
 import org.elasticsearch.common.CheckedSupplier;
 import org.elasticsearch.core.CheckedFunction;
 import org.elasticsearch.search.aggregations.AdaptingAggregator;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.CardinalityUpperBound;
@@ -217,12 +218,12 @@ public class FilterByFilterAggregator extends FiltersAggregator {
      * top level query into account when building the filters.
      */
     @Override
-    protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+    protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
         assert scoreMode().needsScores() == false;
         if (filters().size() == 0) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        Bits live = ctx.reader().getLiveDocs();
+        Bits live = aggCtx.getLeafReaderContext().reader().getLiveDocs();
         if (false == docCountProvider.alwaysOne()) {
             segmentsWithDocCountField++;
         }
@@ -233,10 +234,10 @@ public class FilterByFilterAggregator extends FiltersAggregator {
              * the sub-aggregators opt out of traditional collection.
              */
             segmentsCounted++;
-            collectCount(ctx, live);
+            collectCount(aggCtx.getLeafReaderContext(), live);
         } else {
             segmentsCollected++;
-            collectSubs(ctx, live, sub);
+            collectSubs(aggCtx.getLeafReaderContext(), live, sub);
         }
         return LeafBucketCollector.NO_OP_COLLECTOR;
     }

+ 3 - 2
server/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FiltersAggregator.java

@@ -16,6 +16,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
 import org.elasticsearch.core.CheckedFunction;
 import org.elasticsearch.index.query.QueryBuilder;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.CardinalityUpperBound;
@@ -262,10 +263,10 @@ public abstract class FiltersAggregator extends BucketsAggregator {
         }
 
         @Override
-        protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+        protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
             IntPredicate[] docFilters = new IntPredicate[filters().size()];
             for (int filterOrd = 0; filterOrd < filters().size(); filterOrd++) {
-                docFilters[filterOrd] = filters().get(filterOrd).matchingDocIds(ctx);
+                docFilters[filterOrd] = filters().get(filterOrd).matchingDocIds(aggCtx.getLeafReaderContext());
             }
             return new LeafBucketCollectorBase(sub, null) {
                 @Override

+ 4 - 3
server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregator.java

@@ -8,11 +8,11 @@
 package org.elasticsearch.search.aggregations.bucket.geogrid;
 
 import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.core.Releasables;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.CardinalityUpperBound;
@@ -67,8 +67,9 @@ public abstract class GeoGridAggregator<T extends InternalGeoGrid<?>> extends Bu
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(final LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
-        final SortedNumericDocValues values = valuesSource.longValues(ctx);
+    public LeafBucketCollector getLeafCollector(final AggregationExecutionContext aggCtx, final LeafBucketCollector sub)
+        throws IOException {
+        final SortedNumericDocValues values = valuesSource.longValues(aggCtx.getLeafReaderContext());
         final NumericDocValues singleton = DocValues.unwrapSingleton(values);
         return singleton != null ? getLeafCollector(singleton, sub) : getLeafCollector(values, sub);
     }

+ 4 - 4
server/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregator.java

@@ -7,12 +7,12 @@
  */
 package org.elasticsearch.search.aggregations.bucket.global;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.BulkScorer;
 import org.apache.lucene.search.LeafCollector;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Scorable;
 import org.apache.lucene.search.Weight;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.CardinalityUpperBound;
 import org.elasticsearch.search.aggregations.InternalAggregation;
@@ -35,9 +35,9 @@ public class GlobalAggregator extends BucketsAggregator implements SingleBucketA
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
         // Run sub-aggregations on child documents
-        BulkScorer scorer = weight.bulkScorer(ctx);
+        BulkScorer scorer = weight.bulkScorer(aggCtx.getLeafReaderContext());
         if (scorer == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
@@ -51,7 +51,7 @@ public class GlobalAggregator extends BucketsAggregator implements SingleBucketA
             public void setScorer(Scorable scorer) throws IOException {
                 sub.setScorer(scorer);
             }
-        }, ctx.reader().getLiveDocs());
+        }, aggCtx.getLeafReaderContext().reader().getLiveDocs());
         return LeafBucketCollector.NO_OP_COLLECTOR;
     }
 

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java

@@ -7,7 +7,6 @@
  */
 package org.elasticsearch.search.aggregations.bucket.histogram;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.util.CollectionUtil;
@@ -17,6 +16,7 @@ import org.elasticsearch.common.util.IntArray;
 import org.elasticsearch.common.util.LongArray;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.BucketOrder;
@@ -124,11 +124,11 @@ abstract class AutoDateHistogramAggregator extends DeferableBucketAggregator {
     protected abstract LeafBucketCollector getLeafCollector(SortedNumericDocValues values, LeafBucketCollector sub) throws IOException;
 
     @Override
-    public final LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+    public final LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        return getLeafCollector(valuesSource.longValues(ctx), sub);
+        return getLeafCollector(valuesSource.longValues(aggCtx.getLeafReaderContext()), sub);
     }
 
     protected final InternalAggregation[] buildAggregations(

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java

@@ -9,7 +9,6 @@ package org.elasticsearch.search.aggregations.bucket.histogram;
 
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.util.CollectionUtil;
@@ -20,6 +19,7 @@ import org.elasticsearch.core.Nullable;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.aggregations.AdaptingAggregator;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.BucketOrder;
@@ -273,11 +273,11 @@ class DateHistogramAggregator extends BucketsAggregator implements SizedBucketAg
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        SortedNumericDocValues values = valuesSource.longValues(ctx);
+        SortedNumericDocValues values = valuesSource.longValues(aggCtx.getLeafReaderContext());
         return new LeafBucketCollectorBase(sub, values) {
             @Override
             public void collect(int doc, long owningBucketOrd) throws IOException {

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateRangeHistogramAggregator.java

@@ -7,7 +7,6 @@
  */
 package org.elasticsearch.search.aggregations.bucket.histogram;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CollectionUtil;
@@ -18,6 +17,7 @@ import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
 import org.elasticsearch.index.mapper.RangeFieldMapper;
 import org.elasticsearch.index.mapper.RangeType;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.BucketOrder;
@@ -110,11 +110,11 @@ class DateRangeHistogramAggregator extends BucketsAggregator {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        SortedBinaryDocValues values = valuesSource.bytesValues(ctx);
+        SortedBinaryDocValues values = valuesSource.bytesValues(aggCtx.getLeafReaderContext());
         RangeType rangeType = valuesSource.rangeType();
         return new LeafBucketCollectorBase(sub, values) {
             @Override

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/NumericHistogramAggregator.java

@@ -8,9 +8,9 @@
 
 package org.elasticsearch.search.aggregations.bucket.histogram;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.BucketOrder;
@@ -78,12 +78,12 @@ public class NumericHistogramAggregator extends AbstractHistogramAggregator {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
 
-        final SortedNumericDoubleValues values = valuesSource.doubleValues(ctx);
+        final SortedNumericDoubleValues values = valuesSource.doubleValues(aggCtx.getLeafReaderContext());
         return new LeafBucketCollectorBase(sub, values) {
             @Override
             public void collect(int doc, long owningBucketOrd) throws IOException {

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/RangeHistogramAggregator.java

@@ -8,11 +8,11 @@
 
 package org.elasticsearch.search.aggregations.bucket.histogram;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.util.BytesRef;
 import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
 import org.elasticsearch.index.mapper.RangeFieldMapper;
 import org.elasticsearch.index.mapper.RangeType;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.BucketOrder;
@@ -72,11 +72,11 @@ public class RangeHistogramAggregator extends AbstractHistogramAggregator {
     }
 
     @Override
-    protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+    protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        final SortedBinaryDocValues values = valuesSource.bytesValues(ctx);
+        final SortedBinaryDocValues values = valuesSource.bytesValues(aggCtx.getLeafReaderContext());
         final RangeType rangeType = valuesSource.rangeType();
         return new LeafBucketCollectorBase(sub, values) {
             @Override

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/VariableWidthHistogramAggregator.java

@@ -8,7 +8,6 @@
 
 package org.elasticsearch.search.aggregations.bucket.histogram;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.InPlaceMergeSorter;
@@ -19,6 +18,7 @@ import org.elasticsearch.core.Releasable;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.BucketOrder;
@@ -522,11 +522,11 @@ public class VariableWidthHistogramAggregator extends DeferableBucketAggregator
     }
 
     @Override
-    protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+    protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        final SortedNumericDoubleValues values = valuesSource.doubleValues(ctx);
+        final SortedNumericDoubleValues values = valuesSource.doubleValues(aggCtx.getLeafReaderContext());
         return new LeafBucketCollectorBase(sub, values) {
             @Override
             public void collect(int doc, long bucket) throws IOException {

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregator.java

@@ -7,8 +7,8 @@
  */
 package org.elasticsearch.search.aggregations.bucket.missing;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.elasticsearch.index.fielddata.DocValueBits;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.CardinalityUpperBound;
@@ -43,10 +43,10 @@ public class MissingAggregator extends BucketsAggregator implements SingleBucket
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         final DocValueBits docsWithValue;
         if (valuesSource != null) {
-            docsWithValue = valuesSource.docsWithValue(ctx);
+            docsWithValue = valuesSource.docsWithValue(aggCtx.getLeafReaderContext());
         } else {
             docsWithValue = new DocValueBits() {
                 @Override

+ 6 - 4
server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregator.java

@@ -21,6 +21,7 @@ import org.apache.lucene.search.join.BitSetProducer;
 import org.apache.lucene.util.BitSet;
 import org.elasticsearch.common.lucene.search.Queries;
 import org.elasticsearch.index.mapper.NestedObjectMapper;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.CardinalityUpperBound;
@@ -66,14 +67,15 @@ public class NestedAggregator extends BucketsAggregator implements SingleBucketA
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(final LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
-        IndexReaderContext topLevelContext = ReaderUtil.getTopLevelContext(ctx);
+    public LeafBucketCollector getLeafCollector(final AggregationExecutionContext aggCtx, final LeafBucketCollector sub)
+        throws IOException {
+        IndexReaderContext topLevelContext = ReaderUtil.getTopLevelContext(aggCtx.getLeafReaderContext());
         IndexSearcher searcher = new IndexSearcher(topLevelContext);
         searcher.setQueryCache(null);
         Weight weight = searcher.createWeight(searcher.rewrite(childFilter), ScoreMode.COMPLETE_NO_SCORES, 1f);
-        Scorer childDocsScorer = weight.scorer(ctx);
+        Scorer childDocsScorer = weight.scorer(aggCtx.getLeafReaderContext());
 
-        final BitSet parentDocs = parentFilter.getBitSet(ctx);
+        final BitSet parentDocs = parentFilter.getBitSet(aggCtx.getLeafReaderContext());
         final DocIdSetIterator childDocs = childDocsScorer != null ? childDocsScorer.iterator() : null;
         if (collectsFromSingleBucket) {
             return new LeafBucketCollectorBase(sub, null) {

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregator.java

@@ -7,13 +7,13 @@
  */
 package org.elasticsearch.search.aggregations.bucket.nested;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.join.BitSetProducer;
 import org.apache.lucene.util.BitSet;
 import org.elasticsearch.common.lucene.search.Queries;
 import org.elasticsearch.index.mapper.NestedObjectMapper;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.CardinalityUpperBound;
@@ -55,10 +55,10 @@ public class ReverseNestedAggregator extends BucketsAggregator implements Single
     }
 
     @Override
-    protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         // In ES if parent is deleted, then also the children are deleted, so the child docs this agg receives
         // must belong to parent docs that is alive. For this reason acceptedDocs can be null here.
-        final BitSet parentDocs = parentBitsetProducer.getBitSet(ctx);
+        final BitSet parentDocs = parentBitsetProducer.getBitSet(aggCtx.getLeafReaderContext());
         if (parentDocs == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/bucket/prefix/IpPrefixAggregator.java

@@ -8,11 +8,11 @@
 
 package org.elasticsearch.search.aggregations.bucket.prefix;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CollectionUtil;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.AggregationExecutionException;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
@@ -113,8 +113,8 @@ public final class IpPrefixAggregator extends BucketsAggregator {
     }
 
     @Override
-    protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
-        return new IpPrefixLeafCollector(sub, config.getValuesSource().bytesValues(ctx), ipPrefix);
+    protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
+        return new IpPrefixLeafCollector(sub, config.getValuesSource().bytesValues(aggCtx.getLeafReaderContext()), ipPrefix);
     }
 
     private class IpPrefixLeafCollector extends LeafBucketCollectorBase {

+ 4 - 4
server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/BinaryRangeAggregator.java

@@ -7,12 +7,12 @@
  */
 package org.elasticsearch.search.aggregations.bucket.range;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.util.BytesRef;
 import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.CardinalityUpperBound;
@@ -93,12 +93,12 @@ public final class BinaryRangeAggregator extends BucketsAggregator {
     }
 
     @Override
-    protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+    protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
         if (valuesSource instanceof ValuesSource.Bytes.WithOrdinals) {
-            SortedSetDocValues values = ((ValuesSource.Bytes.WithOrdinals) valuesSource).ordinalsValues(ctx);
+            SortedSetDocValues values = ((ValuesSource.Bytes.WithOrdinals) valuesSource).ordinalsValues(aggCtx.getLeafReaderContext());
             return new SortedSetRangeLeafCollector(values, ranges, sub) {
                 @Override
                 protected void doCollect(LeafBucketCollector sub, int doc, long bucket) throws IOException {
@@ -106,7 +106,7 @@ public final class BinaryRangeAggregator extends BucketsAggregator {
                 }
             };
         } else {
-            SortedBinaryDocValues values = valuesSource.bytesValues(ctx);
+            SortedBinaryDocValues values = valuesSource.bytesValues(aggCtx.getLeafReaderContext());
             return new SortedBinaryRangeLeafCollector(values, ranges, sub) {
                 @Override
                 protected void doCollect(LeafBucketCollector sub, int doc, long bucket) throws IOException {

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

@@ -7,7 +7,6 @@
  */
 package org.elasticsearch.search.aggregations.bucket.range;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.ScorerSupplier;
 import org.elasticsearch.Version;
@@ -23,6 +22,7 @@ import org.elasticsearch.index.mapper.DateFieldMapper.Resolution;
 import org.elasticsearch.index.query.RangeQueryBuilder;
 import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.aggregations.AdaptingAggregator;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -652,8 +652,8 @@ public abstract class RangeAggregator extends BucketsAggregator {
         }
 
         @Override
-        public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
-            final SortedNumericDoubleValues values = ((ValuesSource.Numeric) this.valuesSource).doubleValues(ctx);
+        public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
+            final SortedNumericDoubleValues values = ((ValuesSource.Numeric) this.valuesSource).doubleValues(aggCtx.getLeafReaderContext());
             final NumericDoubleValues singleton = FieldData.unwrapSingleton(values);
 
             if (singleton != null) {

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregator.java

@@ -7,12 +7,12 @@
  */
 package org.elasticsearch.search.aggregations.bucket.sampler;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.misc.search.DiversifiedTopDocsCollector;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.elasticsearch.common.xcontent.LoggingDeprecationHandler;
 import org.elasticsearch.core.Releasables;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.AggregationExecutionException;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
@@ -230,11 +230,11 @@ public class SamplerAggregator extends DeferableBucketAggregator implements Sing
     }
 
     @Override
-    protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+    protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
         if (bdd == null) {
             throw new AggregationExecutionException("Sampler aggregation must be used with child aggregations.");
         }
-        return bdd.getLeafCollector(ctx);
+        return bdd.getLeafCollector(aggCtx);
     }
 
     @Override

+ 5 - 5
server/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/random/RandomSamplerAggregator.java

@@ -8,13 +8,13 @@
 
 package org.elasticsearch.search.aggregations.bucket.sampler.random;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.CollectionTerminatedException;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.Bits;
 import org.elasticsearch.common.CheckedSupplier;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.CardinalityUpperBound;
@@ -83,13 +83,13 @@ public class RandomSamplerAggregator extends BucketsAggregator implements Single
      * allows this aggregation to sample documents in the background. This provides a dramatic speed improvement, especially when a
      * non-trivial {@link RandomSamplerAggregator#topLevelQuery()} is provided.
      *
-     * @param ctx reader context
+     * @param aggCtx aggregation context
      * @param sub collector
      * @return returns {@link LeafBucketCollector#NO_OP_COLLECTOR} if sampling was done. Otherwise, it is a simple pass through collector
      * @throws IOException when building the query or extracting docs fails
      */
     @Override
-    protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+    protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
         // Certain leaf collectors can aggregate values without seeing any documents, even when sampled
         // To handle this, exit early if the sub collector is a no-op
         if (sub.isNoop()) {
@@ -105,13 +105,13 @@ public class RandomSamplerAggregator extends BucketsAggregator implements Single
             };
         }
         // TODO know when sampling would be much slower and skip sampling: https://github.com/elastic/elasticsearch/issues/84353
-        Scorer scorer = weightSupplier.get().scorer(ctx);
+        Scorer scorer = weightSupplier.get().scorer(aggCtx.getLeafReaderContext());
         // This means there are no docs to iterate, possibly due to the fields not existing
         if (scorer == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
         final DocIdSetIterator docIt = scorer.iterator();
-        final Bits liveDocs = ctx.reader().getLiveDocs();
+        final Bits liveDocs = aggCtx.getLeafReaderContext().reader().getLiveDocs();
         try {
             // Iterate every document provided by the scorer iterator
             for (int docId = docIt.nextDoc(); docId != DocIdSetIterator.NO_MORE_DOCS; docId = docIt.nextDoc()) {

+ 6 - 6
server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java

@@ -9,7 +9,6 @@
 package org.elasticsearch.search.aggregations.bucket.terms;
 
 import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.util.ArrayUtil;
@@ -21,6 +20,7 @@ import org.elasticsearch.common.util.LongHash;
 import org.elasticsearch.core.Releasable;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.AggregationExecutionException;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
@@ -107,8 +107,8 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
-        SortedSetDocValues globalOrds = valuesSource.globalOrdinalsValues(ctx);
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
+        SortedSetDocValues globalOrds = valuesSource.globalOrdinalsValues(aggCtx.getLeafReaderContext());
         collectionStrategy.globalOrdsReady(globalOrds);
         SortedDocValues singleValues = DocValues.unwrapSingleton(globalOrds);
         if (singleValues != null) {
@@ -299,12 +299,12 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
         }
 
         @Override
-        public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+        public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
             if (mapping != null) {
                 mapSegmentCountsToGlobalCounts(mapping);
             }
-            final SortedSetDocValues segmentOrds = valuesSource.ordinalsValues(ctx);
-            mapping = valuesSource.globalOrdinalsMapping(ctx);
+            final SortedSetDocValues segmentOrds = valuesSource.ordinalsValues(aggCtx.getLeafReaderContext());
+            mapping = valuesSource.globalOrdinalsMapping(aggCtx.getLeafReaderContext());
             if (segmentOrds.getValueCount() == 0) {
                 segmentsWithoutValues++;
                 return LeafBucketCollector.NO_OP_COLLECTOR;

+ 3 - 2
server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongRareTermsAggregator.java

@@ -13,6 +13,7 @@ import org.elasticsearch.common.util.LongHash;
 import org.elasticsearch.common.util.SetBackedScalingCuckooFilter;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.CardinalityUpperBound;
@@ -64,8 +65,8 @@ public class LongRareTermsAggregator extends AbstractRareTermsAggregator {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
-        SortedNumericDocValues values = getValues(valuesSource, ctx);
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
+        SortedNumericDocValues values = getValues(valuesSource, aggCtx.getLeafReaderContext());
         return new LeafBucketCollectorBase(sub, values) {
             @Override
             public void collect(int docId, long owningBucketOrd) throws IOException {

+ 3 - 2
server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/MapStringTermsAggregator.java

@@ -17,6 +17,7 @@ import org.elasticsearch.core.Releasable;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.BucketOrder;
@@ -85,11 +86,11 @@ public class MapStringTermsAggregator extends AbstractStringTermsAggregator {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
         return resultStrategy.wrapCollector(
             collectorSource.getLeafCollector(
                 includeExclude,
-                ctx,
+                aggCtx.getLeafReaderContext(),
                 sub,
                 this::addRequestCircuitBreakerBytes,
                 (s, doc, owningBucketOrd, bytes) -> {

+ 3 - 2
server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/NumericTermsAggregator.java

@@ -17,6 +17,7 @@ import org.elasticsearch.core.Releasable;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.FieldData;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.BucketOrder;
@@ -81,8 +82,8 @@ public class NumericTermsAggregator extends TermsAggregator {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
-        SortedNumericDocValues values = resultStrategy.getValues(ctx);
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
+        SortedNumericDocValues values = resultStrategy.getValues(aggCtx.getLeafReaderContext());
         return resultStrategy.wrapCollector(new LeafBucketCollectorBase(sub, values) {
             @Override
             public void collect(int doc, long owningBucketOrd) throws IOException {

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringRareTermsAggregator.java

@@ -7,7 +7,6 @@
  */
 package org.elasticsearch.search.aggregations.bucket.terms;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.elasticsearch.common.util.BytesRefHash;
@@ -15,6 +14,7 @@ import org.elasticsearch.common.util.SetBackedScalingCuckooFilter;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.CardinalityUpperBound;
@@ -62,8 +62,8 @@ public class StringRareTermsAggregator extends AbstractRareTermsAggregator {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
-        final SortedBinaryDocValues values = valuesSource.bytesValues(ctx);
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
+        final SortedBinaryDocValues values = valuesSource.bytesValues(aggCtx.getLeafReaderContext());
         return new LeafBucketCollectorBase(sub, values) {
             final BytesRefBuilder previous = new BytesRefBuilder();
 

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/metrics/AbstractHDRPercentilesAggregator.java

@@ -9,7 +9,6 @@
 package org.elasticsearch.search.aggregations.metrics;
 
 import org.HdrHistogram.DoubleHistogram;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.ArrayUtils;
 import org.elasticsearch.common.util.BigArrays;
@@ -17,6 +16,7 @@ import org.elasticsearch.common.util.ObjectArray;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
 import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
@@ -65,11 +65,11 @@ abstract class AbstractHDRPercentilesAggregator extends NumericMetricsAggregator
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        final SortedNumericDoubleValues values = ((ValuesSource.Numeric) valuesSource).doubleValues(ctx);
+        final SortedNumericDoubleValues values = ((ValuesSource.Numeric) valuesSource).doubleValues(aggCtx.getLeafReaderContext());
         return new LeafBucketCollectorBase(sub, values) {
             @Override
             public void collect(int doc, long bucket) throws IOException {

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/metrics/AbstractTDigestPercentilesAggregator.java

@@ -8,7 +8,6 @@
 
 package org.elasticsearch.search.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.ArrayUtils;
 import org.elasticsearch.common.util.BigArrays;
@@ -16,6 +15,7 @@ import org.elasticsearch.common.util.ObjectArray;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
 import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
@@ -64,11 +64,11 @@ abstract class AbstractTDigestPercentilesAggregator extends NumericMetricsAggreg
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        final SortedNumericDoubleValues values = ((ValuesSource.Numeric) valuesSource).doubleValues(ctx);
+        final SortedNumericDoubleValues values = ((ValuesSource.Numeric) valuesSource).doubleValues(aggCtx.getLeafReaderContext());
         return new LeafBucketCollectorBase(sub, values) {
             @Override
             public void collect(int doc, long bucket) throws IOException {

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/metrics/AvgAggregator.java

@@ -7,7 +7,6 @@
  */
 package org.elasticsearch.search.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.DoubleArray;
@@ -15,6 +14,7 @@ import org.elasticsearch.common.util.LongArray;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -60,11 +60,11 @@ class AvgAggregator extends NumericMetricsAggregator.SingleValue {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        final SortedNumericDoubleValues values = valuesSource.doubleValues(ctx);
+        final SortedNumericDoubleValues values = valuesSource.doubleValues(aggCtx.getLeafReaderContext());
         final CompensatedSum kahanSummation = new CompensatedSum(0, 0);
 
         return new LeafBucketCollectorBase(sub, values) {

+ 3 - 2
server/src/main/java/org/elasticsearch/search/aggregations/metrics/CardinalityAggregator.java

@@ -26,6 +26,7 @@ import org.elasticsearch.core.Releasable;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -118,10 +119,10 @@ public class CardinalityAggregator extends NumericMetricsAggregator.SingleValue
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         postCollectLastCollector();
 
-        collector = pickCollector(ctx);
+        collector = pickCollector(aggCtx.getLeafReaderContext());
         return collector;
     }
 

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/metrics/ExtendedStatsAggregator.java

@@ -7,7 +7,6 @@
  */
 package org.elasticsearch.search.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.DoubleArray;
@@ -15,6 +14,7 @@ import org.elasticsearch.common.util.LongArray;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -76,11 +76,11 @@ class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        final SortedNumericDoubleValues values = valuesSource.doubleValues(ctx);
+        final SortedNumericDoubleValues values = valuesSource.doubleValues(aggCtx.getLeafReaderContext());
         final CompensatedSum compensatedSum = new CompensatedSum(0, 0);
         final CompensatedSum compensatedSumOfSqr = new CompensatedSum(0, 0);
         return new LeafBucketCollectorBase(sub, values) {

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsAggregator.java

@@ -8,11 +8,11 @@
 
 package org.elasticsearch.search.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.elasticsearch.common.geo.GeoPoint;
 import org.elasticsearch.common.util.DoubleArray;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.MultiGeoPointValues;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -67,11 +67,11 @@ final class GeoBoundsAggregator extends MetricsAggregator {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        final MultiGeoPointValues values = valuesSource.geoPointValues(ctx);
+        final MultiGeoPointValues values = valuesSource.geoPointValues(aggCtx.getLeafReaderContext());
         return new LeafBucketCollectorBase(sub, values) {
             @Override
             public void collect(int doc, long bucket) throws IOException {

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregator.java

@@ -8,12 +8,12 @@
 
 package org.elasticsearch.search.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.elasticsearch.common.geo.GeoPoint;
 import org.elasticsearch.common.util.DoubleArray;
 import org.elasticsearch.common.util.LongArray;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.MultiGeoPointValues;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -53,11 +53,11 @@ final class GeoCentroidAggregator extends MetricsAggregator {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        final MultiGeoPointValues values = valuesSource.geoPointValues(ctx);
+        final MultiGeoPointValues values = valuesSource.geoPointValues(aggCtx.getLeafReaderContext());
         final CompensatedSum compensatedSumLat = new CompensatedSum(0, 0);
         final CompensatedSum compensatedSumLon = new CompensatedSum(0, 0);
 

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/metrics/GlobalOrdCardinalityAggregator.java

@@ -8,7 +8,6 @@
 
 package org.elasticsearch.search.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.util.BytesRef;
@@ -19,6 +18,7 @@ import org.elasticsearch.common.util.LongArray;
 import org.elasticsearch.common.util.ObjectArray;
 import org.elasticsearch.core.Nullable;
 import org.elasticsearch.core.Releasables;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -68,8 +68,8 @@ public class GlobalOrdCardinalityAggregator extends NumericMetricsAggregator.Sin
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
-        values = valuesSource.globalOrdinalsValues(ctx);
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
+        values = valuesSource.globalOrdinalsValues(aggCtx.getLeafReaderContext());
         return new LeafBucketCollector() {
             @Override
             public void collect(int doc, long bucketOrd) throws IOException {

+ 4 - 4
server/src/main/java/org/elasticsearch/search/aggregations/metrics/MaxAggregator.java

@@ -8,7 +8,6 @@
 package org.elasticsearch.search.aggregations.metrics;
 
 import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.PointValues;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.util.Bits;
@@ -18,6 +17,7 @@ import org.elasticsearch.index.fielddata.NumericDoubleValues;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
 import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.MultiValueMode;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -65,12 +65,12 @@ class MaxAggregator extends NumericMetricsAggregator.SingleValue {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
         if (pointConverter != null) {
-            Number segMax = findLeafMaxValue(ctx.reader(), pointField, pointConverter);
+            Number segMax = findLeafMaxValue(aggCtx.getLeafReaderContext().reader(), pointField, pointConverter);
             if (segMax != null) {
                 /*
                  * There is no parent aggregator (see {@link AggregatorBase#getPointReaderOrNull}
@@ -84,7 +84,7 @@ class MaxAggregator extends NumericMetricsAggregator.SingleValue {
                 return LeafBucketCollector.NO_OP_COLLECTOR;
             }
         }
-        final SortedNumericDoubleValues allValues = valuesSource.doubleValues(ctx);
+        final SortedNumericDoubleValues allValues = valuesSource.doubleValues(aggCtx.getLeafReaderContext());
         final NumericDoubleValues values = MultiValueMode.MAX.select(allValues);
         return new LeafBucketCollectorBase(sub, allValues) {
 

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregator.java

@@ -8,13 +8,13 @@
 
 package org.elasticsearch.search.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.ObjectArray;
 import org.elasticsearch.core.Nullable;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -78,12 +78,12 @@ public class MedianAbsoluteDeviationAggregator extends NumericMetricsAggregator.
     }
 
     @Override
-    protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+    protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
 
-        final SortedNumericDoubleValues values = valuesSource.doubleValues(ctx);
+        final SortedNumericDoubleValues values = valuesSource.doubleValues(aggCtx.getLeafReaderContext());
 
         return new LeafBucketCollectorBase(sub, values) {
             @Override

+ 4 - 4
server/src/main/java/org/elasticsearch/search/aggregations/metrics/MinAggregator.java

@@ -8,7 +8,6 @@
 package org.elasticsearch.search.aggregations.metrics;
 
 import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.PointValues;
 import org.apache.lucene.search.CollectionTerminatedException;
 import org.apache.lucene.search.ScoreMode;
@@ -19,6 +18,7 @@ import org.elasticsearch.index.fielddata.NumericDoubleValues;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
 import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.MultiValueMode;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -66,12 +66,12 @@ public class MinAggregator extends NumericMetricsAggregator.SingleValue {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
         if (pointConverter != null) {
-            Number segMin = findLeafMinValue(ctx.reader(), pointField, pointConverter);
+            Number segMin = findLeafMinValue(aggCtx.getLeafReaderContext().reader(), pointField, pointConverter);
             if (segMin != null) {
                 /*
                  * There is no parent aggregator (see {@link MinAggregator#getPointReaderOrNull}
@@ -84,7 +84,7 @@ public class MinAggregator extends NumericMetricsAggregator.SingleValue {
                 return LeafBucketCollector.NO_OP_COLLECTOR;
             }
         }
-        final SortedNumericDoubleValues allValues = valuesSource.doubleValues(ctx);
+        final SortedNumericDoubleValues allValues = valuesSource.doubleValues(aggCtx.getLeafReaderContext());
         final NumericDoubleValues values = MultiValueMode.MIN.select(allValues);
         return new LeafBucketCollectorBase(sub, allValues) {
             @Override

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricAggregator.java

@@ -8,7 +8,6 @@
 
 package org.elasticsearch.search.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.Scorable;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.io.stream.StreamOutput;
@@ -19,6 +18,7 @@ import org.elasticsearch.core.Releasables;
 import org.elasticsearch.script.Script;
 import org.elasticsearch.script.ScriptedMetricAggContexts;
 import org.elasticsearch.script.ScriptedMetricAggContexts.MapScript;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -90,7 +90,7 @@ class ScriptedMetricAggregator extends MetricsAggregator {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
         // Clear any old leaf scripts so we rebuild them on the new leaf when we first see them.
         for (long i = 0; i < states.size(); i++) {
             State state = states.get(i);
@@ -117,7 +117,7 @@ class ScriptedMetricAggregator extends MetricsAggregator {
                     states.set(owningBucketOrd, state);
                 }
                 if (state.leafMapScript == null) {
-                    state.leafMapScript = state.mapScript.newInstance(ctx);
+                    state.leafMapScript = state.mapScript.newInstance(aggCtx.getLeafReaderContext());
                     state.leafMapScript.setScorer(scorer);
                 }
                 state.leafMapScript.setDocument(doc);

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/metrics/StatsAggregator.java

@@ -7,7 +7,6 @@
  */
 package org.elasticsearch.search.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.DoubleArray;
@@ -15,6 +14,7 @@ import org.elasticsearch.common.util.LongArray;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -65,11 +65,11 @@ class StatsAggregator extends NumericMetricsAggregator.MultiValue {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        final SortedNumericDoubleValues values = valuesSource.doubleValues(ctx);
+        final SortedNumericDoubleValues values = valuesSource.doubleValues(aggCtx.getLeafReaderContext());
         final CompensatedSum kahanSummation = new CompensatedSum(0, 0);
 
         return new LeafBucketCollectorBase(sub, values) {

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/metrics/SumAggregator.java

@@ -7,12 +7,12 @@
  */
 package org.elasticsearch.search.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.DoubleArray;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -55,11 +55,11 @@ public class SumAggregator extends NumericMetricsAggregator.SingleValue {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        final SortedNumericDoubleValues values = valuesSource.doubleValues(ctx);
+        final SortedNumericDoubleValues values = valuesSource.doubleValues(aggCtx.getLeafReaderContext());
         final CompensatedSum kahanSummation = new CompensatedSum(0, 0);
         return new LeafBucketCollectorBase(sub, values) {
             @Override

+ 3 - 3
server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregator.java

@@ -8,7 +8,6 @@
 
 package org.elasticsearch.search.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.FieldDoc;
 import org.apache.lucene.search.LeafCollector;
@@ -32,6 +31,7 @@ import org.elasticsearch.common.util.LongObjectPagedHashMap.Cursor;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.search.SearchHit;
 import org.elasticsearch.search.SearchHits;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -97,7 +97,7 @@ class TopHitsAggregator extends MetricsAggregator {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
         // Create leaf collectors here instead of at the aggregator level. Otherwise in case this collector get invoked
         // when post collecting then we have already replaced the leaf readers on the aggregator level have already been
         // replaced with the next leaf readers and then post collection pushes docids of the previous segment, which
@@ -149,7 +149,7 @@ class TopHitsAggregator extends MetricsAggregator {
 
                 LeafCollector leafCollector = leafCollectors.get(bucket);
                 if (leafCollector == null) {
-                    leafCollector = collectors.collector.getLeafCollector(ctx);
+                    leafCollector = collectors.collector.getLeafCollector(aggCtx.getLeafReaderContext());
                     if (scorer != null) {
                         leafCollector.setScorer(scorer);
                     }

+ 5 - 5
server/src/main/java/org/elasticsearch/search/aggregations/metrics/ValueCountAggregator.java

@@ -7,13 +7,13 @@
  */
 package org.elasticsearch.search.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.LongArray;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.MultiGeoPointValues;
 import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -54,13 +54,13 @@ public class ValueCountAggregator extends NumericMetricsAggregator.SingleValue {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
 
         if (valuesSource instanceof ValuesSource.Numeric) {
-            final SortedNumericDocValues values = ((ValuesSource.Numeric) valuesSource).longValues(ctx);
+            final SortedNumericDocValues values = ((ValuesSource.Numeric) valuesSource).longValues(aggCtx.getLeafReaderContext());
             return new LeafBucketCollectorBase(sub, values) {
 
                 @Override
@@ -73,7 +73,7 @@ public class ValueCountAggregator extends NumericMetricsAggregator.SingleValue {
             };
         }
         if (valuesSource instanceof ValuesSource.Bytes.GeoPoint) {
-            MultiGeoPointValues values = ((ValuesSource.GeoPoint) valuesSource).geoPointValues(ctx);
+            MultiGeoPointValues values = ((ValuesSource.GeoPoint) valuesSource).geoPointValues(aggCtx.getLeafReaderContext());
             return new LeafBucketCollectorBase(sub, null) {
 
                 @Override
@@ -86,7 +86,7 @@ public class ValueCountAggregator extends NumericMetricsAggregator.SingleValue {
             };
         }
         // The following is default collector. Including the keyword FieldType
-        final SortedBinaryDocValues values = valuesSource.bytesValues(ctx);
+        final SortedBinaryDocValues values = valuesSource.bytesValues(aggCtx.getLeafReaderContext());
         return new LeafBucketCollectorBase(sub, values) {
 
             @Override

+ 4 - 4
server/src/main/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregator.java

@@ -7,12 +7,12 @@
  */
 package org.elasticsearch.search.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.DoubleArray;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.AggregationExecutionException;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
@@ -62,12 +62,12 @@ class WeightedAvgAggregator extends NumericMetricsAggregator.SingleValue {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSources == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        final SortedNumericDoubleValues docValues = valuesSources.getField(VALUE_FIELD.getPreferredName(), ctx);
-        final SortedNumericDoubleValues docWeights = valuesSources.getField(WEIGHT_FIELD.getPreferredName(), ctx);
+        final SortedNumericDoubleValues docValues = valuesSources.getField(VALUE_FIELD.getPreferredName(), aggCtx.getLeafReaderContext());
+        final SortedNumericDoubleValues docWeights = valuesSources.getField(WEIGHT_FIELD.getPreferredName(), aggCtx.getLeafReaderContext());
         final CompensatedSum compensatedValueSum = new CompensatedSum(0, 0);
         final CompensatedSum compensatedWeightSum = new CompensatedSum(0, 0);
 

+ 1 - 8
server/src/main/java/org/elasticsearch/search/aggregations/timeseries/TimeSeriesAggregator.java

@@ -8,7 +8,6 @@
 
 package org.elasticsearch.search.aggregations.timeseries;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.util.BytesRef;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper;
@@ -89,13 +88,7 @@ public class TimeSeriesAggregator extends BucketsAggregator {
     }
 
     @Override
-    protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBucketCollector sub) throws IOException {
-        // TODO: remove this method in a follow up PR
-        throw new UnsupportedOperationException("Shouldn't be here");
-    }
-
-    protected LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBucketCollector sub, AggregationExecutionContext aggCtx)
-        throws IOException {
+    protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
         return new LeafBucketCollectorBase(sub, null) {
 
             @Override

+ 1 - 2
server/src/test/java/org/elasticsearch/search/aggregations/AdaptingAggregatorTests.java

@@ -8,7 +8,6 @@
 
 package org.elasticsearch.search.aggregations;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.elasticsearch.core.CheckedFunction;
 import org.elasticsearch.index.mapper.MapperService;
 import org.elasticsearch.index.mapper.MapperServiceTestCase;
@@ -107,7 +106,7 @@ public class AdaptingAggregatorTests extends MapperServiceTestCase {
         }
 
         @Override
-        protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+        protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) {
             // TODO Auto-generated method stub
             return null;
         }

+ 1 - 2
server/src/test/java/org/elasticsearch/search/aggregations/AggregatorBaseTests.java

@@ -9,7 +9,6 @@
 package org.elasticsearch.search.aggregations;
 
 import org.apache.lucene.document.LongPoint;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
@@ -42,7 +41,7 @@ public class AggregatorBaseTests extends MapperServiceTestCase {
         }
 
         @Override
-        protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+        protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) {
             throw new UnsupportedOperationException();
         }
 

+ 2 - 2
server/src/test/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregatorTests.java

@@ -12,11 +12,11 @@ import org.apache.lucene.document.Document;
 import org.apache.lucene.document.SortedNumericDocValuesField;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.tests.index.RandomIndexWriter;
 import org.elasticsearch.index.mapper.NumberFieldMapper;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.AggregatorTestCase;
 import org.elasticsearch.search.aggregations.InternalAggregation;
@@ -46,7 +46,7 @@ public class BucketsAggregatorTests extends AggregatorTestCase {
 
                 return new BucketsAggregator("test", AggregatorFactories.EMPTY, context, null, null, null) {
                     @Override
-                    protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+                    protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) {
                         return null;
                     }
 

+ 1 - 1
test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java

@@ -1286,7 +1286,7 @@ public abstract class AggregatorTestCase extends ESTestCase {
                     throws IOException {
                     return new MetricsAggregator(name, context, parent, metadata) {
                         @Override
-                        protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+                        protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) {
                             return LeafBucketCollector.NO_OP_COLLECTOR;
                         }
 

+ 3 - 3
x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/bucket/histogram/HistoBackedHistogramAggregator.java

@@ -7,9 +7,9 @@
 
 package org.elasticsearch.xpack.analytics.aggregations.bucket.histogram;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.elasticsearch.index.fielddata.HistogramValue;
 import org.elasticsearch.index.fielddata.HistogramValues;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.BucketOrder;
@@ -72,12 +72,12 @@ public class HistoBackedHistogramAggregator extends AbstractHistogramAggregator
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
 
-        final HistogramValues values = valuesSource.getHistogramValues(ctx);
+        final HistogramValues values = valuesSource.getHistogramValues(aggCtx.getLeafReaderContext());
         return new LeafBucketCollectorBase(sub, values) {
             @Override
             public void collect(int doc, long owningBucketOrd) throws IOException {

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

@@ -7,10 +7,10 @@
 
 package org.elasticsearch.xpack.analytics.aggregations.bucket.range;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.elasticsearch.index.fielddata.HistogramValue;
 import org.elasticsearch.index.fielddata.HistogramValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.CardinalityUpperBound;
@@ -113,12 +113,12 @@ public abstract class HistoBackedRangeAggregator extends RangeAggregator {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
         if ((valuesSource instanceof HistogramValuesSource.Histogram) == false) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
         final HistogramValuesSource.Histogram valuesSource = (HistogramValuesSource.Histogram) this.valuesSource;
-        final HistogramValues values = valuesSource.getHistogramValues(ctx);
+        final HistogramValues values = valuesSource.getHistogramValues(aggCtx.getLeafReaderContext());
         return new LeafBucketCollectorBase(sub, values) {
             @Override
             public void collect(int doc, long bucket) throws IOException {

+ 3 - 3
x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/metrics/AbstractHistoBackedHDRPercentilesAggregator.java

@@ -8,7 +8,6 @@
 package org.elasticsearch.xpack.analytics.aggregations.metrics;
 
 import org.HdrHistogram.DoubleHistogram;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.ArrayUtils;
 import org.elasticsearch.common.util.BigArrays;
@@ -17,6 +16,7 @@ import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.HistogramValue;
 import org.elasticsearch.index.fielddata.HistogramValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
 import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
@@ -67,11 +67,11 @@ abstract class AbstractHistoBackedHDRPercentilesAggregator extends NumericMetric
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        final HistogramValues values = ((HistogramValuesSource.Histogram) valuesSource).getHistogramValues(ctx);
+        final HistogramValues values = ((HistogramValuesSource.Histogram) valuesSource).getHistogramValues(aggCtx.getLeafReaderContext());
 
         return new LeafBucketCollectorBase(sub, values) {
             @Override

+ 3 - 3
x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/metrics/AbstractHistoBackedTDigestPercentilesAggregator.java

@@ -7,7 +7,6 @@
 
 package org.elasticsearch.xpack.analytics.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.ArrayUtils;
 import org.elasticsearch.common.util.BigArrays;
@@ -16,6 +15,7 @@ import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.HistogramValue;
 import org.elasticsearch.index.fielddata.HistogramValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
 import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
@@ -67,11 +67,11 @@ abstract class AbstractHistoBackedTDigestPercentilesAggregator extends NumericMe
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        final HistogramValues values = ((HistogramValuesSource.Histogram) valuesSource).getHistogramValues(ctx);
+        final HistogramValues values = ((HistogramValuesSource.Histogram) valuesSource).getHistogramValues(aggCtx.getLeafReaderContext());
 
         return new LeafBucketCollectorBase(sub, values) {
             @Override

+ 3 - 3
x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/metrics/HistoBackedAvgAggregator.java

@@ -6,7 +6,6 @@
  */
 package org.elasticsearch.xpack.analytics.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.DoubleArray;
 import org.elasticsearch.common.util.LongArray;
@@ -14,6 +13,7 @@ import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.HistogramValue;
 import org.elasticsearch.index.fielddata.HistogramValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -65,11 +65,11 @@ public class HistoBackedAvgAggregator extends NumericMetricsAggregator.SingleVal
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        final HistogramValues values = valuesSource.getHistogramValues(ctx);
+        final HistogramValues values = valuesSource.getHistogramValues(aggCtx.getLeafReaderContext());
         final CompensatedSum kahanSummation = new CompensatedSum(0, 0);
 
         return new LeafBucketCollectorBase(sub, values) {

+ 3 - 3
x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/metrics/HistoBackedMaxAggregator.java

@@ -6,13 +6,13 @@
  */
 package org.elasticsearch.xpack.analytics.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.DoubleArray;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.HistogramValue;
 import org.elasticsearch.index.fielddata.HistogramValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -54,12 +54,12 @@ public class HistoBackedMaxAggregator extends NumericMetricsAggregator.SingleVal
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
 
-        final HistogramValues values = valuesSource.getHistogramValues(ctx);
+        final HistogramValues values = valuesSource.getHistogramValues(aggCtx.getLeafReaderContext());
         return new LeafBucketCollectorBase(sub, values) {
 
             @Override

+ 3 - 3
x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/metrics/HistoBackedMinAggregator.java

@@ -6,13 +6,13 @@
  */
 package org.elasticsearch.xpack.analytics.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.DoubleArray;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.HistogramValue;
 import org.elasticsearch.index.fielddata.HistogramValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -54,12 +54,12 @@ public class HistoBackedMinAggregator extends NumericMetricsAggregator.SingleVal
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
 
-        final HistogramValues values = valuesSource.getHistogramValues(ctx);
+        final HistogramValues values = valuesSource.getHistogramValues(aggCtx.getLeafReaderContext());
         return new LeafBucketCollectorBase(sub, values) {
 
             @Override

+ 3 - 3
x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/metrics/HistoBackedSumAggregator.java

@@ -6,13 +6,13 @@
  */
 package org.elasticsearch.xpack.analytics.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.DoubleArray;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.HistogramValue;
 import org.elasticsearch.index.fielddata.HistogramValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -64,11 +64,11 @@ public class HistoBackedSumAggregator extends NumericMetricsAggregator.SingleVal
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        final HistogramValues values = valuesSource.getHistogramValues(ctx);
+        final HistogramValues values = valuesSource.getHistogramValues(aggCtx.getLeafReaderContext());
 
         final CompensatedSum kahanSummation = new CompensatedSum(0, 0);
         return new LeafBucketCollectorBase(sub, values) {

+ 3 - 3
x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/aggregations/metrics/HistoBackedValueCountAggregator.java

@@ -6,11 +6,11 @@
  */
 package org.elasticsearch.xpack.analytics.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.elasticsearch.common.util.LongArray;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.HistogramValue;
 import org.elasticsearch.index.fielddata.HistogramValues;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -52,12 +52,12 @@ public class HistoBackedValueCountAggregator extends NumericMetricsAggregator.Si
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
 
-        final HistogramValues values = valuesSource.getHistogramValues(ctx);
+        final HistogramValues values = valuesSource.getHistogramValues(aggCtx.getLeafReaderContext());
         return new LeafBucketCollectorBase(sub, values) {
             @Override
             public void collect(int doc, long bucket) throws IOException {

+ 6 - 4
x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/boxplot/BoxplotAggregator.java

@@ -7,7 +7,6 @@
 
 package org.elasticsearch.xpack.analytics.boxplot;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.ObjectArray;
@@ -16,6 +15,7 @@ import org.elasticsearch.index.fielddata.HistogramValue;
 import org.elasticsearch.index.fielddata.HistogramValues;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -60,12 +60,14 @@ public class BoxplotAggregator extends NumericMetricsAggregator.MultiValue {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
         if (valuesSource instanceof HistogramValuesSource.Histogram) {
-            final HistogramValues values = ((HistogramValuesSource.Histogram) valuesSource).getHistogramValues(ctx);
+            final HistogramValues values = ((HistogramValuesSource.Histogram) valuesSource).getHistogramValues(
+                aggCtx.getLeafReaderContext()
+            );
             return new LeafBucketCollectorBase(sub, values) {
                 @Override
                 public void collect(int doc, long bucket) throws IOException {
@@ -79,7 +81,7 @@ public class BoxplotAggregator extends NumericMetricsAggregator.MultiValue {
                 }
             };
         } else {
-            final SortedNumericDoubleValues values = ((ValuesSource.Numeric) valuesSource).doubleValues(ctx);
+            final SortedNumericDoubleValues values = ((ValuesSource.Numeric) valuesSource).doubleValues(aggCtx.getLeafReaderContext());
             return new LeafBucketCollectorBase(sub, values) {
                 @Override
                 public void collect(int doc, long bucket) throws IOException {

+ 3 - 2
x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/multiterms/MultiTermsAggregator.java

@@ -23,6 +23,7 @@ import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.BucketOrder;
@@ -183,8 +184,8 @@ class MultiTermsAggregator extends DeferableBucketAggregator {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
-        List<TermValues> termValuesList = termValuesList(ctx);
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
+        List<TermValues> termValuesList = termValuesList(aggCtx.getLeafReaderContext());
 
         return new LeafBucketCollectorBase(sub, values) {
             @Override

+ 3 - 3
x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/rate/HistogramRateAggregator.java

@@ -7,10 +7,10 @@
 
 package org.elasticsearch.xpack.analytics.rate;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.elasticsearch.common.Rounding;
 import org.elasticsearch.index.fielddata.HistogramValue;
 import org.elasticsearch.index.fielddata.HistogramValues;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
 import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
@@ -36,9 +36,9 @@ public class HistogramRateAggregator extends AbstractRateAggregator {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         final CompensatedSum kahanSummation = new CompensatedSum(0, 0);
-        final HistogramValues values = ((HistogramValuesSource.Histogram) valuesSource).getHistogramValues(ctx);
+        final HistogramValues values = ((HistogramValuesSource.Histogram) valuesSource).getHistogramValues(aggCtx.getLeafReaderContext());
         return new LeafBucketCollectorBase(sub, values) {
             @Override
             public void collect(int doc, long bucket) throws IOException {

+ 4 - 4
x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/rate/NumericRateAggregator.java

@@ -7,9 +7,9 @@
 
 package org.elasticsearch.xpack.analytics.rate;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.elasticsearch.common.Rounding;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
 import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
@@ -40,14 +40,14 @@ public class NumericRateAggregator extends AbstractRateAggregator {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         final CompensatedSum kahanSummation = new CompensatedSum(0, 0);
         if (computeWithDocCount) {
             // No field or script has been set at the rate agg. So, rate will be computed based on the doc_counts.
             // This implementation hard-wires the DocCountProvider and reads the _doc_count fields when available.
             // A better approach would be to create a DOC_COUNT ValuesSource type and use that as valuesSource
             // In that case the computeRateOnDocs variable and this branch of the if-statement are not required.
-            docCountProvider.setLeafReaderContext(ctx);
+            docCountProvider.setLeafReaderContext(aggCtx.getLeafReaderContext());
             return new LeafBucketCollectorBase(sub, null) {
                 @Override
                 public void collect(int doc, long bucket) throws IOException {
@@ -66,7 +66,7 @@ public class NumericRateAggregator extends AbstractRateAggregator {
                 }
             };
         } else {
-            final SortedNumericDoubleValues values = ((ValuesSource.Numeric) valuesSource).doubleValues(ctx);
+            final SortedNumericDoubleValues values = ((ValuesSource.Numeric) valuesSource).doubleValues(aggCtx.getLeafReaderContext());
             return new LeafBucketCollectorBase(sub, values) {
                 @Override
                 public void collect(int doc, long bucket) throws IOException {

+ 2 - 2
x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/rate/RateAggregatorFactory.java

@@ -7,8 +7,8 @@
 
 package org.elasticsearch.xpack.analytics.rate;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.elasticsearch.common.Rounding;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -70,7 +70,7 @@ class RateAggregatorFactory extends ValuesSourceAggregatorFactory {
     protected Aggregator createUnmapped(Aggregator parent, Map<String, Object> metadata) throws IOException {
         return new AbstractRateAggregator(name, config, rateUnit, rateMode, context, parent, metadata) {
             @Override
-            public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) {
+            public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) {
                 return LeafBucketCollector.NO_OP_COLLECTOR;
             }
         };

+ 3 - 3
x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/stringstats/StringStatsAggregator.java

@@ -6,7 +6,6 @@
  */
 package org.elasticsearch.xpack.analytics.stringstats;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.util.BytesRef;
 import org.elasticsearch.common.util.BigArrays;
@@ -16,6 +15,7 @@ import org.elasticsearch.common.util.Maps;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -80,11 +80,11 @@ public class StringStatsAggregator extends MetricsAggregator {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        final SortedBinaryDocValues values = valuesSource.bytesValues(ctx);
+        final SortedBinaryDocValues values = valuesSource.bytesValues(aggCtx.getLeafReaderContext());
 
         return new LeafBucketCollectorBase(sub, values) {
             @Override

+ 3 - 2
x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/topmetrics/TopMetricsAggregator.java

@@ -23,6 +23,7 @@ import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.NumericDoubleValues;
 import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.MultiValueMode;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -120,10 +121,10 @@ class TopMetricsAggregator extends NumericMetricsAggregator.MultiValue {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException {
         assert sub.isNoop() : "Expected noop but was " + sub.toString();
 
-        BucketedSort.Leaf leafSort = sort.forLeaf(ctx);
+        BucketedSort.Leaf leafSort = sort.forLeaf(aggCtx.getLeafReaderContext());
 
         return new LeafBucketCollector() {
             @Override

+ 4 - 4
x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/ttest/PairedTTestAggregator.java

@@ -7,10 +7,10 @@
 
 package org.elasticsearch.xpack.analytics.ttest;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.AggregationExecutionException;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -57,12 +57,12 @@ public class PairedTTestAggregator extends TTestAggregator<PairedTTestState> {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSources == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        final SortedNumericDoubleValues docAValues = valuesSources.getField(A_FIELD.getPreferredName(), ctx);
-        final SortedNumericDoubleValues docBValues = valuesSources.getField(B_FIELD.getPreferredName(), ctx);
+        final SortedNumericDoubleValues docAValues = valuesSources.getField(A_FIELD.getPreferredName(), aggCtx.getLeafReaderContext());
+        final SortedNumericDoubleValues docBValues = valuesSources.getField(B_FIELD.getPreferredName(), aggCtx.getLeafReaderContext());
         final CompensatedSum compDiffSum = new CompensatedSum(0, 0);
         final CompensatedSum compDiffSumOfSqr = new CompensatedSum(0, 0);
 

+ 6 - 5
x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/ttest/UnpairedTTestAggregator.java

@@ -15,6 +15,7 @@ import org.elasticsearch.core.Releasables;
 import org.elasticsearch.core.Tuple;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
 import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
@@ -69,19 +70,19 @@ public class UnpairedTTestAggregator extends TTestAggregator<UnpairedTTestState>
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSources == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        final SortedNumericDoubleValues docAValues = valuesSources.getField(A_FIELD.getPreferredName(), ctx);
-        final SortedNumericDoubleValues docBValues = valuesSources.getField(B_FIELD.getPreferredName(), ctx);
+        final SortedNumericDoubleValues docAValues = valuesSources.getField(A_FIELD.getPreferredName(), aggCtx.getLeafReaderContext());
+        final SortedNumericDoubleValues docBValues = valuesSources.getField(B_FIELD.getPreferredName(), aggCtx.getLeafReaderContext());
         final CompensatedSum compSumA = new CompensatedSum(0, 0);
         final CompensatedSum compSumOfSqrA = new CompensatedSum(0, 0);
         final CompensatedSum compSumB = new CompensatedSum(0, 0);
         final CompensatedSum compSumOfSqrB = new CompensatedSum(0, 0);
         final Tuple<Weight, Weight> weights = weightsSupplier.get();
-        final Bits bitsA = getBits(ctx, weights.v1());
-        final Bits bitsB = getBits(ctx, weights.v2());
+        final Bits bitsA = getBits(aggCtx.getLeafReaderContext(), weights.v1());
+        final Bits bitsB = getBits(aggCtx.getLeafReaderContext(), weights.v2());
 
         return new LeafBucketCollectorBase(sub, docAValues) {
 

+ 7 - 4
x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricBackedAvgAggregator.java

@@ -6,7 +6,6 @@
  */
 package org.elasticsearch.xpack.aggregatemetric.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.DoubleArray;
@@ -14,6 +13,7 @@ import org.elasticsearch.common.util.LongArray;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -65,14 +65,17 @@ class AggregateMetricBackedAvgAggregator extends NumericMetricsAggregator.Single
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
         final BigArrays bigArrays = bigArrays();
         // Retrieve aggregate values for metrics sum and value_count
-        final SortedNumericDoubleValues aggregateSums = valuesSource.getAggregateMetricValues(ctx, Metric.sum);
-        final SortedNumericDoubleValues aggregateValueCounts = valuesSource.getAggregateMetricValues(ctx, Metric.value_count);
+        final SortedNumericDoubleValues aggregateSums = valuesSource.getAggregateMetricValues(aggCtx.getLeafReaderContext(), Metric.sum);
+        final SortedNumericDoubleValues aggregateValueCounts = valuesSource.getAggregateMetricValues(
+            aggCtx.getLeafReaderContext(),
+            Metric.value_count
+        );
         final CompensatedSum kahanSummation = new CompensatedSum(0, 0);
         return new LeafBucketCollectorBase(sub, sums) {
             @Override

+ 3 - 3
x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricBackedMaxAggregator.java

@@ -6,7 +6,6 @@
  */
 package org.elasticsearch.xpack.aggregatemetric.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.DoubleArray;
@@ -15,6 +14,7 @@ import org.elasticsearch.index.fielddata.NumericDoubleValues;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
 import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.MultiValueMode;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -57,13 +57,13 @@ class AggregateMetricBackedMaxAggregator extends NumericMetricsAggregator.Single
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
 
         final BigArrays bigArrays = bigArrays();
-        final SortedNumericDoubleValues allValues = valuesSource.getAggregateMetricValues(ctx, Metric.max);
+        final SortedNumericDoubleValues allValues = valuesSource.getAggregateMetricValues(aggCtx.getLeafReaderContext(), Metric.max);
         final NumericDoubleValues values = MultiValueMode.MAX.select(allValues);
         return new LeafBucketCollectorBase(sub, allValues) {
 

+ 3 - 3
x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricBackedMinAggregator.java

@@ -6,7 +6,6 @@
  */
 package org.elasticsearch.xpack.aggregatemetric.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.DoubleArray;
@@ -15,6 +14,7 @@ import org.elasticsearch.index.fielddata.NumericDoubleValues;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
 import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.MultiValueMode;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -57,13 +57,13 @@ class AggregateMetricBackedMinAggregator extends NumericMetricsAggregator.Single
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
 
         final BigArrays bigArrays = bigArrays();
-        final SortedNumericDoubleValues allValues = valuesSource.getAggregateMetricValues(ctx, Metric.min);
+        final SortedNumericDoubleValues allValues = valuesSource.getAggregateMetricValues(aggCtx.getLeafReaderContext(), Metric.min);
         final NumericDoubleValues values = MultiValueMode.MIN.select(allValues);
         return new LeafBucketCollectorBase(sub, allValues) {
 

+ 3 - 3
x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricBackedSumAggregator.java

@@ -6,13 +6,13 @@
  */
 package org.elasticsearch.xpack.aggregatemetric.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.DoubleArray;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
 import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -61,12 +61,12 @@ class AggregateMetricBackedSumAggregator extends NumericMetricsAggregator.Single
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
         final BigArrays bigArrays = bigArrays();
-        final SortedNumericDoubleValues values = valuesSource.getAggregateMetricValues(ctx, Metric.sum);
+        final SortedNumericDoubleValues values = valuesSource.getAggregateMetricValues(aggCtx.getLeafReaderContext(), Metric.sum);
         final CompensatedSum kahanSummation = new CompensatedSum(0, 0);
         return new LeafBucketCollectorBase(sub, values) {
             @Override

+ 3 - 3
x-pack/plugin/mapper-aggregate-metric/src/main/java/org/elasticsearch/xpack/aggregatemetric/aggregations/metrics/AggregateMetricBackedValueCountAggregator.java

@@ -6,11 +6,11 @@
  */
 package org.elasticsearch.xpack.aggregatemetric.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.LongArray;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -55,13 +55,13 @@ class AggregateMetricBackedValueCountAggregator extends NumericMetricsAggregator
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
         final BigArrays bigArrays = bigArrays();
         final SortedNumericDoubleValues values = valuesSource.getAggregateMetricValues(
-            ctx,
+            aggCtx.getLeafReaderContext(),
             AggregateDoubleMetricFieldMapper.Metric.value_count
         );
 

+ 3 - 3
x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/aggs/categorization/CategorizeTextAggregator.java

@@ -9,12 +9,12 @@ package org.elasticsearch.xpack.ml.aggs.categorization;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.util.BytesRef;
 import org.elasticsearch.common.util.BytesRefHash;
 import org.elasticsearch.common.util.ObjectArray;
 import org.elasticsearch.core.Releasables;
 import org.elasticsearch.index.mapper.MappedFieldType;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.CardinalityUpperBound;
@@ -141,7 +141,7 @@ public class CategorizeTextAggregator extends DeferableBucketAggregator {
     }
 
     @Override
-    protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+    protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) {
         return new LeafBucketCollectorBase(sub, null) {
             @Override
             public void collect(int doc, long owningBucketOrd) throws IOException {
@@ -156,7 +156,7 @@ public class CategorizeTextAggregator extends DeferableBucketAggregator {
             }
 
             private void collectFromSource(int doc, long owningBucketOrd, TokenListCategorizer categorizer) throws IOException {
-                sourceLookup.setSegmentAndDocument(ctx, doc);
+                sourceLookup.setSegmentAndDocument(aggCtx.getLeafReaderContext(), doc);
                 Iterator<String> itr = sourceLookup.extractRawValuesWithoutCaching(sourceFieldName).stream().map(obj -> {
                     if (obj instanceof BytesRef) {
                         return fieldType.valueForDisplay(obj).toString();

+ 3 - 3
x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java

@@ -6,11 +6,11 @@
  */
 package org.elasticsearch.xpack.spatial.search.aggregations;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.LongArray;
 import org.elasticsearch.core.Releasables;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -73,11 +73,11 @@ final class GeoLineAggregator extends MetricsAggregator {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         if (valuesSources == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        BucketedSort.Leaf leafSort = sort.forLeaf(ctx);
+        BucketedSort.Leaf leafSort = sort.forLeaf(aggCtx.getLeafReaderContext());
 
         return new LeafBucketCollector() {
             @Override

+ 3 - 3
x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/metrics/GeoShapeBoundsAggregator.java

@@ -7,9 +7,9 @@
 
 package org.elasticsearch.xpack.spatial.search.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.elasticsearch.common.util.DoubleArray;
 import org.elasticsearch.core.Releasables;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -62,11 +62,11 @@ public final class GeoShapeBoundsAggregator extends MetricsAggregator {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        final GeoShapeValues values = valuesSource.geoShapeValues(ctx);
+        final GeoShapeValues values = valuesSource.geoShapeValues(aggCtx.getLeafReaderContext());
         return new LeafBucketCollectorBase(sub, values) {
             @Override
             public void collect(int doc, long bucket) throws IOException {

+ 3 - 3
x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/metrics/GeoShapeCentroidAggregator.java

@@ -7,12 +7,12 @@
 
 package org.elasticsearch.xpack.spatial.search.aggregations.metrics;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.elasticsearch.common.geo.GeoPoint;
 import org.elasticsearch.common.util.ByteArray;
 import org.elasticsearch.common.util.DoubleArray;
 import org.elasticsearch.common.util.LongArray;
 import org.elasticsearch.core.Releasables;
+import org.elasticsearch.search.aggregations.AggregationExecutionContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.LeafBucketCollector;
@@ -61,11 +61,11 @@ public final class GeoShapeCentroidAggregator extends MetricsAggregator {
     }
 
     @Override
-    public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) {
+    public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) {
         if (valuesSource == null) {
             return LeafBucketCollector.NO_OP_COLLECTOR;
         }
-        final GeoShapeValues values = valuesSource.geoShapeValues(ctx);
+        final GeoShapeValues values = valuesSource.geoShapeValues(aggCtx.getLeafReaderContext());
         final CompensatedSum compensatedSumLat = new CompensatedSum(0, 0);
         final CompensatedSum compensatedSumLon = new CompensatedSum(0, 0);
         final CompensatedSum compensatedSumWeight = new CompensatedSum(0, 0);