Explorar o código

Enhance ILM Health Indicator (#96092)

Here we introduce new rules to the ILM Health Indicator. The new rules check all the managed indices by an ILM policy according to the current action and step where they are at any time. For instance, if an index has been on the `rollover` > `wait-for-active-shards` combo for more than 24 hours or the step `wait-for-active-shards` has been retried more than 100 times, the indicator will turn yellow.
Pablo Alcantar Morales %!s(int64=2) %!d(string=hai) anos
pai
achega
d471146b66

+ 5 - 0
docs/changelog/96092.yaml

@@ -0,0 +1,5 @@
+pr: 96092
+summary: Enhance ILM Health Indicator
+area: ILM+SLM
+type: feature
+issues: []

+ 1 - 0
server/src/main/java/org/elasticsearch/health/Diagnosis.java

@@ -44,6 +44,7 @@ public record Diagnosis(Definition definition, @Nullable List<Resource> affected
             INDEX("indices"),
             NODE("nodes"),
             SLM_POLICY("slm_policies"),
+            ILM_POLICY("ilm_policies"),
             FEATURE_STATE("feature_states"),
             SNAPSHOT_REPOSITORY("snapshot_repositories");
 

+ 2 - 2
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/WaitForIndexColorStep.java

@@ -26,9 +26,9 @@ import java.util.function.BiFunction;
 /**
  * Wait Step for index based on color. Optionally derives the index name using the provided prefix (if any).
  */
-class WaitForIndexColorStep extends ClusterStateWaitStep {
+public class WaitForIndexColorStep extends ClusterStateWaitStep {
 
-    static final String NAME = "wait-for-index-color";
+    public static final String NAME = "wait-for-index-color";
 
     private static final Logger logger = LogManager.getLogger(WaitForIndexColorStep.class);
 

+ 1 - 1
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/WaitForNoFollowersStep.java

@@ -37,7 +37,7 @@ public class WaitForNoFollowersStep extends AsyncWaitStep {
 
     private static final Logger logger = LogManager.getLogger(WaitForNoFollowersStep.class);
 
-    static final String NAME = "wait-for-shard-history-leases";
+    public static final String NAME = "wait-for-shard-history-leases";
     static final String CCR_LEASE_KEY = "ccr";
     private static final String WAIT_MESSAGE = "this index is a leader index; waiting for all following indices to cease "
         + "following before proceeding";

+ 334 - 27
x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IlmHealthIndicatorService.java

@@ -7,8 +7,9 @@
 
 package org.elasticsearch.xpack.ilm;
 
-import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.health.Diagnosis;
 import org.elasticsearch.health.HealthIndicatorDetails;
 import org.elasticsearch.health.HealthIndicatorImpact;
@@ -17,16 +18,42 @@ import org.elasticsearch.health.HealthIndicatorService;
 import org.elasticsearch.health.ImpactArea;
 import org.elasticsearch.health.SimpleHealthIndicatorDetails;
 import org.elasticsearch.health.node.HealthInfo;
+import org.elasticsearch.index.Index;
+import org.elasticsearch.xpack.core.ilm.AllocateAction;
+import org.elasticsearch.xpack.core.ilm.DeleteAction;
+import org.elasticsearch.xpack.core.ilm.DeleteStep;
+import org.elasticsearch.xpack.core.ilm.ForceMergeAction;
+import org.elasticsearch.xpack.core.ilm.ForceMergeStep;
 import org.elasticsearch.xpack.core.ilm.IndexLifecycleMetadata;
+import org.elasticsearch.xpack.core.ilm.MigrateAction;
 import org.elasticsearch.xpack.core.ilm.OperationMode;
+import org.elasticsearch.xpack.core.ilm.RolloverAction;
+import org.elasticsearch.xpack.core.ilm.RolloverStep;
+import org.elasticsearch.xpack.core.ilm.SearchableSnapshotAction;
+import org.elasticsearch.xpack.core.ilm.SegmentCountStep;
+import org.elasticsearch.xpack.core.ilm.ShrinkAction;
+import org.elasticsearch.xpack.core.ilm.WaitForActiveShardsStep;
+import org.elasticsearch.xpack.core.ilm.WaitForDataTierStep;
+import org.elasticsearch.xpack.core.ilm.WaitForIndexColorStep;
+import org.elasticsearch.xpack.core.ilm.WaitForNoFollowersStep;
+import org.elasticsearch.xpack.core.ilm.WaitForRolloverReadyStep;
 
+import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.TreeSet;
+import java.util.function.LongSupplier;
+import java.util.stream.Collectors;
 
+import static java.util.stream.Collectors.counting;
+import static java.util.stream.Collectors.groupingBy;
 import static org.elasticsearch.health.HealthStatus.GREEN;
 import static org.elasticsearch.health.HealthStatus.YELLOW;
 import static org.elasticsearch.xpack.core.ilm.LifecycleOperationMetadata.currentILMMode;
+import static org.elasticsearch.xpack.ilm.IlmHealthIndicatorService.RuleConfig.Builder.actionRule;
+import static org.elasticsearch.xpack.ilm.IlmHealthIndicatorService.StepRule.stepRule;
 
 /**
  * This indicator reports health for index lifecycle management component.
@@ -39,7 +66,6 @@ import static org.elasticsearch.xpack.core.ilm.LifecycleOperationMetadata.curren
 public class IlmHealthIndicatorService implements HealthIndicatorService {
 
     public static final String NAME = "ilm";
-
     public static final String HELP_URL = "https://ela.st/fix-ilm";
     public static final Diagnosis ILM_NOT_RUNNING = new Diagnosis(
         new Diagnosis.Definition(
@@ -53,11 +79,103 @@ public class IlmHealthIndicatorService implements HealthIndicatorService {
     );
 
     public static final String AUTOMATION_DISABLED_IMPACT_ID = "automation_disabled";
+    public static final String STAGNATING_INDEX_IMPACT_ID = "stagnating_index";
+    public static final List<HealthIndicatorImpact> AUTOMATION_DISABLED_IMPACT = List.of(
+        new HealthIndicatorImpact(
+            NAME,
+            AUTOMATION_DISABLED_IMPACT_ID,
+            3,
+            "Automatic index lifecycle and data retention management is disabled. The performance and stability of the cluster "
+                + "could be impacted.",
+            List.of(ImpactArea.DEPLOYMENT_MANAGEMENT)
+        )
+    );
+
+    public static final List<HealthIndicatorImpact> STAGNATING_INDEX_IMPACT = List.of(
+        new HealthIndicatorImpact(
+            NAME,
+            STAGNATING_INDEX_IMPACT_ID,
+            3,
+            "Automatic index lifecycle and data retention management cannot make progress on one or more indices. The performance and "
+                + "stability of the indices and/or the cluster could be impacted.",
+            List.of(ImpactArea.DEPLOYMENT_MANAGEMENT)
+        )
+    );
+
+    private static final TimeValue ONE_DAY = TimeValue.timeValueDays(1);
+
+    static final Map<String, RuleConfig> RULES_BY_ACTION_CONFIG = Map.of(
+        RolloverAction.NAME,
+        actionRule(RolloverAction.NAME).stepRules(
+            stepRule(WaitForActiveShardsStep.NAME, ONE_DAY),
+            stepRule(WaitForRolloverReadyStep.NAME, ONE_DAY),
+            stepRule(RolloverStep.NAME, ONE_DAY)
+        ),
+        //
+        MigrateAction.NAME,
+        actionRule(MigrateAction.NAME).maxTimeOnAction(ONE_DAY).noStepRules(),
+        //
+        SearchableSnapshotAction.NAME,
+        actionRule(SearchableSnapshotAction.NAME).maxTimeOnAction(ONE_DAY)
+            .stepRules(
+                stepRule(WaitForDataTierStep.NAME, ONE_DAY),
+                stepRule(WaitForIndexColorStep.NAME, ONE_DAY),
+                // The no-follower step is added here because an `UnfollowAction` is added before the `shrinkAction` in the follower cluster
+                stepRule(WaitForNoFollowersStep.NAME, ONE_DAY)
+            ),
+        //
+        DeleteAction.NAME,
+        actionRule(DeleteAction.NAME).stepRules(stepRule(DeleteStep.NAME, ONE_DAY)),
+        //
+        ShrinkAction.NAME,
+        actionRule(ShrinkAction.NAME).maxTimeOnAction(ONE_DAY)
+            .stepRules(
+                // The no-follower step is added here because an `unfollowAction` is added before the `shrinkAction` in the follower
+                // cluster.
+                stepRule(WaitForNoFollowersStep.NAME, ONE_DAY)
+            ),
+        //
+        AllocateAction.NAME,
+        actionRule(AllocateAction.NAME).maxTimeOnAction(ONE_DAY).noStepRules(),
+        //
+        ForceMergeAction.NAME,
+        actionRule(ForceMergeAction.NAME).maxTimeOnAction(ONE_DAY)
+            .stepRules(
+                stepRule(WaitForIndexColorStep.NAME, ONE_DAY),
+                stepRule(ForceMergeStep.NAME, ONE_DAY),
+                stepRule(SegmentCountStep.NAME, ONE_DAY)
+            )
+        //
+        // The next rule has to be commented because of this issue https://github.com/elastic/elasticsearch/issues/96705
+        // DownsampleAction.NAME,
+        // actionRule(DownsampleAction.NAME).maxTimeOnAction(ONE_DAY).stepRules(stepRule(WaitForNoFollowersStep.NAME, ONE_DAY))
+    );
+
+    public static final Collection<RuleConfig> ILM_RULE_EVALUATOR = RULES_BY_ACTION_CONFIG.values();
+
+    static final Map<String, Diagnosis.Definition> STAGNATING_ACTION_DEFINITIONS = RULES_BY_ACTION_CONFIG.entrySet()
+        .stream()
+        .collect(
+            Collectors.toUnmodifiableMap(
+                Map.Entry::getKey,
+                entry -> new Diagnosis.Definition(
+                    NAME,
+                    "stagnating_action:" + entry.getKey(),
+                    "Some indices have been stagnated on the action [" + entry.getKey() + "] longer than the expected time.",
+                    "Check the current status of the Index Lifecycle Management for every affected index using the "
+                        + "[GET /<affected_index_name>/_ilm/explain] API. Please replace the <affected_index_name> in the API "
+                        + "with the actual index name.",
+                    "https://ela.st/ilm-explain"
+                )
+            )
+        );
 
     private final ClusterService clusterService;
+    private final StagnatingIndicesFinder stagnatingIndicesFinder;
 
-    public IlmHealthIndicatorService(ClusterService clusterService) {
+    public IlmHealthIndicatorService(ClusterService clusterService, StagnatingIndicesFinder stagnatingIndicesFinder) {
         this.clusterService = clusterService;
+        this.stagnatingIndicesFinder = stagnatingIndicesFinder;
     }
 
     @Override
@@ -67,9 +185,9 @@ public class IlmHealthIndicatorService implements HealthIndicatorService {
 
     @Override
     public HealthIndicatorResult calculate(boolean verbose, int maxAffectedResourcesCount, HealthInfo healthInfo) {
-        final ClusterState currentState = clusterService.state();
+        final var currentState = clusterService.state();
         var ilmMetadata = currentState.metadata().custom(IndexLifecycleMetadata.TYPE, IndexLifecycleMetadata.EMPTY);
-        final OperationMode currentMode = currentILMMode(currentState);
+        final var currentMode = currentILMMode(currentState);
         if (ilmMetadata.getPolicyMetadatas().isEmpty()) {
             return createIndicator(
                 GREEN,
@@ -79,39 +197,228 @@ public class IlmHealthIndicatorService implements HealthIndicatorService {
                 Collections.emptyList()
             );
         } else if (currentMode != OperationMode.RUNNING) {
-            List<HealthIndicatorImpact> impacts = Collections.singletonList(
-                new HealthIndicatorImpact(
-                    NAME,
-                    AUTOMATION_DISABLED_IMPACT_ID,
-                    3,
-                    "Automatic index lifecycle and data retention management is disabled. The performance and stability of the cluster "
-                        + "could be impacted.",
-                    List.of(ImpactArea.DEPLOYMENT_MANAGEMENT)
-                )
-            );
             return createIndicator(
                 YELLOW,
                 "Index Lifecycle Management is not running",
                 createDetails(verbose, ilmMetadata, currentMode),
-                impacts,
+                AUTOMATION_DISABLED_IMPACT,
                 List.of(ILM_NOT_RUNNING)
             );
         } else {
-            return createIndicator(
-                GREEN,
-                "Index Lifecycle Management is running",
-                createDetails(verbose, ilmMetadata, currentMode),
-                Collections.emptyList(),
-                Collections.emptyList()
-            );
+            var stagnatingIndices = stagnatingIndicesFinder.find();
+
+            if (stagnatingIndices.isEmpty()) {
+                return createIndicator(
+                    GREEN,
+                    "Index Lifecycle Management is running",
+                    createDetails(verbose, ilmMetadata, currentMode),
+                    Collections.emptyList(),
+                    Collections.emptyList()
+                );
+            } else {
+                return createIndicator(
+                    YELLOW,
+                    (stagnatingIndices.size() > 1 ? stagnatingIndices.size() + " indices have" : "An index has")
+                        + " stayed on the same action longer than expected.",
+                    createDetails(verbose, ilmMetadata, currentMode, stagnatingIndices),
+                    STAGNATING_INDEX_IMPACT,
+                    createDiagnoses(stagnatingIndices, maxAffectedResourcesCount)
+                );
+            }
         }
     }
 
-    private static HealthIndicatorDetails createDetails(boolean verbose, IndexLifecycleMetadata metadata, OperationMode mode) {
-        if (verbose) {
-            return new SimpleHealthIndicatorDetails(Map.of("ilm_status", mode, "policies", metadata.getPolicies().size()));
-        } else {
+    private static HealthIndicatorDetails createDetails(boolean verbose, IndexLifecycleMetadata ilmMetadata, OperationMode currentMode) {
+        return createDetails(verbose, ilmMetadata, currentMode, List.of());
+    }
+
+    private static List<Diagnosis> createDiagnoses(List<IndexMetadata> stagnatingIndices, int maxAffectedResourcesCount) {
+        return stagnatingIndices.stream()
+            .collect(groupingBy(md -> md.getLifecycleExecutionState().action()))
+            .entrySet()
+            .stream()
+            .map(action -> {
+                var affectedIndices = action.getValue()
+                    .stream()
+                    .map(IndexMetadata::getIndex)
+                    .map(Index::getName)
+                    .limit(Math.min(maxAffectedResourcesCount, action.getValue().size()))
+                    .collect(Collectors.toCollection(TreeSet::new));
+                var affectedPolicies = action.getValue()
+                    .stream()
+                    .map(IndexMetadata::getLifecyclePolicyName)
+                    .limit(Math.min(maxAffectedResourcesCount, action.getValue().size()))
+                    .collect(Collectors.toCollection(TreeSet::new));
+                return new Diagnosis(
+                    STAGNATING_ACTION_DEFINITIONS.get(action.getKey()),
+                    List.of(
+                        new Diagnosis.Resource(Diagnosis.Resource.Type.ILM_POLICY, affectedPolicies),
+                        new Diagnosis.Resource(Diagnosis.Resource.Type.INDEX, affectedIndices)
+                    )
+                );
+            })
+            .toList();
+    }
+
+    private static HealthIndicatorDetails createDetails(
+        boolean verbose,
+        IndexLifecycleMetadata metadata,
+        OperationMode mode,
+        List<IndexMetadata> stagnatingIndices
+    ) {
+        if (verbose == false) {
             return HealthIndicatorDetails.EMPTY;
         }
+
+        var details = new HashMap<String, Object>();
+
+        details.put("ilm_status", mode);
+        details.put("policies", metadata.getPolicies().size());
+        details.put("stagnating_indices", stagnatingIndices.size());
+
+        var stagnatingIndicesPerAction = stagnatingIndices.stream()
+            .collect(groupingBy(md -> md.getLifecycleExecutionState().action(), counting()));
+
+        if (stagnatingIndicesPerAction.isEmpty() == false) {
+            RULES_BY_ACTION_CONFIG.forEach((action, value) -> stagnatingIndicesPerAction.putIfAbsent(action, 0L));
+            details.put("stagnating_indices_per_action", stagnatingIndicesPerAction);
+        }
+
+        return new SimpleHealthIndicatorDetails(details);
+    }
+
+    /**
+     * Class in charge of find all the indices that are _potentially_ stagnated at some ILM action. To find the indices, it uses a list of
+     * rules evaluators (Check {@link IlmHealthIndicatorService#RULES_BY_ACTION_CONFIG to the current rules}
+     */
+    static class StagnatingIndicesFinder {
+        private final ClusterService clusterService;
+        private final Collection<RuleConfig> rules;
+        private final LongSupplier nowSupplier;
+
+        StagnatingIndicesFinder(ClusterService clusterService, Collection<RuleConfig> rules, LongSupplier nowSupplier) {
+            this.clusterService = clusterService;
+            this.rules = rules;
+            this.nowSupplier = nowSupplier;
+        }
+
+        /**
+         * @return A list containing the ILM managed indices that are stagnated in any ILM action/step.
+         */
+        public List<IndexMetadata> find() {
+            var metadata = clusterService.state().metadata();
+            var now = nowSupplier.getAsLong();
+            return metadata.indices()
+                .values()
+                .stream()
+                .filter(metadata::isIndexManagedByILM)
+                .filter(md -> isStagnated(rules, now, md))
+                .toList();
+        }
+    }
+
+    static boolean isStagnated(Collection<RuleConfig> rules, Long now, IndexMetadata indexMetadata) {
+        return rules.stream().anyMatch(r -> r.test(now, indexMetadata));
+    }
+
+    @FunctionalInterface
+    public interface RuleConfig {
+
+        boolean test(Long now, IndexMetadata indexMetadata);
+
+        static TimeValue getElapsedTime(Long now, Long currentTime) {
+            return currentTime == null ? TimeValue.ZERO : TimeValue.timeValueMillis(now - currentTime);
+        }
+
+        default RuleConfig and(RuleConfig other) {
+            return (now, indexMetadata) -> test(now, indexMetadata) && other.test(now, indexMetadata);
+        }
+
+        default RuleConfig or(RuleConfig other) {
+            return (now, indexMetadata) -> test(now, indexMetadata) || other.test(now, indexMetadata);
+        }
+
+        /**
+         * Builder class that provides a simple DSL to create composed `RuleConfig`s. It'll enforce that any rule to start with an action
+         * and then optionally define the max-time-on an action or a set of steps. In general, the output rule will be in the form:
+         *
+         *      action-rule AND (step-rule-1 OR step-rule-2 OR step-rule-3 ...)?
+         *
+         * Where the list of `step-rule-xx` could be empty.
+         *
+         * To have a clearer idea of the final shape of the rules, check the methods {@link ActionRule#test} and {@link StepRule#test}
+         */
+        class Builder {
+            private String action;
+            private TimeValue maxTimeOn = null;
+
+            static Builder actionRule(String action) {
+                var builder = new Builder();
+                builder.action = action;
+                return builder;
+            }
+
+            Builder maxTimeOnAction(TimeValue maxTimeOn) {
+                this.maxTimeOn = maxTimeOn;
+                return this;
+            }
+
+            RuleConfig stepRules(StepRule... stepRules) {
+                assert stepRules.length > 0;
+                if (stepRules.length == 1) {
+                    return new ActionRule(action, maxTimeOn).and(stepRules[0]);
+                } else {
+                    RuleConfig stepRule = stepRules[0];
+                    for (var i = 1; i < stepRules.length; i++) {
+                        stepRule = stepRule.or(stepRules[i]);
+                    }
+                    return new ActionRule(action, maxTimeOn).and(stepRule);
+                }
+            }
+
+            RuleConfig noStepRules() {
+                return new ActionRule(action, maxTimeOn);
+            }
+        }
+    }
+
+    /**
+     * Record defining a rule that will check the current action that an ILM-managed index is into.
+     *
+     * @param action    The action against which the rule will be checked.
+     * @param maxTimeOn Maximum time that an index should spend on this action.
+     */
+    record ActionRule(String action, TimeValue maxTimeOn) implements RuleConfig {
+
+        @Override
+        public boolean test(Long now, IndexMetadata indexMetadata) {
+            var currentAction = indexMetadata.getLifecycleExecutionState().action();
+            if (maxTimeOn == null) {
+                return action.equals(currentAction);
+            } else {
+                return action.equals(currentAction)
+                    && maxTimeOn.compareTo(RuleConfig.getElapsedTime(now, indexMetadata.getLifecycleExecutionState().actionTime())) < 0;
+            }
+        }
+    }
+
+    /**
+     * Record defining a rule that will check the current step that an ILM-managed index is into.
+     *
+     * @param step       The step against which the rule will be checked.
+     * @param maxTimeOn  Maximum time that an index should spend on this step.
+     * @param maxRetries Maximum number of times that a step should be retried.
+     */
+    public record StepRule(String step, TimeValue maxTimeOn, long maxRetries) implements RuleConfig {
+        static StepRule stepRule(String name, TimeValue maxTimeOn) {
+            return new StepRule(name, maxTimeOn, 100);
+        }
+
+        @Override
+        public boolean test(Long now, IndexMetadata indexMetadata) {
+            return step.equals(indexMetadata.getLifecycleExecutionState().step())
+                && (maxTimeOn.compareTo(RuleConfig.getElapsedTime(now, indexMetadata.getLifecycleExecutionState().stepTime())) < 0
+                    || indexMetadata.getLifecycleExecutionState().failedStepRetryCount() > maxRetries);
+        }
     }
 }

+ 10 - 1
x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycle.java

@@ -273,7 +273,16 @@ public class IndexLifecycle extends Plugin implements ActionPlugin, HealthPlugin
         );
         snapshotRetentionService.get().init(clusterService);
         components.addAll(Arrays.asList(snapshotLifecycleService.get(), snapshotHistoryStore.get(), snapshotRetentionService.get()));
-        ilmHealthIndicatorService.set(new IlmHealthIndicatorService(clusterService));
+        ilmHealthIndicatorService.set(
+            new IlmHealthIndicatorService(
+                clusterService,
+                new IlmHealthIndicatorService.StagnatingIndicesFinder(
+                    clusterService,
+                    IlmHealthIndicatorService.ILM_RULE_EVALUATOR,
+                    System::currentTimeMillis
+                )
+            )
+        );
         slmHealthIndicatorService.set(new SlmHealthIndicatorService(clusterService));
         reservedLifecycleAction.set(new ReservedLifecycleAction(xContentRegistry, client, XPackPlugin.getSharedLicenseState()));
 

+ 202 - 17
x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/IlmHealthIndicatorServiceTests.java

@@ -7,39 +7,61 @@
 
 package org.elasticsearch.xpack.ilm;
 
+import org.elasticsearch.Version;
 import org.elasticsearch.cluster.ClusterName;
 import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.common.bytes.BytesReference;
+import org.elasticsearch.common.xcontent.LoggingDeprecationHandler;
+import org.elasticsearch.health.Diagnosis;
 import org.elasticsearch.health.HealthIndicatorImpact;
 import org.elasticsearch.health.HealthIndicatorResult;
 import org.elasticsearch.health.ImpactArea;
 import org.elasticsearch.health.SimpleHealthIndicatorDetails;
 import org.elasticsearch.health.node.HealthInfo;
 import org.elasticsearch.test.ESTestCase;
+import org.elasticsearch.xcontent.ToXContent;
+import org.elasticsearch.xcontent.XContentBuilder;
+import org.elasticsearch.xcontent.XContentFactory;
+import org.elasticsearch.xcontent.XContentParser;
+import org.elasticsearch.xcontent.XContentType;
 import org.elasticsearch.xpack.core.ilm.IndexLifecycleMetadata;
 import org.elasticsearch.xpack.core.ilm.LifecyclePolicy;
 import org.elasticsearch.xpack.core.ilm.LifecyclePolicyMetadata;
+import org.elasticsearch.xpack.core.ilm.LifecycleSettings;
 
+import java.io.IOException;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.elasticsearch.cluster.metadata.LifecycleExecutionState.ILM_CUSTOM_METADATA_KEY;
 import static org.elasticsearch.health.HealthStatus.GREEN;
 import static org.elasticsearch.health.HealthStatus.YELLOW;
 import static org.elasticsearch.xpack.core.ilm.OperationMode.RUNNING;
 import static org.elasticsearch.xpack.core.ilm.OperationMode.STOPPED;
 import static org.elasticsearch.xpack.core.ilm.OperationMode.STOPPING;
 import static org.elasticsearch.xpack.ilm.IlmHealthIndicatorService.NAME;
+import static org.elasticsearch.xpack.ilm.IlmHealthIndicatorService.RULES_BY_ACTION_CONFIG;
+import static org.elasticsearch.xpack.ilm.IlmHealthIndicatorService.STAGNATING_ACTION_DEFINITIONS;
+import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoInteractions;
 import static org.mockito.Mockito.when;
 
 public class IlmHealthIndicatorServiceTests extends ESTestCase {
 
-    public void testIsGreenWhenRunningAndPoliciesConfigured() {
+    public void testIsGreenWhenRunningAndPoliciesConfiguredAndNoStagnatingIndices() {
         var clusterState = createClusterStateWith(new IndexLifecycleMetadata(createIlmPolicy(), RUNNING));
-        var service = createIlmHealthIndicatorService(clusterState);
+        var stagnatingIndicesFinder = mockedStagnatingIndicesFinder(List.of());
+        var service = createIlmHealthIndicatorService(clusterState, stagnatingIndicesFinder);
 
         assertThat(
             service.calculate(true, HealthInfo.EMPTY_HEALTH_INFO),
@@ -48,18 +70,143 @@ public class IlmHealthIndicatorServiceTests extends ESTestCase {
                     NAME,
                     GREEN,
                     "Index Lifecycle Management is running",
-                    new SimpleHealthIndicatorDetails(Map.of("ilm_status", RUNNING, "policies", 1)),
-                    Collections.emptyList(),
-                    Collections.emptyList()
+                    new SimpleHealthIndicatorDetails(Map.of("ilm_status", RUNNING, "policies", 1, "stagnating_indices", 0)),
+                    List.of(),
+                    List.of()
                 )
             )
         );
+        verify(stagnatingIndicesFinder, times(1)).find();
+    }
+
+    public void testIsYellowIfThereIsOneStagnatingIndicesAndDetailsEmptyIfNoVerbose() throws IOException {
+        var clusterState = createClusterStateWith(new IndexLifecycleMetadata(createIlmPolicy(), RUNNING));
+        var action = randomAction();
+        var policyName = randomAlphaOfLength(10);
+        var indexName = randomAlphaOfLength(10);
+        var stagnatingIndicesFinder = mockedStagnatingIndicesFinder(List.of(indexMetadata(indexName, policyName, action)));
+        var service = createIlmHealthIndicatorService(clusterState, stagnatingIndicesFinder);
+
+        var indicatorResult = service.calculate(false, HealthInfo.EMPTY_HEALTH_INFO);
+
+        assertEquals(indicatorResult.name(), NAME);
+        assertEquals(indicatorResult.status(), YELLOW);
+        assertEquals(indicatorResult.symptom(), "An index has stayed on the same action longer than expected.");
+        assertEquals(xContentToMap(indicatorResult.details()), Map.of());
+        assertThat(indicatorResult.impacts(), hasSize(1));
+        assertThat(
+            indicatorResult.impacts().get(0),
+            equalTo(
+                new HealthIndicatorImpact(
+                    NAME,
+                    IlmHealthIndicatorService.STAGNATING_INDEX_IMPACT_ID,
+                    3,
+                    "Automatic index lifecycle and data retention management cannot make progress on one or more indices. "
+                        + "The performance and stability of the indices and/or the cluster could be impacted.",
+                    List.of(ImpactArea.DEPLOYMENT_MANAGEMENT)
+                )
+            )
+        );
+        assertThat(indicatorResult.diagnosisList(), hasSize(1));
+        assertEquals(indicatorResult.diagnosisList().get(0).definition(), STAGNATING_ACTION_DEFINITIONS.get(action));
+
+        var affectedResources = indicatorResult.diagnosisList().get(0).affectedResources();
+        assertThat(affectedResources, hasSize(2));
+        assertEquals(affectedResources.get(0).getType(), Diagnosis.Resource.Type.ILM_POLICY);
+        assertThat(affectedResources.get(0).getValues(), hasSize(1));
+        assertThat(affectedResources.get(0).getValues(), containsInAnyOrder(policyName));
+        assertThat(affectedResources.get(1).getValues(), hasSize(1));
+        assertEquals(affectedResources.get(1).getType(), Diagnosis.Resource.Type.INDEX);
+        assertThat(affectedResources.get(1).getValues(), containsInAnyOrder(indexName));
+
+        verify(stagnatingIndicesFinder, times(1)).find();
+    }
+
+    public void testIsYellowIfThereIsOneStagnatingIndices() throws IOException {
+        var clusterState = createClusterStateWith(new IndexLifecycleMetadata(createIlmPolicy(), RUNNING));
+        var action = randomAction();
+        var policyName = randomAlphaOfLength(10);
+        var indexName = randomAlphaOfLength(10);
+        var stagnatingIndicesFinder = mockedStagnatingIndicesFinder(List.of(indexMetadata(indexName, policyName, action)));
+        var service = createIlmHealthIndicatorService(clusterState, stagnatingIndicesFinder);
+
+        var indicatorResult = service.calculate(true, HealthInfo.EMPTY_HEALTH_INFO);
+
+        assertEquals(indicatorResult.name(), NAME);
+        assertEquals(indicatorResult.status(), YELLOW);
+        assertEquals(indicatorResult.symptom(), "An index has stayed on the same action longer than expected.");
+        var expectedILMSummary = new HashMap<>();
+        // Uncomment next line after https://github.com/elastic/elasticsearch/issues/96705 is fixed
+        // expectedILMSummary.put("downsample", 0);
+        expectedILMSummary.put("allocate", 0);
+        expectedILMSummary.put("shrink", 0);
+        expectedILMSummary.put("searchable_snapshot", 0);
+        expectedILMSummary.put("rollover", 0);
+        expectedILMSummary.put("forcemerge", 0);
+        expectedILMSummary.put("delete", 0);
+        expectedILMSummary.put("migrate", 0);
+        expectedILMSummary.put(action, 1);
+
+        assertEquals(
+            xContentToMap(indicatorResult.details()),
+            Map.of(
+                "policies",
+                1,
+                "ilm_status",
+                RUNNING.toString(),
+                "stagnating_indices",
+                1,
+                "stagnating_indices_per_action",
+                expectedILMSummary
+            )
+        );
+        assertThat(indicatorResult.impacts(), hasSize(1));
+        assertThat(
+            indicatorResult.impacts().get(0),
+            equalTo(
+                new HealthIndicatorImpact(
+                    NAME,
+                    IlmHealthIndicatorService.STAGNATING_INDEX_IMPACT_ID,
+                    3,
+                    "Automatic index lifecycle and data retention management cannot make progress on one or more indices. "
+                        + "The performance and stability of the indices and/or the cluster could be impacted.",
+                    List.of(ImpactArea.DEPLOYMENT_MANAGEMENT)
+                )
+            )
+        );
+        assertThat(indicatorResult.diagnosisList(), hasSize(1));
+        assertEquals(indicatorResult.diagnosisList().get(0).definition(), STAGNATING_ACTION_DEFINITIONS.get(action));
+
+        var affectedResources = indicatorResult.diagnosisList().get(0).affectedResources();
+        assertThat(affectedResources, hasSize(2));
+        assertEquals(affectedResources.get(0).getType(), Diagnosis.Resource.Type.ILM_POLICY);
+        assertThat(affectedResources.get(0).getValues(), hasSize(1));
+        assertThat(affectedResources.get(0).getValues(), containsInAnyOrder(policyName));
+        assertThat(affectedResources.get(1).getValues(), hasSize(1));
+        assertEquals(affectedResources.get(1).getType(), Diagnosis.Resource.Type.INDEX);
+        assertThat(affectedResources.get(1).getValues(), containsInAnyOrder(indexName));
+
+        verify(stagnatingIndicesFinder, times(1)).find();
+    }
+
+    private static String randomAction() {
+        return randomFrom(RULES_BY_ACTION_CONFIG.keySet());
+    }
+
+    private static IndexMetadata indexMetadata(String indexName, String policyName, String action) {
+        return IndexMetadata.builder(indexName)
+            .settings(settings(Version.CURRENT).put(LifecycleSettings.LIFECYCLE_NAME, policyName))
+            .putCustom(ILM_CUSTOM_METADATA_KEY, Map.of("action", action))
+            .numberOfShards(randomIntBetween(1, 5))
+            .numberOfReplicas(randomIntBetween(0, 5))
+            .build();
     }
 
     public void testIsYellowWhenNotRunningAndPoliciesConfigured() {
         var status = randomFrom(STOPPED, STOPPING);
         var clusterState = createClusterStateWith(new IndexLifecycleMetadata(createIlmPolicy(), status));
-        var service = createIlmHealthIndicatorService(clusterState);
+        var stagnatingIndicesFinder = mockedStagnatingIndicesFinder(List.of());
+        var service = createIlmHealthIndicatorService(clusterState, stagnatingIndicesFinder);
 
         assertThat(
             service.calculate(true, HealthInfo.EMPTY_HEALTH_INFO),
@@ -68,7 +215,7 @@ public class IlmHealthIndicatorServiceTests extends ESTestCase {
                     NAME,
                     YELLOW,
                     "Index Lifecycle Management is not running",
-                    new SimpleHealthIndicatorDetails(Map.of("ilm_status", status, "policies", 1)),
+                    new SimpleHealthIndicatorDetails(Map.of("ilm_status", status, "policies", 1, "stagnating_indices", 0)),
                     Collections.singletonList(
                         new HealthIndicatorImpact(
                             NAME,
@@ -83,12 +230,14 @@ public class IlmHealthIndicatorServiceTests extends ESTestCase {
                 )
             )
         );
+        verifyNoInteractions(stagnatingIndicesFinder);
     }
 
     public void testIsGreenWhenNotRunningAndNoPolicies() {
         var status = randomFrom(STOPPED, STOPPING);
         var clusterState = createClusterStateWith(new IndexLifecycleMetadata(Map.of(), status));
-        var service = createIlmHealthIndicatorService(clusterState);
+        var stagnatingIndicesFinder = mockedStagnatingIndicesFinder(List.of());
+        var service = createIlmHealthIndicatorService(clusterState, stagnatingIndicesFinder);
 
         assertThat(
             service.calculate(true, HealthInfo.EMPTY_HEALTH_INFO),
@@ -97,17 +246,19 @@ public class IlmHealthIndicatorServiceTests extends ESTestCase {
                     NAME,
                     GREEN,
                     "No Index Lifecycle Management policies configured",
-                    new SimpleHealthIndicatorDetails(Map.of("ilm_status", status, "policies", 0)),
-                    Collections.emptyList(),
-                    Collections.emptyList()
+                    new SimpleHealthIndicatorDetails(Map.of("ilm_status", status, "policies", 0, "stagnating_indices", 0)),
+                    List.of(),
+                    List.of()
                 )
             )
         );
+        verifyNoInteractions(stagnatingIndicesFinder);
     }
 
     public void testIsGreenWhenNoMetadata() {
         var clusterState = createClusterStateWith(null);
-        var service = createIlmHealthIndicatorService(clusterState);
+        var stagnatingIndicesFinder = mockedStagnatingIndicesFinder(List.of());
+        var service = createIlmHealthIndicatorService(clusterState, stagnatingIndicesFinder);
 
         assertThat(
             service.calculate(true, HealthInfo.EMPTY_HEALTH_INFO),
@@ -116,12 +267,13 @@ public class IlmHealthIndicatorServiceTests extends ESTestCase {
                     NAME,
                     GREEN,
                     "No Index Lifecycle Management policies configured",
-                    new SimpleHealthIndicatorDetails(Map.of("ilm_status", RUNNING, "policies", 0)),
-                    Collections.emptyList(),
-                    Collections.emptyList()
+                    new SimpleHealthIndicatorDetails(Map.of("ilm_status", RUNNING, "policies", 0, "stagnating_indices", 0)),
+                    List.of(),
+                    List.of()
                 )
             )
         );
+        verifyNoInteractions(stagnatingIndicesFinder);
     }
 
     // We expose the indicator name and the diagnoses in the x-pack usage API. In order to index them properly in a telemetry index
@@ -132,6 +284,21 @@ public class IlmHealthIndicatorServiceTests extends ESTestCase {
             IlmHealthIndicatorService.ILM_NOT_RUNNING.definition().getUniqueId(),
             equalTo("elasticsearch:health:ilm:diagnosis:ilm_disabled")
         );
+        var definitionIds = STAGNATING_ACTION_DEFINITIONS.values().stream().map(Diagnosis.Definition::getUniqueId).toList();
+
+        assertThat(
+            definitionIds,
+            containsInAnyOrder(
+                "elasticsearch:health:ilm:diagnosis:stagnating_action:rollover",
+                "elasticsearch:health:ilm:diagnosis:stagnating_action:migrate",
+                "elasticsearch:health:ilm:diagnosis:stagnating_action:searchable_snapshot",
+                "elasticsearch:health:ilm:diagnosis:stagnating_action:delete",
+                "elasticsearch:health:ilm:diagnosis:stagnating_action:shrink",
+                "elasticsearch:health:ilm:diagnosis:stagnating_action:forcemerge",
+                "elasticsearch:health:ilm:diagnosis:stagnating_action:allocate"
+                // "elasticsearch:health:ilm:diagnosis:stagnating_action:downsample"
+            )
+        );
     }
 
     private static ClusterState createClusterStateWith(IndexLifecycleMetadata metadata) {
@@ -149,9 +316,27 @@ public class IlmHealthIndicatorServiceTests extends ESTestCase {
         );
     }
 
-    private static IlmHealthIndicatorService createIlmHealthIndicatorService(ClusterState clusterState) {
+    private static IlmHealthIndicatorService createIlmHealthIndicatorService(
+        ClusterState clusterState,
+        IlmHealthIndicatorService.StagnatingIndicesFinder stagnatingIndicesFinder
+    ) {
         var clusterService = mock(ClusterService.class);
         when(clusterService.state()).thenReturn(clusterState);
-        return new IlmHealthIndicatorService(clusterService);
+
+        return new IlmHealthIndicatorService(clusterService, stagnatingIndicesFinder);
+    }
+
+    private static IlmHealthIndicatorService.StagnatingIndicesFinder mockedStagnatingIndicesFinder(List<IndexMetadata> states) {
+        var finder = mock(IlmHealthIndicatorService.StagnatingIndicesFinder.class);
+        when(finder.find()).thenReturn(states);
+        return finder;
+    }
+
+    private Map<String, Object> xContentToMap(ToXContent xcontent) throws IOException {
+        XContentBuilder builder = XContentFactory.jsonBuilder();
+        xcontent.toXContent(builder, ToXContent.EMPTY_PARAMS);
+        XContentParser parser = XContentType.JSON.xContent()
+            .createParser(xContentRegistry(), LoggingDeprecationHandler.INSTANCE, BytesReference.bytes(builder).streamInput());
+        return parser.map();
     }
 }

+ 168 - 0
x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/RuleConfigTests.java

@@ -0,0 +1,168 @@
+/*
+ * 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; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.ilm;
+
+import org.elasticsearch.Version;
+import org.elasticsearch.cluster.metadata.IndexMetadata;
+import org.elasticsearch.core.TimeValue;
+import org.elasticsearch.test.ESTestCase;
+
+import java.util.Map;
+
+import static org.elasticsearch.cluster.metadata.LifecycleExecutionState.ILM_CUSTOM_METADATA_KEY;
+import static org.mockito.Mockito.mock;
+
+public class RuleConfigTests extends ESTestCase {
+    public void testActionRuleConfig() {
+        var actionName = randomAlphaOfLength(30);
+        assertTrue(
+            new IlmHealthIndicatorService.ActionRule(actionName, null).test(
+                randomNonNegativeLong(),
+                metadataAction(actionName, randomNonNegativeLong())
+            )
+        );
+        assertFalse(
+            new IlmHealthIndicatorService.ActionRule(actionName, null).test(
+                randomNonNegativeLong(),
+                metadataAction(randomAlphaOfLength(30), randomNonNegativeLong())
+            )
+        );
+
+        var maxTimeOn = TimeValue.parseTimeValue(randomTimeValue(), "");
+        var rule = new IlmHealthIndicatorService.ActionRule(actionName, maxTimeOn);
+        var now = System.currentTimeMillis();
+
+        assertFalse(rule.test(now, metadataAction(randomAlphaOfLength(30), now)));
+        assertFalse(rule.test(now, metadataAction(actionName, now)));
+        assertTrue(rule.test(now, metadataAction(actionName, now - maxTimeOn.millis() - randomIntBetween(1000, 100000))));
+    }
+
+    public void testStepRuleConfig() {
+        var stepName = randomAlphaOfLength(30);
+        var maxTimeOn = TimeValue.parseTimeValue(randomTimeValue(), "");
+        var maxRetries = randomIntBetween(11, 100);
+        var rule = new IlmHealthIndicatorService.StepRule(stepName, maxTimeOn, maxRetries);
+        var now = System.currentTimeMillis();
+
+        // rule is not for this step
+        assertFalse(rule.test(now, metadataStep(randomAlphaOfLength(30), now, maxRetries + 1)));
+
+        // step still has time to run && can continue retrying
+        assertFalse(rule.test(now, metadataStep(stepName, now, maxRetries - randomIntBetween(0, 10))));
+
+        // step still has run longer than expected
+        assertTrue(
+            rule.test(
+                now,
+                metadataStep(stepName, now - maxTimeOn.millis() - randomIntBetween(1000, 100000), maxRetries - randomIntBetween(0, 10))
+            )
+        );
+
+        // step still has time to run but have retried more than expected
+        assertTrue(rule.test(now, metadataStep(stepName, now, maxRetries + randomIntBetween(1, 10))));
+    }
+
+    public void testRuleChaining() {
+        var mockedMd = mock(IndexMetadata.class);
+        var someLong = randomLong();
+
+        assertTrue(ruleAlwaysReturn(true).test(someLong, mockedMd));
+        assertFalse(ruleAlwaysReturn(false).test(someLong, mockedMd));
+
+        // and
+        assertTrue(ruleAlwaysReturn(true).and(ruleAlwaysReturn(true)).test(someLong, mockedMd));
+        assertFalse(ruleAlwaysReturn(true).and(ruleAlwaysReturn(false)).test(someLong, mockedMd));
+        assertFalse(ruleAlwaysReturn(false).and(ruleAlwaysReturn(false)).test(someLong, mockedMd));
+        assertFalse(ruleAlwaysReturn(false).and(ruleAlwaysReturn(true)).test(someLong, mockedMd));
+
+        // or
+        assertTrue(ruleAlwaysReturn(true).or(ruleAlwaysReturn(true)).test(someLong, mockedMd));
+        assertTrue(ruleAlwaysReturn(true).or(ruleAlwaysReturn(false)).test(someLong, mockedMd));
+        assertFalse(ruleAlwaysReturn(false).or(ruleAlwaysReturn(false)).test(someLong, mockedMd));
+        assertTrue(ruleAlwaysReturn(false).or(ruleAlwaysReturn(true)).test(someLong, mockedMd));
+    }
+
+    public void testGetElapsedTime() {
+        var a = randomLongBetween(1000, 2000);
+        var b = a - randomLongBetween(0, 500);
+        assertEquals(IlmHealthIndicatorService.RuleConfig.getElapsedTime(a, b), TimeValue.timeValueMillis(a - b));
+        assertEquals(IlmHealthIndicatorService.RuleConfig.getElapsedTime(a, null), TimeValue.ZERO);
+    }
+
+    public void testRuleConfigBuilder() {
+        var now = System.currentTimeMillis();
+        var lastExecutionTime = System.currentTimeMillis() - TimeValue.timeValueDays(2).millis();
+        var maxTimeOnStep = TimeValue.timeValueDays(1);
+        var expectedAction = "some-action";
+        var rules = IlmHealthIndicatorService.RuleConfig.Builder.actionRule(expectedAction)
+            .maxTimeOnAction(TimeValue.timeValueDays(1))
+            .stepRules(
+                IlmHealthIndicatorService.StepRule.stepRule("step-1", maxTimeOnStep),
+                IlmHealthIndicatorService.StepRule.stepRule("step-2", maxTimeOnStep),
+                IlmHealthIndicatorService.StepRule.stepRule("step-3", maxTimeOnStep)
+            );
+
+        // An unknown action should not satisfy the conditions
+        assertFalse(rules.test(now, metadataAlwaysOutOfRule(lastExecutionTime, randomAlphaOfLength(30), "step-1")));
+        // The following 3 steps should satisfy the conditions
+        assertTrue(rules.test(now, metadataAlwaysOutOfRule(lastExecutionTime, expectedAction, "step-1")));
+        assertTrue(rules.test(now, metadataAlwaysOutOfRule(lastExecutionTime, expectedAction, "step-2")));
+        assertTrue(rules.test(now, metadataAlwaysOutOfRule(lastExecutionTime, expectedAction, "step-3")));
+        // An unknown step should not satisfy the conditions
+        assertFalse(rules.test(now, metadataAlwaysOutOfRule(lastExecutionTime, expectedAction, randomAlphaOfLength(30))));
+
+        rules = IlmHealthIndicatorService.RuleConfig.Builder.actionRule(expectedAction)
+            .maxTimeOnAction(TimeValue.timeValueDays(1))
+            .noStepRules();
+
+        assertTrue(rules.test(now, metadataAlwaysOutOfRule(lastExecutionTime, expectedAction, randomAlphaOfLength(30))));
+        // An unknown action and step should satisfy the conditions
+        assertFalse(rules.test(now, metadataAlwaysOutOfRule(lastExecutionTime, randomAlphaOfLength(30), randomAlphaOfLength(30))));
+    }
+
+    private IlmHealthIndicatorService.RuleConfig ruleAlwaysReturn(boolean shouldReturn) {
+        return (now, indexMetadata) -> shouldReturn;
+    }
+
+    private IndexMetadata metadataAlwaysOutOfRule(long lastExecutionTime, String action, String step) {
+        return baseBuilder().putCustom(
+            ILM_CUSTOM_METADATA_KEY,
+            Map.of(
+                "action",
+                action,
+                "action_time",
+                String.valueOf(lastExecutionTime),
+                "step",
+                step,
+                "step_time",
+                String.valueOf(lastExecutionTime),
+                "failed_step_retry_count",
+                "200"
+            )
+        ).build();
+    }
+
+    private IndexMetadata metadataStep(String currentStep, long stepTime, long stepRetries) {
+        return baseBuilder().putCustom(
+            ILM_CUSTOM_METADATA_KEY,
+            Map.of("step", currentStep, "step_time", String.valueOf(stepTime), "failed_step_retry_count", String.valueOf(stepRetries))
+        ).build();
+    }
+
+    private IndexMetadata metadataAction(String currentAction, long actionTime) {
+        return baseBuilder().putCustom(ILM_CUSTOM_METADATA_KEY, Map.of("action", currentAction, "action_time", String.valueOf(actionTime)))
+            .build();
+    }
+
+    private IndexMetadata.Builder baseBuilder() {
+        return IndexMetadata.builder("some-index")
+            .settings(settings(Version.CURRENT))
+            .numberOfShards(randomIntBetween(1, 5))
+            .numberOfReplicas(randomIntBetween(0, 5));
+    }
+}

+ 166 - 0
x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/StagnatingIndicesFinderTests.java

@@ -0,0 +1,166 @@
+/*
+ * 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; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.ilm;
+
+import org.elasticsearch.Version;
+import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.cluster.metadata.IndexMetadata;
+import org.elasticsearch.cluster.metadata.LifecycleExecutionState;
+import org.elasticsearch.cluster.metadata.Metadata;
+import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.test.ESTestCase;
+import org.elasticsearch.xpack.core.ilm.LifecycleSettings;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.LongSupplier;
+import java.util.stream.IntStream;
+
+import static org.elasticsearch.cluster.metadata.LifecycleExecutionState.ILM_CUSTOM_METADATA_KEY;
+import static org.elasticsearch.xpack.ilm.IlmHealthIndicatorService.isStagnated;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.hasSize;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class StagnatingIndicesFinderTests extends ESTestCase {
+
+    public void testStagnatingIndicesFinder() {
+        var idxMd1 = randomIndexMetadata();
+        var idxMd2 = randomIndexMetadata();
+        var idxMd3 = randomIndexMetadata();
+        var stagnatingIndices = List.of(idxMd1.indexName, idxMd3.indexName);
+        var mockedTimeSupplier = mock(LongSupplier.class);
+        var instant = (long) randomIntBetween(100000, 200000);
+        var ruleEvaluator = List.<IlmHealthIndicatorService.RuleConfig>of(
+            (now, indexMetadata) -> now == instant && stagnatingIndices.contains(indexMetadata.getIndex().getName())
+        );
+        // Per the evaluator, the timeSupplier _must_ be called only twice
+        when(mockedTimeSupplier.getAsLong()).thenReturn(instant, instant);
+
+        var stagnatedIdx1 = indexMetadataFrom(idxMd1);
+        var stagnatedIdx3 = indexMetadataFrom(idxMd3);
+        var finder = createStagnatingIndicesFinder(
+            ruleEvaluator,
+            mockedTimeSupplier,
+            indexMetadataUnmanaged(randomAlphaOfLength(10)), // non-managed by ILM
+            stagnatedIdx1,                                 // should be stagnated
+            indexMetadataFrom(idxMd2),                       // won't be stagnated
+            stagnatedIdx3,                                 // should be stagnated
+            indexMetadataUnmanaged(randomAlphaOfLength(10))  // non-managed by ILM
+        );
+
+        var foundIndices = finder.find();
+
+        assertThat(foundIndices, hasSize(2));
+        assertThat(foundIndices, containsInAnyOrder(stagnatedIdx1, stagnatedIdx3));
+    }
+
+    public void testStagnatingIndicesEvaluator() {
+        var idxMd1 = randomIndexMetadata();
+        var indexMetadata = indexMetadataFrom(idxMd1);
+        Long moment = 111333111222L;
+        {
+            // no rule matches
+            var executions = randomIntBetween(3, 200);
+            var calls = new AtomicInteger(0);
+            var rules = IntStream.range(0, executions).mapToObj(i -> (IlmHealthIndicatorService.RuleConfig) (now, idxMd) -> {
+                assertEquals(now, moment);
+                assertSame(idxMd, indexMetadata);
+                calls.incrementAndGet();
+                return false;
+            }).toList();
+            assertFalse(isStagnated(rules, moment, indexMetadata));
+            assertEquals(calls.get(), executions);
+        }
+        {
+            var calls = new AtomicReference<>(new ArrayList<Integer>());
+            var rules = List.<IlmHealthIndicatorService.RuleConfig>of((now, idxMd) -> { // will be called
+                assertEquals(now, moment);
+                assertSame(idxMd, indexMetadata);
+                calls.get().add(1);
+                return false;
+            }, (now, idxMd) -> { // will be called and cut the execution
+                assertEquals(now, moment);
+                assertSame(idxMd, indexMetadata);
+                calls.get().add(2);
+                return true;
+            }, (now, idxMd) -> { // won't be called
+                assertEquals(now, moment);
+                assertSame(idxMd, indexMetadata);
+                calls.get().add(3);
+                return true;
+            }, (now, idxMd) -> { // won't be called
+                assertEquals(now, moment);
+                assertSame(idxMd, indexMetadata);
+                calls.get().add(4);
+                return false;
+            });
+
+            assertTrue(isStagnated(rules, moment, indexMetadata));
+            assertEquals(calls.get(), List.of(1, 2));
+        }
+    }
+
+    private static IndexMetadata indexMetadataUnmanaged(String indexName) {
+        return indexMetadataFrom(new IndexMetadataTestCase(indexName, null, null));
+    }
+
+    private static IndexMetadata indexMetadataFrom(IndexMetadataTestCase indexMetadataTestCase) {
+        var settings = settings(Version.CURRENT);
+        var indexMetadataBuilder = IndexMetadata.builder(indexMetadataTestCase.indexName);
+
+        if (indexMetadataTestCase.ilmState != null) {
+            settings.put(LifecycleSettings.LIFECYCLE_NAME, indexMetadataTestCase.policyName);
+            indexMetadataBuilder.putCustom(ILM_CUSTOM_METADATA_KEY, indexMetadataTestCase.ilmState.asMap());
+        }
+
+        return indexMetadataBuilder.settings(settings)
+            .numberOfShards(randomIntBetween(1, 5))
+            .numberOfReplicas(randomIntBetween(0, 5))
+            .build();
+    }
+
+    private IlmHealthIndicatorService.StagnatingIndicesFinder createStagnatingIndicesFinder(
+        Collection<IlmHealthIndicatorService.RuleConfig> evaluator,
+        LongSupplier timeSupplier,
+        IndexMetadata... indicesMetadata
+    ) {
+        var clusterService = mock(ClusterService.class);
+        var state = mock(ClusterState.class);
+        var metadataBuilder = Metadata.builder();
+
+        Arrays.stream(indicesMetadata).forEach(im -> metadataBuilder.put(im, false));
+        when(state.metadata()).thenReturn(metadataBuilder.build());
+
+        when(clusterService.state()).thenReturn(state);
+
+        return new IlmHealthIndicatorService.StagnatingIndicesFinder(clusterService, evaluator, timeSupplier);
+    }
+
+    static IndexMetadataTestCase randomIndexMetadata() {
+        return new IndexMetadataTestCase(
+            randomAlphaOfLength(10),
+            randomAlphaOfLength(10),
+            LifecycleExecutionState.builder()
+                .setPhase(randomAlphaOfLength(5))
+                .setAction(randomAlphaOfLength(10))
+                .setActionTime((long) randomIntBetween(0, 10000))
+                .setStep(randomAlphaOfLength(20))
+                .setStepTime((long) randomIntBetween(0, 10000))
+                .setFailedStepRetryCount(randomIntBetween(0, 1000))
+                .build()
+        );
+    }
+
+    record IndexMetadataTestCase(String indexName, String policyName, LifecycleExecutionState ilmState) {}
+}