Skip to content

Commit 4210a29

Browse files
authored
Search of remote clusters with no shards results in successful status. (#100354) (#100458)
ccs_minimize_roundtrips=true was not checking if no shards were present to search before setting the cluster search status. If the number of shards is zero, then cluster search status should be 'successful'. This aligns ccs_minimize_roundtrips=true with the false setting and how it worked in earlier versions. Backported from #100354 Fixes bug #100350
1 parent 2c9392e commit 4210a29

File tree

4 files changed

+163
-3
lines changed

4 files changed

+163
-3
lines changed

docs/changelog/100354.yaml

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,5 @@
1+
pr: 100354
2+
summary: Search of remote clusters with no shards results in successful status
3+
area: Search
4+
type: bug
5+
issues: []

server/src/internalClusterTest/java/org/elasticsearch/search/ccs/CrossClusterSearchIT.java

Lines changed: 51 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -242,6 +242,57 @@ public void testClusterDetailsAfterCCSWithFailuresOnOneShardOnly() throws Except
242242
assertTrue("should have 'index corrupted' in reason", remoteShardSearchFailure.reason().contains("index corrupted"));
243243
}
244244

245+
// tests bug fix https://github.com/elastic/elasticsearch/issues/100350
246+
public void testClusterDetailsAfterCCSWhereRemoteClusterHasNoShardsToSearch() throws Exception {
247+
Map<String, Object> testClusterInfo = setupTwoClusters();
248+
String localIndex = (String) testClusterInfo.get("local.index");
249+
int localNumShards = (Integer) testClusterInfo.get("local.num_shards");
250+
251+
PlainActionFuture<SearchResponse> queryFuture = new PlainActionFuture<>();
252+
SearchRequest searchRequest = new SearchRequest(localIndex, REMOTE_CLUSTER + ":" + "no_such_index*");
253+
searchRequest.allowPartialSearchResults(false);
254+
boolean minimizeRoundtrips = randomBoolean();
255+
searchRequest.setCcsMinimizeRoundtrips(minimizeRoundtrips);
256+
257+
searchRequest.source(new SearchSourceBuilder().query(new MatchAllQueryBuilder()).size(1000));
258+
client(LOCAL_CLUSTER).search(searchRequest, queryFuture);
259+
260+
assertBusy(() -> assertTrue(queryFuture.isDone()));
261+
262+
SearchResponse searchResponse = queryFuture.get();
263+
assertNotNull(searchResponse);
264+
265+
SearchResponse.Clusters clusters = searchResponse.getClusters();
266+
assertFalse("search cluster results should NOT be marked as partial", clusters.hasPartialResults());
267+
assertThat(clusters.getTotal(), equalTo(2));
268+
assertThat(clusters.getSuccessful(), equalTo(2));
269+
assertThat(clusters.getSkipped(), equalTo(0));
270+
271+
if (minimizeRoundtrips) {
272+
SearchResponse.Cluster localClusterSearchInfo = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY).get();
273+
assertNotNull(localClusterSearchInfo);
274+
assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL));
275+
assertThat(localClusterSearchInfo.getIndexExpression(), equalTo(localIndex));
276+
assertThat(localClusterSearchInfo.getTotalShards(), equalTo(localNumShards));
277+
assertThat(localClusterSearchInfo.getSuccessfulShards(), equalTo(localNumShards));
278+
assertThat(localClusterSearchInfo.getSkippedShards(), equalTo(0));
279+
assertThat(localClusterSearchInfo.getFailedShards(), equalTo(0));
280+
assertThat(localClusterSearchInfo.getFailures().size(), equalTo(0));
281+
assertThat(localClusterSearchInfo.getTook().millis(), greaterThan(0L));
282+
283+
SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get();
284+
assertNotNull(remoteClusterSearchInfo);
285+
assertThat(remoteClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL));
286+
assertThat(remoteClusterSearchInfo.getIndexExpression(), equalTo("no_such_index*"));
287+
assertThat(remoteClusterSearchInfo.getTotalShards(), equalTo(0)); // zero since no shards to search
288+
assertThat(remoteClusterSearchInfo.getSuccessfulShards(), equalTo(0));
289+
assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(0));
290+
assertThat(remoteClusterSearchInfo.getFailedShards(), equalTo(0));
291+
assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(0));
292+
assertNotNull(remoteClusterSearchInfo.getTook());
293+
}
294+
}
295+
245296
public void testClusterDetailsAfterCCSWithFailuresOnRemoteClusterOnly() throws Exception {
246297
Map<String, Object> testClusterInfo = setupTwoClusters();
247298
String localIndex = (String) testClusterInfo.get("local.index");

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

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -800,14 +800,15 @@ private static void ccsClusterInfoUpdate(
800800
) {
801801
/*
802802
* Cluster Status logic:
803-
* 1) FAILED if all shards failed and skip_unavailable=false
804-
* 2) SKIPPED if all shards failed and skip_unavailable=true
803+
* 1) FAILED if total_shards > 0 && all shards failed && skip_unavailable=false
804+
* 2) SKIPPED if total_shards > 0 && all shards failed && skip_unavailable=true
805805
* 3) PARTIAL if it timed out
806806
* 4) PARTIAL if it at least one of the shards succeeded but not all
807807
* 5) SUCCESSFUL if no shards failed (and did not time out)
808808
*/
809809
SearchResponse.Cluster.Status status;
810-
if (searchResponse.getFailedShards() >= searchResponse.getTotalShards()) {
810+
int totalShards = searchResponse.getTotalShards();
811+
if (totalShards > 0 && searchResponse.getFailedShards() >= totalShards) {
811812
if (skipUnavailable) {
812813
status = SearchResponse.Cluster.Status.SKIPPED;
813814
} else {

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

Lines changed: 103 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -583,6 +583,109 @@ public void testClusterDetailsAfterCCSWithFailuresOnOneClusterOnly() throws Exce
583583
}
584584
}
585585

586+
// tests bug fix https://github.com/elastic/elasticsearch/issues/100350
587+
public void testClusterDetailsAfterCCSWhereRemoteClusterHasNoShardsToSearch() throws Exception {
588+
Map<String, Object> testClusterInfo = setupTwoClusters();
589+
String localIndex = (String) testClusterInfo.get("local.index");
590+
int localNumShards = (Integer) testClusterInfo.get("local.num_shards");
591+
592+
SearchListenerPlugin.blockQueryPhase();
593+
594+
SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(localIndex, REMOTE_CLUSTER + ":" + "no_such_index*");
595+
request.setCcsMinimizeRoundtrips(true);
596+
request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1));
597+
request.setKeepOnCompletion(true);
598+
request.getSearchRequest().source(new SearchSourceBuilder().query(new MatchAllQueryBuilder()).size(1000));
599+
600+
AsyncSearchResponse response = submitAsyncSearch(request);
601+
assertNotNull(response.getSearchResponse());
602+
assertTrue(response.isRunning());
603+
604+
{
605+
SearchResponse.Clusters clusters = response.getSearchResponse().getClusters();
606+
assertThat(clusters.getTotal(), equalTo(2));
607+
assertTrue("search cluster results should be marked as partial", clusters.hasPartialResults());
608+
609+
SearchResponse.Cluster localClusterSearchInfo = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY).get();
610+
assertNotNull(localClusterSearchInfo);
611+
assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.RUNNING));
612+
613+
SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get();
614+
assertNotNull(remoteClusterSearchInfo);
615+
assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.RUNNING));
616+
}
617+
618+
SearchListenerPlugin.waitSearchStarted();
619+
SearchListenerPlugin.allowQueryPhase();
620+
621+
assertBusy(() -> {
622+
AsyncStatusResponse statusResponse = getAsyncStatus(response.getId());
623+
assertFalse(statusResponse.isRunning());
624+
assertNotNull(statusResponse.getCompletionStatus());
625+
});
626+
627+
{
628+
AsyncSearchResponse finishedResponse = getAsyncSearch(response.getId());
629+
630+
SearchResponse.Clusters clusters = finishedResponse.getSearchResponse().getClusters();
631+
assertFalse("search cluster results should NOT be marked as partial", clusters.hasPartialResults());
632+
assertThat(clusters.getTotal(), equalTo(2));
633+
assertThat(clusters.getSuccessful(), equalTo(2));
634+
assertThat(clusters.getSkipped(), equalTo(0));
635+
636+
SearchResponse.Cluster localClusterSearchInfo = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY).get();
637+
assertNotNull(localClusterSearchInfo);
638+
assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL));
639+
assertThat(localClusterSearchInfo.getTotalShards(), equalTo(localNumShards));
640+
assertThat(localClusterSearchInfo.getSuccessfulShards(), equalTo(localNumShards));
641+
assertThat(localClusterSearchInfo.getSkippedShards(), equalTo(0));
642+
assertThat(localClusterSearchInfo.getFailedShards(), equalTo(0));
643+
assertThat(localClusterSearchInfo.getFailures().size(), equalTo(0));
644+
assertThat(localClusterSearchInfo.getTook().millis(), greaterThan(0L));
645+
646+
SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get();
647+
assertNotNull(remoteClusterSearchInfo);
648+
assertThat(remoteClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL));
649+
assertThat(remoteClusterSearchInfo.getTotalShards(), equalTo(0)); // zero since no shards to search
650+
assertThat(remoteClusterSearchInfo.getSuccessfulShards(), equalTo(0)); // zero since no shards to search
651+
assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(0));
652+
assertThat(remoteClusterSearchInfo.getFailedShards(), equalTo(0));
653+
assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(0));
654+
assertNotNull(remoteClusterSearchInfo.getTook());
655+
}
656+
657+
// check that the async_search/status response includes the same cluster details
658+
{
659+
AsyncStatusResponse statusResponse = getAsyncStatus(response.getId());
660+
661+
SearchResponse.Clusters clusters = statusResponse.getClusters();
662+
assertFalse("search cluster results should NOT be marked as partial", clusters.hasPartialResults());
663+
assertThat(clusters.getTotal(), equalTo(2));
664+
assertThat(clusters.getSuccessful(), equalTo(2));
665+
assertThat(clusters.getSkipped(), equalTo(0));
666+
667+
SearchResponse.Cluster localClusterSearchInfo = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY).get();
668+
assertNotNull(localClusterSearchInfo);
669+
assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL));
670+
assertThat(localClusterSearchInfo.getTotalShards(), equalTo(localNumShards));
671+
assertThat(localClusterSearchInfo.getSuccessfulShards(), equalTo(localNumShards));
672+
assertThat(localClusterSearchInfo.getSkippedShards(), equalTo(0));
673+
assertThat(localClusterSearchInfo.getFailedShards(), equalTo(0));
674+
assertThat(localClusterSearchInfo.getFailures().size(), equalTo(0));
675+
assertThat(localClusterSearchInfo.getTook().millis(), greaterThan(0L));
676+
677+
SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get();
678+
assertNotNull(remoteClusterSearchInfo);
679+
assertThat(remoteClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL));
680+
assertThat(remoteClusterSearchInfo.getTotalShards(), equalTo(0)); // zero since no shards to search
681+
assertThat(remoteClusterSearchInfo.getSuccessfulShards(), equalTo(0)); // zero since no shards to search
682+
assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(0));
683+
assertThat(remoteClusterSearchInfo.getFailedShards(), equalTo(0));
684+
assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(0));
685+
assertNotNull(remoteClusterSearchInfo.getTook());
686+
}
687+
}
688+
586689
public void testRemoteClusterOnlyCCSSuccessfulResult() throws Exception {
587690
// for remote-only queries, we can't use the SearchListenerPlugin since that listens for search
588691
// stage on the local cluster, so we only test final state of the search response

0 commit comments

Comments
 (0)