Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Do not skip not available shard exception in search response #64337

Merged
merged 9 commits into from
Nov 24, 2020
Original file line number Diff line number Diff line change
Expand Up @@ -36,10 +36,12 @@
import java.util.List;

import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.allOf;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.lessThan;
import static org.hamcrest.Matchers.lessThanOrEqualTo;

@ESIntegTestCase.ClusterScope(minNumDataNodes = 2)
public class SearchRedStateIndexIT extends ESIntegTestCase {
Expand All @@ -52,7 +54,7 @@ public void testAllowPartialsWithRedState() throws Exception {
SearchResponse searchResponse = client().prepareSearch().setSize(0).setAllowPartialSearchResults(true)
.get();
assertThat(RestStatus.OK, equalTo(searchResponse.status()));
assertThat("Expect no shards failed", searchResponse.getFailedShards(), equalTo(0));
assertThat("Expect some shards failed", searchResponse.getFailedShards(), allOf(greaterThan(0), lessThanOrEqualTo(numShards)));
jimczi marked this conversation as resolved.
Show resolved Hide resolved
assertThat("Expect no shards skipped", searchResponse.getSkippedShards(), equalTo(0));
assertThat("Expect subset of shards successful", searchResponse.getSuccessfulShards(), lessThan(numShards));
assertThat("Expected total shards", searchResponse.getTotalShards(), equalTo(numShards));
Expand All @@ -66,7 +68,7 @@ public void testClusterAllowPartialsWithRedState() throws Exception {

SearchResponse searchResponse = client().prepareSearch().setSize(0).get();
assertThat(RestStatus.OK, equalTo(searchResponse.status()));
assertThat("Expect no shards failed", searchResponse.getFailedShards(), equalTo(0));
assertThat("Expect some shards failed", searchResponse.getFailedShards(), allOf(greaterThan(0), lessThanOrEqualTo(numShards)));
assertThat("Expect no shards skipped", searchResponse.getSkippedShards(), equalTo(0));
assertThat("Expect subset of shards successful", searchResponse.getSuccessfulShards(), lessThan(numShards));
assertThat("Expected total shards", searchResponse.getTotalShards(), equalTo(numShards));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,6 @@
import org.elasticsearch.action.support.TransportActions;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
Expand Down Expand Up @@ -235,7 +234,9 @@ private void performPhaseOnShard(final int shardIndex, final SearchShardIterator
* we can continue (cf. InitialSearchPhase#maybeFork).
*/
if (shard == null) {
fork(() -> onShardFailure(shardIndex, null, shardIt, new NoShardAvailableActionException(shardIt.shardId())));
SearchShardTarget unassignedShard = new SearchShardTarget(null, shardIt.shardId(),
shardIt.getClusterAlias(), shardIt.getOriginalIndices());
fork(() -> onShardFailure(shardIndex, unassignedShard, shardIt, new NoShardAvailableActionException(shardIt.shardId())));
} else {
final PendingExecutions pendingExecutions = throttleConcurrentRequests ?
pendingExecutionsPerNode.computeIfAbsent(shard.getNodeId(), n -> new PendingExecutions(maxConcurrentRequestsPerNode))
Expand Down Expand Up @@ -386,14 +387,13 @@ ShardSearchFailure[] buildShardFailures() {
return failures;
}

private void onShardFailure(final int shardIndex, @Nullable SearchShardTarget shard, final SearchShardIterator shardIt, Exception e) {
private void onShardFailure(final int shardIndex, SearchShardTarget shard, final SearchShardIterator shardIt, Exception e) {
// we always add the shard failure for a specific shard instance
// we do make sure to clean it on a successful response from a shard
onShardFailure(shardIndex, shard, e);
final SearchShardTarget nextShard = shardIt.nextOrNull();
final boolean lastShard = nextShard == null;
logger.debug(() -> new ParameterizedMessage("{}: Failed to execute [{}] lastShard [{}]",
shard != null ? shard : shardIt.shardId(), request, lastShard), e);
logger.debug(() -> new ParameterizedMessage("{}: Failed to execute [{}] lastShard [{}]", shard, request, lastShard), e);
if (lastShard) {
if (request.allowPartialSearchResults() == false) {
if (requestCancelled.compareAndSet(false, true)) {
Expand Down Expand Up @@ -437,10 +437,10 @@ protected void onShardGroupFailure(int shardIndex, SearchShardTarget shardTarget
* @param e the failure reason
*/
@Override
public final void onShardFailure(final int shardIndex, @Nullable SearchShardTarget shardTarget, Exception e) {
// we don't aggregate shard failures on non active shards and failures due to the internal cancellation,
public final void onShardFailure(final int shardIndex, SearchShardTarget shardTarget, Exception e) {
// we don't aggregate shard on failures due to the internal cancellation,
// but do keep the header counts right
if (TransportActions.isShardNotAvailableException(e) == false && (requestCancelled.get() && isTaskCancelledException(e)) == false) {
if ((requestCancelled.get() && isTaskCancelledException(e)) == false) {
jimczi marked this conversation as resolved.
Show resolved Hide resolved
AtomicArray<ShardSearchFailure> shardFailures = this.shardFailures.get();
// lazily create shard failures, so we can early build the empty shard failure list in most cases (no failures)
if (shardFailures == null) { // this is double checked locking but it's fine since SetOnce uses a volatile read internally
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -406,18 +406,17 @@ public void testRemoveAsyncIndex() throws Exception {
ensureTaskRemoval(newResp.getId());
}

public void testSearchPhaseFailureNoCause() throws Exception {
public void testSearchPhaseFailure() throws Exception {
SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(indexName);
request.setKeepOnCompletion(true);
request.setWaitForCompletionTimeout(TimeValue.timeValueMinutes(10));
request.getSearchRequest().allowPartialSearchResults(false);
request.getSearchRequest()
// AlreadyClosedException are ignored by the coordinating node
.source(new SearchSourceBuilder().query(new ThrowingQueryBuilder(randomLong(), new AlreadyClosedException("boom"), 0)));
AsyncSearchResponse response = submitAsyncSearch(request);
assertFalse(response.isRunning());
assertTrue(response.isPartial());
assertThat(response.status(), equalTo(RestStatus.SERVICE_UNAVAILABLE));
assertThat(response.status(), equalTo(RestStatus.INTERNAL_SERVER_ERROR));
assertNotNull(response.getFailure());
ensureTaskNotRunning(response.getId());
}
Expand Down