瀏覽代碼

Run `TransportExplainLifecycleAction` on local node (#122885)

This action solely needs the cluster state, it can run on any node.
Additionally, it needs to be cancellable to avoid doing unnecessary work
after a client failure or timeout.

Relates #101805
Niels Bauman 7 月之前
父節點
當前提交
9cecc89fed

+ 5 - 0
docs/changelog/122885.yaml

@@ -0,0 +1,5 @@
+pr: 122885
+summary: Run `TransportExplainLifecycleAction` on local node
+area: ILM+SLM
+type: enhancement
+issues: []

+ 52 - 8
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/ExplainLifecycleRequest.java

@@ -7,15 +7,22 @@
 
 package org.elasticsearch.xpack.core.ilm;
 
+import org.elasticsearch.TransportVersions;
 import org.elasticsearch.action.ActionRequestValidationException;
+import org.elasticsearch.action.IndicesRequest;
 import org.elasticsearch.action.support.IndicesOptions;
-import org.elasticsearch.action.support.master.info.ClusterInfoRequest;
+import org.elasticsearch.action.support.local.LocalClusterStateRequest;
+import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.core.TimeValue;
+import org.elasticsearch.core.UpdateForV10;
+import org.elasticsearch.tasks.CancellableTask;
+import org.elasticsearch.tasks.Task;
+import org.elasticsearch.tasks.TaskId;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Map;
 import java.util.Objects;
 
 /**
@@ -24,26 +31,58 @@ import java.util.Objects;
  * Multiple indices may be queried in the same request using the
  * {@link #indices(String...)} method
  */
-public class ExplainLifecycleRequest extends ClusterInfoRequest<ExplainLifecycleRequest> {
+public class ExplainLifecycleRequest extends LocalClusterStateRequest implements IndicesRequest.Replaceable {
 
+    private String[] indices = Strings.EMPTY_ARRAY;
+    private IndicesOptions indicesOptions;
     private boolean onlyErrors = false;
     private boolean onlyManaged = false;
 
     public ExplainLifecycleRequest(TimeValue masterTimeout) {
-        super(masterTimeout, IndicesOptions.strictExpandOpen());
+        super(masterTimeout);
+        indicesOptions = IndicesOptions.strictExpandOpen();
     }
 
+    /**
+     * NB prior to 9.0 this was a TransportMasterNodeReadAction so for BwC we must remain able to read these requests until
+     * we no longer need to support calling this action remotely.
+     */
+    @UpdateForV10(owner = UpdateForV10.Owner.DATA_MANAGEMENT)
     public ExplainLifecycleRequest(StreamInput in) throws IOException {
         super(in);
+        indices = in.readStringArray();
+        if (in.getTransportVersion().before(TransportVersions.V_8_0_0)) {
+            in.readStringArray();
+        }
+        indicesOptions = IndicesOptions.readIndicesOptions(in);
         onlyErrors = in.readBoolean();
         onlyManaged = in.readBoolean();
     }
 
     @Override
-    public void writeTo(StreamOutput out) throws IOException {
-        super.writeTo(out);
-        out.writeBoolean(onlyErrors);
-        out.writeBoolean(onlyManaged);
+    public ExplainLifecycleRequest indices(String... indices) {
+        this.indices = indices;
+        return this;
+    }
+
+    public ExplainLifecycleRequest indicesOptions(IndicesOptions indicesOptions) {
+        this.indicesOptions = indicesOptions;
+        return this;
+    }
+
+    @Override
+    public String[] indices() {
+        return indices;
+    }
+
+    @Override
+    public IndicesOptions indicesOptions() {
+        return indicesOptions;
+    }
+
+    @Override
+    public boolean includeDataStreams() {
+        return true;
     }
 
     public boolean onlyErrors() {
@@ -69,6 +108,11 @@ public class ExplainLifecycleRequest extends ClusterInfoRequest<ExplainLifecycle
         return null;
     }
 
+    @Override
+    public Task createTask(long id, String type, String action, TaskId parentTaskId, Map<String, String> headers) {
+        return new CancellableTask(id, type, action, "", parentTaskId, headers);
+    }
+
     @Override
     public int hashCode() {
         return Objects.hash(Arrays.hashCode(indices()), indicesOptions(), onlyErrors, onlyManaged);

+ 6 - 13
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/ExplainLifecycleResponse.java

@@ -9,9 +9,8 @@ package org.elasticsearch.xpack.core.ilm;
 
 import org.elasticsearch.action.ActionResponse;
 import org.elasticsearch.common.Strings;
-import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
-import org.elasticsearch.common.util.Maps;
+import org.elasticsearch.core.UpdateForV10;
 import org.elasticsearch.xcontent.ParseField;
 import org.elasticsearch.xcontent.ToXContentObject;
 import org.elasticsearch.xcontent.XContentBuilder;
@@ -32,17 +31,6 @@ public class ExplainLifecycleResponse extends ActionResponse implements ToXConte
 
     private final Map<String, IndexLifecycleExplainResponse> indexResponses;
 
-    public ExplainLifecycleResponse(StreamInput in) throws IOException {
-        super(in);
-        int size = in.readVInt();
-        Map<String, IndexLifecycleExplainResponse> indexResponses = Maps.newMapWithExpectedSize(size);
-        for (int i = 0; i < size; i++) {
-            IndexLifecycleExplainResponse indexResponse = new IndexLifecycleExplainResponse(in);
-            indexResponses.put(indexResponse.getIndex(), indexResponse);
-        }
-        this.indexResponses = indexResponses;
-    }
-
     public ExplainLifecycleResponse(Map<String, IndexLifecycleExplainResponse> indexResponses) {
         this.indexResponses = indexResponses;
     }
@@ -69,6 +57,11 @@ public class ExplainLifecycleResponse extends ActionResponse implements ToXConte
         return builder;
     }
 
+    /**
+     * NB prior to 9.0 this was a TransportMasterNodeReadAction so for BwC we must remain able to write these responses until
+     * we no longer need to support calling this action remotely.
+     */
+    @UpdateForV10(owner = UpdateForV10.Owner.DATA_MANAGEMENT)
     @Override
     public void writeTo(StreamOutput out) throws IOException {
         out.writeCollection(indexResponses.values());

+ 0 - 87
x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/ExplainLifecycleRequestTests.java

@@ -1,87 +0,0 @@
-/*
- * 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.core.ilm;
-
-import org.elasticsearch.action.support.IndicesOptions;
-import org.elasticsearch.common.io.stream.Writeable.Reader;
-import org.elasticsearch.test.AbstractWireSerializingTestCase;
-
-import java.util.Arrays;
-
-public class ExplainLifecycleRequestTests extends AbstractWireSerializingTestCase<ExplainLifecycleRequest> {
-
-    @Override
-    protected ExplainLifecycleRequest createTestInstance() {
-        ExplainLifecycleRequest request = new ExplainLifecycleRequest(TEST_REQUEST_TIMEOUT);
-        if (randomBoolean()) {
-            request.indices(generateRandomStringArray(20, 20, false, false));
-        }
-        if (randomBoolean()) {
-            IndicesOptions indicesOptions = IndicesOptions.fromOptions(
-                randomBoolean(),
-                randomBoolean(),
-                randomBoolean(),
-                randomBoolean(),
-                randomBoolean(),
-                randomBoolean(),
-                randomBoolean(),
-                randomBoolean()
-            );
-            request.indicesOptions(indicesOptions);
-        }
-        if (randomBoolean()) {
-            request.onlyErrors(randomBoolean());
-        }
-        if (randomBoolean()) {
-            request.onlyManaged(randomBoolean());
-        }
-        return request;
-    }
-
-    @Override
-    protected ExplainLifecycleRequest mutateInstance(ExplainLifecycleRequest instance) {
-        String[] indices = instance.indices();
-        IndicesOptions indicesOptions = instance.indicesOptions();
-        boolean onlyErrors = instance.onlyErrors();
-        boolean onlyManaged = instance.onlyManaged();
-        switch (between(0, 3)) {
-            case 0 -> indices = randomValueOtherThanMany(
-                i -> Arrays.equals(i, instance.indices()),
-                () -> generateRandomStringArray(20, 10, false, false)
-            );
-            case 1 -> indicesOptions = randomValueOtherThan(
-                indicesOptions,
-                () -> IndicesOptions.fromOptions(
-                    randomBoolean(),
-                    randomBoolean(),
-                    randomBoolean(),
-                    randomBoolean(),
-                    randomBoolean(),
-                    randomBoolean(),
-                    randomBoolean(),
-                    randomBoolean()
-                )
-            );
-            case 2 -> onlyErrors = onlyErrors == false;
-            case 3 -> onlyManaged = onlyManaged == false;
-            default -> throw new AssertionError("Illegal randomisation branch");
-        }
-        ExplainLifecycleRequest newRequest = new ExplainLifecycleRequest(TEST_REQUEST_TIMEOUT);
-        newRequest.indices(indices);
-        newRequest.indicesOptions(indicesOptions);
-        newRequest.onlyErrors(onlyErrors);
-        newRequest.onlyManaged(onlyManaged);
-        return newRequest;
-    }
-
-    @Override
-    protected Reader<ExplainLifecycleRequest> instanceReader() {
-        return ExplainLifecycleRequest::new;
-    }
-
-}

+ 0 - 96
x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/ExplainLifecycleResponseTests.java

@@ -1,96 +0,0 @@
-/*
- * 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.core.ilm;
-
-import org.elasticsearch.cluster.ClusterModule;
-import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
-import org.elasticsearch.common.io.stream.Writeable;
-import org.elasticsearch.common.util.CollectionUtils;
-import org.elasticsearch.test.AbstractXContentSerializingTestCase;
-import org.elasticsearch.xcontent.ConstructingObjectParser;
-import org.elasticsearch.xcontent.NamedXContentRegistry;
-import org.elasticsearch.xcontent.ParseField;
-import org.elasticsearch.xcontent.XContentParser;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-
-public class ExplainLifecycleResponseTests extends AbstractXContentSerializingTestCase<ExplainLifecycleResponse> {
-
-    @SuppressWarnings("unchecked")
-    private static final ConstructingObjectParser<ExplainLifecycleResponse, Void> PARSER = new ConstructingObjectParser<>(
-        "explain_lifecycle_response",
-        a -> new ExplainLifecycleResponse(
-            ((List<IndexLifecycleExplainResponse>) a[0]).stream()
-                .collect(Collectors.toMap(IndexLifecycleExplainResponse::getIndex, Function.identity()))
-        )
-    );
-    static {
-        PARSER.declareNamedObjects(
-            ConstructingObjectParser.constructorArg(),
-            (p, c, n) -> IndexLifecycleExplainResponse.PARSER.apply(p, c),
-            ExplainLifecycleResponse.INDICES_FIELD
-        );
-    }
-
-    @Override
-    protected ExplainLifecycleResponse createTestInstance() {
-        Map<String, IndexLifecycleExplainResponse> indexResponses = new HashMap<>();
-        long now = System.currentTimeMillis();
-        for (int i = 0; i < randomIntBetween(0, 2); i++) {
-            IndexLifecycleExplainResponse indexResponse = IndexLifecycleExplainResponseTests.randomIndexExplainResponse();
-            // Since the age is calculated from now, we make now constant so that we don't get changes in age during the run of the test:
-            indexResponse.nowSupplier = () -> now;
-            indexResponses.put(indexResponse.getIndex(), indexResponse);
-        }
-        return new ExplainLifecycleResponse(indexResponses);
-    }
-
-    @Override
-    protected Writeable.Reader<ExplainLifecycleResponse> instanceReader() {
-        return ExplainLifecycleResponse::new;
-    }
-
-    @Override
-    protected ExplainLifecycleResponse mutateInstance(ExplainLifecycleResponse response) {
-        Map<String, IndexLifecycleExplainResponse> indexResponses = new HashMap<>(response.getIndexResponses());
-        IndexLifecycleExplainResponse indexResponse = IndexLifecycleExplainResponseTests.randomIndexExplainResponse();
-        indexResponses.put(indexResponse.getIndex(), indexResponse);
-        return new ExplainLifecycleResponse(indexResponses);
-    }
-
-    @Override
-    protected ExplainLifecycleResponse doParseInstance(XContentParser parser) throws IOException {
-        return PARSER.apply(parser, null);
-    }
-
-    @Override
-    protected boolean assertToXContentEquivalence() {
-        return false;
-    }
-
-    protected NamedWriteableRegistry getNamedWriteableRegistry() {
-        return new NamedWriteableRegistry(
-            List.of(new NamedWriteableRegistry.Entry(LifecycleAction.class, MockAction.NAME, MockAction::new))
-        );
-    }
-
-    @Override
-    protected NamedXContentRegistry xContentRegistry() {
-        return new NamedXContentRegistry(
-            CollectionUtils.appendToCopy(
-                ClusterModule.getNamedXWriteables(),
-                new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(MockAction.NAME), MockAction::parse)
-            )
-        );
-    }
-}

+ 6 - 1
x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/RestExplainLifecycleAction.java

@@ -12,6 +12,7 @@ import org.elasticsearch.client.internal.node.NodeClient;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.rest.BaseRestHandler;
 import org.elasticsearch.rest.RestRequest;
+import org.elasticsearch.rest.action.RestCancellableNodeClient;
 import org.elasticsearch.rest.action.RestToXContentListener;
 import org.elasticsearch.xpack.core.ilm.ExplainLifecycleRequest;
 import org.elasticsearch.xpack.core.ilm.action.ExplainLifecycleAction;
@@ -41,6 +42,10 @@ public class RestExplainLifecycleAction extends BaseRestHandler {
         explainLifecycleRequest.indicesOptions(IndicesOptions.fromRequest(restRequest, IndicesOptions.strictExpandOpen()));
         explainLifecycleRequest.onlyManaged(restRequest.paramAsBoolean("only_managed", false));
         explainLifecycleRequest.onlyErrors(restRequest.paramAsBoolean("only_errors", false));
-        return channel -> client.execute(ExplainLifecycleAction.INSTANCE, explainLifecycleRequest, new RestToXContentListener<>(channel));
+        return channel -> new RestCancellableNodeClient(client, restRequest.getHttpChannel()).execute(
+            ExplainLifecycleAction.INSTANCE,
+            explainLifecycleRequest,
+            new RestToXContentListener<>(channel)
+        );
     }
 }

+ 47 - 23
x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportExplainLifecycleAction.java

@@ -10,8 +10,11 @@ package org.elasticsearch.xpack.ilm.action;
 import org.elasticsearch.ElasticsearchParseException;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.action.support.ActionFilters;
-import org.elasticsearch.action.support.master.info.TransportClusterInfoAction;
+import org.elasticsearch.action.support.ChannelActionListener;
+import org.elasticsearch.action.support.local.TransportLocalClusterStateAction;
 import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.cluster.block.ClusterBlockException;
+import org.elasticsearch.cluster.block.ClusterBlockLevel;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
 import org.elasticsearch.cluster.metadata.LifecycleExecutionState;
@@ -22,7 +25,9 @@ import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.bytes.BytesArray;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.core.Nullable;
+import org.elasticsearch.core.UpdateForV10;
 import org.elasticsearch.injection.guice.Inject;
+import org.elasticsearch.tasks.CancellableTask;
 import org.elasticsearch.tasks.Task;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.transport.TransportService;
@@ -34,11 +39,11 @@ import org.elasticsearch.xpack.core.ilm.ErrorStep;
 import org.elasticsearch.xpack.core.ilm.ExplainLifecycleRequest;
 import org.elasticsearch.xpack.core.ilm.ExplainLifecycleResponse;
 import org.elasticsearch.xpack.core.ilm.IndexLifecycleExplainResponse;
+import org.elasticsearch.xpack.core.ilm.IndexLifecycleMetadata;
 import org.elasticsearch.xpack.core.ilm.LifecycleSettings;
 import org.elasticsearch.xpack.core.ilm.PhaseExecutionInfo;
 import org.elasticsearch.xpack.core.ilm.RolloverAction;
 import org.elasticsearch.xpack.core.ilm.action.ExplainLifecycleAction;
-import org.elasticsearch.xpack.ilm.IndexLifecycleService;
 
 import java.io.IOException;
 import java.util.Map;
@@ -47,11 +52,17 @@ import java.util.TreeMap;
 import static org.elasticsearch.index.IndexSettings.LIFECYCLE_ORIGINATION_DATE;
 import static org.elasticsearch.xpack.core.ilm.WaitForRolloverReadyStep.applyDefaultConditions;
 
-public class TransportExplainLifecycleAction extends TransportClusterInfoAction<ExplainLifecycleRequest, ExplainLifecycleResponse> {
+public class TransportExplainLifecycleAction extends TransportLocalClusterStateAction<ExplainLifecycleRequest, ExplainLifecycleResponse> {
 
     private final NamedXContentRegistry xContentRegistry;
-    private final IndexLifecycleService indexLifecycleService;
+    private final IndexNameExpressionResolver indexNameExpressionResolver;
 
+    /**
+     * NB prior to 9.0 this was a TransportMasterNodeReadAction so for BwC it must be registered with the TransportService until
+     * we no longer need to support calling this action remotely.
+     */
+    @UpdateForV10(owner = UpdateForV10.Owner.DATA_MANAGEMENT)
+    @SuppressWarnings("this-escape")
     @Inject
     public TransportExplainLifecycleAction(
         TransportService transportService,
@@ -60,32 +71,43 @@ public class TransportExplainLifecycleAction extends TransportClusterInfoAction<
         ActionFilters actionFilters,
         IndexNameExpressionResolver indexNameExpressionResolver,
         NamedXContentRegistry xContentRegistry,
-        IndexLifecycleService indexLifecycleService,
         ProjectResolver projectResolver
     ) {
         super(
             ExplainLifecycleAction.NAME,
-            transportService,
-            clusterService,
-            threadPool,
             actionFilters,
-            ExplainLifecycleRequest::new,
-            indexNameExpressionResolver,
-            ExplainLifecycleResponse::new,
-            projectResolver
+            transportService.getTaskManager(),
+            clusterService,
+            threadPool.executor(ThreadPool.Names.MANAGEMENT)
         );
         this.xContentRegistry = xContentRegistry;
-        this.indexLifecycleService = indexLifecycleService;
+        this.indexNameExpressionResolver = indexNameExpressionResolver;
+
+        transportService.registerRequestHandler(
+            actionName,
+            executor,
+            false,
+            true,
+            ExplainLifecycleRequest::new,
+            (request, channel, task) -> executeDirect(task, request, new ChannelActionListener<>(channel))
+        );
+
+    }
+
+    @Override
+    protected ClusterBlockException checkBlock(ExplainLifecycleRequest request, ClusterState state) {
+        return state.blocks()
+            .indicesBlockedException(ClusterBlockLevel.METADATA_READ, indexNameExpressionResolver.concreteIndexNames(state, request));
     }
 
     @Override
-    protected void doMasterOperation(
+    protected void localClusterStateOperation(
         Task task,
         ExplainLifecycleRequest request,
-        String[] concreteIndices,
-        ClusterState state,
+        final ClusterState state,
         ActionListener<ExplainLifecycleResponse> listener
     ) {
+        String[] concreteIndices = indexNameExpressionResolver.concreteIndexNames(state, request);
         boolean rolloverOnlyIfHasDocuments = LifecycleSettings.LIFECYCLE_ROLLOVER_ONLY_IF_HAS_DOCUMENTS_SETTING.get(
             state.metadata().settings()
         );
@@ -98,7 +120,6 @@ public class TransportExplainLifecycleAction extends TransportClusterInfoAction<
                     state.metadata(),
                     request.onlyErrors(),
                     request.onlyManaged(),
-                    indexLifecycleService,
                     xContentRegistry,
                     rolloverOnlyIfHasDocuments
                 );
@@ -111,6 +132,8 @@ public class TransportExplainLifecycleAction extends TransportClusterInfoAction<
                 indexResponses.put(indexResponse.getIndex(), indexResponse);
             }
         }
+        // Ensure not cancelled before building XContent.
+        ((CancellableTask) task).ensureNotCancelled();
         listener.onResponse(new ExplainLifecycleResponse(indexResponses));
     }
 
@@ -120,11 +143,11 @@ public class TransportExplainLifecycleAction extends TransportClusterInfoAction<
         Metadata metadata,
         boolean onlyErrors,
         boolean onlyManaged,
-        IndexLifecycleService indexLifecycleService,
         NamedXContentRegistry xContentRegistry,
         boolean rolloverOnlyIfHasDocuments
     ) throws IOException {
-        IndexMetadata indexMetadata = metadata.getProject().index(indexName);
+        final var project = metadata.getProject();
+        IndexMetadata indexMetadata = project.index(indexName);
         Settings idxSettings = indexMetadata.getSettings();
         LifecycleExecutionState lifecycleState = indexMetadata.getLifecycleExecutionState();
         String policyName = indexMetadata.getLifecyclePolicyName();
@@ -167,10 +190,11 @@ public class TransportExplainLifecycleAction extends TransportClusterInfoAction<
         }
 
         final IndexLifecycleExplainResponse indexResponse;
-        if (metadata.getProject().isIndexManagedByILM(indexMetadata)) {
+        if (project.isIndexManagedByILM(indexMetadata)) {
+            final IndexLifecycleMetadata indexLifecycleMetadata = project.custom(IndexLifecycleMetadata.TYPE, IndexLifecycleMetadata.EMPTY);
+            final boolean policyExists = indexLifecycleMetadata.getPolicies().containsKey(policyName);
             // If this is requesting only errors, only include indices in the error step or which are using a nonexistent policy
-            if (onlyErrors == false
-                || (ErrorStep.NAME.equals(lifecycleState.step()) || indexLifecycleService.policyExists(policyName) == false)) {
+            if (onlyErrors == false || (ErrorStep.NAME.equals(lifecycleState.step()) || policyExists == false)) {
                 Long originationDate = idxSettings.getAsLong(LIFECYCLE_ORIGINATION_DATE, -1L);
                 indexResponse = IndexLifecycleExplainResponse.newManagedIndexResponse(
                     indexName,
@@ -180,7 +204,7 @@ public class TransportExplainLifecycleAction extends TransportClusterInfoAction<
                     lifecycleState.phase(),
                     lifecycleState.action(),
                     // treat a missing policy as if the index is in the error step
-                    indexLifecycleService.policyExists(policyName) == false ? ErrorStep.NAME : lifecycleState.step(),
+                    policyExists == false ? ErrorStep.NAME : lifecycleState.step(),
                     lifecycleState.failedStep(),
                     lifecycleState.isAutoRetryableError(),
                     lifecycleState.failedStepRetryCount(),

+ 35 - 27
x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/action/TransportExplainLifecycleActionTests.java

@@ -16,26 +16,30 @@ import org.elasticsearch.xcontent.NamedXContentRegistry;
 import org.elasticsearch.xcontent.ParseField;
 import org.elasticsearch.xpack.core.ilm.ErrorStep;
 import org.elasticsearch.xpack.core.ilm.IndexLifecycleExplainResponse;
+import org.elasticsearch.xpack.core.ilm.IndexLifecycleMetadata;
 import org.elasticsearch.xpack.core.ilm.LifecycleAction;
+import org.elasticsearch.xpack.core.ilm.LifecyclePolicyMetadataTests;
 import org.elasticsearch.xpack.core.ilm.LifecycleSettings;
+import org.elasticsearch.xpack.core.ilm.OperationMode;
 import org.elasticsearch.xpack.core.ilm.RolloverAction;
 import org.elasticsearch.xpack.core.ilm.WaitForRolloverReadyStep;
 import org.elasticsearch.xpack.ilm.IndexLifecycleService;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.Map;
 
 import static org.elasticsearch.cluster.metadata.LifecycleExecutionState.ILM_CUSTOM_METADATA_KEY;
 import static org.elasticsearch.xpack.ilm.action.TransportExplainLifecycleAction.getIndexLifecycleExplainResponse;
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.notNullValue;
 import static org.hamcrest.Matchers.nullValue;
-import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 public class TransportExplainLifecycleActionTests extends ESTestCase {
 
+    private static final String POLICY_NAME = "my-policy";
     public static final String PHASE_DEFINITION = """
         {
           "policy" : "my-policy",
@@ -62,9 +66,6 @@ public class TransportExplainLifecycleActionTests extends ESTestCase {
     public void testGetIndexLifecycleExplainResponse() throws IOException {
         {
             // only errors index
-            IndexLifecycleService indexLifecycleService = mock(IndexLifecycleService.class);
-            when(indexLifecycleService.policyExists("my-policy")).thenReturn(true);
-
             LifecycleExecutionState.Builder errorStepState = LifecycleExecutionState.builder()
                 .setPhase("hot")
                 .setAction("rollover")
@@ -72,19 +73,21 @@ public class TransportExplainLifecycleActionTests extends ESTestCase {
                 .setPhaseDefinition(PHASE_DEFINITION);
             String indexInErrorStep = "index_in_error";
             IndexMetadata indexMetadata = IndexMetadata.builder(indexInErrorStep)
-                .settings(settings(IndexVersion.current()).put(LifecycleSettings.LIFECYCLE_NAME, "my-policy"))
+                .settings(settings(IndexVersion.current()).put(LifecycleSettings.LIFECYCLE_NAME, POLICY_NAME))
                 .numberOfShards(randomIntBetween(1, 5))
                 .numberOfReplicas(randomIntBetween(0, 5))
                 .putCustom(ILM_CUSTOM_METADATA_KEY, errorStepState.build().asMap())
                 .build();
-            Metadata metadata = Metadata.builder().put(indexMetadata, true).build();
+            Metadata metadata = Metadata.builder()
+                .put(indexMetadata, true)
+                .putCustom(IndexLifecycleMetadata.TYPE, createIndexLifecycleMetadata())
+                .build();
 
             IndexLifecycleExplainResponse onlyErrorsResponse = getIndexLifecycleExplainResponse(
                 indexInErrorStep,
                 metadata,
                 true,
                 true,
-                indexLifecycleService,
                 REGISTRY,
                 randomBoolean()
             );
@@ -95,9 +98,6 @@ public class TransportExplainLifecycleActionTests extends ESTestCase {
 
         {
             // only managed index
-            IndexLifecycleService indexLifecycleService = mock(IndexLifecycleService.class);
-            when(indexLifecycleService.policyExists("my-policy")).thenReturn(true);
-
             LifecycleExecutionState.Builder checkRolloverReadyStepState = LifecycleExecutionState.builder()
                 .setPhase("hot")
                 .setAction("rollover")
@@ -106,19 +106,21 @@ public class TransportExplainLifecycleActionTests extends ESTestCase {
 
             String indexInCheckRolloverStep = "index_in_check_rollover";
             IndexMetadata indexMetadata = IndexMetadata.builder(indexInCheckRolloverStep)
-                .settings(settings(IndexVersion.current()).put(LifecycleSettings.LIFECYCLE_NAME, "my-policy"))
+                .settings(settings(IndexVersion.current()).put(LifecycleSettings.LIFECYCLE_NAME, POLICY_NAME))
                 .numberOfShards(randomIntBetween(1, 5))
                 .numberOfReplicas(randomIntBetween(0, 5))
                 .putCustom(ILM_CUSTOM_METADATA_KEY, checkRolloverReadyStepState.build().asMap())
                 .build();
-            Metadata metadata = Metadata.builder().put(indexMetadata, true).build();
+            Metadata metadata = Metadata.builder()
+                .put(indexMetadata, true)
+                .putCustom(IndexLifecycleMetadata.TYPE, createIndexLifecycleMetadata())
+                .build();
 
             IndexLifecycleExplainResponse onlyErrorsResponse = getIndexLifecycleExplainResponse(
                 indexInCheckRolloverStep,
                 metadata,
                 true,
                 true,
-                indexLifecycleService,
                 REGISTRY,
                 randomBoolean()
             );
@@ -129,7 +131,6 @@ public class TransportExplainLifecycleActionTests extends ESTestCase {
                 metadata,
                 false,
                 true,
-                indexLifecycleService,
                 REGISTRY,
                 randomBoolean()
             );
@@ -149,14 +150,16 @@ public class TransportExplainLifecycleActionTests extends ESTestCase {
                 .numberOfShards(randomIntBetween(1, 5))
                 .numberOfReplicas(randomIntBetween(0, 5))
                 .build();
-            Metadata metadata = Metadata.builder().put(indexMetadata, true).build();
+            Metadata metadata = Metadata.builder()
+                .put(indexMetadata, true)
+                .putCustom(IndexLifecycleMetadata.TYPE, createIndexLifecycleMetadata())
+                .build();
 
             IndexLifecycleExplainResponse onlyErrorsResponse = getIndexLifecycleExplainResponse(
                 indexWithMissingPolicy,
                 metadata,
                 true,
                 true,
-                indexLifecycleService,
                 REGISTRY,
                 randomBoolean()
             );
@@ -167,22 +170,21 @@ public class TransportExplainLifecycleActionTests extends ESTestCase {
 
         {
             // not managed index
-            IndexLifecycleService indexLifecycleService = mock(IndexLifecycleService.class);
-            when(indexLifecycleService.policyExists(anyString())).thenReturn(true);
-
             IndexMetadata indexMetadata = IndexMetadata.builder("index")
                 .settings(settings(IndexVersion.current()))
                 .numberOfShards(randomIntBetween(1, 5))
                 .numberOfReplicas(randomIntBetween(0, 5))
                 .build();
-            Metadata metadata = Metadata.builder().put(indexMetadata, true).build();
+            Metadata metadata = Metadata.builder()
+                .put(indexMetadata, true)
+                .putCustom(IndexLifecycleMetadata.TYPE, createIndexLifecycleMetadata())
+                .build();
 
             IndexLifecycleExplainResponse onlyManaged = getIndexLifecycleExplainResponse(
                 "index",
                 metadata,
                 false,
                 true,
-                indexLifecycleService,
                 REGISTRY,
                 randomBoolean()
             );
@@ -191,9 +193,6 @@ public class TransportExplainLifecycleActionTests extends ESTestCase {
 
         {
             // validate addition of default condition with `rolloverOnlyIfHasDocuments` true
-            IndexLifecycleService indexLifecycleService = mock(IndexLifecycleService.class);
-            when(indexLifecycleService.policyExists("my-policy")).thenReturn(true);
-
             LifecycleExecutionState.Builder checkRolloverReadyStepState = LifecycleExecutionState.builder()
                 .setPhase("hot")
                 .setAction("rollover")
@@ -201,19 +200,21 @@ public class TransportExplainLifecycleActionTests extends ESTestCase {
                 .setPhaseDefinition(PHASE_DEFINITION);
             String indexInCheckRolloverStep = "index_in_check_rollover";
             IndexMetadata indexMetadata = IndexMetadata.builder(indexInCheckRolloverStep)
-                .settings(settings(IndexVersion.current()).put(LifecycleSettings.LIFECYCLE_NAME, "my-policy"))
+                .settings(settings(IndexVersion.current()).put(LifecycleSettings.LIFECYCLE_NAME, POLICY_NAME))
                 .numberOfShards(randomIntBetween(1, 5))
                 .numberOfReplicas(randomIntBetween(0, 5))
                 .putCustom(ILM_CUSTOM_METADATA_KEY, checkRolloverReadyStepState.build().asMap())
                 .build();
-            Metadata metadata = Metadata.builder().put(indexMetadata, true).build();
+            Metadata metadata = Metadata.builder()
+                .put(indexMetadata, true)
+                .putCustom(IndexLifecycleMetadata.TYPE, createIndexLifecycleMetadata())
+                .build();
 
             IndexLifecycleExplainResponse response = getIndexLifecycleExplainResponse(
                 indexInCheckRolloverStep,
                 metadata,
                 false,
                 true,
-                indexLifecycleService,
                 REGISTRY,
                 true
             );
@@ -222,4 +223,11 @@ public class TransportExplainLifecycleActionTests extends ESTestCase {
             assertThat(rolloverAction.getConditions().getMinDocs(), is(1L));
         }
     }
+
+    private static IndexLifecycleMetadata createIndexLifecycleMetadata() {
+        return new IndexLifecycleMetadata(
+            Map.of(POLICY_NAME, LifecyclePolicyMetadataTests.createRandomPolicyMetadata(POLICY_NAME)),
+            randomFrom(OperationMode.values())
+        );
+    }
 }