diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index 8475272a5e2cd..bce6d02a8c7fc 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -140,7 +140,7 @@ public final void executeNextPhase(SearchPhase currentPhase, SearchPhase nextPha } else { Boolean allowPartialResults = request.allowPartialSearchResults(); assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults"; - if (allowPartialResults == false && shardFailures.get() != null) { + if (allowPartialResults == false && successfulOps.get() != getNumShards()) { // check if there are actual failures in the atomic array since // successful retries can reset the failures to null ShardOperationFailedException[] shardSearchFailures = buildShardFailures(); @@ -154,6 +154,15 @@ public final void executeNextPhase(SearchPhase currentPhase, SearchPhase nextPha } onPhaseFailure(currentPhase, "Partial shards failure", null); return; + } else { + int discrepancy = getNumShards() - successfulOps.get(); + assert discrepancy > 0 : "discrepancy: " + discrepancy; + if (logger.isDebugEnabled()) { + logger.debug("Partial shards failure (unavailable: {}, successful: {}, skipped: {}, num-shards: {}, phase: {})", + discrepancy, successfulOps.get(), skippedOps.get(), getNumShards(), currentPhase.getName()); + } + onPhaseFailure(currentPhase, "Partial shards failure (" + discrepancy + " shards unavailable)", null); + return; } } if (logger.isTraceEnabled()) { diff --git a/server/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java b/server/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java index 16df17bef1ada..174b164aead26 100644 --- a/server/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java @@ -83,7 +83,7 @@ private AbstractSearchAsyncAction createAction(SearchRequest return null; }; - return new AbstractSearchAsyncAction("test", null, null, nodeIdToConnection, + return new AbstractSearchAsyncAction("test", logger, null, nodeIdToConnection, Collections.singletonMap("foo", new AliasFilter(new MatchAllQueryBuilder())), Collections.singletonMap("foo", 2.0f), Collections.singletonMap("name", Sets.newHashSet("bar", "baz")), null, request, listener, new GroupShardsIterator<>( @@ -239,6 +239,29 @@ public void run() { assertEquals(requestIds, releasedContexts); } + public void testShardNotAvailableWithDisallowPartialFailures() { + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(false); + AtomicReference exception = new AtomicReference<>(); + ActionListener listener = ActionListener.wrap(response -> fail("onResponse should not be called"), exception::set); + int numShards = randomIntBetween(2, 10); + InitialSearchPhase.ArraySearchPhaseResults phaseResults = + new InitialSearchPhase.ArraySearchPhaseResults<>(numShards); + AbstractSearchAsyncAction action = createAction(searchRequest, phaseResults, listener, false, new AtomicLong()); + // skip one to avoid the "all shards failed" failure. + SearchShardIterator skipIterator = new SearchShardIterator(null, null, Collections.emptyList(), null); + skipIterator.resetAndSkip(); + action.skipShard(skipIterator); + // expect at least 2 shards, so onPhaseDone should report failure. + action.onPhaseDone(); + assertThat(exception.get(), instanceOf(SearchPhaseExecutionException.class)); + SearchPhaseExecutionException searchPhaseExecutionException = (SearchPhaseExecutionException)exception.get(); + assertEquals("Partial shards failure (" + (numShards - 1) + " shards unavailable)", + searchPhaseExecutionException.getMessage()); + assertEquals("test", searchPhaseExecutionException.getPhaseName()); + assertEquals(0, searchPhaseExecutionException.shardFailures().length); + assertEquals(0, searchPhaseExecutionException.getSuppressed().length); + } + private static InitialSearchPhase.ArraySearchPhaseResults phaseResults(Set requestIds, List> nodeLookups, int numFailures) {