mirror of
https://github.com/elastic/elasticsearch.git
synced 2025-04-24 15:17:30 -04:00
Add counters to _clusters response for all states (#99566)
To help the user know what the possible cluster states are and to provide an accurate accounting, we added counters summarising `running`, `partial` and `failed` clusters to the `_clusters` section. Changes: - Now in the response is present the number of `running` clusters. - We split up `partial` and `successful` (before was summed up in the `successful` counter). - We now have a counter for `failed` clusters. - Now `total` is always equal to `running` + `skipped` + `failed` + `partial` + `successful`.
This commit is contained in:
parent
292722d1fe
commit
d9c15c526e
13 changed files with 507 additions and 206 deletions
|
@ -1072,8 +1072,26 @@ public class CCSDuelIT extends ESRestTestCase {
|
|||
SearchResponse.Clusters clustersMRTFalse = fanOutSearchResponse.getClusters();
|
||||
|
||||
assertEquals(clustersMRT.getTotal(), clustersMRTFalse.getTotal());
|
||||
assertEquals(clustersMRT.getSuccessful(), clustersMRTFalse.getSuccessful());
|
||||
assertEquals(clustersMRT.getSkipped(), clustersMRTFalse.getSkipped());
|
||||
assertEquals(
|
||||
clustersMRT.getClusterStateCount(SearchResponse.Cluster.Status.SUCCESSFUL),
|
||||
clustersMRTFalse.getClusterStateCount(SearchResponse.Cluster.Status.SUCCESSFUL)
|
||||
);
|
||||
assertEquals(
|
||||
clustersMRT.getClusterStateCount(SearchResponse.Cluster.Status.SKIPPED),
|
||||
clustersMRTFalse.getClusterStateCount(SearchResponse.Cluster.Status.SKIPPED)
|
||||
);
|
||||
assertEquals(
|
||||
clustersMRT.getClusterStateCount(SearchResponse.Cluster.Status.RUNNING),
|
||||
clustersMRTFalse.getClusterStateCount(SearchResponse.Cluster.Status.RUNNING)
|
||||
);
|
||||
assertEquals(
|
||||
clustersMRT.getClusterStateCount(SearchResponse.Cluster.Status.PARTIAL),
|
||||
clustersMRTFalse.getClusterStateCount(SearchResponse.Cluster.Status.PARTIAL)
|
||||
);
|
||||
assertEquals(
|
||||
clustersMRT.getClusterStateCount(SearchResponse.Cluster.Status.FAILED),
|
||||
clustersMRTFalse.getClusterStateCount(SearchResponse.Cluster.Status.FAILED)
|
||||
);
|
||||
|
||||
Map<String, Object> minimizeRoundtripsResponseMap = responseToMap(minimizeRoundtripsSearchResponse);
|
||||
if (clustersMRT.hasClusterObjects() && clustersMRTFalse.hasClusterObjects()) {
|
||||
|
@ -1144,8 +1162,26 @@ public class CCSDuelIT extends ESRestTestCase {
|
|||
SearchResponse.Clusters clustersMRTFalse = fanOutSearchResponse.getClusters();
|
||||
|
||||
assertEquals(clustersMRT.getTotal(), clustersMRTFalse.getTotal());
|
||||
assertEquals(clustersMRT.getSuccessful(), clustersMRTFalse.getSuccessful());
|
||||
assertEquals(clustersMRT.getSkipped(), clustersMRTFalse.getSkipped());
|
||||
assertEquals(
|
||||
clustersMRT.getClusterStateCount(SearchResponse.Cluster.Status.SUCCESSFUL),
|
||||
clustersMRTFalse.getClusterStateCount(SearchResponse.Cluster.Status.SUCCESSFUL)
|
||||
);
|
||||
assertEquals(
|
||||
clustersMRT.getClusterStateCount(SearchResponse.Cluster.Status.SKIPPED),
|
||||
clustersMRTFalse.getClusterStateCount(SearchResponse.Cluster.Status.SKIPPED)
|
||||
);
|
||||
assertEquals(
|
||||
clustersMRT.getClusterStateCount(SearchResponse.Cluster.Status.RUNNING),
|
||||
clustersMRTFalse.getClusterStateCount(SearchResponse.Cluster.Status.RUNNING)
|
||||
);
|
||||
assertEquals(
|
||||
clustersMRT.getClusterStateCount(SearchResponse.Cluster.Status.PARTIAL),
|
||||
clustersMRTFalse.getClusterStateCount(SearchResponse.Cluster.Status.PARTIAL)
|
||||
);
|
||||
assertEquals(
|
||||
clustersMRT.getClusterStateCount(SearchResponse.Cluster.Status.FAILED),
|
||||
clustersMRTFalse.getClusterStateCount(SearchResponse.Cluster.Status.FAILED)
|
||||
);
|
||||
|
||||
Map<String, Object> minimizeRoundtripsResponseMap = responseToMap(minimizeRoundtripsSearchResponse);
|
||||
if (clustersMRT.hasClusterObjects() && clustersMRTFalse.hasClusterObjects()) {
|
||||
|
@ -1229,14 +1265,20 @@ public class CCSDuelIT extends ESRestTestCase {
|
|||
|
||||
private static void assertMultiClusterSearchResponse(SearchResponse searchResponse) {
|
||||
assertEquals(2, searchResponse.getClusters().getTotal());
|
||||
assertEquals(2, searchResponse.getClusters().getSuccessful());
|
||||
// for bwc checks we expect SUCCESSFUL + PARTIAL to be equal to 2
|
||||
int bwcSuccessful = searchResponse.getClusters().getClusterStateCount(SearchResponse.Cluster.Status.SUCCESSFUL);
|
||||
bwcSuccessful += searchResponse.getClusters().getClusterStateCount(SearchResponse.Cluster.Status.PARTIAL);
|
||||
assertEquals(2, bwcSuccessful);
|
||||
assertEquals(0, searchResponse.getClusters().getClusterStateCount(SearchResponse.Cluster.Status.SKIPPED));
|
||||
assertEquals(0, searchResponse.getClusters().getClusterStateCount(SearchResponse.Cluster.Status.RUNNING));
|
||||
assertEquals(0, searchResponse.getClusters().getClusterStateCount(SearchResponse.Cluster.Status.FAILED));
|
||||
assertThat(searchResponse.getTotalShards(), greaterThan(1));
|
||||
assertThat(searchResponse.getSuccessfulShards(), greaterThan(1));
|
||||
}
|
||||
|
||||
private static void assertSingleRemoteClusterSearchResponse(SearchResponse searchResponse) {
|
||||
assertEquals(1, searchResponse.getClusters().getTotal());
|
||||
assertEquals(1, searchResponse.getClusters().getSuccessful());
|
||||
assertEquals(1, searchResponse.getClusters().getClusterStateCount(SearchResponse.Cluster.Status.SUCCESSFUL));
|
||||
assertThat(searchResponse.getTotalShards(), greaterThanOrEqualTo(1));
|
||||
assertThat(searchResponse.getSuccessfulShards(), greaterThanOrEqualTo(1));
|
||||
}
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue