Skip to content

Commit

Permalink
Search of remote clusters with no shards results in successful status. (
Browse files Browse the repository at this point in the history
#100354)

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.

Fixes bug #100350
  • Loading branch information
quux00 authored Oct 6, 2023
1 parent 8b9d413 commit a708501
Show file tree
Hide file tree
Showing 4 changed files with 188 additions and 4 deletions.
5 changes: 5 additions & 0 deletions docs/changelog/100354.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,5 @@
pr: 100354
summary: Search of remote clusters with no shards results in successful status
area: Search
type: bug
issues: []
Original file line number Diff line number Diff line change
Expand Up @@ -271,6 +271,66 @@ public void testClusterDetailsAfterCCSWithFailuresOnOneShardOnly() throws Except
assertOneFailedShard(remoteClusterSearchInfo, remoteNumShards);
}

// tests bug fix https://github.com/elastic/elasticsearch/issues/100350
public void testClusterDetailsAfterCCSWhereRemoteClusterHasNoShardsToSearch() throws Exception {
Map<String, Object> testClusterInfo = setupTwoClusters();
String localIndex = (String) testClusterInfo.get("local.index");
int localNumShards = (Integer) testClusterInfo.get("local.num_shards");

SearchRequest searchRequest = new SearchRequest(localIndex, REMOTE_CLUSTER + ":" + "no_such_index*");
if (randomBoolean()) {
searchRequest = searchRequest.scroll("1m");
}
searchRequest.allowPartialSearchResults(false);
if (randomBoolean()) {
searchRequest.setBatchedReduceSize(randomIntBetween(3, 20));
}
boolean minimizeRoundtrips = randomBoolean();
searchRequest.setCcsMinimizeRoundtrips(minimizeRoundtrips);
boolean dfs = randomBoolean();
if (dfs) {
searchRequest.searchType(SearchType.DFS_QUERY_THEN_FETCH);
}
if (randomBoolean()) {
searchRequest.setPreFilterShardSize(1);
}
searchRequest.source(new SearchSourceBuilder().query(new MatchAllQueryBuilder()).size(10));

SearchResponse searchResponse = client(LOCAL_CLUSTER).search(searchRequest).get();
assertNotNull(searchResponse);

SearchResponse.Clusters clusters = searchResponse.getClusters();
assertFalse("search cluster results should NOT be marked as partial", clusters.hasPartialResults());
assertThat(clusters.getTotal(), equalTo(2));
assertThat(clusters.getClusterStateCount(SearchResponse.Cluster.Status.SUCCESSFUL), equalTo(2));
assertThat(clusters.getClusterStateCount(SearchResponse.Cluster.Status.SKIPPED), equalTo(0));
assertThat(clusters.getClusterStateCount(SearchResponse.Cluster.Status.RUNNING), equalTo(0));
assertThat(clusters.getClusterStateCount(SearchResponse.Cluster.Status.PARTIAL), equalTo(0));
assertThat(clusters.getClusterStateCount(SearchResponse.Cluster.Status.FAILED), equalTo(0));

SearchResponse.Cluster localClusterSearchInfo = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY);
assertNotNull(localClusterSearchInfo);
assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL));
assertThat(localClusterSearchInfo.getIndexExpression(), equalTo(localIndex));
assertThat(localClusterSearchInfo.getTotalShards(), equalTo(localNumShards));
assertThat(localClusterSearchInfo.getSuccessfulShards(), equalTo(localNumShards));
assertThat(localClusterSearchInfo.getSkippedShards(), equalTo(0));
assertThat(localClusterSearchInfo.getFailedShards(), equalTo(0));
assertThat(localClusterSearchInfo.getFailures().size(), equalTo(0));
assertThat(localClusterSearchInfo.getTook().millis(), greaterThan(0L));

SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER);
assertNotNull(remoteClusterSearchInfo);
assertThat(remoteClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL));
assertThat(remoteClusterSearchInfo.getIndexExpression(), equalTo("no_such_index*"));
assertThat(remoteClusterSearchInfo.getTotalShards(), equalTo(0)); // no shards since index does not exist
assertThat(remoteClusterSearchInfo.getSuccessfulShards(), equalTo(0));
assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(0));
assertThat(remoteClusterSearchInfo.getFailedShards(), equalTo(0));
assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(0));
assertNotNull(remoteClusterSearchInfo.getTook());
}

public void testClusterDetailsAfterCCSWithFailuresOnRemoteClusterOnly() throws Exception {
Map<String, Object> testClusterInfo = setupTwoClusters();
String localIndex = (String) testClusterInfo.get("local.index");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -810,15 +810,16 @@ private static void ccsClusterInfoUpdate(
) {
/*
* Cluster Status logic:
* 1) FAILED if all shards failed and skip_unavailable=false
* 2) SKIPPED if all shards failed and skip_unavailable=true
* 1) FAILED if total_shards > 0 && all shards failed && skip_unavailable=false
* 2) SKIPPED if total_shards > 0 && all shards failed && skip_unavailable=true
* 3) PARTIAL if it timed out
* 4) PARTIAL if it at least one of the shards succeeded but not all
* 5) SUCCESSFUL if no shards failed (and did not time out)
*/
clusters.swapCluster(clusterAlias, (k, v) -> {
SearchResponse.Cluster.Status status;
if (searchResponse.getFailedShards() >= searchResponse.getTotalShards()) {
int totalShards = searchResponse.getTotalShards();
if (totalShards > 0 && searchResponse.getFailedShards() >= totalShards) {
if (skipUnavailable) {
status = SearchResponse.Cluster.Status.SKIPPED;
} else {
Expand All @@ -832,7 +833,7 @@ private static void ccsClusterInfoUpdate(
status = SearchResponse.Cluster.Status.SUCCESSFUL;
}
return new SearchResponse.Cluster.Builder(v).setStatus(status)
.setTotalShards(searchResponse.getTotalShards())
.setTotalShards(totalShards)
.setSuccessfulShards(searchResponse.getSuccessfulShards())
.setSkippedShards(searchResponse.getSkippedShards())
.setFailedShards(searchResponse.getFailedShards())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -759,6 +759,124 @@ public void testClusterDetailsAfterCCSWithFailuresOnOneClusterOnly() throws Exce
}
}

// tests bug fix https://github.com/elastic/elasticsearch/issues/100350
public void testClusterDetailsAfterCCSWhereRemoteClusterHasNoShardsToSearch() throws Exception {
Map<String, Object> testClusterInfo = setupTwoClusters();
String localIndex = (String) testClusterInfo.get("local.index");
int localNumShards = (Integer) testClusterInfo.get("local.num_shards");

SearchListenerPlugin.blockQueryPhase();

// query against a missing index on the remote cluster
SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(localIndex, REMOTE_CLUSTER + ":" + "no_such_index*");
request.setCcsMinimizeRoundtrips(randomBoolean());
request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1));
request.setKeepOnCompletion(true);
request.getSearchRequest().source(new SearchSourceBuilder().query(new MatchAllQueryBuilder()).size(10));
if (randomBoolean()) {
request.setBatchedReduceSize(randomIntBetween(2, 256));
}
boolean dfs = randomBoolean();
if (dfs) {
request.getSearchRequest().searchType(SearchType.DFS_QUERY_THEN_FETCH);
}

AsyncSearchResponse response = submitAsyncSearch(request);
assertNotNull(response.getSearchResponse());
assertTrue(response.isRunning());

boolean minimizeRoundtrips = TransportSearchAction.shouldMinimizeRoundtrips(request.getSearchRequest());

assertNotNull(response.getSearchResponse());
assertTrue(response.isRunning());
{
SearchResponse.Clusters clusters = response.getSearchResponse().getClusters();
assertThat(clusters.getTotal(), equalTo(2));
assertTrue("search cluster results should be marked as partial", clusters.hasPartialResults());

SearchResponse.Cluster localClusterSearchInfo = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY);
assertNotNull(localClusterSearchInfo);
assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.RUNNING));

SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER);
assertNotNull(remoteClusterSearchInfo);
assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.RUNNING));
}

SearchListenerPlugin.waitSearchStarted();
SearchListenerPlugin.allowQueryPhase();

waitForSearchTasksToFinish();

{
AsyncSearchResponse finishedResponse = getAsyncSearch(response.getId());
assertFalse(finishedResponse.isPartial());

SearchResponse.Clusters clusters = finishedResponse.getSearchResponse().getClusters();
assertThat(clusters.getTotal(), equalTo(2));
assertThat(clusters.getClusterStateCount(SearchResponse.Cluster.Status.SUCCESSFUL), equalTo(2));
assertThat(clusters.getClusterStateCount(SearchResponse.Cluster.Status.RUNNING), equalTo(0));
assertThat(clusters.getClusterStateCount(SearchResponse.Cluster.Status.PARTIAL), equalTo(0));
assertThat(clusters.getClusterStateCount(SearchResponse.Cluster.Status.SKIPPED), equalTo(0));
assertThat(clusters.getClusterStateCount(SearchResponse.Cluster.Status.FAILED), equalTo(0));

SearchResponse.Cluster localClusterSearchInfo = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY);
assertNotNull(localClusterSearchInfo);
assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL));
assertThat(localClusterSearchInfo.getTotalShards(), equalTo(localNumShards));
assertThat(localClusterSearchInfo.getSuccessfulShards(), equalTo(localNumShards));
assertThat(localClusterSearchInfo.getSkippedShards(), equalTo(0));
assertThat(localClusterSearchInfo.getFailedShards(), equalTo(0));
assertThat(localClusterSearchInfo.getFailures().size(), equalTo(0));
assertThat(localClusterSearchInfo.getTook().millis(), greaterThan(0L));

SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER);
assertThat(remoteClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL));
assertThat(remoteClusterSearchInfo.getTotalShards(), equalTo(0)); // will be zero since index does not index
assertThat(remoteClusterSearchInfo.getSuccessfulShards(), equalTo(0));
assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(0));
assertThat(remoteClusterSearchInfo.getFailedShards(), equalTo(0));
assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(0));

assertNotNull(remoteClusterSearchInfo.getTook());
assertFalse(remoteClusterSearchInfo.isTimedOut());
}
// check that the async_search/status response includes the same cluster details
{
AsyncStatusResponse statusResponse = getAsyncStatus(response.getId());
assertFalse(statusResponse.isPartial());

SearchResponse.Clusters clusters = statusResponse.getClusters();
assertThat(clusters.getTotal(), equalTo(2));
assertThat(clusters.getClusterStateCount(SearchResponse.Cluster.Status.SUCCESSFUL), equalTo(2));
assertThat(clusters.getClusterStateCount(SearchResponse.Cluster.Status.RUNNING), equalTo(0));
assertThat(clusters.getClusterStateCount(SearchResponse.Cluster.Status.PARTIAL), equalTo(0));
assertThat(clusters.getClusterStateCount(SearchResponse.Cluster.Status.SKIPPED), equalTo(0));
assertThat(clusters.getClusterStateCount(SearchResponse.Cluster.Status.FAILED), equalTo(0));

SearchResponse.Cluster localClusterSearchInfo = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY);
assertNotNull(localClusterSearchInfo);
assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL));
assertThat(localClusterSearchInfo.getTotalShards(), equalTo(localNumShards));
assertThat(localClusterSearchInfo.getSuccessfulShards(), equalTo(localNumShards));
assertThat(localClusterSearchInfo.getSkippedShards(), equalTo(0));
assertThat(localClusterSearchInfo.getFailedShards(), equalTo(0));
assertThat(localClusterSearchInfo.getFailures().size(), equalTo(0));
assertThat(localClusterSearchInfo.getTook().millis(), greaterThan(0L));

SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER);
assertThat(remoteClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL));
assertThat(remoteClusterSearchInfo.getTotalShards(), equalTo(0)); // will be zero since index does not index
assertThat(remoteClusterSearchInfo.getSuccessfulShards(), equalTo(0));
assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(0));
assertThat(remoteClusterSearchInfo.getFailedShards(), equalTo(0));
assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(0));

assertNotNull(remoteClusterSearchInfo.getTook());
assertFalse(remoteClusterSearchInfo.isTimedOut());
}
}

public void testCCSWithSearchTimeout() throws Exception {
Map<String, Object> testClusterInfo = setupTwoClusters();
String localIndex = (String) testClusterInfo.get("local.index");
Expand Down

0 comments on commit a708501

Please sign in to comment.