Browse Source

Remove QueryParseContext (#25486)

QueryParseContext is currently only used as a wrapper for an XContentParser, so
this change removes it entirely and changes the appropriate APIs that use it so
far to only accept a parser instead.
Christoph Büscher 8 years ago
parent
commit
f576c987ce
100 changed files with 404 additions and 576 deletions
  1. 4 3
      core/src/main/java/org/elasticsearch/common/ParsingException.java
  2. 0 1
      core/src/main/java/org/elasticsearch/index/query/AbstractQueryBuilder.java
  3. 1 1
      core/src/main/java/org/elasticsearch/index/query/IdsQueryBuilder.java
  4. 1 1
      core/src/main/java/org/elasticsearch/index/query/InnerHitBuilder.java
  5. 1 1
      core/src/main/java/org/elasticsearch/index/query/MatchAllQueryBuilder.java
  6. 1 1
      core/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java
  7. 0 37
      core/src/main/java/org/elasticsearch/index/query/QueryParseContext.java
  8. 0 7
      core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java
  9. 1 1
      core/src/main/java/org/elasticsearch/index/query/TermsQueryBuilder.java
  10. 1 3
      core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java
  11. 1 3
      core/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java
  12. 3 3
      core/src/main/java/org/elasticsearch/search/SearchModule.java
  13. 5 4
      core/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilder.java
  14. 3 3
      core/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java
  15. 8 17
      core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java
  16. 1 3
      core/src/main/java/org/elasticsearch/search/aggregations/InternalOrder.java
  17. 22 26
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregationBuilder.java
  18. 2 4
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregator.java
  19. 3 3
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregationBuilder.java
  20. 3 5
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersAggregationBuilder.java
  21. 4 4
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregationBuilder.java
  22. 3 3
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregationBuilder.java
  23. 4 5
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java
  24. 5 5
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregationBuilder.java
  25. 4 4
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregationBuilder.java
  26. 1 3
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregationBuilder.java
  27. 1 3
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregationBuilder.java
  28. 5 6
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregationBuilder.java
  29. 5 6
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/date/DateRangeAggregationBuilder.java
  30. 6 7
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/GeoDistanceAggregationBuilder.java
  31. 5 6
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/ip/IpRangeAggregationBuilder.java
  32. 4 4
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedAggregationBuilder.java
  33. 1 3
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregationBuilder.java
  34. 17 15
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregationBuilder.java
  35. 17 17
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTextAggregationBuilder.java
  36. 1 3
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/heuristics/GND.java
  37. 1 3
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/heuristics/JLHScore.java
  38. 1 3
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/heuristics/NXYSignificanceHeuristic.java
  39. 1 3
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/heuristics/PercentageScore.java
  40. 1 3
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/heuristics/ScriptHeuristic.java
  41. 1 2
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/heuristics/SignificanceHeuristicParser.java
  42. 7 7
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregationBuilder.java
  43. 3 3
      core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/support/IncludeExclude.java
  44. 4 4
      core/src/main/java/org/elasticsearch/search/aggregations/metrics/avg/AvgAggregationBuilder.java
  45. 4 4
      core/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregationBuilder.java
  46. 4 4
      core/src/main/java/org/elasticsearch/search/aggregations/metrics/geobounds/GeoBoundsAggregationBuilder.java
  47. 4 4
      core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroidAggregationBuilder.java
  48. 4 4
      core/src/main/java/org/elasticsearch/search/aggregations/metrics/max/MaxAggregationBuilder.java
  49. 4 4
      core/src/main/java/org/elasticsearch/search/aggregations/metrics/min/MinAggregationBuilder.java
  50. 6 6
      core/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentileRanksAggregationBuilder.java
  51. 6 6
      core/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentilesAggregationBuilder.java
  52. 1 5
      core/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregationBuilder.java
  53. 4 4
      core/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/StatsAggregationBuilder.java
  54. 4 4
      core/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsAggregationBuilder.java
  55. 4 4
      core/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/SumAggregationBuilder.java
  56. 6 8
      core/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsAggregationBuilder.java
  57. 4 4
      core/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountAggregationBuilder.java
  58. 4 5
      core/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpers.java
  59. 4 5
      core/src/main/java/org/elasticsearch/search/aggregations/pipeline/PipelineAggregator.java
  60. 8 10
      core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/BucketMetricsParser.java
  61. 2 3
      core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/percentile/PercentilesBucketPipelineAggregationBuilder.java
  62. 1 2
      core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/stats/extended/ExtendedStatsBucketParser.java
  63. 3 5
      core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketscript/BucketScriptPipelineAggregationBuilder.java
  64. 3 5
      core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketselector/BucketSelectorPipelineAggregationBuilder.java
  65. 1 3
      core/src/main/java/org/elasticsearch/search/aggregations/pipeline/cumulativesum/CumulativeSumPipelineAggregationBuilder.java
  66. 3 5
      core/src/main/java/org/elasticsearch/search/aggregations/pipeline/derivative/DerivativePipelineAggregationBuilder.java
  67. 2 4
      core/src/main/java/org/elasticsearch/search/aggregations/pipeline/movavg/MovAvgPipelineAggregationBuilder.java
  68. 2 4
      core/src/main/java/org/elasticsearch/search/aggregations/pipeline/serialdiff/SerialDiffPipelineAggregationBuilder.java
  69. 5 6
      core/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceParserHelper.java
  70. 17 20
      core/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java
  71. 3 4
      core/src/main/java/org/elasticsearch/search/collapse/CollapseBuilder.java
  72. 1 3
      core/src/main/java/org/elasticsearch/search/fetch/StoredFieldsContext.java
  73. 3 4
      core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/AbstractHighlighterBuilder.java
  74. 4 5
      core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilder.java
  75. 4 4
      core/src/main/java/org/elasticsearch/search/rescore/QueryRescorerBuilder.java
  76. 2 4
      core/src/main/java/org/elasticsearch/search/rescore/RescoreBuilder.java
  77. 4 5
      core/src/main/java/org/elasticsearch/search/slice/SliceBuilder.java
  78. 2 3
      core/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java
  79. 1 2
      core/src/main/java/org/elasticsearch/search/sort/GeoDistanceSortBuilder.java
  80. 3 4
      core/src/main/java/org/elasticsearch/search/sort/ScoreSortBuilder.java
  81. 3 4
      core/src/main/java/org/elasticsearch/search/sort/ScriptSortBuilder.java
  82. 1 1
      core/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestionBuilder.java
  83. 2 3
      core/src/main/java/org/elasticsearch/search/suggest/completion/context/CategoryContextMapping.java
  84. 2 4
      core/src/main/java/org/elasticsearch/search/suggest/completion/context/CategoryQueryContext.java
  85. 5 7
      core/src/main/java/org/elasticsearch/search/suggest/completion/context/ContextMapping.java
  86. 3 4
      core/src/main/java/org/elasticsearch/search/suggest/completion/context/GeoContextMapping.java
  87. 1 3
      core/src/main/java/org/elasticsearch/search/suggest/completion/context/GeoQueryContext.java
  88. 3 20
      core/src/test/java/org/elasticsearch/index/query/AbstractQueryBuilderTests.java
  89. 9 9
      core/src/test/java/org/elasticsearch/index/query/RangeQueryBuilderTests.java
  90. 9 3
      core/src/test/java/org/elasticsearch/search/SearchModuleTests.java
  91. 2 1
      core/src/test/java/org/elasticsearch/search/SearchServiceTests.java
  92. 1 3
      core/src/test/java/org/elasticsearch/search/aggregations/AggregationCollectorTests.java
  93. 6 14
      core/src/test/java/org/elasticsearch/search/aggregations/AggregatorFactoriesTests.java
  94. 1 3
      core/src/test/java/org/elasticsearch/search/aggregations/BasePipelineAggregationTestCase.java
  95. 2 2
      core/src/test/java/org/elasticsearch/search/aggregations/InternalOrderTests.java
  96. 40 40
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/SignificantTermsSignificanceScoreIT.java
  97. 7 13
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridParserTests.java
  98. 3 6
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/significant/SignificanceHeuristicTests.java
  99. 3 7
      core/src/test/java/org/elasticsearch/search/aggregations/metrics/FiltersTests.java
  100. 1 4
      core/src/test/java/org/elasticsearch/search/aggregations/metrics/TopHitsTests.java

+ 4 - 3
core/src/main/java/org/elasticsearch/common/ParsingException.java

@@ -24,13 +24,14 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentLocation;
-import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.rest.RestStatus;
 
 import java.io.IOException;
 
 /**
- * Exception that can be used when parsing queries with a given {@link QueryParseContext}.
+ * Exception that can be used when parsing queries with a given {@link
+ * XContentParser}.
  * Can contain information about location of the error.
  */
 public class ParsingException extends ElasticsearchException {
@@ -57,7 +58,7 @@ public class ParsingException extends ElasticsearchException {
 
     /**
      * This constructor is provided for use in unit tests where a
-     * {@link QueryParseContext} may not be available
+     * {@link XContentParser} may not be available
      */
     public ParsingException(int line, int col, String msg, Throwable cause) {
         super(msg, cause);

+ 0 - 1
core/src/main/java/org/elasticsearch/index/query/AbstractQueryBuilder.java

@@ -35,7 +35,6 @@ import org.elasticsearch.common.xcontent.NamedXContentRegistry.UnknownNamedObjec
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentLocation;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.common.xcontent.XContentParser.Token;
 import org.elasticsearch.common.xcontent.XContentType;
 
 import java.io.IOException;

+ 1 - 1
core/src/main/java/org/elasticsearch/index/query/IdsQueryBuilder.java

@@ -138,7 +138,7 @@ public class IdsQueryBuilder extends AbstractQueryBuilder<IdsQueryBuilder> {
         builder.endObject();
     }
 
-    private static ObjectParser<IdsQueryBuilder, QueryParseContext> PARSER = new ObjectParser<>(NAME,
+    private static ObjectParser<IdsQueryBuilder, Void> PARSER = new ObjectParser<>(NAME,
             () -> new IdsQueryBuilder());
 
     static {

+ 1 - 1
core/src/main/java/org/elasticsearch/index/query/InnerHitBuilder.java

@@ -53,7 +53,7 @@ public final class InnerHitBuilder extends ToXContentToBytes implements Writeabl
     public static final ParseField IGNORE_UNMAPPED = new ParseField("ignore_unmapped");
     public static final QueryBuilder DEFAULT_INNER_HIT_QUERY = new MatchAllQueryBuilder();
 
-    private static final ObjectParser<InnerHitBuilder, QueryParseContext> PARSER = new ObjectParser<>("inner_hits", InnerHitBuilder::new);
+    private static final ObjectParser<InnerHitBuilder, Void> PARSER = new ObjectParser<>("inner_hits", InnerHitBuilder::new);
 
     static {
         PARSER.declareString(InnerHitBuilder::setName, NAME_FIELD);

+ 1 - 1
core/src/main/java/org/elasticsearch/index/query/MatchAllQueryBuilder.java

@@ -58,7 +58,7 @@ public class MatchAllQueryBuilder extends AbstractQueryBuilder<MatchAllQueryBuil
         builder.endObject();
     }
 
-    private static final ObjectParser<MatchAllQueryBuilder, QueryParseContext> PARSER = new ObjectParser<>(NAME, MatchAllQueryBuilder::new);
+    private static final ObjectParser<MatchAllQueryBuilder, Void> PARSER = new ObjectParser<>(NAME, MatchAllQueryBuilder::new);
 
     static {
         declareStandardFields(PARSER);

+ 1 - 1
core/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java

@@ -1202,7 +1202,7 @@ public class MoreLikeThisQueryBuilder extends AbstractQueryBuilder<MoreLikeThisQ
     }
 
     @Override
-    protected QueryBuilder doRewrite(QueryRewriteContext queryRewriteContext) throws IOException {
+    protected QueryBuilder doRewrite(QueryRewriteContext queryRewriteContext) {
         // TODO this needs heavy cleanups before we can rewrite it
         return this;
     }

+ 0 - 37
core/src/main/java/org/elasticsearch/index/query/QueryParseContext.java

@@ -1,37 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch licenses this file to you under
- * the Apache License, Version 2.0 (the "License"); you may
- * not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.elasticsearch.index.query;
-
-import org.elasticsearch.common.xcontent.XContentParser;
-
-import java.util.Objects;
-
-public class QueryParseContext {
-
-    private final XContentParser parser;
-
-    public QueryParseContext(XContentParser parser) {
-        this.parser = Objects.requireNonNull(parser, "parser cannot be null");
-    }
-
-    public XContentParser parser() {
-        return this.parser;
-    }
-}

+ 0 - 7
core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java

@@ -95,13 +95,6 @@ public class QueryRewriteContext {
         return xContentRegistry;
     }
 
-    /**
-     * Returns a new {@link QueryParseContext} that wraps the provided parser.
-     */
-    public QueryParseContext newParseContext(XContentParser parser) {
-        return new QueryParseContext(parser);
-    }
-
     public long nowInMillis() {
         return nowInMillis.getAsLong();
     }

+ 1 - 1
core/src/main/java/org/elasticsearch/index/query/TermsQueryBuilder.java

@@ -437,7 +437,7 @@ public class TermsQueryBuilder extends AbstractQueryBuilder<TermsQueryBuilder> {
     }
 
     @Override
-    protected QueryBuilder doRewrite(QueryRewriteContext queryRewriteContext) throws IOException {
+    protected QueryBuilder doRewrite(QueryRewriteContext queryRewriteContext) {
         if (this.termsLookup != null) {
             TermsLookup termsLookup = new TermsLookup(this.termsLookup);
             if (termsLookup.index() == null) { // TODO this should go away?

+ 1 - 3
core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java

@@ -31,7 +31,6 @@ import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.xcontent.XContent;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.XContentType;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.rest.BaseRestHandler;
 import org.elasticsearch.rest.RestController;
 import org.elasticsearch.rest.RestRequest;
@@ -91,8 +90,7 @@ public class RestMultiSearchAction extends BaseRestHandler {
 
         parseMultiLineRequest(restRequest, multiRequest.indicesOptions(), allowExplicitIndex, (searchRequest, parser) -> {
             try {
-                final QueryParseContext queryParseContext = new QueryParseContext(parser);
-                searchRequest.source(SearchSourceBuilder.fromXContent(queryParseContext));
+                searchRequest.source(SearchSourceBuilder.fromXContent(parser));
                 multiRequest.add(searchRequest);
             } catch (IOException e) {
                 throw new ElasticsearchParseException("Exception when parsing search request", e);

+ 1 - 3
core/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java

@@ -26,7 +26,6 @@ import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.rest.BaseRestHandler;
 import org.elasticsearch.rest.RestController;
 import org.elasticsearch.rest.RestRequest;
@@ -94,8 +93,7 @@ public class RestSearchAction extends BaseRestHandler {
         }
         searchRequest.indices(Strings.splitStringByCommaToArray(request.param("index")));
         if (requestContentParser != null) {
-            QueryParseContext context = new QueryParseContext(requestContentParser);
-            searchRequest.source().parseXContent(context);
+            searchRequest.source().parseXContent(requestContentParser);
         }
 
         final int batchedReduceSize = request.paramAsInt("batched_reduce_size", searchRequest.getBatchedReduceSize());

+ 3 - 3
core/src/main/java/org/elasticsearch/search/SearchModule.java

@@ -356,7 +356,7 @@ public class SearchModule {
         registerAggregation(new AggregationSpec(FiltersAggregationBuilder.NAME, FiltersAggregationBuilder::new,
                 FiltersAggregationBuilder::parse).addResultReader(InternalFilters::new));
         registerAggregation(new AggregationSpec(AdjacencyMatrixAggregationBuilder.NAME, AdjacencyMatrixAggregationBuilder::new,
-                AdjacencyMatrixAggregationBuilder.getParser()).addResultReader(InternalAdjacencyMatrix::new));
+                AdjacencyMatrixAggregationBuilder::parse).addResultReader(InternalAdjacencyMatrix::new));
         registerAggregation(new AggregationSpec(SamplerAggregationBuilder.NAME, SamplerAggregationBuilder::new,
                 SamplerAggregationBuilder::parse)
                     .addResultReader(InternalSampler.NAME, InternalSampler::new)
@@ -410,7 +410,7 @@ public class SearchModule {
         if (false == transportClient) {
             namedXContents.add(new NamedXContentRegistry.Entry(BaseAggregationBuilder.class, spec.getName(), (p, c) -> {
                 AggregatorFactories.AggParseContext context = (AggregatorFactories.AggParseContext) c;
-                return spec.getParser().parse(context.name, context.queryParseContext);
+                return spec.getParser().parse(context.name, p);
             }));
         }
         namedWriteables.add(
@@ -507,7 +507,7 @@ public class SearchModule {
         if (false == transportClient) {
             namedXContents.add(new NamedXContentRegistry.Entry(BaseAggregationBuilder.class, spec.getName(), (p, c) -> {
                 AggregatorFactories.AggParseContext context = (AggregatorFactories.AggParseContext) c;
-                return spec.getParser().parse(context.name, context.queryParseContext);
+                return spec.getParser().parse(context.name, p);
             }));
         }
         namedWriteables.add(

+ 5 - 4
core/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilder.java

@@ -23,7 +23,7 @@ import org.elasticsearch.action.support.ToXContentToBytes;
 import org.elasticsearch.common.ParseField;
 import org.elasticsearch.common.io.stream.NamedWriteable;
 import org.elasticsearch.common.xcontent.ToXContent;
-import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.search.internal.SearchContext;
 
 import java.io.IOException;
@@ -84,9 +84,10 @@ public abstract class AggregationBuilder
     }
 
     /**
-     * Internal: Registers sub-factories with this factory. The sub-factory will be
-     * responsible for the creation of sub-aggregators under the aggregator
-     * created by this factory. This is only for use by {@link AggregatorFactories#parseAggregators(QueryParseContext)}.
+     * Internal: Registers sub-factories with this factory. The sub-factory will
+     * be responsible for the creation of sub-aggregators under the aggregator
+     * created by this factory. This is only for use by
+     * {@link AggregatorFactories#parseAggregators(XContentParser)}.
      *
      * @param subFactories
      *            The sub-factories

+ 3 - 3
core/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java

@@ -25,7 +25,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
 import org.elasticsearch.common.lease.Releasable;
-import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
 import org.elasticsearch.search.internal.SearchContext;
 
@@ -50,11 +50,11 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
          * aggregation should be skipped (e.g. when trying to aggregate on unmapped fields).
          *
          * @param aggregationName   The name of the aggregation
-         * @param context           The parse context
+         * @param parser            The parser
          * @return                  The resolved aggregator factory or {@code null} in case the aggregation should be skipped
          * @throws java.io.IOException      When parsing fails
          */
-        AggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException;
+        AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException;
     }
 
     /**

+ 8 - 17
core/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java

@@ -25,7 +25,6 @@ import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
 import org.elasticsearch.search.aggregations.support.AggregationPath;
 import org.elasticsearch.search.aggregations.support.AggregationPath.PathElement;
@@ -50,24 +49,18 @@ public class AggregatorFactories {
     public static final Pattern VALID_AGG_NAME = Pattern.compile("[^\\[\\]>]+");
 
     /**
-     * Parses the aggregation request recursively generating aggregator factories in turn.
-     *
-     * @param parseContext   The parse context.
-     *
-     * @return          The parsed aggregator factories.
-     *
-     * @throws IOException When parsing fails for unknown reasons.
+     * Parses the aggregation request recursively generating aggregator
+     * factories in turn.
      */
-    public static AggregatorFactories.Builder parseAggregators(QueryParseContext parseContext) throws IOException {
-        return parseAggregators(parseContext, 0);
+    public static AggregatorFactories.Builder parseAggregators(XContentParser parser) throws IOException {
+        return parseAggregators(parser, 0);
     }
 
-    private static AggregatorFactories.Builder parseAggregators(QueryParseContext parseContext, int level) throws IOException {
+    private static AggregatorFactories.Builder parseAggregators(XContentParser parser, int level) throws IOException {
         Matcher validAggMatcher = VALID_AGG_NAME.matcher("");
         AggregatorFactories.Builder factories = new AggregatorFactories.Builder();
 
         XContentParser.Token token = null;
-        XContentParser parser = parseContext.parser();
         while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
             if (token != XContentParser.Token.FIELD_NAME) {
                 throw new ParsingException(parser.getTokenLocation(),
@@ -111,7 +104,7 @@ public class AggregatorFactories {
                             throw new ParsingException(parser.getTokenLocation(),
                                     "Found two sub aggregation definitions under [" + aggregationName + "]");
                         }
-                        subFactories = parseAggregators(parseContext, level + 1);
+                        subFactories = parseAggregators(parser, level + 1);
                         break;
                     default:
                         if (aggBuilder != null) {
@@ -120,7 +113,7 @@ public class AggregatorFactories {
                         }
 
                         aggBuilder = parser.namedObject(BaseAggregationBuilder.class, fieldName,
-                                new AggParseContext(aggregationName, parseContext));
+                                new AggParseContext(aggregationName));
                     }
                 } else {
                     throw new ParsingException(parser.getTokenLocation(), "Expected [" + XContentParser.Token.START_OBJECT + "] under ["
@@ -156,11 +149,9 @@ public class AggregatorFactories {
      */
     public static final class AggParseContext {
         public final String name;
-        public final QueryParseContext queryParseContext;
 
-        public AggParseContext(String name, QueryParseContext queryParseContext) {
+        public AggParseContext(String name) {
             this.name = name;
-            this.queryParseContext = queryParseContext;
         }
     }
 

+ 1 - 3
core/src/main/java/org/elasticsearch/search/aggregations/InternalOrder.java

@@ -28,7 +28,6 @@ import org.elasticsearch.common.util.Comparators;
 import org.elasticsearch.common.xcontent.XContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket;
 import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator;
 import org.elasticsearch.search.aggregations.support.AggregationPath;
@@ -527,11 +526,10 @@ public class InternalOrder extends BucketOrder {
          * Parse a {@link BucketOrder} from {@link XContent}.
          *
          * @param parser  for parsing {@link XContent} that contains the order.
-         * @param context parsing context.
          * @return bucket ordering strategy
          * @throws IOException on error a {@link XContent} parsing error.
          */
-        public static BucketOrder parseOrderParam(XContentParser parser, QueryParseContext context) throws IOException {
+        public static BucketOrder parseOrderParam(XContentParser parser) throws IOException {
             XContentParser.Token token;
             String orderKey = null;
             boolean orderAsc = false;

+ 22 - 26
core/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregationBuilder.java

@@ -24,12 +24,11 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.IndexSettings;
 import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
-import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
 import org.elasticsearch.search.aggregations.bucket.adjacency.AdjacencyMatrixAggregator.KeyedFilter;
@@ -56,26 +55,23 @@ public class AdjacencyMatrixAggregationBuilder extends AbstractAggregationBuilde
     private List<KeyedFilter> filters;
     private String separator = DEFAULT_SEPARATOR;
 
-    public static Aggregator.Parser getParser() {
-        ObjectParser<AdjacencyMatrixAggregationBuilder, QueryParseContext> parser = new ObjectParser<>(
-                AdjacencyMatrixAggregationBuilder.NAME);
-        parser.declareString(AdjacencyMatrixAggregationBuilder::separator, SEPARATOR_FIELD);
-        parser.declareNamedObjects(AdjacencyMatrixAggregationBuilder::setFiltersAsList, KeyedFilter.PARSER, FILTERS_FIELD);
-        return new Aggregator.Parser() {
-            @Override
-            public AggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-                AdjacencyMatrixAggregationBuilder result = parser.parse(context.parser(),
-                        new AdjacencyMatrixAggregationBuilder(aggregationName), context);
-                result.checkConsistency();
-                return result;
-            }
-        };
+    private static final ObjectParser<AdjacencyMatrixAggregationBuilder, Void> PARSER = new ObjectParser<>(
+            AdjacencyMatrixAggregationBuilder.NAME);
+    static {
+        PARSER.declareString(AdjacencyMatrixAggregationBuilder::separator, SEPARATOR_FIELD);
+        PARSER.declareNamedObjects(AdjacencyMatrixAggregationBuilder::setFiltersAsList, KeyedFilter.PARSER, FILTERS_FIELD);
+    }
+
+    public static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        AdjacencyMatrixAggregationBuilder result = PARSER.parse(parser, new AdjacencyMatrixAggregationBuilder(aggregationName), null);
+        result.checkConsistency();
+        return result;
     }
 
     protected void checkConsistency() {
         if ((filters == null) || (filters.size() == 0)) {
             throw new IllegalStateException("[" + name  + "] is missing : " + FILTERS_FIELD.getPreferredName() + " parameter");
-        }        
+        }
     }
 
 
@@ -96,17 +92,17 @@ public class AdjacencyMatrixAggregationBuilder extends AbstractAggregationBuilde
         // the order of the filters in the request
         Collections.sort(this.filters, Comparator.comparing(KeyedFilter::key));
     }
-    
-   
+
+
     /**
      * @param name
      *            the name of this aggregation
      */
     protected AdjacencyMatrixAggregationBuilder(String name) {
         super(name);
-    }    
-    
-    
+    }
+
+
     /**
      * @param name
      *            the name of this aggregation
@@ -171,8 +167,8 @@ public class AdjacencyMatrixAggregationBuilder extends AbstractAggregationBuilde
      */
     public String separator() {
         return separator;
-    }        
-    
+    }
+
     /**
      * Get the filters. This will be an unmodifiable map
      */
@@ -182,8 +178,8 @@ public class AdjacencyMatrixAggregationBuilder extends AbstractAggregationBuilde
             result.put(keyedFilter.key(), keyedFilter.filter());
         }
         return result;
-    }    
-    
+    }
+
 
     @Override
     protected AggregatorFactory<?> doBuild(SearchContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)

+ 2 - 4
core/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregator.java

@@ -32,7 +32,6 @@ import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.InternalAggregation;
@@ -68,11 +67,10 @@ public class AdjacencyMatrixAggregator extends BucketsAggregator {
         private final String key;
         private final QueryBuilder filter;
 
-        public static final NamedObjectParser<KeyedFilter, QueryParseContext> PARSER =
-                (XContentParser p, QueryParseContext c, String name) ->
+        public static final NamedObjectParser<KeyedFilter, Void> PARSER =
+                (XContentParser p, Void c, String name) ->
                      new KeyedFilter(name, parseInnerQueryBuilder(p));
 
-
         public KeyedFilter(String key, QueryBuilder filter) {
             if (key == null) {
                 throw new IllegalArgumentException("[key] must not be null");

+ 3 - 3
core/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregationBuilder.java

@@ -22,8 +22,8 @@ package org.elasticsearch.search.aggregations.bucket.filter;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -84,8 +84,8 @@ public class FilterAggregationBuilder extends AbstractAggregationBuilder<FilterA
         return builder;
     }
 
-    public static FilterAggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        QueryBuilder filter = parseInnerQueryBuilder(context.parser());
+    public static FilterAggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        QueryBuilder filter = parseInnerQueryBuilder(parser);
         return new FilterAggregationBuilder(aggregationName, filter);
     }
 

+ 3 - 5
core/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersAggregationBuilder.java

@@ -26,7 +26,6 @@ import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -202,9 +201,8 @@ public class FiltersAggregationBuilder extends AbstractAggregationBuilder<Filter
         return builder;
     }
 
-    public static FiltersAggregationBuilder parse(String aggregationName, QueryParseContext context)
+    public static FiltersAggregationBuilder parse(String aggregationName, XContentParser parser)
             throws IOException {
-        XContentParser parser = context.parser();
 
         List<FiltersAggregator.KeyedFilter> keyedFilters = null;
         List<QueryBuilder> nonKeyedFilters = null;
@@ -238,7 +236,7 @@ public class FiltersAggregationBuilder extends AbstractAggregationBuilder<Filter
                         if (token == XContentParser.Token.FIELD_NAME) {
                             key = parser.currentName();
                         } else {
-                            QueryBuilder filter = parseInnerQueryBuilder(context.parser());
+                            QueryBuilder filter = parseInnerQueryBuilder(parser);
                             keyedFilters.add(new FiltersAggregator.KeyedFilter(key, filter));
                         }
                     }
@@ -250,7 +248,7 @@ public class FiltersAggregationBuilder extends AbstractAggregationBuilder<Filter
                 if (FILTERS_FIELD.match(currentFieldName)) {
                     nonKeyedFilters = new ArrayList<>();
                     while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
-                        QueryBuilder filter = parseInnerQueryBuilder(context.parser());
+                        QueryBuilder filter = parseInnerQueryBuilder(parser);
                         nonKeyedFilters.add(filter);
                     }
                 } else {

+ 4 - 4
core/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregationBuilder.java

@@ -28,11 +28,11 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.fielddata.AbstractSortingNumericDocValues;
 import org.elasticsearch.index.fielddata.MultiGeoPointValues;
 import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
 import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
 import org.elasticsearch.search.aggregations.bucket.BucketUtils;
@@ -53,7 +53,7 @@ public class GeoGridAggregationBuilder extends ValuesSourceAggregationBuilder<Va
     public static final int DEFAULT_PRECISION = 5;
     public static final int DEFAULT_MAX_NUM_CELLS = 10000;
 
-    private static final ObjectParser<GeoGridAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<GeoGridAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(GeoGridAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareGeoFields(PARSER, false, false);
@@ -62,8 +62,8 @@ public class GeoGridAggregationBuilder extends ValuesSourceAggregationBuilder<Va
         PARSER.declareInt(GeoGridAggregationBuilder::shardSize, GeoHashGridParams.FIELD_SHARD_SIZE);
     }
 
-    public static GeoGridAggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        return PARSER.parse(context.parser(), new GeoGridAggregationBuilder(aggregationName), context);
+    public static GeoGridAggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        return PARSER.parse(parser, new GeoGridAggregationBuilder(aggregationName), null);
     }
 
     private int precision = DEFAULT_PRECISION;

+ 3 - 3
core/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregationBuilder.java

@@ -22,7 +22,7 @@ package org.elasticsearch.search.aggregations.bucket.global;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -62,8 +62,8 @@ public class GlobalAggregationBuilder extends AbstractAggregationBuilder<GlobalA
         return builder;
     }
 
-    public static GlobalAggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        context.parser().nextToken();
+    public static GlobalAggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        parser.nextToken();
         return new GlobalAggregationBuilder(aggregationName);
     }
 

+ 4 - 5
core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java

@@ -27,7 +27,6 @@ import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
 import org.elasticsearch.search.aggregations.BucketOrder;
@@ -81,7 +80,7 @@ public class DateHistogramAggregationBuilder
         DATE_FIELD_UNITS = unmodifiableMap(dateFieldUnits);
     }
 
-    private static final ObjectParser<DateHistogramAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<DateHistogramAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(DateHistogramAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareNumericFields(PARSER, true, true, true);
@@ -115,12 +114,12 @@ public class DateHistogramAggregationBuilder
         PARSER.declareField(DateHistogramAggregationBuilder::extendedBounds, parser -> ExtendedBounds.PARSER.apply(parser, null),
                 ExtendedBounds.EXTENDED_BOUNDS_FIELD, ObjectParser.ValueType.OBJECT);
 
-        PARSER.declareObjectArray(DateHistogramAggregationBuilder::order, InternalOrder.Parser::parseOrderParam,
+        PARSER.declareObjectArray(DateHistogramAggregationBuilder::order, (p, c) -> InternalOrder.Parser.parseOrderParam(p),
                 Histogram.ORDER_FIELD);
     }
 
-    public static DateHistogramAggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        return PARSER.parse(context.parser(), new DateHistogramAggregationBuilder(aggregationName), context);
+    public static DateHistogramAggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        return PARSER.parse(parser, new DateHistogramAggregationBuilder(aggregationName), null);
     }
 
     private long interval;

+ 5 - 5
core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregationBuilder.java

@@ -24,7 +24,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
 import org.elasticsearch.search.aggregations.BucketOrder;
@@ -59,7 +59,7 @@ public class HistogramAggregationBuilder
         EXTENDED_BOUNDS_PARSER.declareDouble((bounds, d) -> bounds[1] = d, new ParseField("max"));
     }
 
-    private static final ObjectParser<HistogramAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<HistogramAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(HistogramAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareNumericFields(PARSER, true, true, false);
@@ -76,12 +76,12 @@ public class HistogramAggregationBuilder
             histogram.extendedBounds(extendedBounds[0], extendedBounds[1]);
         }, parser -> EXTENDED_BOUNDS_PARSER.apply(parser, null), ExtendedBounds.EXTENDED_BOUNDS_FIELD, ObjectParser.ValueType.OBJECT);
 
-        PARSER.declareObjectArray(HistogramAggregationBuilder::order, InternalOrder.Parser::parseOrderParam,
+        PARSER.declareObjectArray(HistogramAggregationBuilder::order, (p, c) -> InternalOrder.Parser.parseOrderParam(p),
             Histogram.ORDER_FIELD);
     }
 
-    public static HistogramAggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        return PARSER.parse(context.parser(), new HistogramAggregationBuilder(aggregationName), context);
+    public static HistogramAggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        return PARSER.parse(parser, new HistogramAggregationBuilder(aggregationName), null);
     }
 
     private double interval;

+ 4 - 4
core/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregationBuilder.java

@@ -23,7 +23,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
 import org.elasticsearch.search.aggregations.support.ValueType;
@@ -40,14 +40,14 @@ import java.io.IOException;
 public class MissingAggregationBuilder extends ValuesSourceAggregationBuilder<ValuesSource, MissingAggregationBuilder> {
     public static final String NAME = "missing";
 
-    private static final ObjectParser<MissingAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<MissingAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(MissingAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareAnyFields(PARSER, true, true);
     }
 
-    public static MissingAggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        return PARSER.parse(context.parser(), new MissingAggregationBuilder(aggregationName, null), context);
+    public static MissingAggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        return PARSER.parse(parser, new MissingAggregationBuilder(aggregationName, null), null);
     }
 
     public MissingAggregationBuilder(String name, ValueType targetValueType) {

+ 1 - 3
core/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregationBuilder.java

@@ -25,7 +25,6 @@ import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.mapper.ObjectMapper;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregationExecutionException;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
@@ -104,12 +103,11 @@ public class NestedAggregationBuilder extends AbstractAggregationBuilder<NestedA
         return builder;
     }
 
-    public static NestedAggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
+    public static NestedAggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
         String path = null;
 
         XContentParser.Token token;
         String currentFieldName = null;
-        XContentParser parser = context.parser();
         while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
             if (token == XContentParser.Token.FIELD_NAME) {
                 currentFieldName = parser.currentName();

+ 1 - 3
core/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregationBuilder.java

@@ -25,7 +25,6 @@ import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.mapper.ObjectMapper;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.index.query.support.NestedScope;
 import org.elasticsearch.search.SearchParseException;
 import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
@@ -128,12 +127,11 @@ public class ReverseNestedAggregationBuilder extends AbstractAggregationBuilder<
         return builder;
     }
 
-    public static ReverseNestedAggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
+    public static ReverseNestedAggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
         String path = null;
 
         XContentParser.Token token;
         String currentFieldName = null;
-        XContentParser parser = context.parser();
         while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
             if (token == XContentParser.Token.FIELD_NAME) {
                 currentFieldName = parser.currentName();

+ 5 - 6
core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregationBuilder.java

@@ -22,7 +22,6 @@ package org.elasticsearch.search.aggregations.bucket.range;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -37,7 +36,7 @@ import java.io.IOException;
 public class RangeAggregationBuilder extends AbstractRangeBuilder<RangeAggregationBuilder, Range> {
     public static final String NAME = "range";
 
-    private static final ObjectParser<RangeAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<RangeAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(RangeAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareNumericFields(PARSER, true, true, false);
@@ -47,14 +46,14 @@ public class RangeAggregationBuilder extends AbstractRangeBuilder<RangeAggregati
             for (Range range : ranges) {
                 agg.addRange(range);
             }
-        }, RangeAggregationBuilder::parseRange, RangeAggregator.RANGES_FIELD);
+        }, (p, c) -> RangeAggregationBuilder.parseRange(p), RangeAggregator.RANGES_FIELD);
     }
 
-    public static AggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        return PARSER.parse(context.parser(), new RangeAggregationBuilder(aggregationName), context);
+    public static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        return PARSER.parse(parser, new RangeAggregationBuilder(aggregationName), null);
     }
 
-    private static Range parseRange(XContentParser parser, QueryParseContext context) throws IOException {
+    private static Range parseRange(XContentParser parser) throws IOException {
         return Range.fromXContent(parser);
     }
 

+ 5 - 6
core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/date/DateRangeAggregationBuilder.java

@@ -22,7 +22,6 @@ package org.elasticsearch.search.aggregations.bucket.range.date;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -40,7 +39,7 @@ import java.io.IOException;
 public class DateRangeAggregationBuilder extends AbstractRangeBuilder<DateRangeAggregationBuilder, RangeAggregator.Range> {
     public static final String NAME = "date_range";
 
-    private static final ObjectParser<DateRangeAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<DateRangeAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(DateRangeAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareNumericFields(PARSER, true, true, true);
@@ -50,14 +49,14 @@ public class DateRangeAggregationBuilder extends AbstractRangeBuilder<DateRangeA
             for (Range range : ranges) {
                 agg.addRange(range);
             }
-        }, DateRangeAggregationBuilder::parseRange, RangeAggregator.RANGES_FIELD);
+        }, (p, c) -> DateRangeAggregationBuilder.parseRange(p), RangeAggregator.RANGES_FIELD);
     }
 
-    public static AggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        return PARSER.parse(context.parser(), new DateRangeAggregationBuilder(aggregationName), context);
+    public static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        return PARSER.parse(parser, new DateRangeAggregationBuilder(aggregationName), null);
     }
 
-    private static Range parseRange(XContentParser parser, QueryParseContext context) throws IOException {
+    private static Range parseRange(XContentParser parser) throws IOException {
         return Range.fromXContent(parser);
     }
 

+ 6 - 7
core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/GeoDistanceAggregationBuilder.java

@@ -30,7 +30,6 @@ import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.XContentParser.Token;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -54,7 +53,7 @@ public class GeoDistanceAggregationBuilder extends ValuesSourceAggregationBuilde
     static final ParseField UNIT_FIELD = new ParseField("unit");
     static final ParseField DISTANCE_TYPE_FIELD = new ParseField("distance_type");
 
-    private static final ObjectParser<GeoDistanceAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<GeoDistanceAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(GeoDistanceAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareGeoFields(PARSER, true, false);
@@ -65,7 +64,7 @@ public class GeoDistanceAggregationBuilder extends ValuesSourceAggregationBuilde
             for (Range range : ranges) {
                 agg.addRange(range);
             }
-        }, GeoDistanceAggregationBuilder::parseRange, RangeAggregator.RANGES_FIELD);
+        }, (p, c) -> GeoDistanceAggregationBuilder.parseRange(p), RangeAggregator.RANGES_FIELD);
 
         PARSER.declareField(GeoDistanceAggregationBuilder::unit, p -> DistanceUnit.fromString(p.text()),
                 UNIT_FIELD, ObjectParser.ValueType.STRING);
@@ -77,8 +76,8 @@ public class GeoDistanceAggregationBuilder extends ValuesSourceAggregationBuilde
                 ORIGIN_FIELD, ObjectParser.ValueType.OBJECT_ARRAY_OR_STRING);
     }
 
-    public static AggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        GeoDistanceAggregationBuilder builder = PARSER.parse(context.parser(), new GeoDistanceAggregationBuilder(aggregationName), context);
+    public static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        GeoDistanceAggregationBuilder builder = PARSER.parse(parser, new GeoDistanceAggregationBuilder(aggregationName), null);
         if (builder.origin() == null) {
             throw new IllegalArgumentException("Aggregation [" + aggregationName + "] must define an [origin].");
         }
@@ -116,7 +115,7 @@ public class GeoDistanceAggregationBuilder extends ValuesSourceAggregationBuilde
         }
     }
 
-    private static GeoPoint parseGeoPoint(XContentParser parser, QueryParseContext context) throws IOException {
+    private static GeoPoint parseGeoPoint(XContentParser parser) throws IOException {
         Token token = parser.currentToken();
         if (token == XContentParser.Token.VALUE_STRING) {
             GeoPoint point = new GeoPoint();
@@ -164,7 +163,7 @@ public class GeoDistanceAggregationBuilder extends ValuesSourceAggregationBuilde
         throw new IllegalArgumentException("Unexpected token [" + token + "] while parsing geo point");
     }
 
-    private static Range parseRange(XContentParser parser, QueryParseContext context) throws IOException {
+    private static Range parseRange(XContentParser parser) throws IOException {
         String fromAsStr = null;
         String toAsStr = null;
         double from = 0.0;

+ 5 - 6
core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/ip/IpRangeAggregationBuilder.java

@@ -30,7 +30,6 @@ import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.XContentParser.Token;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.script.Script;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
@@ -61,7 +60,7 @@ public final class IpRangeAggregationBuilder
     public static final String NAME = "ip_range";
     private static final ParseField MASK_FIELD = new ParseField("mask");
 
-    private static final ObjectParser<IpRangeAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<IpRangeAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(IpRangeAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareBytesFields(PARSER, false, false);
@@ -70,14 +69,14 @@ public final class IpRangeAggregationBuilder
 
         PARSER.declareObjectArray((agg, ranges) -> {
             for (Range range : ranges) agg.addRange(range);
-        }, IpRangeAggregationBuilder::parseRange, RangeAggregator.RANGES_FIELD);
+        }, (p, c) -> IpRangeAggregationBuilder.parseRange(p), RangeAggregator.RANGES_FIELD);
     }
 
-    public static AggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        return PARSER.parse(context.parser(), new IpRangeAggregationBuilder(aggregationName), context);
+    public static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        return PARSER.parse(parser, new IpRangeAggregationBuilder(aggregationName), null);
     }
 
-    private static Range parseRange(XContentParser parser, QueryParseContext context) throws IOException {
+    private static Range parseRange(XContentParser parser) throws IOException {
         String key = null;
         String from = null;
         String to = null;

+ 4 - 4
core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedAggregationBuilder.java

@@ -23,7 +23,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -43,7 +43,7 @@ public class DiversifiedAggregationBuilder extends ValuesSourceAggregationBuilde
 
     public static final int MAX_DOCS_PER_VALUE_DEFAULT = 1;
 
-    private static final ObjectParser<DiversifiedAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<DiversifiedAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(DiversifiedAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareAnyFields(PARSER, true, false);
@@ -52,8 +52,8 @@ public class DiversifiedAggregationBuilder extends ValuesSourceAggregationBuilde
         PARSER.declareString(DiversifiedAggregationBuilder::executionHint, SamplerAggregator.EXECUTION_HINT_FIELD);
     }
 
-    public static AggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        return PARSER.parse(context.parser(), new DiversifiedAggregationBuilder(aggregationName), context);
+    public static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        return PARSER.parse(parser, new DiversifiedAggregationBuilder(aggregationName), null);
     }
 
     private int shardSize = SamplerAggregationBuilder.DEFAULT_SHARD_SAMPLE_SIZE;

+ 1 - 3
core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregationBuilder.java

@@ -24,7 +24,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -86,12 +85,11 @@ public class SamplerAggregationBuilder extends AbstractAggregationBuilder<Sample
         return builder;
     }
 
-    public static SamplerAggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
+    public static SamplerAggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
         XContentParser.Token token;
         String currentFieldName = null;
         Integer shardSize = null;
 
-        XContentParser parser = context.parser();
         while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
             if (token == XContentParser.Token.FIELD_NAME) {
                 currentFieldName = parser.currentName();

+ 17 - 15
core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregationBuilder.java

@@ -24,8 +24,8 @@ import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.ParseFieldRegistry;
 import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
@@ -62,43 +62,45 @@ public class SignificantTermsAggregationBuilder extends ValuesSourceAggregationB
     static final SignificanceHeuristic DEFAULT_SIGNIFICANCE_HEURISTIC = new JLHScore();
 
     public static Aggregator.Parser getParser(ParseFieldRegistry<SignificanceHeuristicParser> significanceHeuristicParserRegistry) {
-        ObjectParser<SignificantTermsAggregationBuilder, QueryParseContext> parser =
+        ObjectParser<SignificantTermsAggregationBuilder, Void> aggregationParser =
                 new ObjectParser<>(SignificantTermsAggregationBuilder.NAME);
-        ValuesSourceParserHelper.declareAnyFields(parser, true, true);
+        ValuesSourceParserHelper.declareAnyFields(aggregationParser, true, true);
 
-        parser.declareInt(SignificantTermsAggregationBuilder::shardSize, TermsAggregationBuilder.SHARD_SIZE_FIELD_NAME);
+        aggregationParser.declareInt(SignificantTermsAggregationBuilder::shardSize, TermsAggregationBuilder.SHARD_SIZE_FIELD_NAME);
 
-        parser.declareLong(SignificantTermsAggregationBuilder::minDocCount, TermsAggregationBuilder.MIN_DOC_COUNT_FIELD_NAME);
+        aggregationParser.declareLong(SignificantTermsAggregationBuilder::minDocCount, TermsAggregationBuilder.MIN_DOC_COUNT_FIELD_NAME);
 
-        parser.declareLong(SignificantTermsAggregationBuilder::shardMinDocCount, TermsAggregationBuilder.SHARD_MIN_DOC_COUNT_FIELD_NAME);
+        aggregationParser.declareLong(SignificantTermsAggregationBuilder::shardMinDocCount,
+                TermsAggregationBuilder.SHARD_MIN_DOC_COUNT_FIELD_NAME);
 
-        parser.declareInt(SignificantTermsAggregationBuilder::size, TermsAggregationBuilder.REQUIRED_SIZE_FIELD_NAME);
+        aggregationParser.declareInt(SignificantTermsAggregationBuilder::size, TermsAggregationBuilder.REQUIRED_SIZE_FIELD_NAME);
 
-        parser.declareString(SignificantTermsAggregationBuilder::executionHint, TermsAggregationBuilder.EXECUTION_HINT_FIELD_NAME);
+        aggregationParser.declareString(SignificantTermsAggregationBuilder::executionHint,
+                TermsAggregationBuilder.EXECUTION_HINT_FIELD_NAME);
 
-        parser.declareObject(SignificantTermsAggregationBuilder::backgroundFilter,
+        aggregationParser.declareObject(SignificantTermsAggregationBuilder::backgroundFilter,
                 (p, context) -> parseInnerQueryBuilder(p),
                 SignificantTermsAggregationBuilder.BACKGROUND_FILTER);
 
-        parser.declareField((b, v) -> b.includeExclude(IncludeExclude.merge(v, b.includeExclude())),
+        aggregationParser.declareField((b, v) -> b.includeExclude(IncludeExclude.merge(v, b.includeExclude())),
                 IncludeExclude::parseInclude, IncludeExclude.INCLUDE_FIELD, ObjectParser.ValueType.OBJECT_ARRAY_OR_STRING);
 
-        parser.declareField((b, v) -> b.includeExclude(IncludeExclude.merge(b.includeExclude(), v)),
+        aggregationParser.declareField((b, v) -> b.includeExclude(IncludeExclude.merge(b.includeExclude(), v)),
                 IncludeExclude::parseExclude, IncludeExclude.EXCLUDE_FIELD, ObjectParser.ValueType.STRING_ARRAY);
 
         for (String name : significanceHeuristicParserRegistry.getNames()) {
-            parser.declareObject(SignificantTermsAggregationBuilder::significanceHeuristic,
+            aggregationParser.declareObject(SignificantTermsAggregationBuilder::significanceHeuristic,
                     (p, context) -> {
                         SignificanceHeuristicParser significanceHeuristicParser = significanceHeuristicParserRegistry
                                 .lookupReturningNullIfNotFound(name);
-                        return significanceHeuristicParser.parse(context);
+                        return significanceHeuristicParser.parse(p);
                     },
                     new ParseField(name));
         }
         return new Aggregator.Parser() {
             @Override
-            public AggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-                return parser.parse(context.parser(), new SignificantTermsAggregationBuilder(aggregationName, null), context);
+            public AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+                return aggregationParser.parse(parser, new SignificantTermsAggregationBuilder(aggregationName, null), null);
             }
         };
     }

+ 17 - 17
core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTextAggregationBuilder.java

@@ -25,9 +25,9 @@ import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.ParseFieldRegistry;
 import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.query.AbstractQueryBuilder;
 import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregationInitializationException;
@@ -70,55 +70,55 @@ public class SignificantTextAggregationBuilder extends AbstractAggregationBuilde
 
     public static Aggregator.Parser getParser(
             ParseFieldRegistry<SignificanceHeuristicParser> significanceHeuristicParserRegistry) {
-        ObjectParser<SignificantTextAggregationBuilder, QueryParseContext> parser = new ObjectParser<>(
+        ObjectParser<SignificantTextAggregationBuilder, Void> PARSER = new ObjectParser<>(
                 SignificantTextAggregationBuilder.NAME);
 
-        parser.declareInt(SignificantTextAggregationBuilder::shardSize,
+        PARSER.declareInt(SignificantTextAggregationBuilder::shardSize,
                 TermsAggregationBuilder.SHARD_SIZE_FIELD_NAME);
 
-        parser.declareLong(SignificantTextAggregationBuilder::minDocCount,
+        PARSER.declareLong(SignificantTextAggregationBuilder::minDocCount,
                 TermsAggregationBuilder.MIN_DOC_COUNT_FIELD_NAME);
 
-        parser.declareLong(SignificantTextAggregationBuilder::shardMinDocCount,
+        PARSER.declareLong(SignificantTextAggregationBuilder::shardMinDocCount,
                 TermsAggregationBuilder.SHARD_MIN_DOC_COUNT_FIELD_NAME);
 
-        parser.declareInt(SignificantTextAggregationBuilder::size,
+        PARSER.declareInt(SignificantTextAggregationBuilder::size,
                 TermsAggregationBuilder.REQUIRED_SIZE_FIELD_NAME);
 
-        parser.declareString(SignificantTextAggregationBuilder::fieldName, FIELD_NAME);
+        PARSER.declareString(SignificantTextAggregationBuilder::fieldName, FIELD_NAME);
 
-        parser.declareStringArray(SignificantTextAggregationBuilder::sourceFieldNames, SOURCE_FIELDS_NAME);
+        PARSER.declareStringArray(SignificantTextAggregationBuilder::sourceFieldNames, SOURCE_FIELDS_NAME);
 
 
-        parser.declareBoolean(SignificantTextAggregationBuilder::filterDuplicateText,
+        PARSER.declareBoolean(SignificantTextAggregationBuilder::filterDuplicateText,
                 FILTER_DUPLICATE_TEXT_FIELD_NAME);
 
-        parser.declareObject(SignificantTextAggregationBuilder::backgroundFilter,
+        PARSER.declareObject(SignificantTextAggregationBuilder::backgroundFilter,
                 (p, context) -> AbstractQueryBuilder.parseInnerQueryBuilder(p),
                 SignificantTermsAggregationBuilder.BACKGROUND_FILTER);
 
-        parser.declareField((b, v) -> b.includeExclude(IncludeExclude.merge(v, b.includeExclude())),
+        PARSER.declareField((b, v) -> b.includeExclude(IncludeExclude.merge(v, b.includeExclude())),
                 IncludeExclude::parseInclude, IncludeExclude.INCLUDE_FIELD,
                 ObjectParser.ValueType.OBJECT_ARRAY_OR_STRING);
 
-        parser.declareField((b, v) -> b.includeExclude(IncludeExclude.merge(b.includeExclude(), v)),
+        PARSER.declareField((b, v) -> b.includeExclude(IncludeExclude.merge(b.includeExclude(), v)),
                 IncludeExclude::parseExclude, IncludeExclude.EXCLUDE_FIELD,
                 ObjectParser.ValueType.STRING_ARRAY);
 
         for (String name : significanceHeuristicParserRegistry.getNames()) {
-            parser.declareObject(SignificantTextAggregationBuilder::significanceHeuristic,
+            PARSER.declareObject(SignificantTextAggregationBuilder::significanceHeuristic,
                     (p, context) -> {
                         SignificanceHeuristicParser significanceHeuristicParser = significanceHeuristicParserRegistry
                                 .lookupReturningNullIfNotFound(name);
-                        return significanceHeuristicParser.parse(context);
+                        return significanceHeuristicParser.parse(p);
                     }, new ParseField(name));
         }
         return new Aggregator.Parser() {
             @Override
-            public AggregationBuilder parse(String aggregationName, QueryParseContext context)
+            public AggregationBuilder parse(String aggregationName, XContentParser parser)
                     throws IOException {
-                return parser.parse(context.parser(),
-                        new SignificantTextAggregationBuilder(aggregationName, null), context);
+                return PARSER.parse(parser,
+                        new SignificantTextAggregationBuilder(aggregationName, null), null);
             }
         };
     }

+ 1 - 3
core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/heuristics/GND.java

@@ -26,7 +26,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.index.query.QueryShardException;
 
 import java.io.IOException;
@@ -113,8 +112,7 @@ public class GND extends NXYSignificanceHeuristic {
         }
 
         @Override
-        public SignificanceHeuristic parse(QueryParseContext context) throws IOException, QueryShardException {
-            XContentParser parser = context.parser();
+        public SignificanceHeuristic parse(XContentParser parser) throws IOException, QueryShardException {
             String givenName = parser.currentName();
             boolean backgroundIsSuperset = true;
             XContentParser.Token token = parser.nextToken();

+ 1 - 3
core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/heuristics/JLHScore.java

@@ -26,7 +26,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.index.query.QueryShardException;
 
 import java.io.IOException;
@@ -104,9 +103,8 @@ public class JLHScore extends SignificanceHeuristic {
         return builder;
     }
 
-    public static SignificanceHeuristic parse(QueryParseContext context)
+    public static SignificanceHeuristic parse(XContentParser parser)
             throws IOException, QueryShardException {
-        XContentParser parser = context.parser();
         // move to the closing bracket
         if (!parser.nextToken().equals(XContentParser.Token.END_OBJECT)) {
             throw new ElasticsearchParseException(

+ 1 - 3
core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/heuristics/NXYSignificanceHeuristic.java

@@ -27,7 +27,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.index.query.QueryShardException;
 
 import java.io.IOException;
@@ -152,9 +151,8 @@ public abstract class NXYSignificanceHeuristic extends SignificanceHeuristic {
     public abstract static class NXYParser implements SignificanceHeuristicParser {
 
         @Override
-        public SignificanceHeuristic parse(QueryParseContext context)
+        public SignificanceHeuristic parse(XContentParser parser)
                 throws IOException, QueryShardException {
-            XContentParser parser = context.parser();
             String givenName = parser.currentName();
             boolean includeNegatives = false;
             boolean backgroundIsSuperset = true;

+ 1 - 3
core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/heuristics/PercentageScore.java

@@ -26,7 +26,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.index.query.QueryShardException;
 
 import java.io.IOException;
@@ -56,9 +55,8 @@ public class PercentageScore extends SignificanceHeuristic {
         return builder;
     }
 
-    public static SignificanceHeuristic parse(QueryParseContext context)
+    public static SignificanceHeuristic parse(XContentParser parser)
             throws IOException, QueryShardException {
-        XContentParser parser = context.parser();
         // move to the closing bracket
         if (!parser.nextToken().equals(XContentParser.Token.END_OBJECT)) {
             throw new ElasticsearchParseException("failed to parse [percentage] significance heuristic. expected an empty object, but got [{}] instead", parser.currentToken());

+ 1 - 3
core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/heuristics/ScriptHeuristic.java

@@ -26,7 +26,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.index.query.QueryShardContext;
 import org.elasticsearch.index.query.QueryShardException;
 import org.elasticsearch.script.ExecutableScript;
@@ -149,9 +148,8 @@ public class ScriptHeuristic extends SignificanceHeuristic {
         return Objects.equals(script, other.script);
     }
 
-    public static SignificanceHeuristic parse(QueryParseContext context)
+    public static SignificanceHeuristic parse(XContentParser parser)
             throws IOException, QueryShardException {
-        XContentParser parser = context.parser();
         String heuristicName = parser.currentName();
         Script script = null;
         XContentParser.Token token;

+ 1 - 2
core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/heuristics/SignificanceHeuristicParser.java

@@ -22,7 +22,6 @@ package org.elasticsearch.search.aggregations.bucket.significant.heuristics;
 
 import org.elasticsearch.common.ParsingException;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 
 import java.io.IOException;
 
@@ -31,5 +30,5 @@ import java.io.IOException;
  */
 @FunctionalInterface
 public interface SignificanceHeuristicParser {
-    SignificanceHeuristic parse(QueryParseContext context) throws IOException, ParsingException;
+    SignificanceHeuristic parse(XContentParser parser) throws IOException, ParsingException;
 }

+ 7 - 7
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregationBuilder.java

@@ -23,16 +23,16 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
-import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds;
-import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
 import org.elasticsearch.search.aggregations.BucketOrder;
 import org.elasticsearch.search.aggregations.InternalOrder;
 import org.elasticsearch.search.aggregations.InternalOrder.CompoundOrder;
+import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds;
+import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
 import org.elasticsearch.search.aggregations.support.ValueType;
 import org.elasticsearch.search.aggregations.support.ValuesSource;
 import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
@@ -60,7 +60,7 @@ public class TermsAggregationBuilder extends ValuesSourceAggregationBuilder<Valu
     public static final ParseField SHOW_TERM_DOC_COUNT_ERROR = new ParseField("show_term_doc_count_error");
     public static final ParseField ORDER_FIELD = new ParseField("order");
 
-    private static final ObjectParser<TermsAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<TermsAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(TermsAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareAnyFields(PARSER, true, true);
@@ -82,7 +82,7 @@ public class TermsAggregationBuilder extends ValuesSourceAggregationBuilder<Valu
                 (p, c) -> SubAggCollectionMode.parse(p.text()),
                 SubAggCollectionMode.KEY, ObjectParser.ValueType.STRING);
 
-        PARSER.declareObjectArray(TermsAggregationBuilder::order, InternalOrder.Parser::parseOrderParam,
+        PARSER.declareObjectArray(TermsAggregationBuilder::order, (p, c) -> InternalOrder.Parser.parseOrderParam(p),
                 TermsAggregationBuilder.ORDER_FIELD);
 
         PARSER.declareField((b, v) -> b.includeExclude(IncludeExclude.merge(v, b.includeExclude())),
@@ -92,8 +92,8 @@ public class TermsAggregationBuilder extends ValuesSourceAggregationBuilder<Valu
                 IncludeExclude::parseExclude, IncludeExclude.EXCLUDE_FIELD, ObjectParser.ValueType.STRING_ARRAY);
     }
 
-    public static AggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        return PARSER.parse(context.parser(), new TermsAggregationBuilder(aggregationName, null), context);
+    public static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        return PARSER.parse(parser, new TermsAggregationBuilder(aggregationName, null), null);
     }
 
     private BucketOrder order = BucketOrder.compound(BucketOrder.count(false)); // automatically adds tie-breaker key asc order

+ 3 - 3
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/support/IncludeExclude.java

@@ -44,7 +44,6 @@ import org.elasticsearch.common.io.stream.Writeable;
 import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.DocValueFormat;
 
 import java.io.IOException;
@@ -93,7 +92,7 @@ public class IncludeExclude implements Writeable, ToXContent {
         }
     }
 
-    public static IncludeExclude parseInclude(XContentParser parser, QueryParseContext context) throws IOException {
+    public static IncludeExclude parseInclude(XContentParser parser) throws IOException {
         XContentParser.Token token = parser.currentToken();
         if (token == XContentParser.Token.VALUE_STRING) {
             return new IncludeExclude(parser.text(), null);
@@ -128,7 +127,7 @@ public class IncludeExclude implements Writeable, ToXContent {
         }
     }
 
-    public static IncludeExclude parseExclude(XContentParser parser, QueryParseContext context) throws IOException {
+    public static IncludeExclude parseExclude(XContentParser parser) throws IOException {
         XContentParser.Token token = parser.currentToken();
         if (token == XContentParser.Token.VALUE_STRING) {
             return new IncludeExclude(null, parser.text());
@@ -170,6 +169,7 @@ public class IncludeExclude implements Writeable, ToXContent {
             }
         }
 
+        @Override
         public boolean accept(long value) {
             return ((valids == null) || (valids.contains(value))) && ((invalids == null) || (!invalids.contains(value)));
         }

+ 4 - 4
core/src/main/java/org/elasticsearch/search/aggregations/metrics/avg/AvgAggregationBuilder.java

@@ -23,7 +23,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -41,14 +41,14 @@ import java.io.IOException;
 public class AvgAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOnly<ValuesSource.Numeric, AvgAggregationBuilder> {
     public static final String NAME = "avg";
 
-    private static final ObjectParser<AvgAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<AvgAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(AvgAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareNumericFields(PARSER, true, true, false);
     }
 
-    public static AggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        return PARSER.parse(context.parser(), new AvgAggregationBuilder(aggregationName), context);
+    public static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        return PARSER.parse(parser, new AvgAggregationBuilder(aggregationName), null);
     }
 
     public AvgAggregationBuilder(String name) {

+ 4 - 4
core/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregationBuilder.java

@@ -24,7 +24,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -47,7 +47,7 @@ public final class CardinalityAggregationBuilder
     private static final ParseField REHASH = new ParseField("rehash").withAllDeprecated("no replacement - values will always be rehashed");
     public static final ParseField PRECISION_THRESHOLD_FIELD = new ParseField("precision_threshold");
 
-    private static final ObjectParser<CardinalityAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<CardinalityAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(CardinalityAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareAnyFields(PARSER, true, false);
@@ -55,8 +55,8 @@ public final class CardinalityAggregationBuilder
         PARSER.declareLong((b, v) -> {/*ignore*/}, REHASH);
     }
 
-    public static AggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        return PARSER.parse(context.parser(), new CardinalityAggregationBuilder(aggregationName, null), context);
+    public static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        return PARSER.parse(parser, new CardinalityAggregationBuilder(aggregationName, null), null);
     }
 
     private Long precisionThreshold = null;

+ 4 - 4
core/src/main/java/org/elasticsearch/search/aggregations/metrics/geobounds/GeoBoundsAggregationBuilder.java

@@ -23,7 +23,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -41,15 +41,15 @@ import java.util.Objects;
 public class GeoBoundsAggregationBuilder extends ValuesSourceAggregationBuilder<ValuesSource.GeoPoint, GeoBoundsAggregationBuilder> {
     public static final String NAME = "geo_bounds";
 
-    private static final ObjectParser<GeoBoundsAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<GeoBoundsAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(GeoBoundsAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareGeoFields(PARSER, false, false);
         PARSER.declareBoolean(GeoBoundsAggregationBuilder::wrapLongitude, GeoBoundsAggregator.WRAP_LONGITUDE_FIELD);
     }
 
-    public static AggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        return PARSER.parse(context.parser(), new GeoBoundsAggregationBuilder(aggregationName), context);
+    public static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        return PARSER.parse(parser, new GeoBoundsAggregationBuilder(aggregationName), null);
     }
 
     private boolean wrapLongitude = true;

+ 4 - 4
core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroidAggregationBuilder.java

@@ -23,7 +23,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -41,14 +41,14 @@ public class GeoCentroidAggregationBuilder
         extends ValuesSourceAggregationBuilder.LeafOnly<ValuesSource.GeoPoint, GeoCentroidAggregationBuilder> {
     public static final String NAME = "geo_centroid";
 
-    private static final ObjectParser<GeoCentroidAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<GeoCentroidAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(GeoCentroidAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareGeoFields(PARSER, true, false);
     }
 
-    public static AggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        return PARSER.parse(context.parser(), new GeoCentroidAggregationBuilder(aggregationName), context);
+    public static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        return PARSER.parse(parser, new GeoCentroidAggregationBuilder(aggregationName), null);
     }
 
     public GeoCentroidAggregationBuilder(String name) {

+ 4 - 4
core/src/main/java/org/elasticsearch/search/aggregations/metrics/max/MaxAggregationBuilder.java

@@ -23,7 +23,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -41,14 +41,14 @@ import java.io.IOException;
 public class MaxAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOnly<ValuesSource.Numeric, MaxAggregationBuilder> {
     public static final String NAME = "max";
 
-    private static final ObjectParser<MaxAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<MaxAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(MaxAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareNumericFields(PARSER, true, true, false);
     }
 
-    public static AggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        return PARSER.parse(context.parser(), new MaxAggregationBuilder(aggregationName), context);
+    public static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        return PARSER.parse(parser, new MaxAggregationBuilder(aggregationName), null);
     }
 
     public MaxAggregationBuilder(String name) {

+ 4 - 4
core/src/main/java/org/elasticsearch/search/aggregations/metrics/min/MinAggregationBuilder.java

@@ -23,7 +23,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -42,14 +42,14 @@ import java.io.IOException;
 public class MinAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOnly<ValuesSource.Numeric, MinAggregationBuilder> {
     public static final String NAME = "min";
 
-    private static final ObjectParser<MinAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<MinAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(AvgAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareNumericFields(PARSER, true, true, false);
     }
 
-    public static AggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        return PARSER.parse(context.parser(), new MinAggregationBuilder(aggregationName), context);
+    public static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        return PARSER.parse(parser, new MinAggregationBuilder(aggregationName), null);
     }
 
     public MinAggregationBuilder(String name) {

+ 6 - 6
core/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentileRanksAggregationBuilder.java

@@ -24,7 +24,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -53,7 +53,7 @@ public class PercentileRanksAggregationBuilder extends LeafOnly<ValuesSource.Num
         Double compression;
     }
 
-    private static final ObjectParser<TDigestOptions, QueryParseContext> TDIGEST_OPTIONS_PARSER =
+    private static final ObjectParser<TDigestOptions, Void> TDIGEST_OPTIONS_PARSER =
             new ObjectParser<>(PercentilesMethod.TDIGEST.getParseField().getPreferredName(), TDigestOptions::new);
     static {
         TDIGEST_OPTIONS_PARSER.declareDouble((opts, compression) -> opts.compression = compression, new ParseField("compression"));
@@ -63,14 +63,14 @@ public class PercentileRanksAggregationBuilder extends LeafOnly<ValuesSource.Num
         Integer numberOfSigDigits;
     }
 
-    private static final ObjectParser<HDROptions, QueryParseContext> HDR_OPTIONS_PARSER =
+    private static final ObjectParser<HDROptions, Void> HDR_OPTIONS_PARSER =
             new ObjectParser<>(PercentilesMethod.HDR.getParseField().getPreferredName(), HDROptions::new);
     static {
         HDR_OPTIONS_PARSER.declareInt((opts, numberOfSigDigits) -> opts.numberOfSigDigits = numberOfSigDigits,
                 new ParseField("number_of_significant_value_digits"));
     }
 
-    private static final ObjectParser<PercentileRanksAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<PercentileRanksAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(PercentileRanksAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareNumericFields(PARSER, true, false, false);
@@ -96,8 +96,8 @@ public class PercentileRanksAggregationBuilder extends LeafOnly<ValuesSource.Num
         }, HDR_OPTIONS_PARSER::parse, PercentilesMethod.HDR.getParseField(), ObjectParser.ValueType.OBJECT);
     }
 
-    public static AggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        return PARSER.parse(context.parser(), new PercentileRanksAggregationBuilder(aggregationName), context);
+    public static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        return PARSER.parse(parser, new PercentileRanksAggregationBuilder(aggregationName), null);
     }
 
     private double[] values;

+ 6 - 6
core/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentilesAggregationBuilder.java

@@ -24,7 +24,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -58,7 +58,7 @@ public class PercentilesAggregationBuilder extends LeafOnly<ValuesSource.Numeric
         Double compression;
     }
 
-    private static final ObjectParser<TDigestOptions, QueryParseContext> TDIGEST_OPTIONS_PARSER =
+    private static final ObjectParser<TDigestOptions, Void> TDIGEST_OPTIONS_PARSER =
             new ObjectParser<>(PercentilesMethod.TDIGEST.getParseField().getPreferredName(), TDigestOptions::new);
     static {
         TDIGEST_OPTIONS_PARSER.declareDouble((opts, compression) -> opts.compression = compression, COMPRESSION_FIELD);
@@ -68,7 +68,7 @@ public class PercentilesAggregationBuilder extends LeafOnly<ValuesSource.Numeric
         Integer numberOfSigDigits;
     }
 
-    private static final ObjectParser<HDROptions, QueryParseContext> HDR_OPTIONS_PARSER =
+    private static final ObjectParser<HDROptions, Void> HDR_OPTIONS_PARSER =
             new ObjectParser<>(PercentilesMethod.HDR.getParseField().getPreferredName(), HDROptions::new);
     static {
         HDR_OPTIONS_PARSER.declareInt(
@@ -76,7 +76,7 @@ public class PercentilesAggregationBuilder extends LeafOnly<ValuesSource.Numeric
                 NUMBER_SIGNIFICANT_DIGITS_FIELD);
     }
 
-    private static final ObjectParser<PercentilesAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<PercentilesAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(PercentilesAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareNumericFields(PARSER, true, true, false);
@@ -102,8 +102,8 @@ public class PercentilesAggregationBuilder extends LeafOnly<ValuesSource.Numeric
         }, HDR_OPTIONS_PARSER::parse, PercentilesMethod.HDR.getParseField(), ObjectParser.ValueType.OBJECT);
     }
 
-    public static AggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        return PARSER.parse(context.parser(), new PercentilesAggregationBuilder(aggregationName), context);
+    public static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        return PARSER.parse(parser, new PercentilesAggregationBuilder(aggregationName), null);
     }
 
     private double[] percents = DEFAULT_PERCENTS;

+ 1 - 5
core/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregationBuilder.java

@@ -25,11 +25,9 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.index.query.QueryShardContext;
 import org.elasticsearch.script.ExecutableScript;
 import org.elasticsearch.script.Script;
-import org.elasticsearch.script.ScriptContext;
 import org.elasticsearch.script.SearchScript;
 import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
@@ -39,7 +37,6 @@ import org.elasticsearch.search.internal.SearchContext;
 import java.io.IOException;
 import java.util.Map;
 import java.util.Objects;
-import java.util.function.Function;
 
 public class ScriptedMetricAggregationBuilder extends AbstractAggregationBuilder<ScriptedMetricAggregationBuilder> {
     public static final String NAME = "scripted_metric";
@@ -228,7 +225,7 @@ public class ScriptedMetricAggregationBuilder extends AbstractAggregationBuilder
         return builder;
     }
 
-    public static ScriptedMetricAggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
+    public static ScriptedMetricAggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
         Script initScript = null;
         Script mapScript = null;
         Script combineScript = null;
@@ -237,7 +234,6 @@ public class ScriptedMetricAggregationBuilder extends AbstractAggregationBuilder
         XContentParser.Token token;
         String currentFieldName = null;
 
-        XContentParser parser = context.parser();
         while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
             if (token == XContentParser.Token.FIELD_NAME) {
                 currentFieldName = parser.currentName();

+ 4 - 4
core/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/StatsAggregationBuilder.java

@@ -23,7 +23,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -41,14 +41,14 @@ import java.io.IOException;
 public class StatsAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOnly<ValuesSource.Numeric, StatsAggregationBuilder> {
     public static final String NAME = "stats";
 
-    private static final ObjectParser<StatsAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<StatsAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(StatsAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareNumericFields(PARSER, true, true, false);
     }
 
-    public static AggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        return PARSER.parse(context.parser(), new StatsAggregationBuilder(aggregationName), context);
+    public static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        return PARSER.parse(parser, new StatsAggregationBuilder(aggregationName), null);
     }
 
     public StatsAggregationBuilder(String name) {

+ 4 - 4
core/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsAggregationBuilder.java

@@ -23,7 +23,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -43,15 +43,15 @@ public class ExtendedStatsAggregationBuilder
         extends ValuesSourceAggregationBuilder.LeafOnly<ValuesSource.Numeric, ExtendedStatsAggregationBuilder> {
     public static final String NAME = "extended_stats";
 
-    private static final ObjectParser<ExtendedStatsAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<ExtendedStatsAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(ExtendedStatsAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareNumericFields(PARSER, true, true, false);
         PARSER.declareDouble(ExtendedStatsAggregationBuilder::sigma, ExtendedStatsAggregator.SIGMA_FIELD);
     }
 
-    public static AggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        return PARSER.parse(context.parser(), new ExtendedStatsAggregationBuilder(aggregationName), context);
+    public static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        return PARSER.parse(parser, new ExtendedStatsAggregationBuilder(aggregationName), null);
     }
 
     private double sigma = 2.0;

+ 4 - 4
core/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/SumAggregationBuilder.java

@@ -23,7 +23,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -41,14 +41,14 @@ import java.io.IOException;
 public class SumAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOnly<ValuesSource.Numeric, SumAggregationBuilder> {
     public static final String NAME = "sum";
 
-    private static final ObjectParser<SumAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<SumAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(SumAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareNumericFields(PARSER, true, true, false);
     }
 
-    public static AggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        return PARSER.parse(context.parser(), new SumAggregationBuilder(aggregationName), context);
+    public static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        return PARSER.parse(parser, new SumAggregationBuilder(aggregationName), null);
     }
 
     public SumAggregationBuilder(String name) {

+ 6 - 8
core/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsAggregationBuilder.java

@@ -26,7 +26,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.index.query.QueryShardContext;
 import org.elasticsearch.script.Script;
 import org.elasticsearch.script.SearchScript;
@@ -595,11 +594,10 @@ public class TopHitsAggregationBuilder extends AbstractAggregationBuilder<TopHit
         return builder;
     }
 
-    public static TopHitsAggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
+    public static TopHitsAggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
         TopHitsAggregationBuilder factory = new TopHitsAggregationBuilder(aggregationName);
         XContentParser.Token token;
         String currentFieldName = null;
-        XContentParser parser = context.parser();
         while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
             if (token == XContentParser.Token.FIELD_NAME) {
                 currentFieldName = parser.currentName();
@@ -615,10 +613,10 @@ public class TopHitsAggregationBuilder extends AbstractAggregationBuilder<TopHit
                 } else if (SearchSourceBuilder.TRACK_SCORES_FIELD.match(currentFieldName)) {
                     factory.trackScores(parser.booleanValue());
                 } else if (SearchSourceBuilder._SOURCE_FIELD.match(currentFieldName)) {
-                    factory.fetchSource(FetchSourceContext.fromXContent(context.parser()));
+                    factory.fetchSource(FetchSourceContext.fromXContent(parser));
                 } else if (SearchSourceBuilder.STORED_FIELDS_FIELD.match(currentFieldName)) {
                     factory.storedFieldsContext =
-                        StoredFieldsContext.fromXContent(SearchSourceBuilder.STORED_FIELDS_FIELD.getPreferredName(), context);
+                        StoredFieldsContext.fromXContent(SearchSourceBuilder.STORED_FIELDS_FIELD.getPreferredName(), parser);
                 } else if (SearchSourceBuilder.SORT_FIELD.match(currentFieldName)) {
                     factory.sort(parser.text());
                 } else {
@@ -627,7 +625,7 @@ public class TopHitsAggregationBuilder extends AbstractAggregationBuilder<TopHit
                 }
             } else if (token == XContentParser.Token.START_OBJECT) {
                 if (SearchSourceBuilder._SOURCE_FIELD.match(currentFieldName)) {
-                    factory.fetchSource(FetchSourceContext.fromXContent(context.parser()));
+                    factory.fetchSource(FetchSourceContext.fromXContent(parser));
                 } else if (SearchSourceBuilder.SCRIPT_FIELDS_FIELD.match(currentFieldName)) {
                     List<ScriptField> scriptFields = new ArrayList<>();
                     while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
@@ -682,7 +680,7 @@ public class TopHitsAggregationBuilder extends AbstractAggregationBuilder<TopHit
 
                 if (SearchSourceBuilder.STORED_FIELDS_FIELD.match(currentFieldName)) {
                     factory.storedFieldsContext =
-                        StoredFieldsContext.fromXContent(SearchSourceBuilder.STORED_FIELDS_FIELD.getPreferredName(), context);
+                        StoredFieldsContext.fromXContent(SearchSourceBuilder.STORED_FIELDS_FIELD.getPreferredName(), parser);
                 } else if (SearchSourceBuilder.DOCVALUE_FIELDS_FIELD.match(currentFieldName)) {
                     List<String> fieldDataFields = new ArrayList<>();
                     while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
@@ -698,7 +696,7 @@ public class TopHitsAggregationBuilder extends AbstractAggregationBuilder<TopHit
                     List<SortBuilder<?>> sorts = SortBuilder.fromXContent(parser);
                     factory.sorts(sorts);
                 } else if (SearchSourceBuilder._SOURCE_FIELD.match(currentFieldName)) {
-                    factory.fetchSource(FetchSourceContext.fromXContent(context.parser()));
+                    factory.fetchSource(FetchSourceContext.fromXContent(parser));
                 } else {
                     throw new ParsingException(parser.getTokenLocation(), "Unknown key for a " + token + " in [" + currentFieldName + "].",
                             parser.getTokenLocation());

+ 4 - 4
core/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountAggregationBuilder.java

@@ -23,7 +23,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
@@ -40,14 +40,14 @@ import java.io.IOException;
 public class ValueCountAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOnly<ValuesSource, ValueCountAggregationBuilder> {
     public static final String NAME = "value_count";
 
-    private static final ObjectParser<ValueCountAggregationBuilder, QueryParseContext> PARSER;
+    private static final ObjectParser<ValueCountAggregationBuilder, Void> PARSER;
     static {
         PARSER = new ObjectParser<>(ValueCountAggregationBuilder.NAME);
         ValuesSourceParserHelper.declareAnyFields(PARSER, true, true);
     }
 
-    public static AggregationBuilder parse(String aggregationName, QueryParseContext context) throws IOException {
-        return PARSER.parse(context.parser(), new ValueCountAggregationBuilder(aggregationName, null), context);
+    public static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+        return PARSER.parse(parser, new ValueCountAggregationBuilder(aggregationName, null), null);
     }
 
     public ValueCountAggregationBuilder(String name, ValueType targetValueType) {

+ 4 - 5
core/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpers.java

@@ -24,7 +24,6 @@ import org.elasticsearch.common.ParsingException;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentLocation;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.AggregationExecutionException;
 import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation;
 import org.elasticsearch.search.aggregations.InvalidAggregationPathException;
@@ -58,11 +57,11 @@ public class BucketHelpers {
         /**
          * Parse a string GapPolicy into the byte enum
          *
-         * @param context SearchContext this is taking place in
-         * @param text    GapPolicy in string format (e.g. "ignore")
-         * @return        GapPolicy enum
+         * @param text
+         *            GapPolicy in string format (e.g. "ignore")
+         * @return GapPolicy enum
          */
-        public static GapPolicy parse(QueryParseContext context, String text, XContentLocation tokenLocation) {
+        public static GapPolicy parse(String text, XContentLocation tokenLocation) {
             GapPolicy result = null;
             for (GapPolicy policy : values()) {
                 if (policy.parseField.match(text)) {

+ 4 - 5
core/src/main/java/org/elasticsearch/search/aggregations/pipeline/PipelineAggregator.java

@@ -24,7 +24,7 @@ import org.elasticsearch.common.ParseField;
 import org.elasticsearch.common.io.stream.NamedWriteable;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
-import org.elasticsearch.index.query.QueryParseContext;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext;
 import org.elasticsearch.search.aggregations.PipelineAggregationBuilder;
@@ -34,7 +34,7 @@ import java.util.Map;
 
 public abstract class PipelineAggregator implements NamedWriteable {
     /**
-     * Parse the {@link PipelineAggregationBuilder} from a {@link QueryParseContext}.
+     * Parse the {@link PipelineAggregationBuilder} from a {@link XContentParser}.
      */
     @FunctionalInterface
     public interface Parser {
@@ -48,13 +48,12 @@ public abstract class PipelineAggregator implements NamedWriteable {
          *
          * @param pipelineAggregatorName
          *            The name of the pipeline aggregation
-         * @param context
-         *            The search context
+         * @param parser the parser
          * @return The resolved pipeline aggregator factory
          * @throws java.io.IOException
          *             When parsing fails
          */
-        PipelineAggregationBuilder parse(String pipelineAggregatorName, QueryParseContext context)
+        PipelineAggregationBuilder parse(String pipelineAggregatorName, XContentParser parser)
                 throws IOException;
     }
 

+ 8 - 10
core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/BucketMetricsParser.java

@@ -22,7 +22,6 @@ package org.elasticsearch.search.aggregations.pipeline.bucketmetrics;
 import org.elasticsearch.common.ParseField;
 import org.elasticsearch.common.ParsingException;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
 
@@ -44,9 +43,8 @@ public abstract class BucketMetricsParser implements PipelineAggregator.Parser {
     }
 
     @Override
-    public final BucketMetricsPipelineAggregationBuilder<?> parse(String pipelineAggregatorName, QueryParseContext context)
+    public final BucketMetricsPipelineAggregationBuilder<?> parse(String pipelineAggregatorName, XContentParser parser)
             throws IOException {
-        XContentParser parser = context.parser();
         XContentParser.Token token;
         String currentFieldName = null;
         String[] bucketsPaths = null;
@@ -63,9 +61,9 @@ public abstract class BucketMetricsParser implements PipelineAggregator.Parser {
                 } else if (BUCKETS_PATH.match(currentFieldName)) {
                     bucketsPaths = new String[] { parser.text() };
                 } else if (GAP_POLICY.match(currentFieldName)) {
-                    gapPolicy = GapPolicy.parse(context, parser.text(), parser.getTokenLocation());
+                    gapPolicy = GapPolicy.parse(parser.text(), parser.getTokenLocation());
                 } else {
-                    parseToken(pipelineAggregatorName, parser, context, currentFieldName, token, params);
+                    parseToken(pipelineAggregatorName, parser, currentFieldName, token, params);
                 }
             } else if (token == XContentParser.Token.START_ARRAY) {
                 if (BUCKETS_PATH.match(currentFieldName)) {
@@ -76,10 +74,10 @@ public abstract class BucketMetricsParser implements PipelineAggregator.Parser {
                     }
                     bucketsPaths = paths.toArray(new String[paths.size()]);
                 } else {
-                    parseToken(pipelineAggregatorName, parser, context, currentFieldName, token, params);
+                    parseToken(pipelineAggregatorName, parser, currentFieldName, token, params);
                 }
             } else {
-                parseToken(pipelineAggregatorName, parser, context, currentFieldName, token, params);
+                parseToken(pipelineAggregatorName, parser, currentFieldName, token, params);
             }
         }
 
@@ -104,14 +102,14 @@ public abstract class BucketMetricsParser implements PipelineAggregator.Parser {
     protected abstract BucketMetricsPipelineAggregationBuilder<?> buildFactory(String pipelineAggregatorName, String bucketsPaths,
                                                                               Map<String, Object> params);
 
-    protected boolean token(XContentParser parser, QueryParseContext context, String field,
+    protected boolean token(XContentParser parser, String field,
                             XContentParser.Token token, Map<String, Object> params) throws IOException {
         return false;
     }
 
-    private void parseToken(String aggregationName, XContentParser parser, QueryParseContext context, String currentFieldName,
+    private void parseToken(String aggregationName, XContentParser parser, String currentFieldName,
                        XContentParser.Token currentToken, Map<String, Object> params) throws IOException {
-        if (token(parser, context, currentFieldName, currentToken, params) == false) {
+        if (token(parser, currentFieldName, currentToken, params) == false) {
             throw new ParsingException(parser.getTokenLocation(),
                 "Unexpected token " + currentToken + " [" + currentFieldName + "] in [" + aggregationName + "]");
         }

+ 2 - 3
core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/percentile/PercentilesBucketPipelineAggregationBuilder.java

@@ -26,7 +26,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
 import org.elasticsearch.search.aggregations.PipelineAggregationBuilder;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
@@ -136,8 +135,8 @@ public class PercentilesBucketPipelineAggregationBuilder
         }
 
         @Override
-        protected boolean token(XContentParser parser, QueryParseContext context, String field,
-                                XContentParser.Token token, Map<String, Object> params) throws IOException {
+        protected boolean token(XContentParser parser, String field, XContentParser.Token token, Map<String, Object> params)
+                throws IOException {
             if (PERCENTS_FIELD.match(field) && token == XContentParser.Token.START_ARRAY) {
                 DoubleArrayList percents = new DoubleArrayList(10);
                 while (parser.nextToken() != XContentParser.Token.END_ARRAY) {

+ 1 - 2
core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/stats/extended/ExtendedStatsBucketParser.java

@@ -21,7 +21,6 @@ package org.elasticsearch.search.aggregations.pipeline.bucketmetrics.stats.exten
 
 import org.elasticsearch.common.ParseField;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsParser;
 
 import java.io.IOException;
@@ -44,7 +43,7 @@ public class ExtendedStatsBucketParser extends BucketMetricsParser {
     }
 
     @Override
-    protected boolean token(XContentParser parser, QueryParseContext context, String field,
+    protected boolean token(XContentParser parser, String field,
                             XContentParser.Token token, Map<String, Object> params) throws IOException {
         if (SIGMA.match(field) && token == XContentParser.Token.VALUE_NUMBER) {
             params.put(SIGMA.getPreferredName(), parser.doubleValue());

+ 3 - 5
core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketscript/BucketScriptPipelineAggregationBuilder.java

@@ -24,7 +24,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.script.Script;
 import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.aggregations.pipeline.AbstractPipelineAggregationBuilder;
@@ -68,7 +67,7 @@ public class BucketScriptPipelineAggregationBuilder extends AbstractPipelineAggr
     public BucketScriptPipelineAggregationBuilder(StreamInput in) throws IOException {
         super(in, NAME);
         int mapSize = in.readVInt();
-        bucketsPathsMap = new HashMap<String, String>(mapSize);
+        bucketsPathsMap = new HashMap<>(mapSize);
         for (int i = 0; i < mapSize; i++) {
             bucketsPathsMap.put(in.readString(), in.readString());
         }
@@ -157,8 +156,7 @@ public class BucketScriptPipelineAggregationBuilder extends AbstractPipelineAggr
         return builder;
     }
 
-    public static BucketScriptPipelineAggregationBuilder parse(String reducerName, QueryParseContext context) throws IOException {
-        XContentParser parser = context.parser();
+    public static BucketScriptPipelineAggregationBuilder parse(String reducerName, XContentParser parser) throws IOException {
         XContentParser.Token token;
         Script script = null;
         String currentFieldName = null;
@@ -176,7 +174,7 @@ public class BucketScriptPipelineAggregationBuilder extends AbstractPipelineAggr
                     bucketsPathsMap = new HashMap<>();
                     bucketsPathsMap.put("_value", parser.text());
                 } else if (GAP_POLICY.match(currentFieldName)) {
-                    gapPolicy = GapPolicy.parse(context, parser.text(), parser.getTokenLocation());
+                    gapPolicy = GapPolicy.parse(parser.text(), parser.getTokenLocation());
                 } else if (Script.SCRIPT_PARSE_FIELD.match(currentFieldName)) {
                     script = Script.parse(parser);
                 } else {

+ 3 - 5
core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketselector/BucketSelectorPipelineAggregationBuilder.java

@@ -24,7 +24,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.script.Script;
 import org.elasticsearch.search.aggregations.pipeline.AbstractPipelineAggregationBuilder;
 import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
@@ -65,7 +64,7 @@ public class BucketSelectorPipelineAggregationBuilder extends AbstractPipelineAg
     public BucketSelectorPipelineAggregationBuilder(StreamInput in) throws IOException {
         super(in, NAME);
         int mapSize = in.readVInt();
-        bucketsPathsMap = new HashMap<String, String>(mapSize);
+        bucketsPathsMap = new HashMap<>(mapSize);
         for (int i = 0; i < mapSize; i++) {
             bucketsPathsMap.put(in.readString(), in.readString());
         }
@@ -123,8 +122,7 @@ public class BucketSelectorPipelineAggregationBuilder extends AbstractPipelineAg
         return builder;
     }
 
-    public static BucketSelectorPipelineAggregationBuilder parse(String reducerName, QueryParseContext context) throws IOException {
-        XContentParser parser = context.parser();
+    public static BucketSelectorPipelineAggregationBuilder parse(String reducerName, XContentParser parser) throws IOException {
         XContentParser.Token token;
         Script script = null;
         String currentFieldName = null;
@@ -139,7 +137,7 @@ public class BucketSelectorPipelineAggregationBuilder extends AbstractPipelineAg
                     bucketsPathsMap = new HashMap<>();
                     bucketsPathsMap.put("_value", parser.text());
                 } else if (GAP_POLICY.match(currentFieldName)) {
-                    gapPolicy = GapPolicy.parse(context, parser.text(), parser.getTokenLocation());
+                    gapPolicy = GapPolicy.parse(parser.text(), parser.getTokenLocation());
                 } else if (Script.SCRIPT_PARSE_FIELD.match(currentFieldName)) {
                     script = Script.parse(parser);
                 } else {

+ 1 - 3
core/src/main/java/org/elasticsearch/search/aggregations/pipeline/cumulativesum/CumulativeSumPipelineAggregationBuilder.java

@@ -24,7 +24,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
 import org.elasticsearch.search.aggregations.PipelineAggregationBuilder;
@@ -129,9 +128,8 @@ public class CumulativeSumPipelineAggregationBuilder extends AbstractPipelineAgg
         return builder;
     }
 
-    public static CumulativeSumPipelineAggregationBuilder parse(String pipelineAggregatorName, QueryParseContext context)
+    public static CumulativeSumPipelineAggregationBuilder parse(String pipelineAggregatorName, XContentParser parser)
             throws IOException {
-        XContentParser parser = context.parser();
         XContentParser.Token token;
         String currentFieldName = null;
         String[] bucketsPaths = null;

+ 3 - 5
core/src/main/java/org/elasticsearch/search/aggregations/pipeline/derivative/DerivativePipelineAggregationBuilder.java

@@ -27,12 +27,11 @@ import org.elasticsearch.common.rounding.DateTimeUnit;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
 import org.elasticsearch.search.aggregations.PipelineAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregatorFactory;
 import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder;
+import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregatorFactory;
 import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
 import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregatorFactory;
 import org.elasticsearch.search.aggregations.pipeline.AbstractPipelineAggregationBuilder;
@@ -194,8 +193,7 @@ public class DerivativePipelineAggregationBuilder extends AbstractPipelineAggreg
         return builder;
     }
 
-    public static DerivativePipelineAggregationBuilder parse(String pipelineAggregatorName, QueryParseContext context) throws IOException {
-        XContentParser parser = context.parser();
+    public static DerivativePipelineAggregationBuilder parse(String pipelineAggregatorName, XContentParser parser) throws IOException {
         XContentParser.Token token;
         String currentFieldName = null;
         String[] bucketsPaths = null;
@@ -212,7 +210,7 @@ public class DerivativePipelineAggregationBuilder extends AbstractPipelineAggreg
                 } else if (BUCKETS_PATH_FIELD.match(currentFieldName)) {
                     bucketsPaths = new String[] { parser.text() };
                 } else if (GAP_POLICY_FIELD.match(currentFieldName)) {
-                    gapPolicy = GapPolicy.parse(context, parser.text(), parser.getTokenLocation());
+                    gapPolicy = GapPolicy.parse(parser.text(), parser.getTokenLocation());
                 } else if (UNIT_FIELD.match(currentFieldName)) {
                     units = parser.text();
                 } else {

+ 2 - 4
core/src/main/java/org/elasticsearch/search/aggregations/pipeline/movavg/MovAvgPipelineAggregationBuilder.java

@@ -26,7 +26,6 @@ import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ParseFieldRegistry;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.aggregations.AggregatorFactory;
 import org.elasticsearch.search.aggregations.PipelineAggregationBuilder;
@@ -304,8 +303,7 @@ public class MovAvgPipelineAggregationBuilder extends AbstractPipelineAggregatio
 
     public static MovAvgPipelineAggregationBuilder parse(
             ParseFieldRegistry<MovAvgModel.AbstractModelParser> movingAverageMdelParserRegistry,
-            String pipelineAggregatorName, QueryParseContext context) throws IOException {
-        XContentParser parser = context.parser();
+            String pipelineAggregatorName, XContentParser parser) throws IOException {
         XContentParser.Token token;
         String currentFieldName = null;
         String[] bucketsPaths = null;
@@ -344,7 +342,7 @@ public class MovAvgPipelineAggregationBuilder extends AbstractPipelineAggregatio
                 } else if (BUCKETS_PATH.match(currentFieldName)) {
                     bucketsPaths = new String[] { parser.text() };
                 } else if (GAP_POLICY.match(currentFieldName)) {
-                    gapPolicy = GapPolicy.parse(context, parser.text(), parser.getTokenLocation());
+                    gapPolicy = GapPolicy.parse(parser.text(), parser.getTokenLocation());
                 } else if (MODEL.match(currentFieldName)) {
                     model = parser.text();
                 } else {

+ 2 - 4
core/src/main/java/org/elasticsearch/search/aggregations/pipeline/serialdiff/SerialDiffPipelineAggregationBuilder.java

@@ -25,7 +25,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.aggregations.pipeline.AbstractPipelineAggregationBuilder;
 import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
@@ -149,8 +148,7 @@ public class SerialDiffPipelineAggregationBuilder extends AbstractPipelineAggreg
         return builder;
     }
 
-    public static SerialDiffPipelineAggregationBuilder parse(String reducerName, QueryParseContext context) throws IOException {
-        XContentParser parser = context.parser();
+    public static SerialDiffPipelineAggregationBuilder parse(String reducerName, XContentParser parser) throws IOException {
         XContentParser.Token token;
         String currentFieldName = null;
         String[] bucketsPaths = null;
@@ -167,7 +165,7 @@ public class SerialDiffPipelineAggregationBuilder extends AbstractPipelineAggreg
                 } else if (BUCKETS_PATH.match(currentFieldName)) {
                     bucketsPaths = new String[] { parser.text() };
                 } else if (GAP_POLICY.match(currentFieldName)) {
-                    gapPolicy = GapPolicy.parse(context, parser.text(), parser.getTokenLocation());
+                    gapPolicy = GapPolicy.parse(parser.text(), parser.getTokenLocation());
                 } else {
                     throw new ParsingException(parser.getTokenLocation(),
                             "Unknown key for a " + token + " in [" + reducerName + "]: [" + currentFieldName + "].");

+ 5 - 6
core/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceParserHelper.java

@@ -23,7 +23,6 @@ import org.elasticsearch.common.ParseField;
 import org.elasticsearch.common.ParsingException;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.script.Script;
 import org.joda.time.DateTimeZone;
 
@@ -33,31 +32,31 @@ public final class ValuesSourceParserHelper {
     private ValuesSourceParserHelper() {} // utility class, no instantiation
 
     public static void declareAnyFields(
-            ObjectParser<? extends ValuesSourceAggregationBuilder<ValuesSource, ?>, QueryParseContext> objectParser,
+            ObjectParser<? extends ValuesSourceAggregationBuilder<ValuesSource, ?>, Void> objectParser,
             boolean scriptable, boolean formattable) {
         declareFields(objectParser, scriptable, formattable, false, null);
     }
 
     public static void declareNumericFields(
-            ObjectParser<? extends ValuesSourceAggregationBuilder<ValuesSource.Numeric, ?>, QueryParseContext> objectParser,
+            ObjectParser<? extends ValuesSourceAggregationBuilder<ValuesSource.Numeric, ?>, Void> objectParser,
             boolean scriptable, boolean formattable, boolean timezoneAware) {
         declareFields(objectParser, scriptable, formattable, timezoneAware, ValueType.NUMERIC);
     }
 
     public static void declareBytesFields(
-            ObjectParser<? extends ValuesSourceAggregationBuilder<ValuesSource.Bytes, ?>, QueryParseContext> objectParser,
+            ObjectParser<? extends ValuesSourceAggregationBuilder<ValuesSource.Bytes, ?>, Void> objectParser,
             boolean scriptable, boolean formattable) {
         declareFields(objectParser, scriptable, formattable, false, ValueType.STRING);
     }
 
     public static void declareGeoFields(
-            ObjectParser<? extends ValuesSourceAggregationBuilder<ValuesSource.GeoPoint, ?>, QueryParseContext> objectParser,
+            ObjectParser<? extends ValuesSourceAggregationBuilder<ValuesSource.GeoPoint, ?>, Void> objectParser,
             boolean scriptable, boolean formattable) {
         declareFields(objectParser, scriptable, formattable, false, ValueType.GEOPOINT);
     }
 
     private static <VS extends ValuesSource> void declareFields(
-            ObjectParser<? extends ValuesSourceAggregationBuilder<VS, ?>, QueryParseContext> objectParser,
+            ObjectParser<? extends ValuesSourceAggregationBuilder<VS, ?>, Void> objectParser,
             boolean scriptable, boolean formattable, boolean timezoneAware, ValueType targetValueType) {
 
 

+ 17 - 20
core/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java

@@ -36,7 +36,6 @@ import org.elasticsearch.common.xcontent.ToXContentObject;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.index.query.QueryShardContext;
 import org.elasticsearch.script.Script;
 import org.elasticsearch.search.SearchExtBuilder;
@@ -109,9 +108,9 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
     public static final ParseField SLICE = new ParseField("slice");
     public static final ParseField ALL_FIELDS_FIELDS = new ParseField("all_fields");
 
-    public static SearchSourceBuilder fromXContent(QueryParseContext context) throws IOException {
+    public static SearchSourceBuilder fromXContent(XContentParser parser) throws IOException {
         SearchSourceBuilder builder = new SearchSourceBuilder();
-        builder.parseXContent(context);
+        builder.parseXContent(parser);
         return builder;
     }
 
@@ -959,10 +958,9 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
     /**
      * Parse some xContent into this SearchSourceBuilder, overwriting any values specified in the xContent. Use this if you need to set up
      * different defaults than a regular SearchSourceBuilder would have and use
-     * {@link #fromXContent(QueryParseContext)} if you have normal defaults.
+     * {@link #fromXContent(XContentParser)} if you have normal defaults.
      */
-    public void parseXContent(QueryParseContext context) throws IOException {
-        XContentParser parser = context.parser();
+    public void parseXContent(XContentParser parser) throws IOException {
         XContentParser.Token token = parser.currentToken();
         String currentFieldName = null;
         if (token != XContentParser.Token.START_OBJECT && (token = parser.nextToken()) != XContentParser.Token.START_OBJECT) {
@@ -992,10 +990,10 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
                 } else if (TRACK_TOTAL_HITS_FIELD.match(currentFieldName)) {
                     trackTotalHits = parser.booleanValue();
                 } else if (_SOURCE_FIELD.match(currentFieldName)) {
-                    fetchSourceContext = FetchSourceContext.fromXContent(context.parser());
+                    fetchSourceContext = FetchSourceContext.fromXContent(parser);
                 } else if (STORED_FIELDS_FIELD.match(currentFieldName)) {
                     storedFieldsContext =
-                        StoredFieldsContext.fromXContent(SearchSourceBuilder.STORED_FIELDS_FIELD.getPreferredName(), context);
+                        StoredFieldsContext.fromXContent(SearchSourceBuilder.STORED_FIELDS_FIELD.getPreferredName(), parser);
                 } else if (SORT_FIELD.match(currentFieldName)) {
                     sort(parser.text());
                 } else if (PROFILE_FIELD.match(currentFieldName)) {
@@ -1014,7 +1012,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
                 } else if (POST_FILTER_FIELD.match(currentFieldName)) {
                     postQueryBuilder = parseInnerQueryBuilder(parser);
                 } else if (_SOURCE_FIELD.match(currentFieldName)) {
-                    fetchSourceContext = FetchSourceContext.fromXContent(context.parser());
+                    fetchSourceContext = FetchSourceContext.fromXContent(parser);
                 } else if (SCRIPT_FIELDS_FIELD.match(currentFieldName)) {
                     scriptFields = new ArrayList<>();
                     while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
@@ -1035,16 +1033,16 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
                     }
                 } else if (AGGREGATIONS_FIELD.match(currentFieldName)
                         || AGGS_FIELD.match(currentFieldName)) {
-                    aggregations = AggregatorFactories.parseAggregators(context);
+                    aggregations = AggregatorFactories.parseAggregators(parser);
                 } else if (HIGHLIGHT_FIELD.match(currentFieldName)) {
                     highlightBuilder = HighlightBuilder.fromXContent(parser);
                 } else if (SUGGEST_FIELD.match(currentFieldName)) {
-                    suggestBuilder = SuggestBuilder.fromXContent(context.parser());
+                    suggestBuilder = SuggestBuilder.fromXContent(parser);
                 } else if (SORT_FIELD.match(currentFieldName)) {
                     sorts = new ArrayList<>(SortBuilder.fromXContent(parser));
                 } else if (RESCORE_FIELD.match(currentFieldName)) {
                     rescoreBuilders = new ArrayList<>();
-                    rescoreBuilders.add(RescoreBuilder.parseFromXContent(context));
+                    rescoreBuilders.add(RescoreBuilder.parseFromXContent(parser));
                 } else if (EXT_FIELD.match(currentFieldName)) {
                     extBuilders = new ArrayList<>();
                     String extSectionName = null;
@@ -1062,16 +1060,16 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
                         }
                     }
                 } else if (SLICE.match(currentFieldName)) {
-                    sliceBuilder = SliceBuilder.fromXContent(context);
+                    sliceBuilder = SliceBuilder.fromXContent(parser);
                 } else if (COLLAPSE.match(currentFieldName)) {
-                    collapse = CollapseBuilder.fromXContent(context);
+                    collapse = CollapseBuilder.fromXContent(parser);
                 } else {
                     throw new ParsingException(parser.getTokenLocation(), "Unknown key for a " + token + " in [" + currentFieldName + "].",
                             parser.getTokenLocation());
                 }
             } else if (token == XContentParser.Token.START_ARRAY) {
                 if (STORED_FIELDS_FIELD.match(currentFieldName)) {
-                    storedFieldsContext = StoredFieldsContext.fromXContent(STORED_FIELDS_FIELD.getPreferredName(), context);
+                    storedFieldsContext = StoredFieldsContext.fromXContent(STORED_FIELDS_FIELD.getPreferredName(), parser);
                 } else if (DOCVALUE_FIELDS_FIELD.match(currentFieldName)) {
                     docValueFields = new ArrayList<>();
                     while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
@@ -1084,14 +1082,14 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
                     }
                 } else if (INDICES_BOOST_FIELD.match(currentFieldName)) {
                     while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
-                        indexBoosts.add(new IndexBoost(context));
+                        indexBoosts.add(new IndexBoost(parser));
                     }
                 } else if (SORT_FIELD.match(currentFieldName)) {
                     sorts = new ArrayList<>(SortBuilder.fromXContent(parser));
                 } else if (RESCORE_FIELD.match(currentFieldName)) {
                     rescoreBuilders = new ArrayList<>();
                     while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
-                        rescoreBuilders.add(RescoreBuilder.parseFromXContent(context));
+                        rescoreBuilders.add(RescoreBuilder.parseFromXContent(parser));
                     }
                 } else if (STATS_FIELD.match(currentFieldName)) {
                     stats = new ArrayList<>();
@@ -1104,7 +1102,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
                         }
                     }
                 } else if (_SOURCE_FIELD.match(currentFieldName)) {
-                    fetchSourceContext = FetchSourceContext.fromXContent(context.parser());
+                    fetchSourceContext = FetchSourceContext.fromXContent(parser);
                 } else if (SEARCH_AFTER.match(currentFieldName)) {
                     searchAfterBuilder = SearchAfterBuilder.fromXContent(parser);
                 } else if (FIELDS_FIELD.match(currentFieldName)) {
@@ -1276,8 +1274,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
             boost = in.readFloat();
         }
 
-        IndexBoost(QueryParseContext context) throws IOException {
-            XContentParser parser = context.parser();
+        IndexBoost(XContentParser parser) throws IOException {
             XContentParser.Token token = parser.currentToken();
 
             if (token == XContentParser.Token.START_OBJECT) {

+ 3 - 4
core/src/main/java/org/elasticsearch/search/collapse/CollapseBuilder.java

@@ -35,7 +35,6 @@ import org.elasticsearch.index.mapper.KeywordFieldMapper;
 import org.elasticsearch.index.mapper.MappedFieldType;
 import org.elasticsearch.index.mapper.NumberFieldMapper;
 import org.elasticsearch.index.query.InnerHitBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.SearchContextException;
 import org.elasticsearch.search.internal.SearchContext;
 
@@ -52,7 +51,7 @@ public class CollapseBuilder implements Writeable, ToXContentObject {
     public static final ParseField FIELD_FIELD = new ParseField("field");
     public static final ParseField INNER_HITS_FIELD = new ParseField("inner_hits");
     public static final ParseField MAX_CONCURRENT_GROUP_REQUESTS_FIELD = new ParseField("max_concurrent_group_searches");
-    private static final ObjectParser<CollapseBuilder, QueryParseContext> PARSER =
+    private static final ObjectParser<CollapseBuilder, Void> PARSER =
         new ObjectParser<>("collapse", CollapseBuilder::new);
 
     static {
@@ -122,8 +121,8 @@ public class CollapseBuilder implements Writeable, ToXContentObject {
        }
     }
 
-    public static CollapseBuilder fromXContent(QueryParseContext context) throws IOException {
-        CollapseBuilder builder = PARSER.parse(context.parser(), new CollapseBuilder(), context);
+    public static CollapseBuilder fromXContent(XContentParser parser) throws IOException {
+        CollapseBuilder builder = PARSER.parse(parser, new CollapseBuilder(), null);
         return builder;
     }
 

+ 1 - 3
core/src/main/java/org/elasticsearch/search/fetch/StoredFieldsContext.java

@@ -26,7 +26,6 @@ import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.rest.RestRequest;
 
 import java.io.IOException;
@@ -164,8 +163,7 @@ public class StoredFieldsContext implements Writeable {
         return new StoredFieldsContext(fieldNames);
     }
 
-    public static StoredFieldsContext fromXContent(String fieldName, QueryParseContext context) throws IOException {
-        XContentParser parser = context.parser();
+    public static StoredFieldsContext fromXContent(String fieldName, XContentParser parser) throws IOException {
         XContentParser.Token token = parser.currentToken();
 
         if (token == XContentParser.Token.VALUE_STRING) {

+ 3 - 4
core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/AbstractHighlighterBuilder.java

@@ -32,7 +32,6 @@ import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder.BoundaryScannerType;
 import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder.Order;
 
@@ -595,7 +594,7 @@ public abstract class AbstractHighlighterBuilder<HB extends AbstractHighlighterB
     }
 
     static <HB extends AbstractHighlighterBuilder<HB>> BiFunction<XContentParser, HB, HB> setupParser(
-            ObjectParser<HB, QueryParseContext> parser) {
+            ObjectParser<HB, Void> parser) {
         parser.declareStringArray(fromList(String.class, HB::preTags), PRE_TAGS_FIELD);
         parser.declareStringArray(fromList(String.class, HB::postTags), POST_TAGS_FIELD);
         parser.declareString(HB::order, ORDER_FIELD);
@@ -612,14 +611,14 @@ public abstract class AbstractHighlighterBuilder<HB extends AbstractHighlighterB
         parser.declareInt(HB::noMatchSize, NO_MATCH_SIZE_FIELD);
         parser.declareBoolean(HB::forceSource, FORCE_SOURCE_FIELD);
         parser.declareInt(HB::phraseLimit, PHRASE_LIMIT_FIELD);
-        parser.declareObject(HB::options, (XContentParser p, QueryParseContext c) -> {
+        parser.declareObject(HB::options, (XContentParser p, Void c) -> {
             try {
                 return p.map();
             } catch (IOException e) {
                 throw new RuntimeException("Error parsing options", e);
             }
         }, OPTIONS_FIELD);
-        parser.declareObject(HB::highlightQuery, (XContentParser p, QueryParseContext c) -> {
+        parser.declareObject(HB::highlightQuery, (XContentParser p, Void c) -> {
             try {
                 return parseInnerQueryBuilder(p);
             } catch (IOException e) {

+ 4 - 5
core/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilder.java

@@ -30,7 +30,6 @@ import org.elasticsearch.common.xcontent.ObjectParser.NamedObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.index.query.QueryShardContext;
 import org.elasticsearch.search.fetch.subphase.highlight.SearchContextHighlight.FieldOptions;
 
@@ -258,7 +257,7 @@ public class HighlightBuilder extends AbstractHighlighterBuilder<HighlightBuilde
 
     private static final BiFunction<XContentParser, HighlightBuilder, HighlightBuilder> PARSER;
     static {
-        ObjectParser<HighlightBuilder, QueryParseContext> parser = new ObjectParser<>("highlight");
+        ObjectParser<HighlightBuilder, Void> parser = new ObjectParser<>("highlight");
         parser.declareString(HighlightBuilder::tagsSchema, new ParseField("tags_schema"));
         parser.declareString(HighlightBuilder::encoder, ENCODER_FIELD);
         parser.declareNamedObjects(HighlightBuilder::fields, Field.PARSER, (HighlightBuilder hb) -> hb.useExplicitFieldOrder(true),
@@ -417,13 +416,13 @@ public class HighlightBuilder extends AbstractHighlighterBuilder<HighlightBuilde
     }
 
     public static class Field extends AbstractHighlighterBuilder<Field> {
-        static final NamedObjectParser<Field, QueryParseContext> PARSER;
+        static final NamedObjectParser<Field, Void> PARSER;
         static {
-            ObjectParser<Field, QueryParseContext> parser = new ObjectParser<>("highlight_field");
+            ObjectParser<Field, Void> parser = new ObjectParser<>("highlight_field");
             parser.declareInt(Field::fragmentOffset, FRAGMENT_OFFSET_FIELD);
             parser.declareStringArray(fromList(String.class, Field::matchedFields), MATCHED_FIELDS_FIELD);
             BiFunction<XContentParser, Field, Field> decoratedParser = setupParser(parser);
-            PARSER = (XContentParser p, QueryParseContext c, String name) -> decoratedParser.apply(p, new Field(name));
+            PARSER = (XContentParser p, Void c, String name) -> decoratedParser.apply(p, new Field(name));
         }
 
         private final String name;

+ 4 - 4
core/src/main/java/org/elasticsearch/search/rescore/QueryRescorerBuilder.java

@@ -25,8 +25,8 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.index.query.QueryShardContext;
 import org.elasticsearch.search.rescore.QueryRescorer.QueryRescoreContext;
 
@@ -53,7 +53,7 @@ public class QueryRescorerBuilder extends RescoreBuilder<QueryRescorerBuilder> {
     private static ParseField RESCORE_QUERY_WEIGHT_FIELD = new ParseField("rescore_query_weight");
     private static ParseField SCORE_MODE_FIELD = new ParseField("score_mode");
 
-    private static final ObjectParser<InnerBuilder, QueryParseContext> QUERY_RESCORE_PARSER = new ObjectParser<>(NAME, null);
+    private static final ObjectParser<InnerBuilder, Void> QUERY_RESCORE_PARSER = new ObjectParser<>(NAME, null);
 
     static {
         QUERY_RESCORE_PARSER.declareObject(InnerBuilder::setQueryBuilder, (p, c) -> {
@@ -161,8 +161,8 @@ public class QueryRescorerBuilder extends RescoreBuilder<QueryRescorerBuilder> {
         builder.endObject();
     }
 
-    public static QueryRescorerBuilder fromXContent(QueryParseContext parseContext) throws IOException {
-        InnerBuilder innerBuilder = QUERY_RESCORE_PARSER.parse(parseContext.parser(), new InnerBuilder(), parseContext);
+    public static QueryRescorerBuilder fromXContent(XContentParser parser) throws IOException {
+        InnerBuilder innerBuilder = QUERY_RESCORE_PARSER.parse(parser, new InnerBuilder(), null);
         return innerBuilder.build();
     }
 

+ 2 - 4
core/src/main/java/org/elasticsearch/search/rescore/RescoreBuilder.java

@@ -28,7 +28,6 @@ import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.index.query.QueryShardContext;
 import org.elasticsearch.search.rescore.QueryRescorer.QueryRescoreContext;
 
@@ -75,8 +74,7 @@ public abstract class RescoreBuilder<RB extends RescoreBuilder<RB>> extends ToXC
         return windowSize;
     }
 
-    public static RescoreBuilder<?> parseFromXContent(QueryParseContext parseContext) throws IOException {
-        XContentParser parser = parseContext.parser();
+    public static RescoreBuilder<?> parseFromXContent(XContentParser parser) throws IOException {
         String fieldName = null;
         RescoreBuilder<?> rescorer = null;
         Integer windowSize = null;
@@ -93,7 +91,7 @@ public abstract class RescoreBuilder<RB extends RescoreBuilder<RB>> extends ToXC
             } else if (token == XContentParser.Token.START_OBJECT) {
                 // we only have QueryRescorer at this point
                 if (QueryRescorerBuilder.NAME.equals(fieldName)) {
-                    rescorer = QueryRescorerBuilder.fromXContent(parseContext);
+                    rescorer = QueryRescorerBuilder.fromXContent(parser);
                 } else {
                     throw new ParsingException(parser.getTokenLocation(), "rescore doesn't support rescorer with name [" + fieldName + "]");
                 }

+ 4 - 5
core/src/main/java/org/elasticsearch/search/slice/SliceBuilder.java

@@ -30,13 +30,12 @@ import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.fielddata.IndexFieldData;
 import org.elasticsearch.index.fielddata.IndexNumericFieldData;
 import org.elasticsearch.index.mapper.IdFieldMapper;
 import org.elasticsearch.index.mapper.MappedFieldType;
-import org.elasticsearch.index.mapper.MapperService;
 import org.elasticsearch.index.mapper.UidFieldMapper;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.index.query.QueryShardContext;
 
 import java.io.IOException;
@@ -57,7 +56,7 @@ public class SliceBuilder extends ToXContentToBytes implements Writeable {
     public static final ParseField FIELD_FIELD = new ParseField("field");
     public static final ParseField ID_FIELD = new ParseField("id");
     public static final ParseField MAX_FIELD = new ParseField("max");
-    private static final ObjectParser<SliceBuilder, QueryParseContext> PARSER =
+    private static final ObjectParser<SliceBuilder, Void> PARSER =
         new ObjectParser<>("slice", SliceBuilder::new);
 
     static {
@@ -169,8 +168,8 @@ public class SliceBuilder extends ToXContentToBytes implements Writeable {
         builder.field(MAX_FIELD.getPreferredName(), max);
     }
 
-    public static SliceBuilder fromXContent(QueryParseContext context) throws IOException {
-        SliceBuilder builder = PARSER.parse(context.parser(), new SliceBuilder(), context);
+    public static SliceBuilder fromXContent(XContentParser parser) throws IOException {
+        SliceBuilder builder = PARSER.parse(parser, new SliceBuilder(), null);
         return builder;
     }
 

+ 2 - 3
core/src/main/java/org/elasticsearch/search/sort/FieldSortBuilder.java

@@ -32,7 +32,6 @@ import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.N
 import org.elasticsearch.index.fielddata.IndexNumericFieldData;
 import org.elasticsearch.index.mapper.MappedFieldType;
 import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.index.query.QueryShardContext;
 import org.elasticsearch.index.query.QueryShardException;
 import org.elasticsearch.search.DocValueFormat;
@@ -313,7 +312,7 @@ public class FieldSortBuilder extends SortBuilder<FieldSortBuilder> {
     }
 
     /**
-     * Creates a new {@link FieldSortBuilder} from the query held by the {@link QueryParseContext} in
+     * Creates a new {@link FieldSortBuilder} from the query held by the {@link XContentParser} in
      * {@link org.elasticsearch.common.xcontent.XContent} format.
      *
      * @param parser the input parser. The state on the parser contained in this context will be changed as a side effect of this
@@ -325,7 +324,7 @@ public class FieldSortBuilder extends SortBuilder<FieldSortBuilder> {
         return PARSER.parse(parser, new FieldSortBuilder(fieldName), null);
     }
 
-    private static ObjectParser<FieldSortBuilder, QueryParseContext> PARSER = new ObjectParser<>(NAME);
+    private static ObjectParser<FieldSortBuilder, Void> PARSER = new ObjectParser<>(NAME);
 
     static {
         PARSER.declareField(FieldSortBuilder::missing, p -> p.objectText(),  MISSING, ValueType.VALUE);

+ 1 - 2
core/src/main/java/org/elasticsearch/search/sort/GeoDistanceSortBuilder.java

@@ -48,7 +48,6 @@ import org.elasticsearch.index.fielddata.plain.AbstractLatLonPointDVIndexFieldDa
 import org.elasticsearch.index.mapper.MappedFieldType;
 import org.elasticsearch.index.query.GeoValidationMethod;
 import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.index.query.QueryShardContext;
 import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.MultiValueMode;
@@ -384,7 +383,7 @@ public class GeoDistanceSortBuilder extends SortBuilder<GeoDistanceSortBuilder>
     }
 
     /**
-     * Creates a new {@link GeoDistanceSortBuilder} from the query held by the {@link QueryParseContext} in
+     * Creates a new {@link GeoDistanceSortBuilder} from the query held by the {@link XContentParser} in
      * {@link org.elasticsearch.common.xcontent.XContent} format.
      *
      * @param parser the input parser. The state on the parser contained in this context will be changed as a

+ 3 - 4
core/src/main/java/org/elasticsearch/search/sort/ScoreSortBuilder.java

@@ -25,7 +25,6 @@ import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.index.query.QueryShardContext;
 import org.elasticsearch.search.DocValueFormat;
 
@@ -74,7 +73,7 @@ public class ScoreSortBuilder extends SortBuilder<ScoreSortBuilder> {
     }
 
     /**
-     * Creates a new {@link ScoreSortBuilder} from the query held by the {@link QueryParseContext} in
+     * Creates a new {@link ScoreSortBuilder} from the query held by the {@link XContentParser} in
      * {@link org.elasticsearch.common.xcontent.XContent} format.
      *
      * @param parser the input parser. The state on the parser contained in this context will be changed as a side effect of this
@@ -82,11 +81,11 @@ public class ScoreSortBuilder extends SortBuilder<ScoreSortBuilder> {
      * @param fieldName in some sort syntax variations the field name precedes the xContent object that specifies further parameters, e.g.
      *        in '{ "foo": { "order" : "asc"} }'. When parsing the inner object, the field name can be passed in via this argument
      */
-    public static ScoreSortBuilder fromXContent(XContentParser parser, String fieldName) throws IOException {
+    public static ScoreSortBuilder fromXContent(XContentParser parser, String fieldName) {
         return PARSER.apply(parser, null);
     }
 
-    private static ObjectParser<ScoreSortBuilder, QueryParseContext> PARSER = new ObjectParser<>(NAME, ScoreSortBuilder::new);
+    private static ObjectParser<ScoreSortBuilder, Void> PARSER = new ObjectParser<>(NAME, ScoreSortBuilder::new);
 
     static {
         PARSER.declareString((builder, order) -> builder.order(SortOrder.fromString(order)), ORDER_FIELD);

+ 3 - 4
core/src/main/java/org/elasticsearch/search/sort/ScriptSortBuilder.java

@@ -43,7 +43,6 @@ import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
 import org.elasticsearch.index.fielddata.fieldcomparator.BytesRefFieldComparatorSource;
 import org.elasticsearch.index.fielddata.fieldcomparator.DoubleValuesComparatorSource;
 import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.index.query.QueryShardContext;
 import org.elasticsearch.index.query.QueryShardException;
 import org.elasticsearch.script.Script;
@@ -212,7 +211,7 @@ public class ScriptSortBuilder extends SortBuilder<ScriptSortBuilder> {
         return builder;
     }
 
-    private static ConstructingObjectParser<ScriptSortBuilder, QueryParseContext> PARSER = new ConstructingObjectParser<>(NAME,
+    private static ConstructingObjectParser<ScriptSortBuilder, Void> PARSER = new ConstructingObjectParser<>(NAME,
             a -> new ScriptSortBuilder((Script) a[0], (ScriptSortType) a[1]));
 
     static {
@@ -226,7 +225,7 @@ public class ScriptSortBuilder extends SortBuilder<ScriptSortBuilder> {
     }
 
     /**
-     * Creates a new {@link ScriptSortBuilder} from the query held by the {@link QueryParseContext} in
+     * Creates a new {@link ScriptSortBuilder} from the query held by the {@link XContentParser} in
      * {@link org.elasticsearch.common.xcontent.XContent} format.
      *
      * @param parser the input parser. The state on the parser contained in this context will be changed as a side effect of this
@@ -234,7 +233,7 @@ public class ScriptSortBuilder extends SortBuilder<ScriptSortBuilder> {
      * @param elementName in some sort syntax variations the field name precedes the xContent object that specifies further parameters, e.g.
      *        in '{ "foo": { "order" : "asc"} }'. When parsing the inner object, the field name can be passed in via this argument
      */
-    public static ScriptSortBuilder fromXContent(XContentParser parser, String elementName) throws IOException {
+    public static ScriptSortBuilder fromXContent(XContentParser parser, String elementName) {
         return PARSER.apply(parser, null);
     }
 

+ 1 - 1
core/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestionBuilder.java

@@ -276,7 +276,7 @@ public class CompletionSuggestionBuilder extends SuggestionBuilder<CompletionSug
                             if (currentToken == XContentParser.Token.FIELD_NAME) {
                                 currentFieldName = contextParser.currentName();
                                 final ContextMapping mapping = contextMappings.get(currentFieldName);
-                                queryContexts.put(currentFieldName, mapping.parseQueryContext(context.newParseContext(contextParser)));
+                                queryContexts.put(currentFieldName, mapping.parseQueryContext(contextParser));
                             }
                         }
                         suggestionContext.setQueryContexts(queryContexts);

+ 2 - 3
core/src/main/java/org/elasticsearch/search/suggest/completion/context/CategoryContextMapping.java

@@ -28,7 +28,6 @@ import org.elasticsearch.common.xcontent.XContentParser.Token;
 import org.elasticsearch.index.mapper.KeywordFieldMapper;
 import org.elasticsearch.index.mapper.ParseContext;
 import org.elasticsearch.index.mapper.ParseContext.Document;
-import org.elasticsearch.index.query.QueryParseContext;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -148,8 +147,8 @@ public class CategoryContextMapping extends ContextMapping<CategoryQueryContext>
     }
 
     @Override
-    protected CategoryQueryContext fromXContent(QueryParseContext context) throws IOException {
-        return CategoryQueryContext.fromXContent(context);
+    protected CategoryQueryContext fromXContent(XContentParser parser) throws IOException {
+        return CategoryQueryContext.fromXContent(parser);
     }
 
     /**

+ 2 - 4
core/src/main/java/org/elasticsearch/search/suggest/completion/context/CategoryQueryContext.java

@@ -26,7 +26,6 @@ import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 
 import java.io.IOException;
 import java.util.Objects;
@@ -105,8 +104,7 @@ public final class CategoryQueryContext implements ToXContent {
         CATEGORY_PARSER.declareBoolean(Builder::setPrefix, new ParseField(CONTEXT_PREFIX));
     }
 
-    public static CategoryQueryContext fromXContent(QueryParseContext context) throws IOException {
-        XContentParser parser = context.parser();
+    public static CategoryQueryContext fromXContent(XContentParser parser) throws IOException {
         XContentParser.Token token = parser.currentToken();
         Builder builder = builder();
         if (token == XContentParser.Token.START_OBJECT) {
@@ -114,7 +112,7 @@ public final class CategoryQueryContext implements ToXContent {
                 CATEGORY_PARSER.parse(parser, builder, null);
             } catch(ParsingException e) {
                 throw new ElasticsearchParseException("category context must be a string, number or boolean");
-            } 
+            }
         } else if (token == XContentParser.Token.VALUE_STRING || token == XContentParser.Token.VALUE_BOOLEAN
                 || token == XContentParser.Token.VALUE_NUMBER) {
             builder.setCategory(parser.text());

+ 5 - 7
core/src/main/java/org/elasticsearch/search/suggest/completion/context/ContextMapping.java

@@ -27,7 +27,6 @@ import org.elasticsearch.common.xcontent.XContentParser.Token;
 import org.elasticsearch.common.xcontent.json.JsonXContent;
 import org.elasticsearch.index.mapper.CompletionFieldMapper;
 import org.elasticsearch.index.mapper.ParseContext;
-import org.elasticsearch.index.query.QueryParseContext;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -100,23 +99,22 @@ public abstract class ContextMapping<T extends ToXContent> implements ToXContent
     /**
      * Prototype for the query context
      */
-    protected abstract T fromXContent(QueryParseContext context) throws IOException;
+    protected abstract T fromXContent(XContentParser context) throws IOException;
 
     /**
      * Parses query contexts for this mapper
      */
-    public final List<InternalQueryContext> parseQueryContext(QueryParseContext context) throws IOException, ElasticsearchParseException {
+    public final List<InternalQueryContext> parseQueryContext(XContentParser parser) throws IOException, ElasticsearchParseException {
         List<T> queryContexts = new ArrayList<>();
-        XContentParser parser = context.parser();
         Token token = parser.nextToken();
         if (token == Token.START_ARRAY) {
             while (parser.nextToken() != Token.END_ARRAY) {
-                queryContexts.add(fromXContent(context));
+                queryContexts.add(fromXContent(parser));
             }
         } else {
-            queryContexts.add(fromXContent(context));
+            queryContexts.add(fromXContent(parser));
         }
-        
+
         return toInternalQueryContexts(queryContexts);
     }
 

+ 3 - 4
core/src/main/java/org/elasticsearch/search/suggest/completion/context/GeoContextMapping.java

@@ -29,11 +29,10 @@ import org.elasticsearch.common.unit.DistanceUnit;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.XContentParser.Token;
-import org.elasticsearch.index.mapper.GeoPointFieldMapper;
 import org.elasticsearch.index.mapper.FieldMapper;
+import org.elasticsearch.index.mapper.GeoPointFieldMapper;
 import org.elasticsearch.index.mapper.ParseContext;
 import org.elasticsearch.index.mapper.ParseContext.Document;
-import org.elasticsearch.index.query.QueryParseContext;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -225,8 +224,8 @@ public class GeoContextMapping extends ContextMapping<GeoQueryContext> {
     }
 
     @Override
-    protected GeoQueryContext fromXContent(QueryParseContext context) throws IOException {
-        return GeoQueryContext.fromXContent(context);
+    protected GeoQueryContext fromXContent(XContentParser parser) throws IOException {
+        return GeoQueryContext.fromXContent(parser);
     }
 
     /**

+ 1 - 3
core/src/main/java/org/elasticsearch/search/suggest/completion/context/GeoQueryContext.java

@@ -27,7 +27,6 @@ import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.query.QueryParseContext;
 
 import java.io.IOException;
 import java.util.Collections;
@@ -124,8 +123,7 @@ public final class GeoQueryContext implements ToXContent {
         GEO_CONTEXT_PARSER.declareDouble(GeoQueryContext.Builder::setLon, new ParseField("lon"));
     }
 
-    public static GeoQueryContext fromXContent(QueryParseContext context) throws IOException {
-        XContentParser parser = context.parser();
+    public static GeoQueryContext fromXContent(XContentParser parser) throws IOException {
         XContentParser.Token token = parser.currentToken();
         GeoQueryContext.Builder builder = new Builder();
         if (token == XContentParser.Token.START_OBJECT) {

+ 3 - 20
core/src/test/java/org/elasticsearch/index/query/QueryParseContextTests.java → core/src/test/java/org/elasticsearch/index/query/AbstractQueryBuilderTests.java

@@ -20,17 +20,13 @@
 package org.elasticsearch.index.query;
 
 import org.elasticsearch.common.ParsingException;
-import org.elasticsearch.common.logging.DeprecationLogger;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.util.concurrent.ThreadContext;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.json.JsonXContent;
 import org.elasticsearch.search.SearchModule;
 import org.elasticsearch.test.ESTestCase;
-import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Before;
 import org.junit.BeforeClass;
 
 import java.io.IOException;
@@ -38,7 +34,8 @@ import java.io.IOException;
 import static java.util.Collections.emptyList;
 import static org.elasticsearch.index.query.AbstractQueryBuilder.parseInnerQueryBuilder;
 
-public class QueryParseContextTests extends ESTestCase {
+public class AbstractQueryBuilderTests extends ESTestCase {
+
     private static NamedXContentRegistry xContentRegistry;
 
     @BeforeClass
@@ -51,20 +48,6 @@ public class QueryParseContextTests extends ESTestCase {
         xContentRegistry = null;
     }
 
-    private ThreadContext threadContext;
-
-    @Before
-    public void beforeTest() throws IOException {
-        this.threadContext = new ThreadContext(Settings.EMPTY);
-        DeprecationLogger.setThreadContext(threadContext);
-    }
-
-    @After
-    public void teardown() throws IOException {
-        DeprecationLogger.removeThreadContext(this.threadContext);
-        this.threadContext.close();
-    }
-
     public void testParseInnerQueryBuilder() throws IOException {
         QueryBuilder query = new MatchQueryBuilder("foo", "bar");
         String source = query.toString();
@@ -91,7 +74,6 @@ public class QueryParseContextTests extends ESTestCase {
 
         source = "{ \"foo\" : \"bar\" }";
         try (XContentParser parser = createParser(JsonXContent.jsonXContent, source)) {
-            QueryParseContext context = new QueryParseContext(parser);
             ParsingException exception = expectThrows(ParsingException.class, () ->  parseInnerQueryBuilder(parser));
             assertEquals("[foo] query malformed, no start_object after query name", exception.getMessage());
         }
@@ -107,4 +89,5 @@ public class QueryParseContextTests extends ESTestCase {
     protected NamedXContentRegistry xContentRegistry() {
         return xContentRegistry;
     }
+
 }

+ 9 - 9
core/src/test/java/org/elasticsearch/index/query/RangeQueryBuilderTests.java

@@ -224,7 +224,7 @@ public class RangeQueryBuilderTests extends AbstractQueryTestCase<RangeQueryBuil
     /**
      * Specifying a timezone together with a numeric range query should throw an exception.
      */
-    public void testToQueryNonDateWithTimezone() throws QueryShardException, IOException {
+    public void testToQueryNonDateWithTimezone() throws QueryShardException {
         RangeQueryBuilder query = new RangeQueryBuilder(INT_FIELD_NAME);
         query.from(1).to(10).timeZone("UTC");
         QueryShardException e = expectThrows(QueryShardException.class, () -> query.toQuery(createShardContext()));
@@ -234,7 +234,7 @@ public class RangeQueryBuilderTests extends AbstractQueryTestCase<RangeQueryBuil
     /**
      * Specifying a timezone together with an unmapped field should throw an exception.
      */
-    public void testToQueryUnmappedWithTimezone() throws QueryShardException, IOException {
+    public void testToQueryUnmappedWithTimezone() throws QueryShardException {
         RangeQueryBuilder query = new RangeQueryBuilder("bogus_field");
         query.from(1).to(10).timeZone("UTC");
         QueryShardException e = expectThrows(QueryShardException.class, () -> query.toQuery(createShardContext()));
@@ -410,7 +410,7 @@ public class RangeQueryBuilderTests extends AbstractQueryTestCase<RangeQueryBuil
         String fieldName = randomAlphaOfLengthBetween(1, 20);
         RangeQueryBuilder query = new RangeQueryBuilder(fieldName) {
             @Override
-            protected MappedFieldType.Relation getRelation(QueryRewriteContext queryRewriteContext) throws IOException {
+            protected MappedFieldType.Relation getRelation(QueryRewriteContext queryRewriteContext) {
                 return Relation.WITHIN;
             }
         };
@@ -431,7 +431,7 @@ public class RangeQueryBuilderTests extends AbstractQueryTestCase<RangeQueryBuil
         String fieldName = randomAlphaOfLengthBetween(1, 20);
         RangeQueryBuilder query = new RangeQueryBuilder(fieldName) {
             @Override
-            protected MappedFieldType.Relation getRelation(QueryRewriteContext queryRewriteContext) throws IOException {
+            protected MappedFieldType.Relation getRelation(QueryRewriteContext queryRewriteContext) {
                 return Relation.WITHIN;
             }
         };
@@ -456,7 +456,7 @@ public class RangeQueryBuilderTests extends AbstractQueryTestCase<RangeQueryBuil
         String fieldName = randomAlphaOfLengthBetween(1, 20);
         RangeQueryBuilder query = new RangeQueryBuilder(fieldName) {
             @Override
-            protected MappedFieldType.Relation getRelation(QueryRewriteContext queryRewriteContext) throws IOException {
+            protected MappedFieldType.Relation getRelation(QueryRewriteContext queryRewriteContext) {
                 return Relation.DISJOINT;
             }
         };
@@ -473,7 +473,7 @@ public class RangeQueryBuilderTests extends AbstractQueryTestCase<RangeQueryBuil
         String fieldName = randomAlphaOfLengthBetween(1, 20);
         RangeQueryBuilder query = new RangeQueryBuilder(fieldName) {
             @Override
-            protected MappedFieldType.Relation getRelation(QueryRewriteContext queryRewriteContext) throws IOException {
+            protected MappedFieldType.Relation getRelation(QueryRewriteContext queryRewriteContext) {
                 return Relation.INTERSECTS;
             }
         };
@@ -490,7 +490,7 @@ public class RangeQueryBuilderTests extends AbstractQueryTestCase<RangeQueryBuil
         String fieldName = randomAlphaOfLengthBetween(1, 20);
         RangeQueryBuilder query = new RangeQueryBuilder(fieldName) {
             @Override
-            protected MappedFieldType.Relation getRelation(QueryRewriteContext queryRewriteContext) throws IOException {
+            protected MappedFieldType.Relation getRelation(QueryRewriteContext queryRewriteContext) {
                 return Relation.INTERSECTS;
             }
         };
@@ -499,7 +499,7 @@ public class RangeQueryBuilderTests extends AbstractQueryTestCase<RangeQueryBuil
         assertThat(rewritten, sameInstance(query));
     }
 
-    public void testParseFailsWithMultipleFields() throws IOException {
+    public void testParseFailsWithMultipleFields() {
         String json =
                 "{\n" +
                 "    \"range\": {\n" +
@@ -517,7 +517,7 @@ public class RangeQueryBuilderTests extends AbstractQueryTestCase<RangeQueryBuil
         assertEquals("[range] query doesn't support multiple fields, found [age] and [price]", e.getMessage());
     }
 
-    public void testParseFailsWithMultipleFieldsWhenOneIsDate() throws IOException {
+    public void testParseFailsWithMultipleFieldsWhenOneIsDate() {
         String json =
                 "{\n" +
                 "    \"range\": {\n" +

+ 9 - 3
core/src/test/java/org/elasticsearch/search/SearchModuleTests.java

@@ -24,8 +24,8 @@ import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.index.query.TermQueryBuilder;
 import org.elasticsearch.index.query.functionscore.GaussDecayFunctionBuilder;
 import org.elasticsearch.plugins.SearchPlugin;
@@ -91,6 +91,7 @@ public class SearchModuleTests extends ModuleTestCase {
                 () -> new SearchModule(Settings.EMPTY, false, singletonList(registersDupeHighlighter)));
 
         SearchPlugin registersDupeSuggester = new SearchPlugin() {
+            @Override
             public List<SearchPlugin.SuggesterSpec<?>> getSuggesters() {
                 return singletonList(new SuggesterSpec<>("term", TermSuggestionBuilder::new, TermSuggestionBuilder::fromXContent));
             }
@@ -136,6 +137,7 @@ public class SearchModuleTests extends ModuleTestCase {
                 singletonList(registersDupeFetchSubPhase)));
 
         SearchPlugin registersDupeQuery = new SearchPlugin() {
+            @Override
             public List<SearchPlugin.QuerySpec<?>> getQueries() {
                 return singletonList(new QuerySpec<>(TermQueryBuilder.NAME, TermQueryBuilder::new, TermQueryBuilder::fromXContent));
             }
@@ -144,6 +146,7 @@ public class SearchModuleTests extends ModuleTestCase {
                 new SearchModule(Settings.EMPTY, false, singletonList(registersDupeQuery)).getNamedXContents()));
 
         SearchPlugin registersDupeAggregation = new SearchPlugin() {
+            @Override
             public List<AggregationSpec> getAggregations() {
                 return singletonList(new AggregationSpec(TermsAggregationBuilder.NAME, TermsAggregationBuilder::new,
                         TermsAggregationBuilder::parse));
@@ -153,6 +156,7 @@ public class SearchModuleTests extends ModuleTestCase {
                 singletonList(registersDupeAggregation)).getNamedXContents()));
 
         SearchPlugin registersDupePipelineAggregation = new SearchPlugin() {
+            @Override
             public List<PipelineAggregationSpec> getPipelineAggregations() {
                 return singletonList(new PipelineAggregationSpec(
                         DerivativePipelineAggregationBuilder.NAME,
@@ -229,6 +233,7 @@ public class SearchModuleTests extends ModuleTestCase {
 
     public void testRegisterAggregation() {
         SearchModule module = new SearchModule(Settings.EMPTY, false, singletonList(new SearchPlugin() {
+            @Override
             public List<AggregationSpec> getAggregations() {
                 return singletonList(new AggregationSpec("test", TestAggregationBuilder::new, TestAggregationBuilder::fromXContent));
             }
@@ -243,6 +248,7 @@ public class SearchModuleTests extends ModuleTestCase {
 
     public void testRegisterPipelineAggregation() {
         SearchModule module = new SearchModule(Settings.EMPTY, false, singletonList(new SearchPlugin() {
+            @Override
             public List<PipelineAggregationSpec> getPipelineAggregations() {
                 return singletonList(new PipelineAggregationSpec("test",
                         TestPipelineAggregationBuilder::new, TestPipelineAggregator::new, TestPipelineAggregationBuilder::fromXContent));
@@ -344,7 +350,7 @@ public class SearchModuleTests extends ModuleTestCase {
             return false;
         }
 
-        private static TestAggregationBuilder fromXContent(String name, QueryParseContext c) {
+        private static TestAggregationBuilder fromXContent(String name, XContentParser p) {
             return null;
         }
     }
@@ -389,7 +395,7 @@ public class SearchModuleTests extends ModuleTestCase {
             return false;
         }
 
-        private static TestPipelineAggregationBuilder fromXContent(String name, QueryParseContext c) {
+        private static TestPipelineAggregationBuilder fromXContent(String name, XContentParser p) {
             return null;
         }
     }

+ 2 - 1
core/src/test/java/org/elasticsearch/search/SearchServiceTests.java

@@ -19,6 +19,7 @@
 package org.elasticsearch.search;
 
 import com.carrotsearch.hppc.IntArrayList;
+
 import org.apache.lucene.search.Query;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.elasticsearch.action.ActionListener;
@@ -272,7 +273,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
         }
 
         @Override
-        protected QueryBuilder doRewrite(QueryRewriteContext queryShardContext) throws IOException {
+        protected QueryBuilder doRewrite(QueryRewriteContext queryShardContext) {
             throw new IllegalStateException("Fail on rewrite phase");
         }
 

+ 1 - 3
core/src/test/java/org/elasticsearch/search/aggregations/AggregationCollectorTests.java

@@ -22,7 +22,6 @@ package org.elasticsearch.search.aggregations;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.json.JsonXContent;
 import org.elasticsearch.index.IndexService;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.internal.SearchContext;
 import org.elasticsearch.test.ESSingleNodeTestCase;
 
@@ -58,10 +57,9 @@ public class AggregationCollectorTests extends ESSingleNodeTestCase {
 
     private boolean needsScores(IndexService index, String agg) throws IOException {
         XContentParser aggParser = createParser(JsonXContent.jsonXContent, agg);
-        QueryParseContext parseContext = new QueryParseContext(aggParser);
         aggParser.nextToken();
         SearchContext context = createSearchContext(index);
-        final AggregatorFactories factories = AggregatorFactories.parseAggregators(parseContext).build(context, null);
+        final AggregatorFactories factories = AggregatorFactories.parseAggregators(aggParser).build(context, null);
         final Aggregator[] aggregators = factories.createTopLevelAggregators();
         assertEquals(1, aggregators.length);
         return aggregators[0].needsScores();

+ 6 - 14
core/src/test/java/org/elasticsearch/search/aggregations/AggregatorFactoriesTests.java

@@ -26,8 +26,6 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.json.JsonXContent;
 import org.elasticsearch.env.Environment;
-import org.elasticsearch.index.query.QueryParseContext;
-import org.elasticsearch.script.ScriptService;
 import org.elasticsearch.search.SearchModule;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilders;
 import org.elasticsearch.test.AbstractQueryTestCase;
@@ -96,9 +94,8 @@ public class AggregatorFactoriesTests extends ESTestCase {
                     .endObject()
                 .endObject();
         XContentParser parser = createParser(source);
-        QueryParseContext parseContext = new QueryParseContext(parser);
         assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
-        Exception e = expectThrows(ParsingException.class, () -> AggregatorFactories.parseAggregators(parseContext));
+        Exception e = expectThrows(ParsingException.class, () -> AggregatorFactories.parseAggregators(parser));
         assertThat(e.toString(), containsString("Found two aggregation type definitions in [in_stock]: [filter] and [terms]"));
     }
 
@@ -129,9 +126,8 @@ public class AggregatorFactoriesTests extends ESTestCase {
                     .endObject()
                 .endObject();
         XContentParser parser = createParser(source);
-        QueryParseContext parseContext = new QueryParseContext(parser);
         assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
-        Exception e = expectThrows(ParsingException.class, () -> AggregatorFactories.parseAggregators(parseContext));
+        Exception e = expectThrows(ParsingException.class, () -> AggregatorFactories.parseAggregators(parser));
         assertThat(e.toString(), containsString("Found two sub aggregation definitions under [by_date]"));
     }
 
@@ -164,9 +160,8 @@ public class AggregatorFactoriesTests extends ESTestCase {
                     .endObject()
                 .endObject();
         XContentParser parser = createParser(source);
-        QueryParseContext parseContext = new QueryParseContext(parser);
         assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
-        Exception e = expectThrows(ParsingException.class, () -> AggregatorFactories.parseAggregators(parseContext));
+        Exception e = expectThrows(ParsingException.class, () -> AggregatorFactories.parseAggregators(parser));
         assertThat(e.toString(), containsString("Invalid aggregation name [" + name + "]"));
     }
 
@@ -188,9 +183,8 @@ public class AggregatorFactoriesTests extends ESTestCase {
                     .endObject()
                 .endObject();
         XContentParser parser = createParser(source);
-        QueryParseContext parseContext = new QueryParseContext(parser);
         assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
-        Exception e = expectThrows(ParsingException.class, () -> AggregatorFactories.parseAggregators(parseContext));
+        Exception e = expectThrows(ParsingException.class, () -> AggregatorFactories.parseAggregators(parser));
         assertThat(e.toString(), containsString("Two sibling aggregations cannot have the same name: [" + name + "]"));
     }
 
@@ -213,9 +207,8 @@ public class AggregatorFactoriesTests extends ESTestCase {
                     .endObject()
                 .endObject();
         XContentParser parser = createParser(source);
-        QueryParseContext parseContext = new QueryParseContext(parser);
         assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
-        Exception e = expectThrows(ParsingException.class, () -> AggregatorFactories.parseAggregators(parseContext));
+        Exception e = expectThrows(ParsingException.class, () -> AggregatorFactories.parseAggregators(parser));
         assertThat(e.toString(), containsString("Expected [START_OBJECT] under [field], but got a [VALUE_STRING] in [cardinality]"));
     }
 
@@ -238,9 +231,8 @@ public class AggregatorFactoriesTests extends ESTestCase {
                     .endObject()
                 .endObject();
         XContentParser parser = createParser(source);
-        QueryParseContext parseContext = new QueryParseContext(parser);
         assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
-        Exception e = expectThrows(ParsingException.class, () -> AggregatorFactories.parseAggregators(parseContext));
+        Exception e = expectThrows(ParsingException.class, () -> AggregatorFactories.parseAggregators(parser));
         assertThat(e.toString(), containsString("Expected [START_OBJECT] under [field], but got a [VALUE_STRING] in [tag_count]"));
     }
 

+ 1 - 3
core/src/test/java/org/elasticsearch/search/aggregations/BasePipelineAggregationTestCase.java

@@ -32,7 +32,6 @@ import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.XContentType;
 import org.elasticsearch.env.Environment;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.indices.IndicesModule;
 import org.elasticsearch.search.SearchModule;
 import org.elasticsearch.search.aggregations.pipeline.AbstractPipelineAggregationBuilder;
@@ -117,9 +116,8 @@ public abstract class BasePipelineAggregationTestCase<AF extends AbstractPipelin
     }
 
     protected PipelineAggregationBuilder parse(XContentParser parser) throws IOException {
-        QueryParseContext parseContext = new QueryParseContext(parser);
         assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
-        AggregatorFactories.Builder parsed = AggregatorFactories.parseAggregators(parseContext);
+        AggregatorFactories.Builder parsed = AggregatorFactories.parseAggregators(parser);
         assertThat(parsed.getAggregatorFactories(), hasSize(0));
         assertThat(parsed.getPipelineAggregatorFactories(), hasSize(1));
         PipelineAggregationBuilder newAgg = parsed.getPipelineAggregatorFactories().get(0);

+ 2 - 2
core/src/test/java/org/elasticsearch/search/aggregations/InternalOrderTests.java

@@ -67,12 +67,12 @@ public class InternalOrderTests extends AbstractSerializingTestCase<BucketOrder>
     protected BucketOrder doParseInstance(XContentParser parser) throws IOException {
         Token token = parser.nextToken();
         if (token == Token.START_OBJECT) {
-            return InternalOrder.Parser.parseOrderParam(parser, null);
+            return InternalOrder.Parser.parseOrderParam(parser);
         }
         if (token == Token.START_ARRAY) {
             List<BucketOrder> orders = new ArrayList<>();
             while (parser.nextToken() == Token.START_OBJECT) {
-                orders.add(InternalOrder.Parser.parseOrderParam(parser, null));
+                orders.add(InternalOrder.Parser.parseOrderParam(parser));
             }
             return BucketOrder.compound(orders);
         }

+ 40 - 40
core/src/test/java/org/elasticsearch/search/aggregations/bucket/SignificantTermsSignificanceScoreIT.java

@@ -27,9 +27,9 @@ import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentFactory;
+import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.XContentType;
 import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.index.query.QueryShardException;
 import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.plugins.SearchPlugin;
@@ -116,7 +116,7 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
                                     .significanceHeuristic(new SimpleHeuristic())
                                     .minDocCount(1)
                             )
-                    );                        
+                    );
         }else
         {
             request = client().prepareSearch(INDEX_NAME).setTypes(DOC_TYPE)
@@ -128,9 +128,9 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
                                     .significanceHeuristic(new SimpleHeuristic())
                                     .minDocCount(1)
                             )
-                    );            
+                    );
         }
-        
+
         SearchResponse response = request.execute().actionGet();
         assertSearchResponse(response);
         StringTerms classes = response.getAggregations().get("class");
@@ -177,7 +177,7 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
         @Override
         public List<SearchExtensionSpec<SignificanceHeuristic, SignificanceHeuristicParser>> getSignificanceHeuristics() {
             return singletonList(new SearchExtensionSpec<SignificanceHeuristic, SignificanceHeuristicParser>(SimpleHeuristic.NAME,
-                    SimpleHeuristic::new, (context) -> SimpleHeuristic.parse(context)));
+                    SimpleHeuristic::new, (parser) -> SimpleHeuristic.parse(parser)));
         }
 
         @Override
@@ -257,9 +257,9 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
             return subsetFreq / subsetSize > supersetFreq / supersetSize ? 2.0 : 1.0;
         }
 
-        public static SignificanceHeuristic parse(QueryParseContext context)
+        public static SignificanceHeuristic parse(XContentParser parser)
                 throws IOException, QueryShardException {
-            context.parser().nextToken();
+            parser.nextToken();
             return new SimpleHeuristic();
         }
     }
@@ -268,23 +268,23 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
         String type = randomBoolean() ? "text" : "long";
         String settings = "{\"index.number_of_shards\": 1, \"index.number_of_replicas\": 0}";
         SharedSignificantTermsTestMethods.index01Docs(type, settings, this);
-        
+
         SearchRequestBuilder request;
         if ("text".equals(type) && randomBoolean() ) {
             // Use significant_text on text fields but occasionally run with alternative of
             // significant_terms on legacy fieldData=true too.
             request = client().prepareSearch(INDEX_NAME).setTypes(DOC_TYPE)
                     .addAggregation(terms("class").field(CLASS_FIELD)
-                            .subAggregation(significantText("sig_terms", TEXT_FIELD)));            
+                            .subAggregation(significantText("sig_terms", TEXT_FIELD)));
         } else {
             request = client().prepareSearch(INDEX_NAME).setTypes(DOC_TYPE)
                     .addAggregation(terms("class").field(CLASS_FIELD)
-                            .subAggregation(significantTerms("sig_terms").field(TEXT_FIELD)));            
+                            .subAggregation(significantTerms("sig_terms").field(TEXT_FIELD)));
         }
-        
+
         SearchResponse response = request.execute().actionGet();
-        
-        
+
+
         assertSearchResponse(response);
         StringTerms classes = response.getAggregations().get("class");
         assertThat(classes.getBuckets().size(), equalTo(2));
@@ -368,8 +368,8 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
             indexRequestBuilderList.add(client().prepareIndex(INDEX_NAME, DOC_TYPE, "1").setSource(TEXT_FIELD, text, CLASS_FIELD, "1"));
         }
         indexRandom(true, false, indexRequestBuilderList);
-        
-        
+
+
         SearchRequestBuilder request;
         if (randomBoolean() ) {
             request = client().prepareSearch(INDEX_NAME).setTypes(DOC_TYPE)
@@ -379,7 +379,7 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
                         .subAggregation(
                                 significantTerms("sig_terms")
                                         .field(TEXT_FIELD)
-                                        .minDocCount(1)));            
+                                        .minDocCount(1)));
         }else
         {
             request = client().prepareSearch(INDEX_NAME).setTypes(DOC_TYPE)
@@ -388,9 +388,9 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
                             .field(CLASS_FIELD)
                             .subAggregation(
                                     significantText("sig_terms", TEXT_FIELD)
-                                            .minDocCount(1)));            
+                                            .minDocCount(1)));
         }
-        
+
         request.execute().actionGet();
 
     }
@@ -409,7 +409,7 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
     // 2. filter buckets and set the background to the other class and set is_background false
     // both should yield exact same result
     public void testBackgroundVsSeparateSet(SignificanceHeuristic significanceHeuristicExpectingSuperset,
-                                            SignificanceHeuristic significanceHeuristicExpectingSeparateSets, 
+                                            SignificanceHeuristic significanceHeuristicExpectingSeparateSets,
                                             String type) throws Exception {
 
         final boolean useSigText = randomBoolean() && type.equals("text");
@@ -422,7 +422,7 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
                                     significantText("sig_terms", TEXT_FIELD)
                                             .minDocCount(1)
                                             .significanceHeuristic(
-                                                    significanceHeuristicExpectingSuperset)));            
+                                                    significanceHeuristicExpectingSuperset)));
         }else
         {
             request1 = client().prepareSearch(INDEX_NAME).setTypes(DOC_TYPE)
@@ -433,12 +433,12 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
                                             .field(TEXT_FIELD)
                                             .minDocCount(1)
                                             .significanceHeuristic(
-                                                    significanceHeuristicExpectingSuperset)));                        
+                                                    significanceHeuristicExpectingSuperset)));
         }
-        
+
         SearchResponse response1 = request1.execute().actionGet();
         assertSearchResponse(response1);
-        
+
         SearchRequestBuilder request2;
         if (useSigText) {
             request2 = client().prepareSearch(INDEX_NAME).setTypes(DOC_TYPE)
@@ -451,7 +451,7 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
                             .subAggregation(significantText("sig_terms", TEXT_FIELD)
                                     .minDocCount(1)
                                     .backgroundFilter(QueryBuilders.termQuery(CLASS_FIELD, "0"))
-                                    .significanceHeuristic(significanceHeuristicExpectingSeparateSets)));            
+                                    .significanceHeuristic(significanceHeuristicExpectingSeparateSets)));
         }else
         {
             request2 = client().prepareSearch(INDEX_NAME).setTypes(DOC_TYPE)
@@ -466,9 +466,9 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
                                     .field(TEXT_FIELD)
                                     .minDocCount(1)
                                     .backgroundFilter(QueryBuilders.termQuery(CLASS_FIELD, "0"))
-                                    .significanceHeuristic(significanceHeuristicExpectingSeparateSets)));                        
+                                    .significanceHeuristic(significanceHeuristicExpectingSeparateSets)));
         }
-        
+
         SearchResponse response2 = request2.execute().actionGet();
 
         StringTerms classes = response1.getAggregations().get("class");
@@ -513,17 +513,17 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
                             .field("text")
                             .executionHint(randomExecutionHint())
                             .significanceHeuristic(heuristic)
-                            .minDocCount(1).shardSize(1000).size(1000)));            
+                            .minDocCount(1).shardSize(1000).size(1000)));
         }else
         {
             request = client().prepareSearch("test")
                     .addAggregation(terms("class").field("class").subAggregation(significantText("mySignificantTerms", "text")
                             .significanceHeuristic(heuristic)
-                            .minDocCount(1).shardSize(1000).size(1000)));                        
-        }        
+                            .minDocCount(1).shardSize(1000).size(1000)));
+        }
         SearchResponse response = request.execute().actionGet();
-        assertSearchResponse(response);        
-        
+        assertSearchResponse(response);
+
         assertSearchResponse(response);
         StringTerms classes = response.getAggregations().get("class");
         assertThat(classes.getBuckets().size(), equalTo(2));
@@ -580,14 +580,14 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
         String type = randomBoolean() ? "text" : "long";
         indexRandomFrequencies01(type);
         ScriptHeuristic scriptHeuristic = getScriptSignificanceHeuristic();
-        
+
         SearchRequestBuilder request;
         if ("text".equals(type) && randomBoolean()) {
             request = client().prepareSearch(INDEX_NAME)
                     .addAggregation(terms("class").field(CLASS_FIELD)
                             .subAggregation(significantText("mySignificantTerms", TEXT_FIELD)
                             .significanceHeuristic(scriptHeuristic)
-                            .minDocCount(1).shardSize(2).size(2)));            
+                            .minDocCount(1).shardSize(2).size(2)));
         }else
         {
             request = client().prepareSearch(INDEX_NAME)
@@ -596,10 +596,10 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
                             .field(TEXT_FIELD)
                             .executionHint(randomExecutionHint())
                             .significanceHeuristic(scriptHeuristic)
-                            .minDocCount(1).shardSize(2).size(2)));                        
-        }        
+                            .minDocCount(1).shardSize(2).size(2)));
+        }
         SearchResponse response = request.execute().actionGet();
-        assertSearchResponse(response);           
+        assertSearchResponse(response);
         for (Terms.Bucket classBucket : ((Terms) response.getAggregations().get("class")).getBuckets()) {
             SignificantTerms sigTerms = classBucket.getAggregations().get("mySignificantTerms");
             for (SignificantTerms.Bucket bucket : sigTerms.getBuckets()) {
@@ -666,8 +666,8 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
 
         // Test that a request using a script does not get cached
         ScriptHeuristic scriptHeuristic = getScriptSignificanceHeuristic();
-        boolean useSigText = randomBoolean();        
-        SearchResponse r; 
+        boolean useSigText = randomBoolean();
+        SearchResponse r;
         if (useSigText) {
             r = client().prepareSearch("cache_test_idx").setSize(0)
                     .addAggregation(significantText("foo", "s").significanceHeuristic(scriptHeuristic)).get();
@@ -696,7 +696,7 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
         assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
                 .getMissCount(), equalTo(1L));
     }
-    
-    
+
+
 
 }

+ 7 - 13
core/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridParserTests.java

@@ -21,7 +21,6 @@ package org.elasticsearch.search.aggregations.bucket.geogrid;
 import org.elasticsearch.common.ParsingException;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.json.JsonXContent;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.test.ESTestCase;
 
 import static org.hamcrest.Matchers.instanceOf;
@@ -29,33 +28,30 @@ import static org.hamcrest.Matchers.instanceOf;
 public class GeoHashGridParserTests extends ESTestCase {
     public void testParseValidFromInts() throws Exception {
         int precision = randomIntBetween(1, 12);
-        XContentParser stParser = createParser(JsonXContent.jsonXContent, 
+        XContentParser stParser = createParser(JsonXContent.jsonXContent,
                 "{\"field\":\"my_loc\", \"precision\":" + precision + ", \"size\": 500, \"shard_size\": 550}");
-        QueryParseContext parseContext = new QueryParseContext(stParser);
         XContentParser.Token token = stParser.nextToken();
         assertSame(XContentParser.Token.START_OBJECT, token);
         // can create a factory
-        assertNotNull(GeoGridAggregationBuilder.parse("geohash_grid", parseContext));
+        assertNotNull(GeoGridAggregationBuilder.parse("geohash_grid", stParser));
     }
 
     public void testParseValidFromStrings() throws Exception {
         int precision = randomIntBetween(1, 12);
-        XContentParser stParser = createParser(JsonXContent.jsonXContent, 
+        XContentParser stParser = createParser(JsonXContent.jsonXContent,
                 "{\"field\":\"my_loc\", \"precision\":\"" + precision + "\", \"size\": \"500\", \"shard_size\": \"550\"}");
-        QueryParseContext parseContext = new QueryParseContext(stParser);
         XContentParser.Token token = stParser.nextToken();
         assertSame(XContentParser.Token.START_OBJECT, token);
         // can create a factory
-        assertNotNull(GeoGridAggregationBuilder.parse("geohash_grid", parseContext));
+        assertNotNull(GeoGridAggregationBuilder.parse("geohash_grid", stParser));
     }
 
     public void testParseErrorOnNonIntPrecision() throws Exception {
         XContentParser stParser = createParser(JsonXContent.jsonXContent, "{\"field\":\"my_loc\", \"precision\":\"2.0\"}");
-        QueryParseContext parseContext = new QueryParseContext(stParser);
         XContentParser.Token token = stParser.nextToken();
         assertSame(XContentParser.Token.START_OBJECT, token);
         try {
-            GeoGridAggregationBuilder.parse("geohash_grid", parseContext);
+            GeoGridAggregationBuilder.parse("geohash_grid", stParser);
             fail();
         } catch (ParsingException ex) {
             assertThat(ex.getCause(), instanceOf(NumberFormatException.class));
@@ -65,11 +61,10 @@ public class GeoHashGridParserTests extends ESTestCase {
 
     public void testParseErrorOnBooleanPrecision() throws Exception {
         XContentParser stParser = createParser(JsonXContent.jsonXContent, "{\"field\":\"my_loc\", \"precision\":false}");
-        QueryParseContext parseContext = new QueryParseContext(stParser);
         XContentParser.Token token = stParser.nextToken();
         assertSame(XContentParser.Token.START_OBJECT, token);
         try {
-            GeoGridAggregationBuilder.parse("geohash_grid", parseContext);
+            GeoGridAggregationBuilder.parse("geohash_grid", stParser);
             fail();
         } catch (IllegalArgumentException ex) {
             assertEquals("[geohash_grid] precision doesn't support values of type: VALUE_BOOLEAN", ex.getMessage());
@@ -78,11 +73,10 @@ public class GeoHashGridParserTests extends ESTestCase {
 
     public void testParseErrorOnPrecisionOutOfRange() throws Exception {
         XContentParser stParser = createParser(JsonXContent.jsonXContent, "{\"field\":\"my_loc\", \"precision\":\"13\"}");
-        QueryParseContext parseContext = new QueryParseContext(stParser);
         XContentParser.Token token = stParser.nextToken();
         assertSame(XContentParser.Token.START_OBJECT, token);
         try {
-            GeoGridAggregationBuilder.parse("geohash_grid", parseContext);
+            GeoGridAggregationBuilder.parse("geohash_grid", stParser);
             fail();
         } catch (ParsingException ex) {
             assertThat(ex.getCause(), instanceOf(IllegalArgumentException.class));

+ 3 - 6
core/src/test/java/org/elasticsearch/search/aggregations/bucket/significant/SignificanceHeuristicTests.java

@@ -33,7 +33,6 @@ import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.json.JsonXContent;
 import org.elasticsearch.index.Index;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.DocValueFormat;
 import org.elasticsearch.search.SearchModule;
 import org.elasticsearch.search.SearchShardTarget;
@@ -267,9 +266,8 @@ public class SignificanceHeuristicTests extends ESTestCase {
         try {
             XContentParser stParser = createParser(JsonXContent.jsonXContent,
                     "{\"field\":\"text\", " + faultyHeuristicDefinition + ",\"min_doc_count\":200}");
-            QueryParseContext parseContext = new QueryParseContext(stParser);
             stParser.nextToken();
-            SignificantTermsAggregationBuilder.getParser(significanceHeuristicParserRegistry).parse("testagg", parseContext);
+            SignificantTermsAggregationBuilder.getParser(significanceHeuristicParserRegistry).parse("testagg", stParser);
             fail();
         } catch (ParsingException e) {
             assertThat(e.getCause().getMessage(), containsString(expectedError));
@@ -286,14 +284,13 @@ public class SignificanceHeuristicTests extends ESTestCase {
         return parseSignificanceHeuristic(significanceHeuristicParserRegistry, stParser);
     }
 
-    private SignificanceHeuristic parseSignificanceHeuristic(
+    private static SignificanceHeuristic parseSignificanceHeuristic(
             ParseFieldRegistry<SignificanceHeuristicParser> significanceHeuristicParserRegistry,
             XContentParser stParser) throws IOException {
-        QueryParseContext parseContext = new QueryParseContext(stParser);
         stParser.nextToken();
         SignificantTermsAggregationBuilder aggregatorFactory =
                 (SignificantTermsAggregationBuilder) SignificantTermsAggregationBuilder.getParser(
-                significanceHeuristicParserRegistry).parse("testagg", parseContext);
+                significanceHeuristicParserRegistry).parse("testagg", stParser);
         stParser.nextToken();
         assertThat(aggregatorFactory.getBucketCountThresholds().getMinDocCount(), equalTo(200L));
         assertThat(stParser.currentToken(), equalTo(null));

+ 3 - 7
core/src/test/java/org/elasticsearch/search/aggregations/metrics/FiltersTests.java

@@ -26,7 +26,6 @@ import org.elasticsearch.common.xcontent.XContentType;
 import org.elasticsearch.index.query.MatchNoneQueryBuilder;
 import org.elasticsearch.index.query.QueryBuilder;
 import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.aggregations.BaseAggregationTestCase;
 import org.elasticsearch.search.aggregations.bucket.filters.FiltersAggregationBuilder;
 import org.elasticsearch.search.aggregations.bucket.filters.FiltersAggregator.KeyedFilter;
@@ -87,8 +86,7 @@ public class FiltersTests extends BaseAggregationTestCase<FiltersAggregationBuil
         builder.endObject();
         XContentParser parser = createParser(shuffleXContent(builder));
         parser.nextToken();
-        QueryParseContext context = new QueryParseContext(parser);
-        FiltersAggregationBuilder filters = FiltersAggregationBuilder.parse("agg_name", context);
+        FiltersAggregationBuilder filters = FiltersAggregationBuilder.parse("agg_name", parser);
         // The other bucket is disabled by default
         assertFalse(filters.otherBucket());
 
@@ -99,8 +97,7 @@ public class FiltersTests extends BaseAggregationTestCase<FiltersAggregationBuil
         builder.endObject();
         parser = createParser(shuffleXContent(builder));
         parser.nextToken();
-        context = new QueryParseContext(parser);
-        filters = FiltersAggregationBuilder.parse("agg_name", context);
+        filters = FiltersAggregationBuilder.parse("agg_name", parser);
         // but setting a key enables it automatically
         assertTrue(filters.otherBucket());
 
@@ -112,8 +109,7 @@ public class FiltersTests extends BaseAggregationTestCase<FiltersAggregationBuil
         builder.endObject();
         parser = createParser(shuffleXContent(builder));
         parser.nextToken();
-        context = new QueryParseContext(parser);
-        filters = FiltersAggregationBuilder.parse("agg_name", context);
+        filters = FiltersAggregationBuilder.parse("agg_name", parser);
         // unless the other bucket is explicitly disabled
         assertFalse(filters.otherBucket());
     }

+ 1 - 4
core/src/test/java/org/elasticsearch/search/aggregations/metrics/TopHitsTests.java

@@ -21,8 +21,6 @@ package org.elasticsearch.search.aggregations.metrics;
 
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.json.JsonXContent;
-import org.elasticsearch.index.query.QueryParseContext;
-import org.elasticsearch.script.Script;
 import org.elasticsearch.search.aggregations.AggregationInitializationException;
 import org.elasticsearch.search.aggregations.AggregatorFactories;
 import org.elasticsearch.search.aggregations.BaseAggregationTestCase;
@@ -189,9 +187,8 @@ public class TopHitsTests extends BaseAggregationTestCase<TopHitsAggregationBuil
             "    }\n" +
             "}";
         XContentParser parser = createParser(JsonXContent.jsonXContent, source);
-        QueryParseContext parseContext = new QueryParseContext(parser);
         assertSame(XContentParser.Token.START_OBJECT, parser.nextToken());
-        Exception e = expectThrows(AggregationInitializationException.class, () -> AggregatorFactories.parseAggregators(parseContext));
+        Exception e = expectThrows(AggregationInitializationException.class, () -> AggregatorFactories.parseAggregators(parser));
         assertThat(e.toString(), containsString("Aggregator [top_tags_hits] of type [top_hits] cannot accept sub-aggregations"));
     }
 

Some files were not shown because too many files changed in this diff