Replace SearchContextException with SearchException (#47046)

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

Relates #46523
This commit is contained in:
Jim Ferenczi 2019-09-26 14:21:23 +02:00 committed by GitHub
parent 95e2ca741e
commit 73a09b34b8
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
26 changed files with 98 additions and 122 deletions

View File

@ -44,7 +44,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;
@ -58,6 +57,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;
@ -67,7 +67,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),

View File

@ -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

View File

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

View File

@ -263,7 +263,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);
}
}
}

View File

@ -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);
}
}

View File

@ -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) {

View File

@ -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);

View File

@ -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);
}

View File

@ -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) {

View File

@ -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;

View File

@ -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);
}
}

View File

@ -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 {

View File

@ -52,6 +52,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;
@ -229,7 +230,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()) {
@ -266,7 +267,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();
@ -419,12 +420,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);
}
}
}

View File

@ -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 {

View File

@ -46,7 +46,7 @@ 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().getType() + "#"
throw new FetchPhaseExecutionException(context.shardTarget(), "Failed to explain doc [" + hitContext.hit().getType() + "#"
+ hitContext.hit().getId() + "]", e);
} finally {
context.clearReleasables(SearchContext.Lifetime.COLLECTION);

View File

@ -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);
}
}

View File

@ -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)) });

View File

@ -123,7 +123,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);
}

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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;
@ -300,7 +300,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}");
}
{
@ -920,7 +920,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:

View File

@ -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 {
@ -790,7 +787,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);

View File

@ -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."));
}

View File

@ -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"));
}

View File

@ -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;
@ -128,18 +128,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"));
}

View File

@ -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;