Skip to content

Commit

Permalink
Do not skip not available shard exception in search response
Browse files Browse the repository at this point in the history
 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 elastic#47700
  • Loading branch information
jimczi committed Oct 29, 2020
1 parent b56cd1e commit 2310c2f
Show file tree
Hide file tree
Showing 2 changed files with 12 additions and 10 deletions.
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)));
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) {
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

0 comments on commit 2310c2f

Please sign in to comment.