Skip to content

Commit 0cc3973

Browse files
authored
Return partial failures if search was cancelled. (#65467)
In #63520, we started cancelling searches that encounter shard failures and don't allow partial results. In this case we return an 'all shards failed' response, since there are no successful responses. This PR proposes to return a 'partial shards failure' instead. The reasoning: it's misleading to claim that 'all shards failed' when we only know at least one shard failed (and preemptively cancelled the rest).
1 parent c2fda05 commit 0cc3973

File tree

3 files changed

+12
-12
lines changed

3 files changed

+12
-12
lines changed

modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/reindex/35_search_failures.yml

Lines changed: 0 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1,8 +1,5 @@
11
---
22
"Response format for search failures":
3-
- skip:
4-
version: "all"
5-
reason: "awaits fix of https://github.com/elastic/elasticsearch/pull/64382"
63
- do:
74
indices.create:
85
index: source

server/src/internalClusterTest/java/org/elasticsearch/search/SearchCancellationIT.java

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -67,6 +67,7 @@
6767
import static org.elasticsearch.search.SearchCancellationIT.ScriptedBlockPlugin.SCRIPT_NAME;
6868
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertFailures;
6969
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
70+
import static org.hamcrest.Matchers.containsString;
7071
import static org.hamcrest.Matchers.equalTo;
7172
import static org.hamcrest.Matchers.greaterThan;
7273
import static org.hamcrest.Matchers.hasSize;
@@ -311,12 +312,12 @@ public void testCancelFailedSearchWhenPartialResultDisallowed() throws Exception
311312
.build());
312313
indexTestData();
313314
Thread searchThread = new Thread(() -> {
314-
expectThrows(Exception.class, () -> {
315+
SearchPhaseExecutionException e = expectThrows(SearchPhaseExecutionException.class, () ->
315316
client().prepareSearch("test")
316317
.setSearchType(SearchType.QUERY_THEN_FETCH)
317318
.setQuery(scriptQuery(new Script(ScriptType.INLINE, "mockscript", SCRIPT_NAME, Collections.emptyMap())))
318-
.setAllowPartialSearchResults(false).setSize(1000).get();
319-
});
319+
.setAllowPartialSearchResults(false).setSize(1000).get());
320+
assertThat(e.getMessage(), containsString("Partial shards failure"));
320321
});
321322
searchThread.start();
322323
try {

server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java

Lines changed: 8 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -314,13 +314,16 @@ public boolean isForceExecution() {
314314
});
315315
}
316316

317+
317318
@Override
318319
public final void executeNextPhase(SearchPhase currentPhase, SearchPhase nextPhase) {
319-
/* This is the main search phase transition where we move to the next phase. At this point we check if there is
320-
* at least one successful operation left and if so we move to the next phase. If not we immediately fail the
321-
* search phase as "all shards failed"*/
322-
if (successfulOps.get() == 0) { // we have 0 successful results that means we shortcut stuff and return a failure
323-
final ShardOperationFailedException[] shardSearchFailures = ExceptionsHelper.groupBy(buildShardFailures());
320+
/* This is the main search phase transition where we move to the next phase. If all shards
321+
* failed or if there was a failure and partial results are not allowed, then we immediately
322+
* fail. Otherwise we continue to the next phase.
323+
*/
324+
ShardOperationFailedException[] shardSearchFailures = buildShardFailures();
325+
if (shardSearchFailures.length == getNumShards()) {
326+
shardSearchFailures = ExceptionsHelper.groupBy(shardSearchFailures);
324327
Throwable cause = shardSearchFailures.length == 0 ? null :
325328
ElasticsearchException.guessRootCauses(shardSearchFailures[0].getCause())[0];
326329
logger.debug(() -> new ParameterizedMessage("All shards failed for phase: [{}]", getName()), cause);
@@ -331,7 +334,6 @@ public final void executeNextPhase(SearchPhase currentPhase, SearchPhase nextPha
331334
if (allowPartialResults == false && successfulOps.get() != getNumShards()) {
332335
// check if there are actual failures in the atomic array since
333336
// successful retries can reset the failures to null
334-
ShardOperationFailedException[] shardSearchFailures = buildShardFailures();
335337
if (shardSearchFailures.length > 0) {
336338
if (logger.isDebugEnabled()) {
337339
int numShardFailures = shardSearchFailures.length;

0 commit comments

Comments
 (0)