Fail a DFS_QUERY_THEN_FETCH request if all shards failed the QUERY phase.
Today, if some shards pass the DFS phase but all of them fail the QUERY phase, the response will only consist of failed shards. We should throw an exception instead in order to be consistent with the QUERY_THEN_FETCH type.
This commit is contained in:
parent
cb8139a583
commit
d8880f2906
|
@ -22,10 +22,7 @@ package org.elasticsearch.action.search.type;
|
||||||
import com.carrotsearch.hppc.IntArrayList;
|
import com.carrotsearch.hppc.IntArrayList;
|
||||||
import org.apache.lucene.search.ScoreDoc;
|
import org.apache.lucene.search.ScoreDoc;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
import org.elasticsearch.action.search.ReduceSearchPhaseException;
|
import org.elasticsearch.action.search.*;
|
||||||
import org.elasticsearch.action.search.SearchOperationThreading;
|
|
||||||
import org.elasticsearch.action.search.SearchRequest;
|
|
||||||
import org.elasticsearch.action.search.SearchResponse;
|
|
||||||
import org.elasticsearch.cluster.ClusterService;
|
import org.elasticsearch.cluster.ClusterService;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
import org.elasticsearch.common.inject.Inject;
|
import org.elasticsearch.common.inject.Inject;
|
||||||
|
@ -170,7 +167,11 @@ public class TransportSearchDfsQueryThenFetchAction extends TransportSearchTypeA
|
||||||
this.addShardFailure(shardIndex, dfsResult.shardTarget(), t);
|
this.addShardFailure(shardIndex, dfsResult.shardTarget(), t);
|
||||||
successfulOps.decrementAndGet();
|
successfulOps.decrementAndGet();
|
||||||
if (counter.decrementAndGet() == 0) {
|
if (counter.decrementAndGet() == 0) {
|
||||||
executeFetchPhase();
|
if (successfulOps.get() == 0) {
|
||||||
|
listener.onFailure(new SearchPhaseExecutionException("query", "all shards failed", buildShardFailures()));
|
||||||
|
} else {
|
||||||
|
executeFetchPhase();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue