Disallow partial results when shard unavailable (#45739)

Searching with `allowPartialSearchResults=false` could still return
partial search results during recovery. If a shard copy fails
with a "shard not available" exception, the failure would be ignored and
a partial result returned. The one case where this is known to happen
is when a shard copy is recovering when searching, since
`IllegalIndexShardStateException` is considered a "shard not available"
exception.

Relates to #42612
This commit is contained in:
Henning Andersen 2019-08-27 14:17:15 +02:00 committed by Henning Andersen
parent 873ad3f942
commit 300e717e42
2 changed files with 34 additions and 2 deletions

View File

@ -140,7 +140,7 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
} 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 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
}
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()) {

View File

@ -83,7 +83,7 @@ public class AbstractSearchAsyncActionTests extends ESTestCase {
return null;
};
return new AbstractSearchAsyncAction<SearchPhaseResult>("test", null, null, nodeIdToConnection,
return new AbstractSearchAsyncAction<SearchPhaseResult>("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 class AbstractSearchAsyncActionTests extends ESTestCase {
assertEquals(requestIds, releasedContexts);
}
public void testShardNotAvailableWithDisallowPartialFailures() {
SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(false);
AtomicReference<Exception> exception = new AtomicReference<>();
ActionListener<SearchResponse> listener = ActionListener.wrap(response -> fail("onResponse should not be called"), exception::set);
int numShards = randomIntBetween(2, 10);
InitialSearchPhase.ArraySearchPhaseResults<SearchPhaseResult> phaseResults =
new InitialSearchPhase.ArraySearchPhaseResults<>(numShards);
AbstractSearchAsyncAction<SearchPhaseResult> 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<SearchPhaseResult> phaseResults(Set<Long> requestIds,
List<Tuple<String, String>> nodeLookups,
int numFailures) {