Explorar el Código

Collect health API stats (#91559)

This PR introduces the collectors of the health API telemetry. Our
target telemetry has the following shape:

```
{
    "invocations": {
      "total": 22,
      "verbose_true": 12,
      "verbose_false": 10
    },
    "statuses": {
      "green": 10,
      "yellow": 4,
      "red": 8,
      "values": ["green", "yellow", "red"]
    },
    "indicators": {
      "red" : {
        "master_stability": 2,
        "ilm":2,
        "slm": 4,
        "values": ["master_stability", "ilm", "slm"]
      },
      "yellow": {
        "disk": 1,
        "shards_availability": 1,
        "master_stability": 2,
        "values": ["disk", "shards_availability", "master_stability"]
      }
    },
    "diagnoses": {
      "red": {
        "elasticsearch:health:shards_availability:primary_unassigned": 1,
        "elasticsearch:health:disk:add_disk_capacity_master_nodes": 3,
        "values": ["elasticsearch:health:shards_availability:primary_unassigned", "elasticsearch:health:disk:add_disk_capacity_master_nodes"]
      },
      "yellow": {
        "elasticsearch:health:disk:add_disk_capacity_data_nodes": 1,
        "values": [""elasticsearch:health:disk:add_disk_capacity_data_nodes"]
      }
    }
  }
```

This PR introduces the thread safe `Counters` class and the
`HealthApiStats` which keeps keeps of the metrics above based on the
health api responses that it encounters. The `HealthApiStatsAction`
collects the `HealthApiStats` of all nodes.

Part of: #90877
Mary Gouseti hace 2 años
padre
commit
8e9a403c65

+ 5 - 0
docs/changelog/91559.yaml

@@ -0,0 +1,5 @@
+pr: 91559
+summary: Health api stats
+area: Health
+type: enhancement
+issues: []

+ 37 - 0
server/src/internalClusterTest/java/org/elasticsearch/health/GetHealthActionIT.java

@@ -15,11 +15,13 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
 import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
+import org.elasticsearch.common.metrics.Counters;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.env.NodeEnvironment;
 import org.elasticsearch.health.node.HealthInfo;
+import org.elasticsearch.health.stats.HealthApiStatsAction;
 import org.elasticsearch.plugins.HealthPlugin;
 import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.repositories.RepositoriesService;
@@ -33,15 +35,18 @@ import org.elasticsearch.xcontent.NamedXContentRegistry;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
+import java.util.Set;
 import java.util.concurrent.ExecutionException;
 import java.util.function.Supplier;
 import java.util.stream.Stream;
 
 import static org.elasticsearch.common.util.CollectionUtils.appendToCopy;
 import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
 import static org.hamcrest.Matchers.instanceOf;
 
 @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST)
@@ -204,6 +209,38 @@ public class GetHealthActionIT extends ESIntegTestCase {
                 assertThat(exception.getCause(), instanceOf(ResourceNotFoundException.class));
             }
 
+            // Check health api stats
+            {
+                HealthApiStatsAction.Response response = client.execute(HealthApiStatsAction.INSTANCE, new HealthApiStatsAction.Request())
+                    .get();
+                Counters stats = response.getStats();
+                assertThat(stats.get("invocations.total"), equalTo(4L));
+                assertThat(stats.get("invocations.verbose_true"), equalTo(2L));
+                assertThat(stats.get("invocations.verbose_false"), equalTo(2L));
+                assertThat(
+                    stats.get("invocations.verbose_true") + stats.get("invocations.verbose_false"),
+                    equalTo(stats.get("invocations.total"))
+                );
+                HealthStatus mostSevereHealthStatus = HealthStatus.merge(
+                    Stream.of(ilmIndicatorStatus, slmIndicatorStatus, clusterCoordinationIndicatorStatus)
+                );
+                assertThat(stats.get("statuses." + mostSevereHealthStatus.xContentValue()), greaterThanOrEqualTo(2L));
+                assertThat(stats.get("statuses." + ilmIndicatorStatus.xContentValue()), greaterThanOrEqualTo(2L));
+                String label = "indicators." + ilmIndicatorStatus.xContentValue() + ".ilm";
+                if (ilmIndicatorStatus != HealthStatus.GREEN) {
+                    assertThat(stats.get(label), greaterThanOrEqualTo(4L));
+                } else {
+                    expectThrows(IllegalArgumentException.class, () -> stats.get(label));
+                }
+                Set<HealthStatus> expectedStatuses = new HashSet<>();
+                expectedStatuses.add(ilmIndicatorStatus);
+                expectedStatuses.add(mostSevereHealthStatus);
+                assertThat(response.getStatuses(), equalTo(expectedStatuses));
+                if (mostSevereHealthStatus != HealthStatus.GREEN || ilmIndicatorStatus != HealthStatus.GREEN) {
+                    assertThat(response.getIndicators().isEmpty(), equalTo(mostSevereHealthStatus == HealthStatus.GREEN));
+                }
+            }
+
         } finally {
             updateClusterSettings(
                 Settings.builder()

+ 1 - 0
server/src/main/java/module-info.java

@@ -218,6 +218,7 @@ module org.elasticsearch.server {
     exports org.elasticsearch.health;
     exports org.elasticsearch.health.node;
     exports org.elasticsearch.health.node.selection;
+    exports org.elasticsearch.health.stats;
     exports org.elasticsearch.http;
     exports org.elasticsearch.index;
     exports org.elasticsearch.index.analysis;

+ 3 - 0
server/src/main/java/org/elasticsearch/action/ActionModule.java

@@ -269,6 +269,8 @@ import org.elasticsearch.health.GetHealthAction;
 import org.elasticsearch.health.RestGetHealthAction;
 import org.elasticsearch.health.node.FetchHealthInfoCacheAction;
 import org.elasticsearch.health.node.UpdateHealthInfoCacheAction;
+import org.elasticsearch.health.stats.HealthApiStatsAction;
+import org.elasticsearch.health.stats.HealthApiStatsTransportAction;
 import org.elasticsearch.index.seqno.GlobalCheckpointSyncAction;
 import org.elasticsearch.index.seqno.RetentionLeaseActions;
 import org.elasticsearch.indices.SystemIndices;
@@ -566,6 +568,7 @@ public class ActionModule extends AbstractModule {
         actions.register(CancelTasksAction.INSTANCE, TransportCancelTasksAction.class);
         actions.register(GetHealthAction.INSTANCE, GetHealthAction.TransportAction.class);
         actions.register(PrevalidateNodeRemovalAction.INSTANCE, TransportPrevalidateNodeRemovalAction.class);
+        actions.register(HealthApiStatsAction.INSTANCE, HealthApiStatsTransportAction.class);
 
         actions.register(AddVotingConfigExclusionsAction.INSTANCE, TransportAddVotingConfigExclusionsAction.class);
         actions.register(ClearVotingConfigExclusionsAction.INSTANCE, TransportClearVotingConfigExclusionsAction.class);

+ 139 - 0
server/src/main/java/org/elasticsearch/common/metrics/Counters.java

@@ -0,0 +1,139 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+
+package org.elasticsearch.common.metrics;
+
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.io.stream.Writeable;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Simple usage stat counters based on longs. Internally this is a map mapping from String which is the counter's label to a CounterMetric
+ * which is the value. This class also provides a helper method that converts the counters to a nested map, using the "." in the label as a
+ * splitter. This allows the stats producer to not worry about how the map is actually nested.
+
+ * IMPORTANT: if the consumer of the metrics will make use of the nested map, it is the responsibility of the producer to provide labels
+ * that will not have conflicts, which means that there no counter will have a label which is a substring of the label of another counter.
+ * For example, the counters `foo: 1` and `foo.bar: 3` cannot co-exist in a nested map.
+ */
+public class Counters implements Writeable {
+
+    private final ConcurrentMap<String, CounterMetric> counters = new ConcurrentHashMap<>();
+
+    public Counters(StreamInput in) throws IOException {
+        int numCounters = in.readVInt();
+        for (int i = 0; i < numCounters; i++) {
+            inc(in.readString(), in.readVLong());
+        }
+    }
+
+    public Counters(String... names) {
+        for (String name : names) {
+            counters.put(name, new CounterMetric());
+        }
+    }
+
+    /**
+     * Increment the counter by one
+     * @param name Name of the counter
+     */
+    public void inc(String name) {
+        inc(name, 1);
+    }
+
+    /**
+     * Increment the counter by configured number
+     * @param name The name of the counter
+     * @param count Incremental value
+     */
+    public void inc(String name, long count) {
+        counters.computeIfAbsent(name, ignored -> new CounterMetric()).inc(count);
+    }
+
+    public long get(String name) {
+        if (counters.containsKey(name)) {
+            return counters.get(name).count();
+        }
+        throw new IllegalArgumentException("Counter with name " + name + " does not exist.");
+    }
+
+    public long size() {
+        return counters.size();
+    }
+
+    public boolean hasCounters() {
+        return counters.isEmpty() == false;
+    }
+
+    /**
+     * Convert the counters to a nested map, using the "." as a splitter to create a nested map. For example, the counters `foo.bar`: 2,
+     * `foo.baz`: 1, `foobar`: 5 would become:
+     * {
+     *     "foo": {
+     *         "bar": 2,
+     *         "baz": 1
+     *     },
+     *     "foobar": 5
+     * }
+     * @return A nested map with all the current configured counters
+     * @throws IllegalStateException if there is a conflict in a path of two counters for example `foo`: 1 and `foo.bar`: 1.
+     */
+    @SuppressWarnings("unchecked")
+    public Map<String, Object> toNestedMap() {
+        Map<String, Object> root = new HashMap<>();
+        for (var counter : counters.entrySet()) {
+            Map<String, Object> currentLevel = root;
+            String[] parts = counter.getKey().split("\\.");
+            for (int i = 0; i < parts.length - 1; i++) {
+                if (currentLevel.get(parts[i]) == null || currentLevel.get(parts[i]) instanceof Map) {
+                    currentLevel = (Map<String, Object>) currentLevel.computeIfAbsent(parts[i], k -> new HashMap<String, Object>());
+                } else {
+                    throw new IllegalStateException(
+                        "Failed to convert counter '" + counter.getKey() + "' because '" + parts[i] + "' is already a leaf."
+                    );
+                }
+            }
+            String leaf = parts[parts.length - 1];
+            if (currentLevel.containsKey(leaf)) {
+                throw new IllegalStateException(
+                    "Failed to convert counter '" + counter.getKey() + "' because this is the path of another metric."
+                );
+            } else {
+                currentLevel.put(leaf, counter.getValue().count());
+            }
+        }
+        return root;
+    }
+
+    @Override
+    public void writeTo(StreamOutput out) throws IOException {
+        out.writeVInt(counters.size());
+        for (var entry : counters.entrySet()) {
+            out.writeString(entry.getKey());
+            out.writeVLong(entry.getValue().count());
+        }
+    }
+
+    public static Counters merge(List<Counters> counters) {
+        Counters result = new Counters();
+        for (Counters c : counters) {
+            for (var entry : c.counters.entrySet()) {
+                result.inc(entry.getKey(), entry.getValue().count());
+            }
+        }
+
+        return result;
+    }
+}

+ 6 - 2
server/src/main/java/org/elasticsearch/health/Diagnosis.java

@@ -140,7 +140,11 @@ public record Diagnosis(Definition definition, @Nullable List<Resource> affected
      * @param action A description of the action to be taken to remedy the problem
      * @param helpURL Optional evergreen url to a help document
      */
-    public record Definition(String indicatorName, String id, String cause, String action, String helpURL) {}
+    public record Definition(String indicatorName, String id, String cause, String action, String helpURL) {
+        public String getUniqueId() {
+            return HEALTH_API_ID_PREFIX + indicatorName + ":diagnosis:" + id;
+        }
+    }
 
     @Override
     public Iterator<? extends ToXContent> toXContentChunked() {
@@ -152,7 +156,7 @@ public record Diagnosis(Definition definition, @Nullable List<Resource> affected
         }
         return Iterators.concat(Iterators.single((ToXContent) (builder, params) -> {
             builder.startObject();
-            builder.field("id", HEALTH_API_ID_PREFIX + definition.indicatorName + ":diagnosis:" + definition.id);
+            builder.field("id", definition.getUniqueId());
             builder.field("cause", definition.cause);
             builder.field("action", definition.action);
             builder.field("help_url", definition.helpURL);

+ 14 - 13
server/src/main/java/org/elasticsearch/health/GetHealthAction.java

@@ -23,6 +23,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ChunkedToXContent;
 import org.elasticsearch.core.Nullable;
+import org.elasticsearch.health.stats.HealthApiStats;
 import org.elasticsearch.tasks.CancellableTask;
 import org.elasticsearch.tasks.Task;
 import org.elasticsearch.tasks.TaskId;
@@ -81,6 +82,10 @@ public class GetHealthAction extends ActionType<GetHealthAction.Response> {
                 .orElseThrow(() -> new NoSuchElementException("Indicator [" + name + "] is not found"));
         }
 
+        public List<HealthIndicatorResult> getIndicatorResults() {
+            return indicators;
+        }
+
         @Override
         public void writeTo(StreamOutput out) throws IOException {
             throw new AssertionError("GetHealthAction should not be sent over the wire.");
@@ -169,6 +174,7 @@ public class GetHealthAction extends ActionType<GetHealthAction.Response> {
         private final ClusterService clusterService;
         private final HealthService healthService;
         private final NodeClient client;
+        private final HealthApiStats healthApiStats;
 
         @Inject
         public TransportAction(
@@ -176,29 +182,24 @@ public class GetHealthAction extends ActionType<GetHealthAction.Response> {
             TransportService transportService,
             ClusterService clusterService,
             HealthService healthService,
-            NodeClient client
+            NodeClient client,
+            HealthApiStats healthApiStats
         ) {
             super(NAME, actionFilters, transportService.getTaskManager());
             this.clusterService = clusterService;
             this.healthService = healthService;
             this.client = client;
+            this.healthApiStats = healthApiStats;
         }
 
         @Override
         protected void doExecute(Task task, Request request, ActionListener<Response> responseListener) {
             assert task instanceof CancellableTask;
-            healthService.getHealth(
-                client,
-                request.indicatorName,
-                request.verbose,
-                responseListener.map(
-                    healthIndicatorResults -> new Response(
-                        clusterService.getClusterName(),
-                        healthIndicatorResults,
-                        request.indicatorName == null
-                    )
-                )
-            );
+            healthService.getHealth(client, request.indicatorName, request.verbose, responseListener.map(healthIndicatorResults -> {
+                Response response = new Response(clusterService.getClusterName(), healthIndicatorResults, request.indicatorName == null);
+                healthApiStats.track(request.verbose, response);
+                return response;
+            }));
         }
     }
 }

+ 12 - 0
server/src/main/java/org/elasticsearch/health/HealthStatus.java

@@ -9,6 +9,7 @@
 package org.elasticsearch.health;
 
 import org.elasticsearch.cluster.coordination.CoordinationDiagnosticsService;
+import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
 
@@ -29,6 +30,17 @@ public enum HealthStatus implements Writeable {
         this.value = value;
     }
 
+    public static HealthStatus read(StreamInput in) throws IOException {
+        byte value = in.readByte();
+        return switch (value) {
+            case 0 -> GREEN;
+            case 1 -> UNKNOWN;
+            case 2 -> YELLOW;
+            case 3 -> RED;
+            default -> throw new IllegalArgumentException("unknown value for health status [" + value + "]");
+        };
+    }
+
     @Override
     public void writeTo(StreamOutput out) throws IOException {
         out.writeByte(value);

+ 107 - 0
server/src/main/java/org/elasticsearch/health/stats/HealthApiStats.java

@@ -0,0 +1,107 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+
+package org.elasticsearch.health.stats;
+
+import org.elasticsearch.common.metrics.Counters;
+import org.elasticsearch.core.Strings;
+import org.elasticsearch.health.Diagnosis;
+import org.elasticsearch.health.GetHealthAction;
+import org.elasticsearch.health.HealthIndicatorResult;
+import org.elasticsearch.health.HealthStatus;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+/**
+ * This class tracks the health api calls and counts the statuses that have been encountered along with the unhealthy indicators and
+ * diagnoses.
+ */
+public class HealthApiStats {
+
+    private static final String TOTAL_INVOCATIONS = "invocations.total";
+    private static final String VERBOSE_TRUE = "invocations.verbose_true";
+    private static final String VERBOSE_FALSE = "invocations.verbose_false";
+    private final Function<HealthStatus, String> statusLabel = status -> Strings.format("statuses.%s", status.xContentValue());
+    private final BiFunction<HealthStatus, String, String> indicatorLabel = (status, indicator) -> Strings.format(
+        "indicators.%s.%s",
+        status.xContentValue(),
+        indicator
+    );
+    private final BiFunction<HealthStatus, String, String> diagnosisLabel = (status, diagnosis) -> Strings.format(
+        "diagnoses.%s.%s",
+        status.xContentValue(),
+        diagnosis
+    );
+
+    private final Set<HealthStatus> statuses = ConcurrentHashMap.newKeySet();
+    private final ConcurrentMap<HealthStatus, Set<String>> indicators = new ConcurrentHashMap<>();
+    private final ConcurrentMap<HealthStatus, Set<String>> diagnoses = new ConcurrentHashMap<>();
+    private final Counters stats = new Counters(TOTAL_INVOCATIONS);
+
+    public HealthApiStats() {}
+
+    public void track(boolean verbose, GetHealthAction.Response response) {
+        stats.inc(TOTAL_INVOCATIONS);
+        if (verbose) {
+            stats.inc(VERBOSE_TRUE);
+        } else {
+            stats.inc(VERBOSE_FALSE);
+        }
+
+        // The response status could be null because of a drill-down API call, in this case
+        // we can use the status of the drilled down indicator
+        HealthStatus status = response.getStatus() != null
+            ? response.getStatus()
+            : response.getIndicatorResults().stream().map(HealthIndicatorResult::status).findFirst().orElse(null);
+        if (status != null) {
+            stats.inc(statusLabel.apply(status));
+            statuses.add(status);
+        }
+
+        if (status != HealthStatus.GREEN) {
+            for (HealthIndicatorResult indicator : response.getIndicatorResults()) {
+                if (indicator.status() != HealthStatus.GREEN) {
+                    stats.inc(indicatorLabel.apply(indicator.status(), indicator.name()));
+                    indicators.computeIfAbsent(indicator.status(), k -> ConcurrentHashMap.newKeySet()).add(indicator.name());
+                    if (indicator.diagnosisList() != null) {
+                        for (Diagnosis diagnosis : indicator.diagnosisList()) {
+                            stats.inc(diagnosisLabel.apply(indicator.status(), diagnosis.definition().getUniqueId()));
+                            diagnoses.computeIfAbsent(indicator.status(), k -> ConcurrentHashMap.newKeySet())
+                                .add(diagnosis.definition().getUniqueId());
+                        }
+                    }
+                }
+            }
+        }
+    }
+
+    public boolean hasCounters() {
+        return stats.hasCounters();
+    }
+
+    public Counters getStats() {
+        return stats;
+    }
+
+    public Map<HealthStatus, Set<String>> getIndicators() {
+        return Map.copyOf(indicators);
+    }
+
+    public Map<HealthStatus, Set<String>> getDiagnoses() {
+        return Map.copyOf(diagnoses);
+    }
+
+    public Set<HealthStatus> getStatuses() {
+        return Set.copyOf(statuses);
+    }
+}

+ 193 - 0
server/src/main/java/org/elasticsearch/health/stats/HealthApiStatsAction.java

@@ -0,0 +1,193 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+
+package org.elasticsearch.health.stats;
+
+import org.elasticsearch.action.ActionRequestValidationException;
+import org.elasticsearch.action.ActionType;
+import org.elasticsearch.action.FailedNodeException;
+import org.elasticsearch.action.support.nodes.BaseNodeResponse;
+import org.elasticsearch.action.support.nodes.BaseNodesRequest;
+import org.elasticsearch.action.support.nodes.BaseNodesResponse;
+import org.elasticsearch.cluster.ClusterName;
+import org.elasticsearch.cluster.node.DiscoveryNode;
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.metrics.Counters;
+import org.elasticsearch.core.Nullable;
+import org.elasticsearch.health.HealthStatus;
+import org.elasticsearch.transport.TransportRequest;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+/**
+ * This class collects the stats of the health API from every node
+ */
+public class HealthApiStatsAction extends ActionType<HealthApiStatsAction.Response> {
+
+    public static final HealthApiStatsAction INSTANCE = new HealthApiStatsAction();
+    public static final String NAME = "cluster:monitor/health_api/stats";
+
+    private HealthApiStatsAction() {
+        super(NAME, Response::new);
+    }
+
+    public static class Request extends BaseNodesRequest<Request> {
+
+        public Request() {
+            super((String[]) null);
+        }
+
+        public Request(StreamInput in) throws IOException {
+            super(in);
+        }
+
+        @Override
+        public ActionRequestValidationException validate() {
+            return null;
+        }
+
+        @Override
+        public void writeTo(StreamOutput out) throws IOException {
+            super.writeTo(out);
+        }
+
+        @Override
+        public String toString() {
+            return "health_api_stats";
+        }
+
+        public static class Node extends TransportRequest {
+
+            public Node(StreamInput in) throws IOException {
+                super(in);
+            }
+
+            public Node(Request ignored) {}
+
+            @Override
+            public void writeTo(StreamOutput out) throws IOException {
+                super.writeTo(out);
+            }
+        }
+    }
+
+    public static class Response extends BaseNodesResponse<Response.Node> {
+
+        public Response(StreamInput in) throws IOException {
+            super(in);
+        }
+
+        public Response(ClusterName clusterName, List<Node> nodes, List<FailedNodeException> failures) {
+            super(clusterName, nodes, failures);
+        }
+
+        @Override
+        public void writeTo(StreamOutput out) throws IOException {
+            super.writeTo(out);
+        }
+
+        @Override
+        protected List<Node> readNodesFrom(StreamInput in) throws IOException {
+            return in.readList(Node::new);
+        }
+
+        @Override
+        protected void writeNodesTo(StreamOutput out, List<Node> nodes) throws IOException {
+            out.writeList(nodes);
+        }
+
+        public Counters getStats() {
+            List<Counters> counters = getNodes().stream().map(Node::getStats).filter(Objects::nonNull).toList();
+            return Counters.merge(counters);
+        }
+
+        public Set<HealthStatus> getStatuses() {
+            Set<HealthStatus> statuses = new HashSet<>();
+            for (Node node : getNodes()) {
+                statuses.addAll(node.statuses);
+            }
+            return statuses;
+        }
+
+        public Map<HealthStatus, Set<String>> getIndicators() {
+            Map<HealthStatus, Set<String>> indicators = new HashMap<>();
+            for (Node node : getNodes()) {
+                for (HealthStatus status : node.indicators.keySet()) {
+                    indicators.computeIfAbsent(status, s -> new HashSet<>()).addAll(node.indicators.get(status));
+                }
+            }
+            return indicators;
+        }
+
+        public Map<HealthStatus, Set<String>> getDiagnoses() {
+            Map<HealthStatus, Set<String>> diagnoses = new HashMap<>();
+            for (Node node : getNodes()) {
+                for (HealthStatus status : node.diagnoses.keySet()) {
+                    diagnoses.computeIfAbsent(status, s -> new HashSet<>()).addAll(node.diagnoses.get(status));
+                }
+            }
+            return diagnoses;
+        }
+
+        public static class Node extends BaseNodeResponse {
+            @Nullable
+            private Counters stats;
+            private Set<HealthStatus> statuses = Set.of();
+            private Map<HealthStatus, Set<String>> indicators = Map.of();
+            private Map<HealthStatus, Set<String>> diagnoses = Map.of();
+
+            public Node(StreamInput in) throws IOException {
+                super(in);
+                stats = in.readOptionalWriteable(Counters::new);
+                statuses = in.readSet(HealthStatus::read);
+                indicators = in.readMap(HealthStatus::read, input -> input.readSet(StreamInput::readString));
+                diagnoses = in.readMap(HealthStatus::read, input -> input.readSet(StreamInput::readString));
+            }
+
+            public Node(DiscoveryNode node) {
+                super(node);
+            }
+
+            public Counters getStats() {
+                return stats;
+            }
+
+            public void setStats(Counters stats) {
+                this.stats = stats;
+            }
+
+            public void setStatuses(Set<HealthStatus> statuses) {
+                this.statuses = statuses;
+            }
+
+            public void setIndicators(Map<HealthStatus, Set<String>> indicators) {
+                this.indicators = indicators;
+            }
+
+            public void setDiagnoses(Map<HealthStatus, Set<String>> diagnoses) {
+                this.diagnoses = diagnoses;
+            }
+
+            @Override
+            public void writeTo(StreamOutput out) throws IOException {
+                super.writeTo(out);
+                out.writeOptionalWriteable(stats);
+                out.writeCollection(statuses);
+                out.writeMap(indicators, StreamOutput::writeEnum, (o, v) -> o.writeCollection(v, StreamOutput::writeString));
+                out.writeMap(diagnoses, StreamOutput::writeEnum, (o, v) -> o.writeCollection(v, StreamOutput::writeString));
+            }
+        }
+    }
+}

+ 87 - 0
server/src/main/java/org/elasticsearch/health/stats/HealthApiStatsTransportAction.java

@@ -0,0 +1,87 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+package org.elasticsearch.health.stats;
+
+import org.elasticsearch.action.FailedNodeException;
+import org.elasticsearch.action.support.ActionFilters;
+import org.elasticsearch.action.support.nodes.TransportNodesAction;
+import org.elasticsearch.cluster.node.DiscoveryNode;
+import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.common.inject.Inject;
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.tasks.Task;
+import org.elasticsearch.threadpool.ThreadPool;
+import org.elasticsearch.transport.TransportService;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Performs the health api stats operation.
+ */
+public class HealthApiStatsTransportAction extends TransportNodesAction<
+    HealthApiStatsAction.Request,
+    HealthApiStatsAction.Response,
+    HealthApiStatsAction.Request.Node,
+    HealthApiStatsAction.Response.Node> {
+
+    private final HealthApiStats healthApiStats;
+
+    @Inject
+    public HealthApiStatsTransportAction(
+        TransportService transportService,
+        ClusterService clusterService,
+        ThreadPool threadPool,
+        ActionFilters actionFilters,
+        HealthApiStats healthApiStats
+    ) {
+        super(
+            HealthApiStatsAction.NAME,
+            threadPool,
+            clusterService,
+            transportService,
+            actionFilters,
+            HealthApiStatsAction.Request::new,
+            HealthApiStatsAction.Request.Node::new,
+            ThreadPool.Names.MANAGEMENT,
+            HealthApiStatsAction.Response.Node.class
+        );
+        this.healthApiStats = healthApiStats;
+    }
+
+    @Override
+    protected HealthApiStatsAction.Response newResponse(
+        HealthApiStatsAction.Request request,
+        List<HealthApiStatsAction.Response.Node> nodes,
+        List<FailedNodeException> failures
+    ) {
+        return new HealthApiStatsAction.Response(clusterService.getClusterName(), nodes, failures);
+    }
+
+    @Override
+    protected HealthApiStatsAction.Request.Node newNodeRequest(HealthApiStatsAction.Request request) {
+        return new HealthApiStatsAction.Request.Node(request);
+    }
+
+    @Override
+    protected HealthApiStatsAction.Response.Node newNodeResponse(StreamInput in, DiscoveryNode node) throws IOException {
+        return new HealthApiStatsAction.Response.Node(in);
+    }
+
+    @Override
+    protected HealthApiStatsAction.Response.Node nodeOperation(HealthApiStatsAction.Request.Node request, Task task) {
+        HealthApiStatsAction.Response.Node statsResponse = new HealthApiStatsAction.Response.Node(clusterService.localNode());
+        if (healthApiStats.hasCounters()) {
+            statsResponse.setStats(healthApiStats.getStats());
+        }
+        statsResponse.setStatuses(healthApiStats.getStatuses());
+        statsResponse.setIndicators(healthApiStats.getIndicators());
+        statsResponse.setDiagnoses(healthApiStats.getDiagnoses());
+        return statsResponse;
+    }
+}

+ 3 - 0
server/src/main/java/org/elasticsearch/node/Node.java

@@ -113,6 +113,7 @@ import org.elasticsearch.health.node.DiskHealthIndicatorService;
 import org.elasticsearch.health.node.HealthInfoCache;
 import org.elasticsearch.health.node.LocalHealthMonitor;
 import org.elasticsearch.health.node.selection.HealthNodeTaskExecutor;
+import org.elasticsearch.health.stats.HealthApiStats;
 import org.elasticsearch.http.HttpServerTransport;
 import org.elasticsearch.index.IndexSettingProvider;
 import org.elasticsearch.index.IndexSettingProviders;
@@ -1001,6 +1002,7 @@ public class Node implements Closeable {
             HealthMetadataService healthMetadataService = HealthMetadataService.create(clusterService, settings);
             LocalHealthMonitor localHealthMonitor = LocalHealthMonitor.create(settings, clusterService, nodeService, threadPool, client);
             HealthInfoCache nodeHealthOverview = HealthInfoCache.create(clusterService);
+            HealthApiStats healthApiStats = new HealthApiStats();
 
             modules.add(b -> {
                 b.bind(Node.class).toInstance(this);
@@ -1089,6 +1091,7 @@ public class Node implements Closeable {
                 b.bind(HealthMetadataService.class).toInstance(healthMetadataService);
                 b.bind(LocalHealthMonitor.class).toInstance(localHealthMonitor);
                 b.bind(HealthInfoCache.class).toInstance(nodeHealthOverview);
+                b.bind(HealthApiStats.class).toInstance(healthApiStats);
                 b.bind(Tracer.class).toInstance(tracer);
                 b.bind(FileSettingsService.class).toInstance(fileSettingsService);
                 b.bind(WriteLoadForecaster.class).toInstance(writeLoadForecaster);

+ 78 - 0
server/src/test/java/org/elasticsearch/common/metrics/CountersTests.java

@@ -0,0 +1,78 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+
+package org.elasticsearch.common.metrics;
+
+import org.elasticsearch.test.ESTestCase;
+
+import java.util.List;
+import java.util.Map;
+
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasEntry;
+import static org.hamcrest.Matchers.hasKey;
+import static org.hamcrest.Matchers.instanceOf;
+
+public class CountersTests extends ESTestCase {
+
+    @SuppressWarnings("unchecked")
+    public void testCounters() {
+        Counters counters = new Counters();
+        counters.inc("f", 200);
+        counters.inc("foo.bar");
+        counters.inc("foo.baz");
+        counters.inc("foo.baz");
+        assertThat(counters.get("f"), equalTo(200L));
+        assertThat(counters.get("foo.bar"), equalTo(1L));
+        assertThat(counters.get("foo.baz"), equalTo(2L));
+        expectThrows(IllegalArgumentException.class, () -> counters.get("foo"));
+        Map<String, Object> map = counters.toNestedMap();
+        assertThat(map, hasEntry("f", 200L));
+        assertThat(map, hasKey("foo"));
+        assertThat(map.get("foo"), instanceOf(Map.class));
+        Map<String, Object> fooMap = (Map<String, Object>) map.get("foo");
+        assertThat(fooMap, hasEntry("bar", 1L));
+        assertThat(fooMap, hasEntry("baz", 2L));
+    }
+
+    public void testMerging() {
+        Counters counters1 = new Counters();
+        counters1.inc("f", 200);
+        counters1.inc("foo.bar");
+        counters1.inc("foo.baz");
+        counters1.inc("foo.baz");
+        Counters counters2 = new Counters();
+        Counters counters3 = new Counters();
+        counters3.inc("foo.bar", 2);
+        counters3.inc("bar.foo");
+        Counters mergedCounters = Counters.merge(List.of(counters1, counters2, counters3));
+        assertThat(mergedCounters.get("f"), equalTo(200L));
+        assertThat(mergedCounters.get("foo.bar"), equalTo(3L));
+        assertThat(mergedCounters.get("foo.baz"), equalTo(2L));
+        assertThat(mergedCounters.get("bar.foo"), equalTo(1L));
+    }
+
+    public void testPathConflictNestedMapConversion() {
+        Counters counters = new Counters();
+        counters.inc("foo.bar");
+        counters.inc("foo");
+        counters.inc("foo.baz");
+        assertThat(counters.get("foo.bar"), equalTo(1L));
+        assertThat(counters.get("foo"), equalTo(1L));
+
+        try {
+            counters.toNestedMap();
+            fail("Expected an IllegalStateException but got no exception");
+        } catch (IllegalStateException e) {
+            assertThat(e.getMessage(), containsString("Failed to convert counter 'foo"));
+        } catch (Exception e) {
+            fail("Expected an IllegalStateException but got " + e.getClass().getName());
+        }
+    }
+}

+ 20 - 2
server/src/test/java/org/elasticsearch/health/HealthStatusTests.java

@@ -8,7 +8,8 @@
 
 package org.elasticsearch.health;
 
-import org.elasticsearch.test.ESTestCase;
+import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.test.AbstractWireSerializingTestCase;
 
 import java.util.ArrayList;
 import java.util.stream.Stream;
@@ -17,8 +18,9 @@ import static org.elasticsearch.health.HealthStatus.GREEN;
 import static org.elasticsearch.health.HealthStatus.RED;
 import static org.elasticsearch.health.HealthStatus.UNKNOWN;
 import static org.elasticsearch.health.HealthStatus.YELLOW;
+import static org.hamcrest.Matchers.equalTo;
 
-public class HealthStatusTests extends ESTestCase {
+public class HealthStatusTests extends AbstractWireSerializingTestCase<HealthStatus> {
 
     public void testAllGreenStatuses() {
         assertEquals(GREEN, HealthStatus.merge(randomStatusesContaining(GREEN)));
@@ -54,4 +56,20 @@ public class HealthStatusTests extends ESTestCase {
         }
         return result.stream();
     }
+
+    @Override
+    protected Writeable.Reader<HealthStatus> instanceReader() {
+        return HealthStatus::read;
+    }
+
+    @Override
+    protected HealthStatus createTestInstance() {
+        return randomFrom(HealthStatus.values());
+    }
+
+    @Override
+    protected void assertEqualInstances(HealthStatus expectedInstance, HealthStatus newInstance) {
+        assertThat(newInstance, equalTo(expectedInstance));
+        assertThat(newInstance.hashCode(), equalTo(expectedInstance.hashCode()));
+    }
 }

+ 98 - 0
server/src/test/java/org/elasticsearch/health/stats/HealthApiStatsActionResponseTests.java

@@ -0,0 +1,98 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+
+package org.elasticsearch.health.stats;
+
+import org.elasticsearch.Version;
+import org.elasticsearch.cluster.ClusterName;
+import org.elasticsearch.cluster.node.DiscoveryNode;
+import org.elasticsearch.common.metrics.Counters;
+import org.elasticsearch.health.HealthStatus;
+import org.elasticsearch.test.ESTestCase;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.Matchers.equalTo;
+
+public class HealthApiStatsActionResponseTests extends ESTestCase {
+
+    public void testMerging() {
+
+        HealthApiStatsAction.Response.Node nodeResponse1 = new HealthApiStatsAction.Response.Node(
+            new DiscoveryNode("remote_node", buildNewFakeTransportAddress(), Version.CURRENT)
+        );
+        {
+            Counters counters = new Counters();
+            counters.inc("merged.metric", randomIntBetween(1, 10));
+            counters.inc("only.one.metric", randomIntBetween(1, 10));
+            nodeResponse1.setStats(counters);
+            nodeResponse1.setStatuses(Set.of(HealthStatus.RED, HealthStatus.GREEN));
+            nodeResponse1.setIndicators(Map.of(HealthStatus.RED, Set.of("one_red_indicator", "shared_red_indicator")));
+            nodeResponse1.setDiagnoses(
+                Map.of(HealthStatus.RED, Set.of("one:red:diagnosis"), HealthStatus.YELLOW, Set.of("one:yellow:diagnosis"))
+            );
+        }
+        HealthApiStatsAction.Response.Node nodeResponse2 = new HealthApiStatsAction.Response.Node(
+            new DiscoveryNode("remote_node", buildNewFakeTransportAddress(), Version.CURRENT)
+        );
+
+        HealthApiStatsAction.Response.Node nodeResponse3 = new HealthApiStatsAction.Response.Node(
+            new DiscoveryNode("remote_node", buildNewFakeTransportAddress(), Version.CURRENT)
+        );
+        {
+            Counters counters = new Counters();
+            counters.inc("merged.metric", randomIntBetween(1, 10));
+            counters.inc("only.third.metric", randomIntBetween(1, 10));
+            nodeResponse3.setStats(counters);
+            nodeResponse3.setStatuses(Set.of(HealthStatus.YELLOW, HealthStatus.GREEN));
+            nodeResponse3.setIndicators(
+                Map.of(HealthStatus.RED, Set.of("shared_red_indicator"), HealthStatus.YELLOW, Set.of("one_yellow_indicator"))
+            );
+            nodeResponse3.setDiagnoses(
+                Map.of(HealthStatus.RED, Set.of("another:red:diagnosis"), HealthStatus.YELLOW, Set.of("another:yellow:diagnosis"))
+            );
+        }
+        HealthApiStatsAction.Response response = new HealthApiStatsAction.Response(
+            ClusterName.DEFAULT,
+            List.of(nodeResponse1, nodeResponse2, nodeResponse3),
+            List.of()
+        );
+        Counters stats = response.getStats();
+        assertThat(
+            stats.get("merged.metric"),
+            equalTo(nodeResponse1.getStats().get("merged.metric") + nodeResponse3.getStats().get("merged.metric"))
+        );
+        assertThat(stats.get("only.one.metric"), equalTo(nodeResponse1.getStats().get("only.one.metric")));
+        assertThat(stats.get("only.third.metric"), equalTo(nodeResponse3.getStats().get("only.third.metric")));
+        assertThat(response.getStatuses(), equalTo(Set.of(HealthStatus.RED, HealthStatus.YELLOW, HealthStatus.GREEN)));
+        assertThat(
+            response.getIndicators(),
+            equalTo(
+                Map.of(
+                    HealthStatus.RED,
+                    Set.of("shared_red_indicator", "one_red_indicator"),
+                    HealthStatus.YELLOW,
+                    Set.of("one_yellow_indicator")
+                )
+            )
+        );
+        assertThat(
+            response.getDiagnoses(),
+            equalTo(
+                Map.of(
+                    HealthStatus.RED,
+                    Set.of("one:red:diagnosis", "another:red:diagnosis"),
+                    HealthStatus.YELLOW,
+                    Set.of("one:yellow:diagnosis", "another:yellow:diagnosis")
+                )
+            )
+        );
+    }
+}

+ 187 - 0
server/src/test/java/org/elasticsearch/health/stats/HealthApiStatsTests.java

@@ -0,0 +1,187 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+
+package org.elasticsearch.health.stats;
+
+import org.elasticsearch.cluster.ClusterName;
+import org.elasticsearch.health.Diagnosis;
+import org.elasticsearch.health.GetHealthAction;
+import org.elasticsearch.health.HealthIndicatorResult;
+import org.elasticsearch.health.HealthStatus;
+import org.elasticsearch.test.ESTestCase;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.elasticsearch.health.HealthStatus.GREEN;
+import static org.elasticsearch.health.HealthStatus.RED;
+import static org.elasticsearch.health.HealthStatus.YELLOW;
+import static org.hamcrest.Matchers.equalTo;
+
+public class HealthApiStatsTests extends ESTestCase {
+
+    public void testEmptyCounters() {
+        HealthApiStats healthApiStats = new HealthApiStats();
+        assertThat(healthApiStats.getStats().get("invocations.total"), equalTo(0L));
+        Map<String, Object> metricMap = healthApiStats.getStats().toNestedMap();
+        assertThat(metricMap.containsKey("statuses"), equalTo(false));
+        assertThat(metricMap.containsKey("indicators"), equalTo(false));
+        assertThat(metricMap.containsKey("diagnoses"), equalTo(false));
+    }
+
+    @SuppressWarnings("unchecked")
+    public void testCounters() {
+        HealthApiStats healthApiStats = new HealthApiStats();
+
+        {
+            // All green
+            var indicators = List.of(
+                createHealthIndicatorResult("network_latency", GREEN),
+                createHealthIndicatorResult("shards_availability", GREEN)
+            );
+            GetHealthAction.Response response = new GetHealthAction.Response(ClusterName.DEFAULT, indicators, true);
+
+            healthApiStats.track(false, response);
+            assertThat(healthApiStats.getStats().get("invocations.total"), equalTo(1L));
+            assertThat(healthApiStats.getStats().get("invocations.verbose_false"), equalTo(1L));
+            expectThrows(IllegalArgumentException.class, () -> healthApiStats.getStats().get("invocations.verbose_true"));
+            assertThat(healthApiStats.getStats().get("statuses.green"), equalTo(1L));
+            Map<String, Object> metricMap = healthApiStats.getStats().toNestedMap();
+            assertThat(((Map<String, Object>) metricMap.get("statuses")).size(), equalTo(1));
+            assertThat(metricMap.containsKey("indicators"), equalTo(false));
+            assertThat(metricMap.containsKey("diagnoses"), equalTo(false));
+            assertThat(healthApiStats.getStatuses(), equalTo(Set.of(GREEN)));
+            assertThat(healthApiStats.getIndicators().isEmpty(), equalTo(true));
+            assertThat(healthApiStats.getDiagnoses().isEmpty(), equalTo(true));
+        }
+        {
+            // Yellow no diagnosis
+            var indicators = List.of(
+                createHealthIndicatorResult("network_latency", GREEN),
+                createHealthIndicatorResult("shards_availability", YELLOW)
+            );
+            GetHealthAction.Response response = new GetHealthAction.Response(ClusterName.DEFAULT, indicators, true);
+
+            healthApiStats.track(true, response);
+            assertThat(healthApiStats.getStats().get("invocations.total"), equalTo(2L));
+            assertThat(healthApiStats.getStats().get("invocations.verbose_true"), equalTo(1L));
+            assertThat(healthApiStats.getStats().get("invocations.verbose_false"), equalTo(1L));
+            assertThat(healthApiStats.getStats().get("statuses.green"), equalTo(1L));
+            assertThat(healthApiStats.getStats().get("statuses.yellow"), equalTo(1L));
+            assertThat(healthApiStats.getStats().get("indicators.yellow.shards_availability"), equalTo(1L));
+            Map<String, Object> metricMap = healthApiStats.getStats().toNestedMap();
+            assertThat(metricMap.containsKey("diagnoses"), equalTo(false));
+            assertThat(healthApiStats.getStatuses(), equalTo(Set.of(GREEN, YELLOW)));
+            assertThat(healthApiStats.getIndicators().get(YELLOW), equalTo(Set.of("shards_availability")));
+            assertThat(healthApiStats.getDiagnoses().isEmpty(), equalTo(true));
+        }
+        {
+            // Yellow and Red with diagnoses
+            var indicators = List.of(
+                createHealthIndicatorResult("network_latency", GREEN),
+                createHealthIndicatorResult("disk", YELLOW, "add_disk_capacity_data_nodes"),
+                createHealthIndicatorResult("shards_availability", RED, "primary_unassigned")
+            );
+            GetHealthAction.Response response = new GetHealthAction.Response(ClusterName.DEFAULT, indicators, true);
+
+            healthApiStats.track(true, response);
+            assertThat(healthApiStats.getStats().get("invocations.total"), equalTo(3L));
+            assertThat(healthApiStats.getStats().get("invocations.verbose_true"), equalTo(2L));
+            assertThat(healthApiStats.getStats().get("invocations.verbose_false"), equalTo(1L));
+            assertThat(healthApiStats.getStats().get("statuses.green"), equalTo(1L));
+            assertThat(healthApiStats.getStats().get("statuses.yellow"), equalTo(1L));
+            assertThat(healthApiStats.getStats().get("indicators.yellow.shards_availability"), equalTo(1L));
+            assertThat(healthApiStats.getStats().get("indicators.yellow.disk"), equalTo(1L));
+            assertThat(healthApiStats.getStats().get("statuses.red"), equalTo(1L));
+            assertThat(healthApiStats.getStats().get("indicators.red.shards_availability"), equalTo(1L));
+            assertThat(
+                healthApiStats.getStats().get("diagnoses.yellow.elasticsearch:health:disk:diagnosis:add_disk_capacity_data_nodes"),
+                equalTo(1L)
+            );
+            assertThat(
+                healthApiStats.getStats().get("diagnoses.red.elasticsearch:health:shards_availability:diagnosis:primary_unassigned"),
+                equalTo(1L)
+            );
+            assertThat(healthApiStats.getStatuses(), equalTo(Set.of(GREEN, YELLOW, RED)));
+            assertThat(healthApiStats.getIndicators().get(YELLOW), equalTo(Set.of("shards_availability", "disk")));
+            assertThat(healthApiStats.getIndicators().get(RED), equalTo(Set.of("shards_availability")));
+            assertThat(
+                healthApiStats.getDiagnoses().get(YELLOW),
+                equalTo(Set.of("elasticsearch:health:disk:diagnosis:add_disk_capacity_data_nodes"))
+            );
+            assertThat(
+                healthApiStats.getDiagnoses().get(RED),
+                equalTo(Set.of("elasticsearch:health:shards_availability:diagnosis:primary_unassigned"))
+            );
+        }
+        {
+            // Increment counters
+            var indicators = List.of(
+                createHealthIndicatorResult("network_latency", GREEN),
+                createHealthIndicatorResult("disk", RED, "add_disk_capacity_data_nodes"),
+                createHealthIndicatorResult("shards_availability", RED, "primary_unassigned")
+            );
+            GetHealthAction.Response response = new GetHealthAction.Response(ClusterName.DEFAULT, indicators, true);
+
+            healthApiStats.track(true, response);
+            assertThat(healthApiStats.getStats().get("invocations.total"), equalTo(4L));
+            assertThat(healthApiStats.getStats().get("invocations.verbose_true"), equalTo(3L));
+            assertThat(healthApiStats.getStats().get("invocations.verbose_false"), equalTo(1L));
+            assertThat(healthApiStats.getStats().get("statuses.green"), equalTo(1L));
+            assertThat(healthApiStats.getStats().get("statuses.yellow"), equalTo(1L));
+            assertThat(healthApiStats.getStats().get("indicators.yellow.disk"), equalTo(1L));
+            assertThat(healthApiStats.getStats().get("indicators.yellow.shards_availability"), equalTo(1L));
+            assertThat(healthApiStats.getStats().get("statuses.red"), equalTo(2L));
+            assertThat(healthApiStats.getStats().get("indicators.red.shards_availability"), equalTo(2L));
+            assertThat(healthApiStats.getStats().get("indicators.red.disk"), equalTo(1L));
+            assertThat(
+                healthApiStats.getStats().get("diagnoses.yellow.elasticsearch:health:disk:diagnosis:add_disk_capacity_data_nodes"),
+                equalTo(1L)
+            );
+            assertThat(
+                healthApiStats.getStats().get("diagnoses.red.elasticsearch:health:shards_availability:diagnosis:primary_unassigned"),
+                equalTo(2L)
+            );
+            assertThat(
+                healthApiStats.getStats().get("diagnoses.red.elasticsearch:health:disk:diagnosis:add_disk_capacity_data_nodes"),
+                equalTo(1L)
+            );
+            assertThat(healthApiStats.getStatuses(), equalTo(Set.of(GREEN, YELLOW, RED)));
+            assertThat(healthApiStats.getIndicators().get(YELLOW), equalTo(Set.of("shards_availability", "disk")));
+            assertThat(healthApiStats.getIndicators().get(RED), equalTo(Set.of("shards_availability", "disk")));
+            assertThat(
+                healthApiStats.getDiagnoses().get(YELLOW),
+                equalTo(Set.of("elasticsearch:health:disk:diagnosis:add_disk_capacity_data_nodes"))
+            );
+            assertThat(
+                healthApiStats.getDiagnoses().get(RED),
+                equalTo(
+                    Set.of(
+                        "elasticsearch:health:shards_availability:diagnosis:primary_unassigned",
+                        "elasticsearch:health:disk:diagnosis:add_disk_capacity_data_nodes"
+                    )
+                )
+            );
+        }
+    }
+
+    private HealthIndicatorResult createHealthIndicatorResult(String indicator, HealthStatus status, String... diagnosisIds) {
+        return new HealthIndicatorResult(
+            indicator,
+            status,
+            null,
+            null,
+            null,
+            Arrays.stream(diagnosisIds)
+                .map(diagnosisId -> new Diagnosis(new Diagnosis.Definition(indicator, diagnosisId, "", "", ""), null))
+                .toList()
+        );
+    }
+}

+ 1 - 0
x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java

@@ -257,6 +257,7 @@ public class Constants {
         "cluster:monitor/fetch/health/info",
         "cluster:monitor/health",
         "cluster:monitor/health_api",
+        "cluster:monitor/health_api/stats",
         "cluster:monitor/update/health/info",
         "cluster:monitor/ingest/geoip/stats",
         "cluster:monitor/main",