From 3dc467823c640f9f78a5027779b233c3cff52a4f Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Wed, 12 Feb 2025 12:20:49 +0100 Subject: [PATCH 1/3] Handle search timeout in SuggestPhase Whenever a search timeout is set to a search request, the timeout may be triggered by the suggest phase via exitable directory reader. In that case, the exception that is thrown by the timeout check needs to be handled, instead of returned back to the user. Instead of handling the timeout in each phase, this commit handles it as part of QueryPhase for both SuggestPhase and RescorePhase. For rescore phase, one integration test that is time dependent is also rewritten to remove the time dependency and moved from QueryRescorerIT to SearchTimeoutIT. Closes #122186 --- .../elasticsearch/search/SearchTimeoutIT.java | 241 +++++++++++++++++- .../search/functionscore/QueryRescorerIT.java | 17 -- .../search/query/QueryPhase.java | 27 +- .../search/rescore/RescorePhase.java | 10 +- .../search/suggest/SuggestPhase.java | 1 - .../search/query/QueryPhaseTimeoutTests.java | 129 ++++++++++ 6 files changed, 390 insertions(+), 35 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/SearchTimeoutIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/SearchTimeoutIT.java index f45c37715f7cc..f79321ef8d0de 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/SearchTimeoutIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/SearchTimeoutIT.java @@ -14,6 +14,7 @@ import org.apache.lucene.search.ConstantScoreScorer; import org.apache.lucene.search.ConstantScoreWeight; import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.Explanation; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.LeafCollector; import org.apache.lucene.search.Query; @@ -22,8 +23,10 @@ import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.ScorerSupplier; +import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.Weight; import org.apache.lucene.util.Bits; +import org.apache.lucene.util.CharsRefBuilder; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.TransportVersion; import org.elasticsearch.action.search.SearchRequestBuilder; @@ -33,12 +36,23 @@ import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.query.AbstractQueryBuilder; import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.QueryRewriteContext; import org.elasticsearch.index.query.SearchExecutionContext; +import org.elasticsearch.index.query.TermQueryBuilder; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.SearchPlugin; import org.elasticsearch.search.aggregations.bucket.terms.StringTerms; import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder; import org.elasticsearch.search.internal.ContextIndexSearcher; +import org.elasticsearch.search.rescore.RescoreContext; +import org.elasticsearch.search.rescore.Rescorer; +import org.elasticsearch.search.rescore.RescorerBuilder; +import org.elasticsearch.search.suggest.SortBy; +import org.elasticsearch.search.suggest.SuggestBuilder; +import org.elasticsearch.search.suggest.Suggester; +import org.elasticsearch.search.suggest.SuggestionSearchContext; +import org.elasticsearch.search.suggest.term.TermSuggestion; +import org.elasticsearch.search.suggest.term.TermSuggestionBuilder; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; import org.elasticsearch.xcontent.XContentBuilder; @@ -58,7 +72,7 @@ public class SearchTimeoutIT extends ESIntegTestCase { @Override protected Collection> nodePlugins() { - return Collections.singleton(BulkScorerTimeoutQueryPlugin.class); + return Collections.singleton(SearchTimeoutPlugin.class); } @Override @@ -72,6 +86,9 @@ protected void setupSuiteScopeCluster() throws Exception { indexRandom(true, "test", randomIntBetween(20, 50)); } + /** + * Test the scenario where the query times out before starting to collect documents, verify that partial hits are not returned + */ public void testTopHitsTimeoutBeforeCollecting() { // setting the timeout is necessary only because we check that if a TimeExceededException is thrown, a timeout was set SearchRequestBuilder searchRequestBuilder = prepareSearch("test").setTimeout(new TimeValue(10, TimeUnit.SECONDS)) @@ -88,6 +105,9 @@ public void testTopHitsTimeoutBeforeCollecting() { }); } + /** + * Test the scenario where the query times out while collecting documents, verify that partial hits results are returned + */ public void testTopHitsTimeoutWhileCollecting() { // setting the timeout is necessary only because we check that if a TimeExceededException is thrown, a timeout was set SearchRequestBuilder searchRequestBuilder = prepareSearch("test").setTimeout(new TimeValue(10, TimeUnit.SECONDS)) @@ -103,6 +123,9 @@ public void testTopHitsTimeoutWhileCollecting() { }); } + /** + * Test the scenario where the query times out before starting to collect documents, verify that partial aggs results are not returned + */ public void testAggsTimeoutBeforeCollecting() { SearchRequestBuilder searchRequestBuilder = prepareSearch("test").setSize(0) // setting the timeout is necessary only because we check that if a TimeExceededException is thrown, a timeout was set @@ -123,6 +146,9 @@ public void testAggsTimeoutBeforeCollecting() { }); } + /** + * Test the scenario where the query times out while collecting documents, verify that partial aggs results are returned + */ public void testAggsTimeoutWhileCollecting() { SearchRequestBuilder searchRequestBuilder = prepareSearch("test").setSize(0) // setting the timeout is necessary only because we check that if a TimeExceededException is thrown, a timeout was set @@ -145,6 +171,56 @@ public void testAggsTimeoutWhileCollecting() { }); } + /** + * Test the scenario where the suggest phase (part of the query phase) times out, yet there are results + * available coming from executing the query and aggs on each shard. + */ + public void testSuggestTimeoutWithPartialResults() { + SuggestBuilder suggestBuilder = new SuggestBuilder(); + suggestBuilder.setGlobalText("text"); + TimeoutSuggestionBuilder timeoutSuggestionBuilder = new TimeoutSuggestionBuilder(); + suggestBuilder.addSuggestion("suggest", timeoutSuggestionBuilder); + SearchRequestBuilder searchRequestBuilder = prepareSearch("test").suggest(suggestBuilder) + .addAggregation(new TermsAggregationBuilder("terms").field("field.keyword")); + ElasticsearchAssertions.assertResponse(searchRequestBuilder, searchResponse -> { + assertThat(searchResponse.isTimedOut(), equalTo(true)); + assertEquals(0, searchResponse.getShardFailures().length); + assertEquals(0, searchResponse.getFailedShards()); + assertThat(searchResponse.getSuccessfulShards(), greaterThan(0)); + assertEquals(searchResponse.getSuccessfulShards(), searchResponse.getTotalShards()); + assertThat(searchResponse.getHits().getTotalHits().value(), greaterThan(0L)); + assertThat(searchResponse.getHits().getHits().length, greaterThan(0)); + StringTerms terms = searchResponse.getAggregations().get("terms"); + assertEquals(1, terms.getBuckets().size()); + StringTerms.Bucket bucket = terms.getBuckets().get(0); + assertEquals("value", bucket.getKeyAsString()); + assertThat(bucket.getDocCount(), greaterThan(0L)); + }); + } + + /** + * Test the scenario where the rescore phase (part of the query phase) times out, yet there are results + * available coming from executing the query and aggs on each shard. + */ + public void testRescoreTimeoutWithPartialResults() { + SearchRequestBuilder searchRequestBuilder = prepareSearch("test").setRescorer(new TimeoutRescorerBuilder()) + .addAggregation(new TermsAggregationBuilder("terms").field("field.keyword")); + ElasticsearchAssertions.assertResponse(searchRequestBuilder, searchResponse -> { + assertThat(searchResponse.isTimedOut(), equalTo(true)); + assertEquals(0, searchResponse.getShardFailures().length); + assertEquals(0, searchResponse.getFailedShards()); + assertThat(searchResponse.getSuccessfulShards(), greaterThan(0)); + assertEquals(searchResponse.getSuccessfulShards(), searchResponse.getTotalShards()); + assertThat(searchResponse.getHits().getTotalHits().value(), greaterThan(0L)); + assertThat(searchResponse.getHits().getHits().length, greaterThan(0)); + StringTerms terms = searchResponse.getAggregations().get("terms"); + assertEquals(1, terms.getBuckets().size()); + StringTerms.Bucket bucket = terms.getBuckets().get(0); + assertEquals("value", bucket.getKeyAsString()); + assertThat(bucket.getDocCount(), greaterThan(0L)); + }); + } + public void testPartialResultsIntolerantTimeoutBeforeCollecting() { ElasticsearchException ex = expectThrows( ElasticsearchException.class, @@ -171,13 +247,67 @@ public void testPartialResultsIntolerantTimeoutWhileCollecting() { assertEquals(429, ex.status().getStatus()); } - public static final class BulkScorerTimeoutQueryPlugin extends Plugin implements SearchPlugin { + public void testPartialResultsIntolerantTimeoutWhileSuggestingOnly() { + SuggestBuilder suggestBuilder = new SuggestBuilder(); + suggestBuilder.setGlobalText("text"); + TimeoutSuggestionBuilder timeoutSuggestionBuilder = new TimeoutSuggestionBuilder(); + suggestBuilder.addSuggestion("suggest", timeoutSuggestionBuilder); + ElasticsearchException ex = expectThrows( + ElasticsearchException.class, + prepareSearch("test").suggest(suggestBuilder).setAllowPartialSearchResults(false) // this line causes timeouts to report + // failures + ); + assertTrue(ex.toString().contains("Time exceeded")); + assertEquals(429, ex.status().getStatus()); + } + + public void testPartialResultsIntolerantTimeoutWhileSuggesting() { + SuggestBuilder suggestBuilder = new SuggestBuilder(); + suggestBuilder.setGlobalText("text"); + TimeoutSuggestionBuilder timeoutSuggestionBuilder = new TimeoutSuggestionBuilder(); + suggestBuilder.addSuggestion("suggest", timeoutSuggestionBuilder); + ElasticsearchException ex = expectThrows( + ElasticsearchException.class, + prepareSearch("test").setQuery(new TermQueryBuilder("field", "value")) + .suggest(suggestBuilder) + .setAllowPartialSearchResults(false) // this line causes timeouts to report failures + ); + assertTrue(ex.toString().contains("Time exceeded")); + assertEquals(429, ex.status().getStatus()); + } + + public void testPartialResultsIntolerantTimeoutWhileRescoring() { + ElasticsearchException ex = expectThrows( + ElasticsearchException.class, + prepareSearch("test").setQuery(new TermQueryBuilder("field", "value")) + .setRescorer(new TimeoutRescorerBuilder()) + .setAllowPartialSearchResults(false) // this line causes timeouts to report failures + ); + assertTrue(ex.toString().contains("Time exceeded")); + assertEquals(429, ex.status().getStatus()); + } + + public static final class SearchTimeoutPlugin extends Plugin implements SearchPlugin { @Override public List> getQueries() { return Collections.singletonList(new QuerySpec("timeout", BulkScorerTimeoutQuery::new, parser -> { throw new UnsupportedOperationException(); })); } + + @Override + public List> getSuggesters() { + return Collections.singletonList(new SuggesterSpec<>("timeout", TimeoutSuggestionBuilder::new, parser -> { + throw new UnsupportedOperationException(); + }, TermSuggestion::new)); + } + + @Override + public List> getRescorers() { + return Collections.singletonList(new RescorerSpec<>("timeout", TimeoutRescorerBuilder::new, parser -> { + throw new UnsupportedOperationException(); + })); + } } /** @@ -315,4 +445,111 @@ public TransportVersion getMinimalSupportedVersion() { return null; } } + + /** + * Suggestion builder that triggers a timeout as part of its execution + */ + private static final class TimeoutSuggestionBuilder extends TermSuggestionBuilder { + TimeoutSuggestionBuilder() { + super("field"); + } + + TimeoutSuggestionBuilder(StreamInput in) throws IOException { + super(in); + } + + @Override + public String getWriteableName() { + return "timeout"; + } + + @Override + public SuggestionSearchContext.SuggestionContext build(SearchExecutionContext context) { + return new TimeoutSuggestionContext(new TimeoutSuggester((ContextIndexSearcher) context.searcher()), context); + } + } + + private static final class TimeoutSuggester extends Suggester { + private final ContextIndexSearcher contextIndexSearcher; + + TimeoutSuggester(ContextIndexSearcher contextIndexSearcher) { + this.contextIndexSearcher = contextIndexSearcher; + } + + @Override + protected TermSuggestion innerExecute( + String name, + TimeoutSuggestionContext suggestion, + IndexSearcher searcher, + CharsRefBuilder spare + ) { + contextIndexSearcher.throwTimeExceededException(); + assert false; + return new TermSuggestion(name, suggestion.getSize(), SortBy.SCORE); + } + + @Override + protected TermSuggestion emptySuggestion(String name, TimeoutSuggestionContext suggestion, CharsRefBuilder spare) { + return new TermSuggestion(name, suggestion.getSize(), SortBy.SCORE); + } + } + + private static final class TimeoutSuggestionContext extends SuggestionSearchContext.SuggestionContext { + TimeoutSuggestionContext(Suggester suggester, SearchExecutionContext searchExecutionContext) { + super(suggester, searchExecutionContext); + } + } + + private static final class TimeoutRescorerBuilder extends RescorerBuilder { + TimeoutRescorerBuilder() { + super(); + } + + TimeoutRescorerBuilder(StreamInput in) throws IOException { + super(in); + } + + @Override + protected void doWriteTo(StreamOutput out) {} + + @Override + protected void doXContent(XContentBuilder builder, Params params) {} + + @Override + protected RescoreContext innerBuildContext(int windowSize, SearchExecutionContext context) throws IOException { + return new RescoreContext(10, new Rescorer() { + @Override + public TopDocs rescore(TopDocs topDocs, IndexSearcher searcher, RescoreContext rescoreContext) { + ((ContextIndexSearcher) context.searcher()).throwTimeExceededException(); + assert false; + return null; + } + + @Override + public Explanation explain( + int topLevelDocId, + IndexSearcher searcher, + RescoreContext rescoreContext, + Explanation sourceExplanation + ) { + throw new UnsupportedOperationException(); + } + }); + } + + @Override + public String getWriteableName() { + return "timeout"; + } + + @Override + public TransportVersion getMinimalSupportedVersion() { + return null; + } + + @Override + public RescorerBuilder rewrite(QueryRewriteContext ctx) { + return this; + } + } } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/functionscore/QueryRescorerIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/functionscore/QueryRescorerIT.java index fbdcfe26d28ee..0ba4c13c352c1 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/functionscore/QueryRescorerIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/functionscore/QueryRescorerIT.java @@ -24,7 +24,6 @@ import org.elasticsearch.common.lucene.search.function.ScoreFunction; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings.Builder; -import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.query.Operator; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; @@ -994,22 +993,6 @@ public void testRescoreAfterCollapseRandom() throws Exception { }); } - public void testRescoreWithTimeout() throws Exception { - // no dummy docs since merges can change scores while we run queries. - int numDocs = indexRandomNumbers("whitespace", -1, false); - - String intToEnglish = English.intToEnglish(between(0, numDocs - 1)); - String query = intToEnglish.split(" ")[0]; - assertResponse( - prepareSearch().setSearchType(SearchType.QUERY_THEN_FETCH) - .setQuery(QueryBuilders.matchQuery("field1", query).operator(Operator.OR)) - .setSize(10) - .addRescorer(new QueryRescorerBuilder(functionScoreQuery(new TestTimedScoreFunctionBuilder())).windowSize(100)) - .setTimeout(TimeValue.timeValueMillis(10)), - r -> assertTrue(r.isTimedOut()) - ); - } - @Override protected Collection> nodePlugins() { return List.of(TestTimedQueryPlugin.class); diff --git a/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java b/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java index 3036a295d459a..8ad52c4f9bb59 100644 --- a/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java +++ b/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java @@ -126,7 +126,15 @@ static void executeRank(SearchContext searchContext) throws QueryPhaseExecutionE static void executeQuery(SearchContext searchContext) throws QueryPhaseExecutionException { if (searchContext.hasOnlySuggest()) { - SuggestPhase.execute(searchContext); + try { + SuggestPhase.execute(searchContext); + } catch (ContextIndexSearcher.TimeExceededException timeExceededException) { + SearchTimeoutException.handleTimeout( + searchContext.request().allowPartialSearchResults(), + searchContext.shardTarget(), + searchContext.queryResult() + ); + } searchContext.queryResult().topDocs(new TopDocsAndMaxScore(Lucene.EMPTY_TOP_DOCS, Float.NaN), new DocValueFormat[0]); return; } @@ -142,11 +150,18 @@ static void executeQuery(SearchContext searchContext) throws QueryPhaseExecution addCollectorsAndSearch(searchContext); - RescorePhase.execute(searchContext); - SuggestPhase.execute(searchContext); - - if (searchContext.getProfilers() != null) { - searchContext.queryResult().profileResults(searchContext.getProfilers().buildQueryPhaseResults()); + try { + RescorePhase.execute(searchContext); + SuggestPhase.execute(searchContext); + if (searchContext.getProfilers() != null) { + searchContext.queryResult().profileResults(searchContext.getProfilers().buildQueryPhaseResults()); + } + } catch (ContextIndexSearcher.TimeExceededException timeExceededException) { + SearchTimeoutException.handleTimeout( + searchContext.request().allowPartialSearchResults(), + searchContext.shardTarget(), + searchContext.queryResult() + ); } } diff --git a/server/src/main/java/org/elasticsearch/search/rescore/RescorePhase.java b/server/src/main/java/org/elasticsearch/search/rescore/RescorePhase.java index f8b348b383f01..7223da3c6101b 100644 --- a/server/src/main/java/org/elasticsearch/search/rescore/RescorePhase.java +++ b/server/src/main/java/org/elasticsearch/search/rescore/RescorePhase.java @@ -18,9 +18,7 @@ import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore; import org.elasticsearch.common.util.Maps; import org.elasticsearch.lucene.grouping.TopFieldGroups; -import org.elasticsearch.search.internal.ContextIndexSearcher; import org.elasticsearch.search.internal.SearchContext; -import org.elasticsearch.search.query.SearchTimeoutException; import org.elasticsearch.search.sort.ShardDocSortField; import org.elasticsearch.search.sort.SortAndFormats; @@ -72,7 +70,7 @@ public static void execute(SearchContext context) { assert topDocsSortedByScore(topDocs) : "topdocs should be sorted after rescore"; ctx.setCancellationChecker(null); } - /** + /* * Since rescorers are building top docs with score only, we must reconstruct the {@link TopFieldGroups} * or {@link TopFieldDocs} using their original version before rescoring. */ @@ -86,12 +84,6 @@ public static void execute(SearchContext context) { .topDocs(new TopDocsAndMaxScore(topDocs, topDocs.scoreDocs[0].score), context.queryResult().sortValueFormats()); } catch (IOException e) { throw new ElasticsearchException("Rescore Phase Failed", e); - } catch (ContextIndexSearcher.TimeExceededException e) { - SearchTimeoutException.handleTimeout( - context.request().allowPartialSearchResults(), - context.shardTarget(), - context.queryResult() - ); } } diff --git a/server/src/main/java/org/elasticsearch/search/suggest/SuggestPhase.java b/server/src/main/java/org/elasticsearch/search/suggest/SuggestPhase.java index d63e0717ca7ac..272855bacd544 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/SuggestPhase.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/SuggestPhase.java @@ -56,5 +56,4 @@ public static void execute(SearchContext context) { throw new ElasticsearchException("I/O exception during suggest phase", e); } } - } diff --git a/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTimeoutTests.java b/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTimeoutTests.java index b417f7adbc8b7..56f8d957e6e83 100644 --- a/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTimeoutTests.java +++ b/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTimeoutTests.java @@ -27,6 +27,7 @@ import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.LeafCollector; +import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.QueryVisitor; import org.apache.lucene.search.Scorable; @@ -38,14 +39,29 @@ import org.apache.lucene.tests.index.RandomIndexWriter; import org.apache.lucene.tests.util.LuceneTestCase; import org.apache.lucene.util.Bits; +import org.apache.lucene.util.CharsRefBuilder; +import org.elasticsearch.action.OriginalIndices; +import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchShardTask; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.text.Text; import org.elasticsearch.core.CheckedConsumer; +import org.elasticsearch.index.query.MatchAllQueryBuilder; import org.elasticsearch.index.query.ParsedQuery; +import org.elasticsearch.index.query.SearchExecutionContext; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.ContextIndexSearcher; import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.search.internal.ShardSearchRequest; +import org.elasticsearch.search.suggest.Suggest; +import org.elasticsearch.search.suggest.SuggestBuilder; +import org.elasticsearch.search.suggest.Suggester; +import org.elasticsearch.search.suggest.SuggestionSearchContext; import org.elasticsearch.test.TestSearchContext; +import org.hamcrest.Matchers; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -275,6 +291,119 @@ private TestSearchContext createSearchContext(Query query, int size) throws IOEx return context; } + public void testSuggestOnlyWithTimeout() throws Exception { + SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder().suggest(new SuggestBuilder()); + try (SearchContext context = createSearchContextWithSuggestTimeout(searchSourceBuilder)) { + assertTrue(context.hasOnlySuggest()); + QueryPhase.execute(context); + assertTrue(context.queryResult().searchTimedOut()); + assertNull(context.queryResult().suggest()); + assertNotNull(context.queryResult().topDocs()); + assertEquals(0, context.queryResult().topDocs().topDocs.totalHits.value()); + } + } + + public void testSuggestAndQueryWithSuggestTimeout() throws Exception { + SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder().suggest(new SuggestBuilder()).query(new MatchAllQueryBuilder()); + try (SearchContext context = createSearchContextWithSuggestTimeout(searchSourceBuilder)) { + context.parsedQuery(new ParsedQuery(new MatchAllDocsQuery())); + assertFalse(context.hasOnlySuggest()); + QueryPhase.execute(context); + assertNotNull(context.queryResult().topDocs()); + assertThat(context.queryResult().topDocs().topDocs.totalHits.value(), Matchers.greaterThan(0L)); + assertTrue(context.queryResult().searchTimedOut()); + assertNull(context.queryResult().suggest()); + } + } + + private TestSearchContext createSearchContextWithSuggestTimeout(SearchSourceBuilder searchSourceBuilder) throws IOException { + ContextIndexSearcher contextIndexSearcher = newContextSearcher(reader); + SuggestionSearchContext suggestionSearchContext = new SuggestionSearchContext(); + suggestionSearchContext.addSuggestion("suggestion", new TestSuggestionContext(new TestSuggester(contextIndexSearcher), null)); + TestSearchContext context = new TestSearchContext(null, indexShard, contextIndexSearcher) { + @Override + public SuggestionSearchContext suggest() { + return suggestionSearchContext; + } + + @Override + public ShardSearchRequest request() { + SearchRequest searchRequest = new SearchRequest(); + searchRequest.allowPartialSearchResults(true); + searchRequest.source(searchSourceBuilder); + return new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + indexShard.shardId(), + 0, + 1, + AliasFilter.EMPTY, + 1F, + 0, + null + ); + } + }; + context.setTask(new SearchShardTask(123L, "", "", "", null, Collections.emptyMap())); + return context; + } + + private static final class TestSuggester extends Suggester { + private final ContextIndexSearcher contextIndexSearcher; + + TestSuggester(ContextIndexSearcher contextIndexSearcher) { + this.contextIndexSearcher = contextIndexSearcher; + } + + @Override + protected TestSuggestion innerExecute( + String name, + TestSuggestionContext suggestion, + IndexSearcher searcher, + CharsRefBuilder spare + ) { + contextIndexSearcher.throwTimeExceededException(); + assert false; + return new TestSuggestion(); + } + + @Override + protected TestSuggestion emptySuggestion(String name, TestSuggestionContext suggestion, CharsRefBuilder spare) { + return new TestSuggestion(); + } + } + + private static final class TestSuggestionContext extends SuggestionSearchContext.SuggestionContext { + TestSuggestionContext(Suggester suggester, SearchExecutionContext searchExecutionContext) { + super(suggester, searchExecutionContext); + } + } + + private static final class TestSuggestion extends Suggest.Suggestion< + Suggest.Suggestion.Entry> { + TestSuggestion() { + super("suggestion", 10); + } + + @Override + protected Entry newEntry(StreamInput in) { + return new TestSuggestionEntry(); + } + + @Override + public String getWriteableName() { + return "suggestion"; + } + } + + private static final class TestSuggestionEntry extends Suggest.Suggestion.Entry { + @Override + protected Option newOption(StreamInput in) { + return new Option(new Text("text"), 1f) { + }; + } + } + private static class Score extends Scorable { float score; From 66bf77342672435097e72575480a551cbf0334d9 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Wed, 12 Feb 2025 12:27:20 +0100 Subject: [PATCH 2/3] Update docs/changelog/122357.yaml --- docs/changelog/122357.yaml | 6 ++++++ 1 file changed, 6 insertions(+) create mode 100644 docs/changelog/122357.yaml diff --git a/docs/changelog/122357.yaml b/docs/changelog/122357.yaml new file mode 100644 index 0000000000000..7648002c9356f --- /dev/null +++ b/docs/changelog/122357.yaml @@ -0,0 +1,6 @@ +pr: 122357 +summary: Handle search timeout in `SuggestPhase` +area: Search +type: bug +issues: + - 122186 From ccc8d0ee33d45c58cb7ba4958a7f41e88f3ce733 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Thu, 13 Feb 2025 16:16:07 +0100 Subject: [PATCH 3/3] iter --- .../elasticsearch/search/query/QueryPhaseTimeoutTests.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTimeoutTests.java b/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTimeoutTests.java index 56f8d957e6e83..6b38e05bdc4e3 100644 --- a/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTimeoutTests.java +++ b/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTimeoutTests.java @@ -309,7 +309,6 @@ public void testSuggestAndQueryWithSuggestTimeout() throws Exception { context.parsedQuery(new ParsedQuery(new MatchAllDocsQuery())); assertFalse(context.hasOnlySuggest()); QueryPhase.execute(context); - assertNotNull(context.queryResult().topDocs()); assertThat(context.queryResult().topDocs().topDocs.totalHits.value(), Matchers.greaterThan(0L)); assertTrue(context.queryResult().searchTimedOut()); assertNull(context.queryResult().suggest()); @@ -363,8 +362,7 @@ protected TestSuggestion innerExecute( CharsRefBuilder spare ) { contextIndexSearcher.throwTimeExceededException(); - assert false; - return new TestSuggestion(); + throw new AssertionError("should have thrown TimeExceededException"); } @Override