Skip to content

Commit

Permalink
Ignoring unavailable shards during search request execution with igno…
Browse files Browse the repository at this point in the history
…re_available parameter

Signed-off-by: Ankit Jain <akjain@amazon.com>
  • Loading branch information
jainankitk committed Apr 19, 2024
1 parent 84679de commit 4cdb973
Show file tree
Hide file tree
Showing 3 changed files with 24 additions and 2 deletions.
1 change: 1 addition & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/),
- [BWC and API enforcement] Enforcing the presence of API annotations at build time ([#12872](https://github.com/opensearch-project/OpenSearch/pull/12872))
- Improve built-in secure transports support ([#12907](https://github.com/opensearch-project/OpenSearch/pull/12907))
- Update links to documentation in rest-api-spec ([#13043](https://github.com/opensearch-project/OpenSearch/pull/13043))
- Ignoring unavailable shards during search request execution with ignore_available parameter ([#13298](https://github.com/opensearch-project/OpenSearch/pull/13298))
- Refactoring globMatch using simpleMatchWithNormalizedStrings from Regex ([#13104](https://github.com/opensearch-project/OpenSearch/pull/13104))

### Deprecated
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -425,8 +425,10 @@ public final void executeNextPhase(SearchPhase currentPhase, SearchPhase nextPha
currentPhase.getName()
);
}
onPhaseFailure(currentPhase, "Partial shards failure (" + discrepancy + " shards unavailable)", null);
return;
if (!request.indicesOptions().ignoreUnavailable()) {
onPhaseFailure(currentPhase, "Partial shards failure (" + discrepancy + " shards unavailable)", null);
return;
}
}
}
if (logger.isTraceEnabled()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,7 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
Expand Down Expand Up @@ -442,6 +443,24 @@ public void testShardNotAvailableWithDisallowPartialFailures() {
assertEquals(0, searchPhaseExecutionException.getSuppressed().length);
}

public void testShardNotAvailableWithIgnoreUnavailable() {
SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(false)
.indicesOptions(new IndicesOptions(EnumSet.of(IndicesOptions.Option.IGNORE_UNAVAILABLE), IndicesOptions.WildcardStates.NONE));
AtomicReference<Exception> exception = new AtomicReference<>();
ActionListener<SearchResponse> listener = ActionListener.wrap(response -> {}, exception::set);
int numShards = randomIntBetween(2, 10);
ArraySearchPhaseResults<SearchPhaseResult> phaseResults = new 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);

// Validate no exception is thrown
action.executeNextPhase(action, createFetchSearchPhase());
action.sendSearchResponse(InternalSearchResponse.empty(), phaseResults.results);
}

private static ArraySearchPhaseResults<SearchPhaseResult> phaseResults(
Set<ShardSearchContextId> contextIds,
List<Tuple<String, String>> nodeLookups,
Expand Down

0 comments on commit 4cdb973

Please sign in to comment.