Skip to content

Commit 6d22901

Browse files
authored
Do not skip not available shard exception in search response (#64337)
Today search responses do not report failures for shard that were not available for the search. So if one shard is not assigned on a search over 5 shards, the search response will report: ``` "_shards": { "total": 5, "successful": 4, "skipped": 0, "failed": 0 } ``` If all shards are unassigned, we report a generic search phase exception with no cause. It's easy to spot that `successful` is less than `total` in the response but not reporting the failure is misleading for users. This change removes the special handling of not available shards exception in search responses and treat them as any other failure that could occur on a shard. These exceptions will count in the `failed` section and will be reported in details in the `shard_failures` section. If all shards are unavailable, the search API will now return 404 NOT_FOUND as an indication that the search failed because it couldn't find any of the resources. Closes #47700
1 parent 5b143b4 commit 6d22901

File tree

5 files changed

+32
-42
lines changed

5 files changed

+32
-42
lines changed

server/src/internalClusterTest/java/org/elasticsearch/search/basic/SearchRedStateIndexIT.java

Lines changed: 13 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -20,9 +20,11 @@
2020
package org.elasticsearch.search.basic;
2121

2222

23+
import org.elasticsearch.action.NoShardAvailableActionException;
2324
import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse;
2425
import org.elasticsearch.action.search.SearchPhaseExecutionException;
2526
import org.elasticsearch.action.search.SearchResponse;
27+
import org.elasticsearch.action.search.ShardSearchFailure;
2628
import org.elasticsearch.cluster.ClusterState;
2729
import org.elasticsearch.cluster.health.ClusterHealthStatus;
2830
import org.elasticsearch.cluster.routing.ShardRouting;
@@ -36,10 +38,13 @@
3638
import java.util.List;
3739

3840
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
41+
import static org.hamcrest.Matchers.allOf;
3942
import static org.hamcrest.Matchers.containsString;
4043
import static org.hamcrest.Matchers.equalTo;
4144
import static org.hamcrest.Matchers.greaterThan;
45+
import static org.hamcrest.Matchers.instanceOf;
4246
import static org.hamcrest.Matchers.lessThan;
47+
import static org.hamcrest.Matchers.lessThanOrEqualTo;
4348

4449
@ESIntegTestCase.ClusterScope(minNumDataNodes = 2)
4550
public class SearchRedStateIndexIT extends ESIntegTestCase {
@@ -52,10 +57,13 @@ public void testAllowPartialsWithRedState() throws Exception {
5257
SearchResponse searchResponse = client().prepareSearch().setSize(0).setAllowPartialSearchResults(true)
5358
.get();
5459
assertThat(RestStatus.OK, equalTo(searchResponse.status()));
55-
assertThat("Expect no shards failed", searchResponse.getFailedShards(), equalTo(0));
60+
assertThat("Expect some shards failed", searchResponse.getFailedShards(), allOf(greaterThan(0), lessThanOrEqualTo(numShards)));
5661
assertThat("Expect no shards skipped", searchResponse.getSkippedShards(), equalTo(0));
5762
assertThat("Expect subset of shards successful", searchResponse.getSuccessfulShards(), lessThan(numShards));
5863
assertThat("Expected total shards", searchResponse.getTotalShards(), equalTo(numShards));
64+
for (ShardSearchFailure failure : searchResponse.getShardFailures()) {
65+
assertThat(failure.getCause(), instanceOf(NoShardAvailableActionException.class));
66+
}
5967
}
6068

6169
public void testClusterAllowPartialsWithRedState() throws Exception {
@@ -66,10 +74,13 @@ public void testClusterAllowPartialsWithRedState() throws Exception {
6674

6775
SearchResponse searchResponse = client().prepareSearch().setSize(0).get();
6876
assertThat(RestStatus.OK, equalTo(searchResponse.status()));
69-
assertThat("Expect no shards failed", searchResponse.getFailedShards(), equalTo(0));
77+
assertThat("Expect some shards failed", searchResponse.getFailedShards(), allOf(greaterThan(0), lessThanOrEqualTo(numShards)));
7078
assertThat("Expect no shards skipped", searchResponse.getSkippedShards(), equalTo(0));
7179
assertThat("Expect subset of shards successful", searchResponse.getSuccessfulShards(), lessThan(numShards));
7280
assertThat("Expected total shards", searchResponse.getTotalShards(), equalTo(numShards));
81+
for (ShardSearchFailure failure : searchResponse.getShardFailures()) {
82+
assertThat(failure.getCause(), instanceOf(NoShardAvailableActionException.class));
83+
}
7384
}
7485

7586

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

Lines changed: 18 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,6 @@
3232
import org.elasticsearch.action.support.TransportActions;
3333
import org.elasticsearch.cluster.ClusterState;
3434
import org.elasticsearch.cluster.routing.GroupShardsIterator;
35-
import org.elasticsearch.common.Nullable;
3635
import org.elasticsearch.common.lease.Releasable;
3736
import org.elasticsearch.common.lease.Releasables;
3837
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
@@ -235,7 +234,9 @@ private void performPhaseOnShard(final int shardIndex, final SearchShardIterator
235234
* we can continue (cf. InitialSearchPhase#maybeFork).
236235
*/
237236
if (shard == null) {
238-
fork(() -> onShardFailure(shardIndex, null, shardIt, new NoShardAvailableActionException(shardIt.shardId())));
237+
SearchShardTarget unassignedShard = new SearchShardTarget(null, shardIt.shardId(),
238+
shardIt.getClusterAlias(), shardIt.getOriginalIndices());
239+
fork(() -> onShardFailure(shardIndex, unassignedShard, shardIt, new NoShardAvailableActionException(shardIt.shardId())));
239240
} else {
240241
final PendingExecutions pendingExecutions = throttleConcurrentRequests ?
241242
pendingExecutionsPerNode.computeIfAbsent(shard.getNodeId(), n -> new PendingExecutions(maxConcurrentRequestsPerNode))
@@ -386,14 +387,13 @@ ShardSearchFailure[] buildShardFailures() {
386387
return failures;
387388
}
388389

389-
private void onShardFailure(final int shardIndex, @Nullable SearchShardTarget shard, final SearchShardIterator shardIt, Exception e) {
390+
private void onShardFailure(final int shardIndex, SearchShardTarget shard, final SearchShardIterator shardIt, Exception e) {
390391
// we always add the shard failure for a specific shard instance
391392
// we do make sure to clean it on a successful response from a shard
392393
onShardFailure(shardIndex, shard, e);
393394
final SearchShardTarget nextShard = shardIt.nextOrNull();
394395
final boolean lastShard = nextShard == null;
395-
logger.debug(() -> new ParameterizedMessage("{}: Failed to execute [{}] lastShard [{}]",
396-
shard != null ? shard : shardIt.shardId(), request, lastShard), e);
396+
logger.debug(() -> new ParameterizedMessage("{}: Failed to execute [{}] lastShard [{}]", shard, request, lastShard), e);
397397
if (lastShard) {
398398
if (request.allowPartialSearchResults() == false) {
399399
if (requestCancelled.compareAndSet(false, true)) {
@@ -437,10 +437,15 @@ protected void onShardGroupFailure(int shardIndex, SearchShardTarget shardTarget
437437
* @param e the failure reason
438438
*/
439439
@Override
440-
public final void onShardFailure(final int shardIndex, @Nullable SearchShardTarget shardTarget, Exception e) {
441-
// we don't aggregate shard failures on non active shards and failures due to the internal cancellation,
440+
public final void onShardFailure(final int shardIndex, SearchShardTarget shardTarget, Exception e) {
441+
if (TransportActions.isShardNotAvailableException(e)) {
442+
// Groups shard not available exceptions under a generic exception that returns a SERVICE_UNAVAILABLE(503)
443+
// temporary error.
444+
e = new NoShardAvailableActionException(shardTarget.getShardId(), e.getMessage());
445+
}
446+
// we don't aggregate shard on failures due to the internal cancellation,
442447
// but do keep the header counts right
443-
if (TransportActions.isShardNotAvailableException(e) == false && (requestCancelled.get() && isTaskCancelledException(e)) == false) {
448+
if ((requestCancelled.get() && isTaskCancelledException(e)) == false) {
444449
AtomicArray<ShardSearchFailure> shardFailures = this.shardFailures.get();
445450
// lazily create shard failures, so we can early build the empty shard failure list in most cases (no failures)
446451
if (shardFailures == null) { // this is double checked locking but it's fine since SetOnce uses a volatile read internally
@@ -545,7 +550,11 @@ public final SearchRequest getRequest() {
545550

546551
protected final SearchResponse buildSearchResponse(InternalSearchResponse internalSearchResponse, ShardSearchFailure[] failures,
547552
String scrollId, String searchContextId) {
548-
return new SearchResponse(internalSearchResponse, scrollId, getNumShards(), successfulOps.get(),
553+
int numSuccess = successfulOps.get();
554+
int numFailures = failures.length;
555+
assert numSuccess + numFailures == getNumShards()
556+
: "numSuccess(" + numSuccess + ") + numFailures(" + numFailures + ") != totalShards(" + getNumShards() + ")";
557+
return new SearchResponse(internalSearchResponse, scrollId, getNumShards(), numSuccess,
549558
skippedOps.get(), buildTookInMillis(), failures, clusters, searchContextId);
550559
}
551560

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

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -204,8 +204,6 @@ public int getSkippedShards() {
204204
* The failed number of shards the search was executed on.
205205
*/
206206
public int getFailedShards() {
207-
// we don't return totalShards - successfulShards, we don't count "no shards available" as a failed shard, just don't
208-
// count it in the successful counter
209207
return shardFailures.length;
210208
}
211209

server/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java

Lines changed: 0 additions & 27 deletions
Original file line numberDiff line numberDiff line change
@@ -160,33 +160,6 @@ public void testBuildShardSearchTransportRequest() {
160160
assertEquals(clusterAlias, shardSearchTransportRequest.getClusterAlias());
161161
}
162162

163-
public void testBuildSearchResponse() {
164-
SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(randomBoolean());
165-
ArraySearchPhaseResults<SearchPhaseResult> phaseResults = new ArraySearchPhaseResults<>(10);
166-
AbstractSearchAsyncAction<SearchPhaseResult> action = createAction(searchRequest,
167-
phaseResults, null, false, new AtomicLong());
168-
InternalSearchResponse internalSearchResponse = InternalSearchResponse.empty();
169-
SearchResponse searchResponse = action.buildSearchResponse(internalSearchResponse, action.buildShardFailures(), null, null);
170-
assertSame(searchResponse.getAggregations(), internalSearchResponse.aggregations());
171-
assertSame(searchResponse.getSuggest(), internalSearchResponse.suggest());
172-
assertSame(searchResponse.getProfileResults(), internalSearchResponse.profile());
173-
assertSame(searchResponse.getHits(), internalSearchResponse.hits());
174-
}
175-
176-
public void testBuildSearchResponseAllowPartialFailures() {
177-
SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true);
178-
final ArraySearchPhaseResults<SearchPhaseResult> queryResult = new ArraySearchPhaseResults<>(10);
179-
AbstractSearchAsyncAction<SearchPhaseResult> action = createAction(searchRequest, queryResult, null, false, new AtomicLong());
180-
action.onShardFailure(0, new SearchShardTarget("node", new ShardId("index", "index-uuid", 0), null, OriginalIndices.NONE),
181-
new IllegalArgumentException());
182-
InternalSearchResponse internalSearchResponse = InternalSearchResponse.empty();
183-
SearchResponse searchResponse = action.buildSearchResponse(internalSearchResponse, action.buildShardFailures(), null, null);
184-
assertSame(searchResponse.getAggregations(), internalSearchResponse.aggregations());
185-
assertSame(searchResponse.getSuggest(), internalSearchResponse.suggest());
186-
assertSame(searchResponse.getProfileResults(), internalSearchResponse.profile());
187-
assertSame(searchResponse.getHits(), internalSearchResponse.hits());
188-
}
189-
190163
public void testSendSearchResponseDisallowPartialFailures() {
191164
SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(false);
192165
AtomicReference<Exception> exception = new AtomicReference<>();

x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/AsyncSearchActionIT.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -447,13 +447,12 @@ public void testRemoveAsyncIndex() throws Exception {
447447
ensureTaskRemoval(newResp.getId());
448448
}
449449

450-
public void testSearchPhaseFailureNoCause() throws Exception {
450+
public void testSearchPhaseFailure() throws Exception {
451451
SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(indexName);
452452
request.setKeepOnCompletion(true);
453453
request.setWaitForCompletionTimeout(TimeValue.timeValueMinutes(10));
454454
request.getSearchRequest().allowPartialSearchResults(false);
455455
request.getSearchRequest()
456-
// AlreadyClosedException are ignored by the coordinating node
457456
.source(new SearchSourceBuilder().query(new ThrowingQueryBuilder(randomLong(), new AlreadyClosedException("boom"), 0)));
458457
AsyncSearchResponse response = submitAsyncSearch(request);
459458
assertFalse(response.isRunning());

0 commit comments

Comments
 (0)