Browse Source

Replace SearchContextException with SearchException (#46965)

This commit removes the SearchContextException in favor of a simpler
SearchException that doesn't leak the SearchContext.

Relates #46523
Jim Ferenczi 6 years ago
parent
commit
7daa1be8d2
26 changed files with 99 additions and 122 deletions
  1. 2 2
      modules/rank-eval/src/test/java/org/elasticsearch/index/rankeval/RankEvalResponseTests.java
  2. 6 2
      server/src/main/java/org/elasticsearch/ElasticsearchException.java
  3. 8 0
      server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java
  4. 1 1
      server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java
  5. 0 41
      server/src/main/java/org/elasticsearch/search/SearchContextException.java
  6. 5 6
      server/src/main/java/org/elasticsearch/search/SearchParseException.java
  7. 15 14
      server/src/main/java/org/elasticsearch/search/SearchService.java
  8. 1 1
      server/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java
  9. 3 2
      server/src/main/java/org/elasticsearch/search/aggregations/AggregatorBase.java
  10. 1 1
      server/src/main/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricAggregatorFactory.java
  11. 1 1
      server/src/main/java/org/elasticsearch/search/dfs/DfsPhase.java
  12. 7 7
      server/src/main/java/org/elasticsearch/search/dfs/DfsPhaseExecutionException.java
  13. 5 4
      server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java
  14. 7 7
      server/src/main/java/org/elasticsearch/search/fetch/FetchPhaseExecutionException.java
  15. 2 1
      server/src/main/java/org/elasticsearch/search/fetch/subphase/ExplainFetchSubPhase.java
  16. 1 1
      server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/FastVectorHighlighter.java
  17. 4 2
      server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/PlainHighlighter.java
  18. 1 1
      server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/UnifiedHighlighter.java
  19. 2 2
      server/src/main/java/org/elasticsearch/search/query/QueryPhase.java
  20. 7 7
      server/src/main/java/org/elasticsearch/search/query/QueryPhaseExecutionException.java
  21. 3 3
      server/src/test/java/org/elasticsearch/ElasticsearchExceptionTests.java
  22. 3 6
      server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java
  23. 3 3
      server/src/test/java/org/elasticsearch/search/searchafter/SearchAfterIT.java
  24. 2 2
      server/src/test/java/org/elasticsearch/search/slice/SearchSliceIT.java
  25. 5 5
      server/src/test/java/org/elasticsearch/search/source/MetadataFetchingIT.java
  26. 4 0
      test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java

+ 2 - 2
modules/rank-eval/src/test/java/org/elasticsearch/index/rankeval/RankEvalResponseTests.java

@@ -42,7 +42,6 @@ import org.elasticsearch.search.SearchHit;
 import org.elasticsearch.search.SearchParseException;
 import org.elasticsearch.search.SearchShardTarget;
 import org.elasticsearch.test.ESTestCase;
-import org.elasticsearch.test.TestSearchContext;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -56,6 +55,7 @@ import java.util.function.Predicate;
 
 import static java.util.Collections.singleton;
 import static org.elasticsearch.common.xcontent.XContentHelper.toXContent;
+import static org.elasticsearch.test.TestSearchContext.SHARD_TARGET;
 import static org.elasticsearch.test.XContentTestUtils.insertRandomFields;
 import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertToXContentEquivalent;
 import static org.hamcrest.Matchers.instanceOf;
@@ -65,7 +65,7 @@ public class RankEvalResponseTests extends ESTestCase {
     private static final Exception[] RANDOM_EXCEPTIONS = new Exception[] {
             new ClusterBlockException(singleton(NoMasterBlockService.NO_MASTER_BLOCK_WRITES)),
             new CircuitBreakingException("Data too large", 123, 456, CircuitBreaker.Durability.PERMANENT),
-            new SearchParseException(new TestSearchContext(null), "Parse failure", new XContentLocation(12, 98)),
+            new SearchParseException(SHARD_TARGET, "Parse failure", new XContentLocation(12, 98)),
             new IllegalArgumentException("Closed resource", new RuntimeException("Resource")),
             new SearchPhaseExecutionException("search", "all shards failed",
                     new ShardSearchFailure[] { new ShardSearchFailure(new ParsingException(1, 2, "foobar", null),

+ 6 - 2
server/src/main/java/org/elasticsearch/ElasticsearchException.java

@@ -36,6 +36,7 @@ import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.rest.RestStatus;
+import org.elasticsearch.search.SearchException;
 import org.elasticsearch.search.aggregations.MultiBucketConsumerService;
 import org.elasticsearch.transport.TcpTransport;
 
@@ -283,6 +284,10 @@ public class ElasticsearchException extends RuntimeException implements ToXConte
     public static ElasticsearchException readException(StreamInput input, int id) throws IOException {
         CheckedFunction<StreamInput, ? extends ElasticsearchException, IOException> elasticsearchException = ID_TO_SUPPLIER.get(id);
         if (elasticsearchException == null) {
+            if (id == 127 && input.getVersion().before(Version.V_7_5_0)) {
+                // was SearchContextException
+                return new SearchException(input);
+            }
             throw new IllegalStateException("unknown exception for id: " + id);
         }
         return elasticsearchException.apply(input);
@@ -965,8 +970,7 @@ public class ElasticsearchException extends RuntimeException implements ToXConte
                 TcpTransport.HttpRequestOnTransportException::new, 125, UNKNOWN_VERSION_ADDED),
         MAPPER_PARSING_EXCEPTION(org.elasticsearch.index.mapper.MapperParsingException.class,
                 org.elasticsearch.index.mapper.MapperParsingException::new, 126, UNKNOWN_VERSION_ADDED),
-        SEARCH_CONTEXT_EXCEPTION(org.elasticsearch.search.SearchContextException.class,
-                org.elasticsearch.search.SearchContextException::new, 127, UNKNOWN_VERSION_ADDED),
+        // 127 used to be org.elasticsearch.search.SearchContextException
         SEARCH_SOURCE_BUILDER_EXCEPTION(org.elasticsearch.search.builder.SearchSourceBuilderException.class,
                 org.elasticsearch.search.builder.SearchSourceBuilderException::new, 128, UNKNOWN_VERSION_ADDED),
         // 129 was EngineClosedException

+ 8 - 0
server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java

@@ -445,4 +445,12 @@ public class QueryShardContext extends QueryRewriteContext {
     public BigArrays bigArrays() {
         return bigArrays;
     }
+
+    public SimilarityService getSimilarityService() {
+        return similarityService;
+    }
+
+    public BitsetFilterCache getBitsetFilterCache() {
+        return bitsetFilterCache;
+    }
 }

+ 1 - 1
server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java

@@ -260,7 +260,7 @@ final class DefaultSearchContext extends SearchContext {
             try {
                 this.query = searcher.rewrite(query);
             } catch (IOException e) {
-                throw new QueryPhaseExecutionException(this, "Failed to rewrite main query", e);
+                throw new QueryPhaseExecutionException(shardTarget, "Failed to rewrite main query", e);
             }
         }
     }

+ 0 - 41
server/src/main/java/org/elasticsearch/search/SearchContextException.java

@@ -1,41 +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.search;
-
-import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.search.internal.SearchContext;
-
-import java.io.IOException;
-
-public class SearchContextException extends SearchException {
-
-    public SearchContextException(SearchContext context, String msg) {
-        super(context.shardTarget(), msg);
-    }
-
-    public SearchContextException(SearchContext context, String msg, Throwable t) {
-        super(context.shardTarget(), msg, t);
-    }
-
-    public SearchContextException(StreamInput in) throws IOException {
-        super(in);
-    }
-
-}

+ 5 - 6
server/src/main/java/org/elasticsearch/search/SearchParseException.java

@@ -25,22 +25,21 @@ import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentLocation;
 import org.elasticsearch.rest.RestStatus;
-import org.elasticsearch.search.internal.SearchContext;
 
 import java.io.IOException;
 
-public class SearchParseException extends SearchContextException {
+public class SearchParseException extends SearchException {
 
     public static final int UNKNOWN_POSITION = -1;
     private final int lineNumber;
     private final int columnNumber;
 
-    public SearchParseException(SearchContext context, String msg, @Nullable XContentLocation location) {
-        this(context, msg, location, null);
+    public SearchParseException(SearchShardTarget shardTarget, String msg, @Nullable XContentLocation location) {
+        this(shardTarget, msg, location, null);
     }
 
-    public SearchParseException(SearchContext context, String msg, @Nullable XContentLocation location, Throwable cause) {
-        super(context, msg, cause);
+    public SearchParseException(SearchShardTarget shardTarget, String msg, @Nullable XContentLocation location, Throwable cause) {
+        super(shardTarget, msg, cause);
         int lineNumber = UNKNOWN_POSITION;
         int columnNumber = UNKNOWN_POSITION;
         if (location != null) {

+ 15 - 14
server/src/main/java/org/elasticsearch/search/SearchService.java

@@ -730,11 +730,12 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
         }
     }
 
-    private void parseSource(DefaultSearchContext context, SearchSourceBuilder source) throws SearchContextException {
+    private void parseSource(DefaultSearchContext context, SearchSourceBuilder source) throws SearchException {
         // nothing to parse...
         if (source == null) {
             return;
         }
+        SearchShardTarget shardTarget = context.shardTarget();
         QueryShardContext queryShardContext = context.getQueryShardContext();
         context.from(source.from());
         context.size(source.size());
@@ -760,14 +761,14 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
                     context.sort(optionalSort.get());
                 }
             } catch (IOException e) {
-                throw new SearchContextException(context, "failed to create sort elements", e);
+                throw new SearchException(shardTarget, "failed to create sort elements", e);
             }
         }
         context.trackScores(source.trackScores());
         if (source.trackTotalHitsUpTo() != null
                 && source.trackTotalHitsUpTo() != SearchContext.TRACK_TOTAL_HITS_ACCURATE
                 && context.scrollContext() != null) {
-            throw new SearchContextException(context, "disabling [track_total_hits] is not allowed in a scroll context");
+            throw new SearchException(shardTarget, "disabling [track_total_hits] is not allowed in a scroll context");
         }
         if (source.trackTotalHitsUpTo() != null) {
             context.trackTotalHitsUpTo(source.trackTotalHitsUpTo());
@@ -794,7 +795,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
             try {
                 context.suggest(source.suggest().build(queryShardContext));
             } catch (IOException e) {
-                throw new SearchContextException(context, "failed to create SuggestionSearchContext", e);
+                throw new SearchException(shardTarget, "failed to create SuggestionSearchContext", e);
             }
         }
         if (source.rescores() != null) {
@@ -803,7 +804,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
                     context.addRescore(rescore.buildContext(queryShardContext));
                 }
             } catch (IOException e) {
-                throw new SearchContextException(context, "failed to create RescoreSearchContext", e);
+                throw new SearchException(shardTarget, "failed to create RescoreSearchContext", e);
             }
         }
         if (source.explain() != null) {
@@ -834,7 +835,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
             try {
                 context.highlight(highlightBuilder.build(queryShardContext));
             } catch (IOException e) {
-                throw new SearchContextException(context, "failed to create SearchContextHighlighter", e);
+                throw new SearchException(shardTarget, "failed to create SearchContextHighlighter", e);
             }
         }
         if (source.scriptFields() != null && source.size() != 0) {
@@ -869,10 +870,10 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
         }
         if (source.searchAfter() != null && source.searchAfter().length > 0) {
             if (context.scrollContext() != null) {
-                throw new SearchContextException(context, "`search_after` cannot be used in a scroll context.");
+                throw new SearchException(shardTarget, "`search_after` cannot be used in a scroll context.");
             }
             if (context.from() > 0) {
-                throw new SearchContextException(context, "`from` parameter must be set to 0 when `search_after` is used.");
+                throw new SearchException(shardTarget, "`from` parameter must be set to 0 when `search_after` is used.");
             }
             FieldDoc fieldDoc = SearchAfterBuilder.buildFieldDoc(context.sort(), source.searchAfter());
             context.searchAfter(fieldDoc);
@@ -880,7 +881,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
 
         if (source.slice() != null) {
             if (context.scrollContext() == null) {
-                throw new SearchContextException(context, "`slice` cannot be used outside of a scroll context");
+                throw new SearchException(shardTarget, "`slice` cannot be used outside of a scroll context");
             }
             context.sliceBuilder(source.slice());
         }
@@ -888,10 +889,10 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
         if (source.storedFields() != null) {
             if (source.storedFields().fetchFields() == false) {
                 if (context.version()) {
-                    throw new SearchContextException(context, "`stored_fields` cannot be disabled if version is requested");
+                    throw new SearchException(shardTarget, "`stored_fields` cannot be disabled if version is requested");
                 }
                 if (context.sourceRequested()) {
-                    throw new SearchContextException(context, "`stored_fields` cannot be disabled if _source is requested");
+                    throw new SearchException(shardTarget, "`stored_fields` cannot be disabled if _source is requested");
                 }
             }
             context.storedFieldsContext(source.storedFields());
@@ -899,13 +900,13 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
 
         if (source.collapse() != null) {
             if (context.scrollContext() != null) {
-                throw new SearchContextException(context, "cannot use `collapse` in a scroll context");
+                throw new SearchException(shardTarget, "cannot use `collapse` in a scroll context");
             }
             if (context.searchAfter() != null) {
-                throw new SearchContextException(context, "cannot use `collapse` in conjunction with `search_after`");
+                throw new SearchException(shardTarget, "cannot use `collapse` in conjunction with `search_after`");
             }
             if (context.rescore() != null && context.rescore().isEmpty() == false) {
-                throw new SearchContextException(context, "cannot use `collapse` in conjunction with `rescore`");
+                throw new SearchException(shardTarget, "cannot use `collapse` in conjunction with `rescore`");
             }
             final CollapseContext collapseContext = source.collapse().build(queryShardContext);
             context.collapse(collapseContext);

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

@@ -116,7 +116,7 @@ public class AggregationPhase implements SearchPhase {
                 globalsCollector.preCollection();
                 context.searcher().search(query, collector);
             } catch (Exception e) {
-                throw new QueryPhaseExecutionException(context, "Failed to execute global aggregators", e);
+                throw new QueryPhaseExecutionException(context.shardTarget(), "Failed to execute global aggregators", e);
             } finally {
                 context.clearReleasables(SearchContext.Lifetime.COLLECTION);
             }

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

@@ -23,6 +23,7 @@ import org.apache.lucene.search.ScoreMode;
 import org.elasticsearch.common.breaker.CircuitBreaker;
 import org.elasticsearch.common.breaker.CircuitBreakingException;
 import org.elasticsearch.indices.breaker.CircuitBreakerService;
+import org.elasticsearch.search.SearchShardTarget;
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
 import org.elasticsearch.search.internal.SearchContext;
 import org.elasticsearch.search.internal.SearchContext.Lifetime;
@@ -76,11 +77,11 @@ public abstract class AggregatorBase extends Aggregator {
         assert factories != null : "sub-factories provided to BucketAggregator must not be null, use AggragatorFactories.EMPTY instead";
         this.subAggregators = factories.createSubAggregators(context, this);
         context.addReleasable(this, Lifetime.PHASE);
+        final SearchShardTarget shardTarget = context.shardTarget();
         // Register a safeguard to highlight any invalid construction logic (call to this constructor without subsequent preCollection call)
         collectableSubAggregators = new BucketCollector() {
             void badState(){
-                throw new QueryPhaseExecutionException(AggregatorBase.this.context,
-                        "preCollection not called on new Aggregator before use", null);
+                throw new QueryPhaseExecutionException(shardTarget, "preCollection not called on new Aggregator before use", null);
             }
             @Override
             public LeafBucketCollector getLeafCollector(LeafReaderContext reader) {

+ 1 - 1
server/src/main/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricAggregatorFactory.java

@@ -134,7 +134,7 @@ class ScriptedMetricAggregatorFactory extends AggregatorFactory {
             || original instanceof Boolean) {
             clone = original;
         } else {
-            throw new SearchParseException(context,
+            throw new SearchParseException(context.shardTarget(),
                 "Can only clone primitives, String, ArrayList, and HashMap. Found: " + original.getClass().getCanonicalName(), null);
         }
         return clone;

+ 1 - 1
server/src/main/java/org/elasticsearch/search/dfs/DfsPhase.java

@@ -95,7 +95,7 @@ public class DfsPhase implements SearchPhase {
                     .fieldStatistics(fieldStatistics)
                     .maxDoc(context.searcher().getIndexReader().maxDoc());
         } catch (Exception e) {
-            throw new DfsPhaseExecutionException(context, "Exception during dfs phase", e);
+            throw new DfsPhaseExecutionException(context.shardTarget(), "Exception during dfs phase", e);
         }
     }
 

+ 7 - 7
server/src/main/java/org/elasticsearch/search/dfs/DfsPhaseExecutionException.java

@@ -20,19 +20,19 @@
 package org.elasticsearch.search.dfs;
 
 import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.search.SearchContextException;
-import org.elasticsearch.search.internal.SearchContext;
+import org.elasticsearch.search.SearchException;
+import org.elasticsearch.search.SearchShardTarget;
 
 import java.io.IOException;
 
-public class DfsPhaseExecutionException extends SearchContextException {
+public class DfsPhaseExecutionException extends SearchException {
 
-    public DfsPhaseExecutionException(SearchContext context, String msg, Throwable t) {
-        super(context, "Dfs Failed [" + msg + "]", t);
+    public DfsPhaseExecutionException(SearchShardTarget shardTarget, String msg, Throwable t) {
+        super(shardTarget, "Dfs Failed [" + msg + "]", t);
     }
 
-    public DfsPhaseExecutionException(SearchContext context, String msg) {
-        super(context, "Dfs Failed [" + msg + "]");
+    public DfsPhaseExecutionException(SearchShardTarget shardTarget, String msg) {
+        super(shardTarget, "Dfs Failed [" + msg + "]");
     }
 
     public DfsPhaseExecutionException(StreamInput in) throws IOException {

+ 5 - 4
server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java

@@ -50,6 +50,7 @@ import org.elasticsearch.index.mapper.Uid;
 import org.elasticsearch.search.SearchHit;
 import org.elasticsearch.search.SearchHits;
 import org.elasticsearch.search.SearchPhase;
+import org.elasticsearch.search.SearchShardTarget;
 import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
 import org.elasticsearch.search.fetch.subphase.InnerHitsContext;
 import org.elasticsearch.search.fetch.subphase.InnerHitsFetchSubPhase;
@@ -225,7 +226,7 @@ public class FetchPhase implements SearchPhase {
                                                        int subDocId,
                                                        Map<String, Set<String>> storedToRequestedFields,
                                                        LeafReaderContext subReaderContext) {
-        loadStoredFields(context, subReaderContext, fieldsVisitor, subDocId);
+        loadStoredFields(context.shardTarget(), subReaderContext, fieldsVisitor, subDocId);
         fieldsVisitor.postProcess(context.mapperService());
 
         if (fieldsVisitor.fields().isEmpty()) {
@@ -262,7 +263,7 @@ public class FetchPhase implements SearchPhase {
         final boolean needSource = context.sourceRequested() || context.highlight() != null;
         if (needSource || (context instanceof InnerHitsContext.InnerHitSubContext == false)) {
             FieldsVisitor rootFieldsVisitor = new FieldsVisitor(needSource);
-            loadStoredFields(context, subReaderContext, rootFieldsVisitor, rootSubDocId);
+            loadStoredFields(context.shardTarget(), subReaderContext, rootFieldsVisitor, rootSubDocId);
             rootFieldsVisitor.postProcess(context.mapperService());
             uid = rootFieldsVisitor.uid();
             source = rootFieldsVisitor.source();
@@ -402,12 +403,12 @@ public class FetchPhase implements SearchPhase {
         return nestedIdentity;
     }
 
-    private void loadStoredFields(SearchContext searchContext, LeafReaderContext readerContext, FieldsVisitor fieldVisitor, int docId) {
+    private void loadStoredFields(SearchShardTarget shardTarget, LeafReaderContext readerContext, FieldsVisitor fieldVisitor, int docId) {
         fieldVisitor.reset();
         try {
             readerContext.reader().document(docId, fieldVisitor);
         } catch (IOException e) {
-            throw new FetchPhaseExecutionException(searchContext, "Failed to fetch doc id [" + docId + "]", e);
+            throw new FetchPhaseExecutionException(shardTarget, "Failed to fetch doc id [" + docId + "]", e);
         }
     }
 }

+ 7 - 7
server/src/main/java/org/elasticsearch/search/fetch/FetchPhaseExecutionException.java

@@ -20,19 +20,19 @@
 package org.elasticsearch.search.fetch;
 
 import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.search.SearchContextException;
-import org.elasticsearch.search.internal.SearchContext;
+import org.elasticsearch.search.SearchException;
+import org.elasticsearch.search.SearchShardTarget;
 
 import java.io.IOException;
 
-public class FetchPhaseExecutionException extends SearchContextException {
+public class FetchPhaseExecutionException extends SearchException {
 
-    public FetchPhaseExecutionException(SearchContext context, String msg, Throwable t) {
-        super(context, "Fetch Failed [" + msg + "]", t);
+    public FetchPhaseExecutionException(SearchShardTarget shardTarget, String msg, Throwable t) {
+        super(shardTarget, "Fetch Failed [" + msg + "]", t);
     }
 
-    public FetchPhaseExecutionException(SearchContext context, String msg) {
-        super(context, "Fetch Failed [" + msg + "]");
+    public FetchPhaseExecutionException(SearchShardTarget shardTarget, String msg) {
+        super(shardTarget, "Fetch Failed [" + msg + "]");
     }
 
     public FetchPhaseExecutionException(StreamInput in) throws IOException {

+ 2 - 1
server/src/main/java/org/elasticsearch/search/fetch/subphase/ExplainFetchSubPhase.java

@@ -46,7 +46,8 @@ public final class ExplainFetchSubPhase implements FetchSubPhase {
             // we use the top level doc id, since we work with the top level searcher
             hitContext.hit().explanation(explanation);
         } catch (IOException e) {
-            throw new FetchPhaseExecutionException(context, "Failed to explain doc [" + hitContext.hit().getId() + "]", e);
+            throw new FetchPhaseExecutionException(context.shardTarget(),
+                "Failed to explain doc [" + hitContext.hit().getId() + "]", e);
         } finally {
             context.clearReleasables(SearchContext.Lifetime.COLLECTION);
         }

+ 1 - 1
server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/FastVectorHighlighter.java

@@ -203,7 +203,7 @@ public class FastVectorHighlighter implements Highlighter {
             return null;
 
         } catch (Exception e) {
-            throw new FetchPhaseExecutionException(context,
+            throw new FetchPhaseExecutionException(context.shardTarget(),
                 "Failed to highlight field [" + highlighterContext.fieldName + "]", e);
         }
     }

+ 4 - 2
server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/PlainHighlighter.java

@@ -139,7 +139,8 @@ public class PlainHighlighter implements Highlighter {
                 // the plain highlighter will parse the source and try to analyze it.
                 return null;
             } else {
-                throw new FetchPhaseExecutionException(context, "Failed to highlight field [" + highlighterContext.fieldName + "]", e);
+                throw new FetchPhaseExecutionException(context.shardTarget(),
+                    "Failed to highlight field [" + highlighterContext.fieldName + "]", e);
             }
         }
         if (field.fieldOptions().scoreOrdered()) {
@@ -178,7 +179,8 @@ public class PlainHighlighter implements Highlighter {
             try {
                 end = findGoodEndForNoHighlightExcerpt(noMatchSize, analyzer, fieldType.name(), fieldContents);
             } catch (Exception e) {
-                throw new FetchPhaseExecutionException(context, "Failed to highlight field [" + highlighterContext.fieldName + "]", e);
+                throw new FetchPhaseExecutionException(context.shardTarget(),
+                    "Failed to highlight field [" + highlighterContext.fieldName + "]", e);
             }
             if (end > 0) {
                 return new HighlightField(highlighterContext.fieldName, new Text[] { new Text(fieldContents.substring(0, end)) });

+ 1 - 1
server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/UnifiedHighlighter.java

@@ -122,7 +122,7 @@ public class UnifiedHighlighter implements Highlighter {
                 }
             }
         } catch (IOException e) {
-            throw new FetchPhaseExecutionException(context,
+            throw new FetchPhaseExecutionException(context.shardTarget(),
                 "Failed to highlight field [" + highlighterContext.fieldName + "]", e);
         }
 

+ 2 - 2
server/src/main/java/org/elasticsearch/search/query/QueryPhase.java

@@ -275,7 +275,7 @@ public class QueryPhase implements SearchPhase {
 
                 if (searchContext.request().allowPartialSearchResults() == false) {
                     // Can't rethrow TimeExceededException because not serializable
-                    throw new QueryPhaseExecutionException(searchContext, "Time exceeded");
+                    throw new QueryPhaseExecutionException(searchContext.shardTarget(), "Time exceeded");
                 }
                 queryResult.searchTimedOut(true);
             } finally {
@@ -302,7 +302,7 @@ public class QueryPhase implements SearchPhase {
             }
             return topDocsFactory.shouldRescore();
         } catch (Exception e) {
-            throw new QueryPhaseExecutionException(searchContext, "Failed to execute main query", e);
+            throw new QueryPhaseExecutionException(searchContext.shardTarget(), "Failed to execute main query", e);
         }
     }
 

+ 7 - 7
server/src/main/java/org/elasticsearch/search/query/QueryPhaseExecutionException.java

@@ -20,22 +20,22 @@
 package org.elasticsearch.search.query;
 
 import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.search.SearchContextException;
-import org.elasticsearch.search.internal.SearchContext;
+import org.elasticsearch.search.SearchException;
+import org.elasticsearch.search.SearchShardTarget;
 
 import java.io.IOException;
 
-public class QueryPhaseExecutionException extends SearchContextException {
+public class QueryPhaseExecutionException extends SearchException {
 
-    public QueryPhaseExecutionException(SearchContext context, String msg, Throwable cause) {
-        super(context, "Query Failed [" + msg + "]", cause);
+    public QueryPhaseExecutionException(SearchShardTarget shardTarget, String msg, Throwable cause) {
+        super(shardTarget, "Query Failed [" + msg + "]", cause);
     }
 
     public QueryPhaseExecutionException(StreamInput in) throws IOException {
         super(in);
     }
 
-    public QueryPhaseExecutionException(SearchContext context, String msg) {
-        super(context, msg);
+    public QueryPhaseExecutionException(SearchShardTarget shardTarget, String msg) {
+        super(shardTarget, msg);
     }
 }

+ 3 - 3
server/src/test/java/org/elasticsearch/ElasticsearchExceptionTests.java

@@ -57,7 +57,6 @@ import org.elasticsearch.search.SearchContextMissingException;
 import org.elasticsearch.search.SearchParseException;
 import org.elasticsearch.search.SearchShardTarget;
 import org.elasticsearch.test.ESTestCase;
-import org.elasticsearch.test.TestSearchContext;
 import org.elasticsearch.transport.RemoteTransportException;
 
 import java.io.EOFException;
@@ -73,6 +72,7 @@ import java.util.Map;
 import static java.util.Collections.emptyList;
 import static java.util.Collections.singleton;
 import static java.util.Collections.singletonList;
+import static org.elasticsearch.test.TestSearchContext.SHARD_TARGET;
 import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertToXContentEquivalent;
 import static org.hamcrest.CoreMatchers.hasItem;
 import static org.hamcrest.CoreMatchers.hasItems;
@@ -307,7 +307,7 @@ public class ElasticsearchExceptionTests extends ESTestCase {
                     "\"caused_by\":{\"type\":\"illegal_argument_exception\",\"reason\":\"foo\"}}");
         }
         {
-            ElasticsearchException e = new SearchParseException(new TestSearchContext(null), "foo", new XContentLocation(1,0));
+            ElasticsearchException e = new SearchParseException(SHARD_TARGET, "foo", new XContentLocation(1,0));
             assertExceptionAsJson(e, "{\"type\":\"search_parse_exception\",\"reason\":\"foo\",\"line\":1,\"col\":0}");
         }
         {
@@ -927,7 +927,7 @@ public class ElasticsearchExceptionTests extends ESTestCase {
                 expected = new ElasticsearchException("Elasticsearch exception [type=parsing_exception, reason=Unknown identifier]");
                 break;
             case 2:
-                actual = new SearchParseException(new TestSearchContext(null), "Parse failure", new XContentLocation(12, 98));
+                actual = new SearchParseException(SHARD_TARGET, "Parse failure", new XContentLocation(12, 98));
                 expected = new ElasticsearchException("Elasticsearch exception [type=search_parse_exception, reason=Parse failure]");
                 break;
             case 3:

+ 3 - 6
server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java

@@ -79,13 +79,11 @@ import org.elasticsearch.search.SearchException;
 import org.elasticsearch.search.SearchParseException;
 import org.elasticsearch.search.SearchShardTarget;
 import org.elasticsearch.search.aggregations.MultiBucketConsumerService;
-import org.elasticsearch.search.internal.SearchContext;
 import org.elasticsearch.snapshots.Snapshot;
 import org.elasticsearch.snapshots.SnapshotException;
 import org.elasticsearch.snapshots.SnapshotId;
 import org.elasticsearch.snapshots.SnapshotInProgressException;
 import org.elasticsearch.test.ESTestCase;
-import org.elasticsearch.test.TestSearchContext;
 import org.elasticsearch.test.VersionUtils;
 import org.elasticsearch.transport.ActionNotFoundTransportException;
 import org.elasticsearch.transport.ActionTransportException;
@@ -121,6 +119,7 @@ import static java.lang.reflect.Modifier.isInterface;
 import static java.util.Collections.emptyMap;
 import static java.util.Collections.emptySet;
 import static java.util.Collections.singleton;
+import static org.elasticsearch.test.TestSearchContext.SHARD_TARGET;
 import static org.hamcrest.Matchers.greaterThanOrEqualTo;
 
 public class ExceptionSerializationTests extends ESTestCase {
@@ -388,12 +387,10 @@ public class ExceptionSerializationTests extends ESTestCase {
     }
 
     public void testSearchParseException() throws IOException {
-        SearchContext ctx = new TestSearchContext(null);
-        SearchParseException ex = serialize(new SearchParseException(ctx, "foo", new XContentLocation(66, 666)));
+        SearchParseException ex = serialize(new SearchParseException(SHARD_TARGET, "foo", new XContentLocation(66, 666)));
         assertEquals("foo", ex.getMessage());
         assertEquals(66, ex.getLineNumber());
         assertEquals(666, ex.getColumnNumber());
-        assertEquals(ctx.shardTarget(), ex.shard());
     }
 
     public void testIllegalIndexShardStateException() throws IOException {
@@ -789,7 +786,7 @@ public class ExceptionSerializationTests extends ESTestCase {
         ids.put(124, null);
         ids.put(125, TcpTransport.HttpRequestOnTransportException.class);
         ids.put(126, org.elasticsearch.index.mapper.MapperParsingException.class);
-        ids.put(127, org.elasticsearch.search.SearchContextException.class);
+        ids.put(127, null); // was org.elasticsearch.search.SearchContextException.class
         ids.put(128, org.elasticsearch.search.builder.SearchSourceBuilderException.class);
         ids.put(129, null); // was org.elasticsearch.index.engine.EngineClosedException.class
         ids.put(130, org.elasticsearch.action.NoShardAvailableActionException.class);

+ 3 - 3
server/src/test/java/org/elasticsearch/search/searchafter/SearchAfterIT.java

@@ -27,7 +27,7 @@ import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.action.search.ShardSearchFailure;
 import org.elasticsearch.common.UUIDs;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.search.SearchContextException;
+import org.elasticsearch.search.SearchException;
 import org.elasticsearch.search.SearchHit;
 import org.elasticsearch.search.sort.SortOrder;
 import org.elasticsearch.test.ESIntegTestCase;
@@ -67,7 +67,7 @@ public class SearchAfterIT extends ESIntegTestCase {
         } catch (SearchPhaseExecutionException e) {
             assertTrue(e.shardFailures().length > 0);
             for (ShardSearchFailure failure : e.shardFailures()) {
-                assertThat(failure.getCause().getClass(), Matchers.equalTo(SearchContextException.class));
+                assertThat(failure.getCause().getClass(), Matchers.equalTo(SearchException.class));
                 assertThat(failure.getCause().getMessage(), Matchers.equalTo("`search_after` cannot be used in a scroll context."));
             }
         }
@@ -83,7 +83,7 @@ public class SearchAfterIT extends ESIntegTestCase {
         } catch (SearchPhaseExecutionException e) {
             assertTrue(e.shardFailures().length > 0);
             for (ShardSearchFailure failure : e.shardFailures()) {
-                assertThat(failure.getCause().getClass(), Matchers.equalTo(SearchContextException.class));
+                assertThat(failure.getCause().getClass(), Matchers.equalTo(SearchException.class));
                 assertThat(failure.getCause().getMessage(),
                         Matchers.equalTo("`from` parameter must be set to 0 when `search_after` is used."));
             }

+ 2 - 2
server/src/test/java/org/elasticsearch/search/slice/SearchSliceIT.java

@@ -31,7 +31,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.common.xcontent.XContentType;
 import org.elasticsearch.search.Scroll;
-import org.elasticsearch.search.SearchContextException;
+import org.elasticsearch.search.SearchException;
 import org.elasticsearch.search.SearchHit;
 import org.elasticsearch.search.sort.SortBuilders;
 import org.elasticsearch.test.ESIntegTestCase;
@@ -205,7 +205,7 @@ public class SearchSliceIT extends ESIntegTestCase {
                 .slice(new SliceBuilder("invalid_random_int", 0, 10))
                 .get());
         Throwable rootCause = findRootCause(exc);
-        assertThat(rootCause.getClass(), equalTo(SearchContextException.class));
+        assertThat(rootCause.getClass(), equalTo(SearchException.class));
         assertThat(rootCause.getMessage(),
             equalTo("`slice` cannot be used outside of a scroll context"));
     }

+ 5 - 5
server/src/test/java/org/elasticsearch/search/source/MetadataFetchingIT.java

@@ -25,7 +25,7 @@ import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.index.query.InnerHitBuilder;
 import org.elasticsearch.index.query.NestedQueryBuilder;
 import org.elasticsearch.index.query.TermQueryBuilder;
-import org.elasticsearch.search.SearchContextException;
+import org.elasticsearch.search.SearchException;
 import org.elasticsearch.search.SearchHits;
 import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
 import org.elasticsearch.test.ESIntegTestCase;
@@ -122,18 +122,18 @@ public class MetadataFetchingIT extends ESIntegTestCase {
         {
             SearchPhaseExecutionException exc = expectThrows(SearchPhaseExecutionException.class,
                 () -> client().prepareSearch("test").setFetchSource(true).storedFields("_none_").get());
-            Throwable rootCause = ExceptionsHelper.unwrap(exc, SearchContextException.class);
+            Throwable rootCause = ExceptionsHelper.unwrap(exc, SearchException.class);
             assertNotNull(rootCause);
-            assertThat(rootCause.getClass(), equalTo(SearchContextException.class));
+            assertThat(rootCause.getClass(), equalTo(SearchException.class));
             assertThat(rootCause.getMessage(),
                 equalTo("`stored_fields` cannot be disabled if _source is requested"));
         }
         {
             SearchPhaseExecutionException exc = expectThrows(SearchPhaseExecutionException.class,
                 () -> client().prepareSearch("test").storedFields("_none_").setVersion(true).get());
-            Throwable rootCause = ExceptionsHelper.unwrap(exc, SearchContextException.class);
+            Throwable rootCause = ExceptionsHelper.unwrap(exc, SearchException.class);
             assertNotNull(rootCause);
-            assertThat(rootCause.getClass(), equalTo(SearchContextException.class));
+            assertThat(rootCause.getClass(), equalTo(SearchException.class));
             assertThat(rootCause.getMessage(),
                 equalTo("`stored_fields` cannot be disabled if version is requested"));
         }

+ 4 - 0
test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java

@@ -21,6 +21,7 @@ package org.elasticsearch.test;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.FieldDoc;
 import org.apache.lucene.search.Query;
+import org.elasticsearch.action.OriginalIndices;
 import org.elasticsearch.action.search.SearchTask;
 import org.elasticsearch.action.search.SearchType;
 import org.elasticsearch.common.unit.TimeValue;
@@ -35,6 +36,7 @@ import org.elasticsearch.index.query.InnerHitContextBuilder;
 import org.elasticsearch.index.query.ParsedQuery;
 import org.elasticsearch.index.query.QueryShardContext;
 import org.elasticsearch.index.shard.IndexShard;
+import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.index.similarity.SimilarityService;
 import org.elasticsearch.search.SearchExtBuilder;
 import org.elasticsearch.search.SearchShardTarget;
@@ -64,6 +66,8 @@ import java.util.List;
 import java.util.Map;
 
 public class TestSearchContext extends SearchContext {
+    public static final SearchShardTarget SHARD_TARGET =
+        new SearchShardTarget("test", new ShardId("test", "test", 0), null, OriginalIndices.NONE);
 
     final BigArrays bigArrays;
     final IndexService indexService;