Răsfoiți Sursa

Implement ILM/settings operator handlers (#88097)

Relates to #86224
Nikola Grcevski 3 ani în urmă
părinte
comite
fc93f77d3b
15 a modificat fișierele cu 753 adăugiri și 4 ștergeri
  1. 2 0
      server/src/main/java/module-info.java
  2. 20 0
      server/src/main/java/org/elasticsearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java
  3. 30 0
      server/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeAction.java
  4. 90 0
      server/src/main/java/org/elasticsearch/immutablestate/action/ImmutableClusterSettingsAction.java
  5. 100 0
      server/src/test/java/org/elasticsearch/immutablestate/action/ImmutableClusterSettingsActionTests.java
  6. 7 0
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/action/DeleteLifecycleAction.java
  7. 4 0
      x-pack/plugin/ilm/src/main/java/module-info.java
  8. 32 0
      x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/ILMImmutableStateHandlerProvider.java
  9. 5 0
      x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycle.java
  10. 111 0
      x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/ImmutableLifecycleAction.java
  11. 20 0
      x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportDeleteLifecycleAction.java
  12. 37 4
      x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportPutLifecycleAction.java
  13. 212 0
      x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/action/ImmutableILMStateControllerTests.java
  14. 35 0
      x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/action/TransportDeleteLifecycleActionTests.java
  15. 48 0
      x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/action/TransportPutLifecycleActionTests.java

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

@@ -358,4 +358,6 @@ module org.elasticsearch.server {
         with
             org.elasticsearch.cluster.coordination.NodeToolCliProvider,
             org.elasticsearch.index.shard.ShardToolCliProvider;
+
+    uses org.elasticsearch.immutablestate.ImmutableClusterStateHandlerProvider;
 }

+ 20 - 0
server/src/main/java/org/elasticsearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java

@@ -29,11 +29,13 @@ import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.core.SuppressForbidden;
+import org.elasticsearch.immutablestate.action.ImmutableClusterSettingsAction;
 import org.elasticsearch.tasks.Task;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.transport.TransportService;
 
 import java.util.HashSet;
+import java.util.Optional;
 import java.util.Set;
 
 import static org.elasticsearch.common.settings.AbstractScopedSettings.ARCHIVED_SETTINGS_PREFIX;
@@ -128,6 +130,17 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeAct
         return true;
     }
 
+    @Override
+    protected Optional<String> immutableStateHandlerName() {
+        return Optional.of(ImmutableClusterSettingsAction.NAME);
+    }
+
+    @Override
+    protected Set<String> modifiedKeys(ClusterUpdateSettingsRequest request) {
+        Settings allSettings = Settings.builder().put(request.persistentSettings()).put(request.transientSettings()).build();
+        return allSettings.keySet();
+    }
+
     private static final String UPDATE_TASK_SOURCE = "cluster_update_settings";
     private static final String REROUTE_TASK_SOURCE = "reroute_after_cluster_update_settings";
 
@@ -243,6 +256,13 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeAct
             this.request = request;
         }
 
+        /**
+         * Used by the immutable state handler {@link ImmutableClusterSettingsAction}
+         */
+        public ClusterUpdateSettingsTask(final ClusterSettings clusterSettings, ClusterUpdateSettingsRequest request) {
+            this(clusterSettings, Priority.IMMEDIATE, request, null);
+        }
+
         @Override
         public ClusterState execute(final ClusterState currentState) {
             final ClusterState clusterState = updater.updateSettings(

+ 30 - 0
server/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeAction.java

@@ -42,6 +42,9 @@ import org.elasticsearch.transport.RemoteTransportException;
 import org.elasticsearch.transport.TransportException;
 import org.elasticsearch.transport.TransportService;
 
+import java.util.Collections;
+import java.util.Optional;
+import java.util.Set;
 import java.util.function.Predicate;
 
 import static org.elasticsearch.core.Strings.format;
@@ -142,6 +145,33 @@ public abstract class TransportMasterNodeAction<Request extends MasterNodeReques
         }
     }
 
+    /**
+     * Override this method if the master node action also has an {@link org.elasticsearch.immutablestate.ImmutableClusterStateHandler}
+     * interaction.
+     * <p>
+     * We need to check if certain settings or entities are allowed to be modified by the master node
+     * action, depending on if they are set as immutable in 'operator' mode (file based settings, modules, plugins).
+     *
+     * @return an Optional of the {@link org.elasticsearch.immutablestate.ImmutableClusterStateHandler} name
+     */
+    protected Optional<String> immutableStateHandlerName() {
+        return Optional.empty();
+    }
+
+    /**
+     * Override this method to return the keys of the cluster state or cluster entities that are modified by
+     * the Request object.
+     * <p>
+     * This method is used by the immutable state handler logic (see {@link org.elasticsearch.immutablestate.ImmutableClusterStateHandler})
+     * to verify if the keys don't conflict with an existing key set as immutable.
+     *
+     * @param request the TransportMasterNode request
+     * @return set of String keys intended to be modified/set/deleted by this request
+     */
+    protected Set<String> modifiedKeys(Request request) {
+        return Collections.emptySet();
+    }
+
     @Override
     protected void doExecute(Task task, final Request request, ActionListener<Response> listener) {
         ClusterState state = clusterService.state();

+ 90 - 0
server/src/main/java/org/elasticsearch/immutablestate/action/ImmutableClusterSettingsAction.java

@@ -0,0 +1,90 @@
+/*
+ * 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.immutablestate.action;
+
+import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest;
+import org.elasticsearch.action.admin.cluster.settings.TransportClusterUpdateSettingsAction;
+import org.elasticsearch.client.internal.Requests;
+import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.common.settings.ClusterSettings;
+import org.elasticsearch.immutablestate.ImmutableClusterStateHandler;
+import org.elasticsearch.immutablestate.TransformState;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.elasticsearch.common.util.Maps.asMap;
+
+/**
+ * This Action is the immutable state save version of RestClusterUpdateSettingsAction
+ * <p>
+ * It is used by the ImmutableClusterStateController to update the persistent cluster settings.
+ * Since transient cluster settings are deprecated, this action doesn't support updating transient cluster settings.
+ */
+public class ImmutableClusterSettingsAction implements ImmutableClusterStateHandler<ClusterUpdateSettingsRequest> {
+
+    public static final String NAME = "cluster_settings";
+
+    private final ClusterSettings clusterSettings;
+
+    public ImmutableClusterSettingsAction(ClusterSettings clusterSettings) {
+        this.clusterSettings = clusterSettings;
+    }
+
+    @Override
+    public String name() {
+        return NAME;
+    }
+
+    @SuppressWarnings("unchecked")
+    private ClusterUpdateSettingsRequest prepare(Object input, Set<String> previouslySet) {
+        final ClusterUpdateSettingsRequest clusterUpdateSettingsRequest = Requests.clusterUpdateSettingsRequest();
+
+        Map<String, ?> source = asMap(input);
+        Map<String, Object> persistentSettings = new HashMap<>();
+        Set<String> toDelete = new HashSet<>(previouslySet);
+
+        source.forEach((k, v) -> {
+            persistentSettings.put(k, v);
+            toDelete.remove(k);
+        });
+
+        toDelete.forEach(k -> persistentSettings.put(k, null));
+
+        clusterUpdateSettingsRequest.persistentSettings(persistentSettings);
+        return clusterUpdateSettingsRequest;
+    }
+
+    @Override
+    public TransformState transform(Object input, TransformState prevState) {
+        ClusterUpdateSettingsRequest request = prepare(input, prevState.keys());
+
+        // allow empty requests, this is how we clean up settings
+        if (request.persistentSettings().isEmpty() == false) {
+            validate(request);
+        }
+
+        ClusterState state = prevState.state();
+
+        TransportClusterUpdateSettingsAction.ClusterUpdateSettingsTask updateSettingsTask =
+            new TransportClusterUpdateSettingsAction.ClusterUpdateSettingsTask(clusterSettings, request);
+
+        state = updateSettingsTask.execute(state);
+        Set<String> currentKeys = request.persistentSettings()
+            .keySet()
+            .stream()
+            .filter(k -> request.persistentSettings().hasValue(k))
+            .collect(Collectors.toSet());
+
+        return new TransformState(state, currentKeys);
+    }
+}

+ 100 - 0
server/src/test/java/org/elasticsearch/immutablestate/action/ImmutableClusterSettingsActionTests.java

@@ -0,0 +1,100 @@
+/*
+ * 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.immutablestate.action;
+
+import org.elasticsearch.cluster.ClusterName;
+import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.common.settings.ClusterSettings;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.immutablestate.TransformState;
+import org.elasticsearch.test.ESTestCase;
+import org.elasticsearch.xcontent.XContentParser;
+import org.elasticsearch.xcontent.XContentParserConfiguration;
+import org.elasticsearch.xcontent.XContentType;
+
+import java.util.Collections;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+
+public class ImmutableClusterSettingsActionTests extends ESTestCase {
+
+    private TransformState processJSON(ImmutableClusterSettingsAction action, TransformState prevState, String json) throws Exception {
+        try (XContentParser parser = XContentType.JSON.xContent().createParser(XContentParserConfiguration.EMPTY, json)) {
+            return action.transform(parser.map(), prevState);
+        }
+    }
+
+    public void testValidation() throws Exception {
+        ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
+
+        ClusterState state = ClusterState.builder(new ClusterName("elasticsearch")).build();
+        TransformState prevState = new TransformState(state, Collections.emptySet());
+        ImmutableClusterSettingsAction action = new ImmutableClusterSettingsAction(clusterSettings);
+
+        String badPolicyJSON = """
+            {
+                "indices.recovery.min_bytes_per_sec": "50mb"
+            }""";
+
+        assertEquals(
+            "persistent setting [indices.recovery.min_bytes_per_sec], not recognized",
+            expectThrows(IllegalArgumentException.class, () -> processJSON(action, prevState, badPolicyJSON)).getMessage()
+        );
+    }
+
+    public void testSetUnsetSettings() throws Exception {
+        ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
+
+        ClusterState state = ClusterState.builder(new ClusterName("elasticsearch")).build();
+        TransformState prevState = new TransformState(state, Collections.emptySet());
+        ImmutableClusterSettingsAction action = new ImmutableClusterSettingsAction(clusterSettings);
+
+        String emptyJSON = "";
+
+        TransformState updatedState = processJSON(action, prevState, emptyJSON);
+        assertEquals(0, updatedState.keys().size());
+        assertEquals(prevState.state(), updatedState.state());
+
+        String settingsJSON = """
+            {
+                "indices.recovery.max_bytes_per_sec": "50mb",
+                "cluster": {
+                     "remote": {
+                         "cluster_one": {
+                             "seeds": [
+                                 "127.0.0.1:9300"
+                             ]
+                         }
+                     }
+                }
+            }""";
+
+        prevState = updatedState;
+        updatedState = processJSON(action, prevState, settingsJSON);
+        assertThat(updatedState.keys(), containsInAnyOrder("indices.recovery.max_bytes_per_sec", "cluster.remote.cluster_one.seeds"));
+        assertEquals("50mb", updatedState.state().metadata().persistentSettings().get("indices.recovery.max_bytes_per_sec"));
+        assertEquals("[127.0.0.1:9300]", updatedState.state().metadata().persistentSettings().get("cluster.remote.cluster_one.seeds"));
+
+        String oneSettingJSON = """
+            {
+                "indices.recovery.max_bytes_per_sec": "25mb"
+            }""";
+
+        prevState = updatedState;
+        updatedState = processJSON(action, prevState, oneSettingJSON);
+        assertThat(updatedState.keys(), containsInAnyOrder("indices.recovery.max_bytes_per_sec"));
+        assertEquals("25mb", updatedState.state().metadata().persistentSettings().get("indices.recovery.max_bytes_per_sec"));
+        assertNull(updatedState.state().metadata().persistentSettings().get("cluster.remote.cluster_one.seeds"));
+
+        prevState = updatedState;
+        updatedState = processJSON(action, prevState, emptyJSON);
+        assertEquals(0, updatedState.keys().size());
+        assertNull(updatedState.state().metadata().persistentSettings().get("indices.recovery.max_bytes_per_sec"));
+    }
+}

+ 7 - 0
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/action/DeleteLifecycleAction.java

@@ -18,6 +18,8 @@ import org.elasticsearch.xcontent.ParseField;
 import java.io.IOException;
 import java.util.Objects;
 
+import static org.elasticsearch.core.Strings.format;
+
 public class DeleteLifecycleAction extends ActionType<AcknowledgedResponse> {
     public static final DeleteLifecycleAction INSTANCE = new DeleteLifecycleAction();
     public static final String NAME = "cluster:admin/ilm/delete";
@@ -75,6 +77,11 @@ public class DeleteLifecycleAction extends ActionType<AcknowledgedResponse> {
             return Objects.equals(policyName, other.policyName);
         }
 
+        @Override
+        public String toString() {
+            return format("delete lifecycle policy [%s]", policyName);
+        }
+
     }
 
 }

+ 4 - 0
x-pack/plugin/ilm/src/main/java/module-info.java

@@ -1,3 +1,5 @@
+import org.elasticsearch.xpack.ilm.ILMImmutableStateHandlerProvider;
+
 /*
  * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
  * or more contributor license agreements. Licensed under the Elastic License
@@ -16,4 +18,6 @@ module org.elasticsearch.ilm {
     exports org.elasticsearch.xpack.ilm to org.elasticsearch.server;
     exports org.elasticsearch.xpack.slm.action to org.elasticsearch.server;
     exports org.elasticsearch.xpack.slm to org.elasticsearch.server;
+
+    provides org.elasticsearch.immutablestate.ImmutableClusterStateHandlerProvider with ILMImmutableStateHandlerProvider;
 }

+ 32 - 0
x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/ILMImmutableStateHandlerProvider.java

@@ -0,0 +1,32 @@
+/*
+ * 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.immutablestate.ImmutableClusterStateHandler;
+import org.elasticsearch.immutablestate.ImmutableClusterStateHandlerProvider;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * ILM Provider implementation for the {@link ImmutableClusterStateHandlerProvider} service interface
+ */
+public class ILMImmutableStateHandlerProvider implements ImmutableClusterStateHandlerProvider {
+    private static final Set<ImmutableClusterStateHandler<?>> handlers = ConcurrentHashMap.newKeySet();
+
+    @Override
+    public Collection<ImmutableClusterStateHandler<?>> handlers() {
+        return handlers;
+    }
+
+    public static void registerHandlers(ImmutableClusterStateHandler<?>... stateHandlers) {
+        handlers.addAll(Arrays.asList(stateHandlers));
+    }
+}

+ 5 - 0
x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycle.java

@@ -83,6 +83,7 @@ import org.elasticsearch.xpack.core.slm.action.GetSnapshotLifecycleStatsAction;
 import org.elasticsearch.xpack.core.slm.action.PutSnapshotLifecycleAction;
 import org.elasticsearch.xpack.core.slm.action.StartSLMAction;
 import org.elasticsearch.xpack.core.slm.action.StopSLMAction;
+import org.elasticsearch.xpack.ilm.action.ImmutableLifecycleAction;
 import org.elasticsearch.xpack.ilm.action.RestDeleteLifecycleAction;
 import org.elasticsearch.xpack.ilm.action.RestExplainLifecycleAction;
 import org.elasticsearch.xpack.ilm.action.RestGetLifecycleAction;
@@ -267,6 +268,10 @@ public class IndexLifecycle extends Plugin implements ActionPlugin, HealthPlugin
         components.addAll(Arrays.asList(snapshotLifecycleService.get(), snapshotHistoryStore.get(), snapshotRetentionService.get()));
         ilmHealthIndicatorService.set(new IlmHealthIndicatorService(clusterService));
         slmHealthIndicatorService.set(new SlmHealthIndicatorService(clusterService));
+
+        ILMImmutableStateHandlerProvider.registerHandlers(
+            new ImmutableLifecycleAction(xContentRegistry, client, XPackPlugin.getSharedLicenseState())
+        );
         return components;
     }
 

+ 111 - 0
x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/ImmutableLifecycleAction.java

@@ -0,0 +1,111 @@
+/*
+ * 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.action;
+
+import org.elasticsearch.client.internal.Client;
+import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.immutablestate.ImmutableClusterStateHandler;
+import org.elasticsearch.immutablestate.TransformState;
+import org.elasticsearch.license.XPackLicenseState;
+import org.elasticsearch.xcontent.NamedXContentRegistry;
+import org.elasticsearch.xcontent.XContentParser;
+import org.elasticsearch.xcontent.XContentParserConfiguration;
+import org.elasticsearch.xpack.core.ilm.LifecyclePolicy;
+import org.elasticsearch.xpack.core.ilm.action.PutLifecycleAction;
+import org.elasticsearch.xpack.core.template.LifecyclePolicyConfig;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.elasticsearch.common.util.Maps.asMap;
+import static org.elasticsearch.common.xcontent.XContentHelper.mapToXContentParser;
+
+/**
+ * This {@link org.elasticsearch.immutablestate.ImmutableClusterStateHandler} is responsible for immutable state
+ * CRUD operations on ILM policies in, e.g. file based settings.
+ * <p>
+ * Internally it uses {@link TransportPutLifecycleAction} and
+ * {@link TransportDeleteLifecycleAction} to add, update and delete ILM policies.
+ */
+public class ImmutableLifecycleAction implements ImmutableClusterStateHandler<LifecyclePolicy> {
+
+    private final NamedXContentRegistry xContentRegistry;
+    private final Client client;
+    private final XPackLicenseState licenseState;
+
+    public static final String NAME = "ilm";
+
+    public ImmutableLifecycleAction(NamedXContentRegistry xContentRegistry, Client client, XPackLicenseState licenseState) {
+        this.xContentRegistry = xContentRegistry;
+        this.client = client;
+        this.licenseState = licenseState;
+    }
+
+    @Override
+    public String name() {
+        return NAME;
+    }
+
+    @SuppressWarnings("unchecked")
+    public Collection<PutLifecycleAction.Request> prepare(Object input) throws IOException {
+        List<PutLifecycleAction.Request> result = new ArrayList<>();
+
+        Map<String, ?> source = asMap(input);
+
+        for (String name : source.keySet()) {
+            Map<String, ?> content = (Map<String, ?>) source.get(name);
+            var config = XContentParserConfiguration.EMPTY.withRegistry(LifecyclePolicyConfig.DEFAULT_X_CONTENT_REGISTRY);
+            try (XContentParser parser = mapToXContentParser(config, content)) {
+                LifecyclePolicy policy = LifecyclePolicy.parse(parser, name);
+                PutLifecycleAction.Request request = new PutLifecycleAction.Request(policy);
+                validate(request);
+                result.add(request);
+            }
+        }
+
+        return result;
+    }
+
+    @Override
+    public TransformState transform(Object source, TransformState prevState) throws Exception {
+        var requests = prepare(source);
+
+        ClusterState state = prevState.state();
+
+        for (var request : requests) {
+            TransportPutLifecycleAction.UpdateLifecyclePolicyTask task = new TransportPutLifecycleAction.UpdateLifecyclePolicyTask(
+                request,
+                licenseState,
+                xContentRegistry,
+                client
+            );
+
+            state = task.execute(state);
+        }
+
+        Set<String> entities = requests.stream().map(r -> r.getPolicy().getName()).collect(Collectors.toSet());
+
+        Set<String> toDelete = new HashSet<>(prevState.keys());
+        toDelete.removeAll(entities);
+
+        for (var policyToDelete : toDelete) {
+            TransportDeleteLifecycleAction.DeleteLifecyclePolicyTask task = new TransportDeleteLifecycleAction.DeleteLifecyclePolicyTask(
+                policyToDelete
+            );
+            state = task.execute(state);
+        }
+
+        return new TransformState(state, entities);
+    }
+}

+ 20 - 0
x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportDeleteLifecycleAction.java

@@ -31,6 +31,8 @@ import org.elasticsearch.xpack.core.ilm.action.DeleteLifecycleAction;
 import org.elasticsearch.xpack.core.ilm.action.DeleteLifecycleAction.Request;
 
 import java.util.List;
+import java.util.Optional;
+import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
@@ -70,6 +72,14 @@ public class TransportDeleteLifecycleAction extends TransportMasterNodeAction<Re
             this.request = request;
         }
 
+        /**
+         * Used by the {@link org.elasticsearch.immutablestate.ImmutableClusterStateHandler} for ILM
+         * {@link ImmutableLifecycleAction}
+         */
+        DeleteLifecyclePolicyTask(String policyName) {
+            this(new Request(policyName), null);
+        }
+
         @Override
         public ClusterState execute(ClusterState currentState) {
             String policyToDelete = request.getPolicyName();
@@ -107,4 +117,14 @@ public class TransportDeleteLifecycleAction extends TransportMasterNodeAction<Re
     protected ClusterBlockException checkBlock(Request request, ClusterState state) {
         return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE);
     }
+
+    @Override
+    protected Optional<String> immutableStateHandlerName() {
+        return Optional.of(ImmutableLifecycleAction.NAME);
+    }
+
+    @Override
+    protected Set<String> modifiedKeys(Request request) {
+        return Set.of(request.getPolicyName());
+    }
 }

+ 37 - 4
x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportPutLifecycleAction.java

@@ -43,8 +43,11 @@ import org.elasticsearch.xpack.core.ilm.action.PutLifecycleAction.Request;
 import org.elasticsearch.xpack.core.slm.SnapshotLifecycleMetadata;
 
 import java.time.Instant;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
@@ -121,6 +124,7 @@ public class TransportPutLifecycleAction extends TransportMasterNodeAction<Reque
         private final Map<String, String> filteredHeaders;
         private final NamedXContentRegistry xContentRegistry;
         private final Client client;
+        private final boolean verboseLogging;
 
         public UpdateLifecyclePolicyTask(
             Request request,
@@ -136,6 +140,23 @@ public class TransportPutLifecycleAction extends TransportMasterNodeAction<Reque
             this.filteredHeaders = filteredHeaders;
             this.xContentRegistry = xContentRegistry;
             this.client = client;
+            this.verboseLogging = true;
+        }
+
+        /**
+         * Used by the {@link org.elasticsearch.immutablestate.ImmutableClusterStateHandler} for ILM
+         * {@link ImmutableLifecycleAction}
+         * <p>
+         * It disables verbose logging and has no filtered headers.
+         */
+        UpdateLifecyclePolicyTask(Request request, XPackLicenseState licenseState, NamedXContentRegistry xContentRegistry, Client client) {
+            super(request, null);
+            this.request = request;
+            this.licenseState = licenseState;
+            this.filteredHeaders = Collections.emptyMap();
+            this.xContentRegistry = xContentRegistry;
+            this.client = client;
+            this.verboseLogging = false;
         }
 
         @Override
@@ -161,10 +182,12 @@ public class TransportPutLifecycleAction extends TransportMasterNodeAction<Reque
                 Instant.now().toEpochMilli()
             );
             LifecyclePolicyMetadata oldPolicy = newPolicies.put(lifecyclePolicyMetadata.getName(), lifecyclePolicyMetadata);
-            if (oldPolicy == null) {
-                logger.info("adding index lifecycle policy [{}]", request.getPolicy().getName());
-            } else {
-                logger.info("updating index lifecycle policy [{}]", request.getPolicy().getName());
+            if (verboseLogging) {
+                if (oldPolicy == null) {
+                    logger.info("adding index lifecycle policy [{}]", request.getPolicy().getName());
+                } else {
+                    logger.info("updating index lifecycle policy [{}]", request.getPolicy().getName());
+                }
             }
             IndexLifecycleMetadata newMetadata = new IndexLifecycleMetadata(newPolicies, currentMetadata.getOperationMode());
             stateBuilder.metadata(Metadata.builder(currentState.getMetadata()).putCustom(IndexLifecycleMetadata.TYPE, newMetadata).build());
@@ -285,4 +308,14 @@ public class TransportPutLifecycleAction extends TransportMasterNodeAction<Reque
     protected ClusterBlockException checkBlock(Request request, ClusterState state) {
         return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE);
     }
+
+    @Override
+    protected Optional<String> immutableStateHandlerName() {
+        return Optional.of(ImmutableLifecycleAction.NAME);
+    }
+
+    @Override
+    protected Set<String> modifiedKeys(Request request) {
+        return Set.of(request.getPolicy().getName());
+    }
 }

+ 212 - 0
x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/action/ImmutableILMStateControllerTests.java

@@ -0,0 +1,212 @@
+/*
+ * 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.action;
+
+import org.elasticsearch.client.internal.Client;
+import org.elasticsearch.cluster.ClusterModule;
+import org.elasticsearch.cluster.ClusterName;
+import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.immutablestate.TransformState;
+import org.elasticsearch.license.XPackLicenseState;
+import org.elasticsearch.test.ESTestCase;
+import org.elasticsearch.xcontent.NamedXContentRegistry;
+import org.elasticsearch.xcontent.ParseField;
+import org.elasticsearch.xcontent.XContentParseException;
+import org.elasticsearch.xcontent.XContentParser;
+import org.elasticsearch.xcontent.XContentParserConfiguration;
+import org.elasticsearch.xcontent.XContentType;
+import org.elasticsearch.xpack.core.ilm.AllocateAction;
+import org.elasticsearch.xpack.core.ilm.DeleteAction;
+import org.elasticsearch.xpack.core.ilm.ForceMergeAction;
+import org.elasticsearch.xpack.core.ilm.FreezeAction;
+import org.elasticsearch.xpack.core.ilm.IndexLifecycleMetadata;
+import org.elasticsearch.xpack.core.ilm.LifecycleAction;
+import org.elasticsearch.xpack.core.ilm.LifecycleType;
+import org.elasticsearch.xpack.core.ilm.MigrateAction;
+import org.elasticsearch.xpack.core.ilm.ReadOnlyAction;
+import org.elasticsearch.xpack.core.ilm.RolloverAction;
+import org.elasticsearch.xpack.core.ilm.RollupILMAction;
+import org.elasticsearch.xpack.core.ilm.SearchableSnapshotAction;
+import org.elasticsearch.xpack.core.ilm.SetPriorityAction;
+import org.elasticsearch.xpack.core.ilm.ShrinkAction;
+import org.elasticsearch.xpack.core.ilm.TimeseriesLifecycleType;
+import org.elasticsearch.xpack.core.ilm.UnfollowAction;
+import org.elasticsearch.xpack.core.ilm.WaitForSnapshotAction;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class ImmutableILMStateControllerTests extends ESTestCase {
+
+    protected NamedXContentRegistry xContentRegistry() {
+        List<NamedXContentRegistry.Entry> entries = new ArrayList<>(ClusterModule.getNamedXWriteables());
+        entries.addAll(
+            Arrays.asList(
+                new NamedXContentRegistry.Entry(
+                    LifecycleType.class,
+                    new ParseField(TimeseriesLifecycleType.TYPE),
+                    (p) -> TimeseriesLifecycleType.INSTANCE
+                ),
+                new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(AllocateAction.NAME), AllocateAction::parse),
+                new NamedXContentRegistry.Entry(
+                    LifecycleAction.class,
+                    new ParseField(WaitForSnapshotAction.NAME),
+                    WaitForSnapshotAction::parse
+                ),
+                new NamedXContentRegistry.Entry(
+                    LifecycleAction.class,
+                    new ParseField(SearchableSnapshotAction.NAME),
+                    SearchableSnapshotAction::parse
+                ),
+                new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(DeleteAction.NAME), DeleteAction::parse),
+                new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(ForceMergeAction.NAME), ForceMergeAction::parse),
+                new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(ReadOnlyAction.NAME), ReadOnlyAction::parse),
+                new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(RolloverAction.NAME), RolloverAction::parse),
+                new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(ShrinkAction.NAME), ShrinkAction::parse),
+                new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(FreezeAction.NAME), FreezeAction::parse),
+                new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(SetPriorityAction.NAME), SetPriorityAction::parse),
+                new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(MigrateAction.NAME), MigrateAction::parse),
+                new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(UnfollowAction.NAME), UnfollowAction::parse),
+                new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(RollupILMAction.NAME), RollupILMAction::parse)
+            )
+        );
+        return new NamedXContentRegistry(entries);
+    }
+
+    private TransformState processJSON(ImmutableLifecycleAction action, TransformState prevState, String json) throws Exception {
+        try (XContentParser parser = XContentType.JSON.xContent().createParser(XContentParserConfiguration.EMPTY, json)) {
+            return action.transform(parser.map(), prevState);
+        }
+    }
+
+    public void testValidationFails() {
+        Client client = mock(Client.class);
+        when(client.settings()).thenReturn(Settings.EMPTY);
+        final ClusterName clusterName = new ClusterName("elasticsearch");
+
+        ClusterState state = ClusterState.builder(clusterName).build();
+        ImmutableLifecycleAction action = new ImmutableLifecycleAction(xContentRegistry(), client, mock(XPackLicenseState.class));
+        TransformState prevState = new TransformState(state, Collections.emptySet());
+
+        String badPolicyJSON = """
+            {
+                "my_timeseries_lifecycle": {
+                    "phase": {
+                        "warm": {
+                            "min_age": "10s",
+                            "actions": {
+                            }
+                        }
+                    }
+                }
+            }""";
+
+        assertEquals(
+            "[1:2] [lifecycle_policy] unknown field [phase] did you mean [phases]?",
+            expectThrows(XContentParseException.class, () -> processJSON(action, prevState, badPolicyJSON)).getMessage()
+        );
+    }
+
+    public void testActionAddRemove() throws Exception {
+        Client client = mock(Client.class);
+        when(client.settings()).thenReturn(Settings.EMPTY);
+        final ClusterName clusterName = new ClusterName("elasticsearch");
+
+        ClusterState state = ClusterState.builder(clusterName).build();
+
+        ImmutableLifecycleAction action = new ImmutableLifecycleAction(xContentRegistry(), client, mock(XPackLicenseState.class));
+
+        String emptyJSON = "";
+
+        TransformState prevState = new TransformState(state, Collections.emptySet());
+
+        TransformState updatedState = processJSON(action, prevState, emptyJSON);
+        assertEquals(0, updatedState.keys().size());
+        assertEquals(prevState.state(), updatedState.state());
+
+        String twoPoliciesJSON = """
+            {
+                "my_timeseries_lifecycle": {
+                    "phases": {
+                        "warm": {
+                            "min_age": "10s",
+                            "actions": {
+                            }
+                        }
+                    }
+                },
+                "my_timeseries_lifecycle1": {
+                    "phases": {
+                        "warm": {
+                            "min_age": "10s",
+                            "actions": {
+                            }
+                        },
+                        "delete": {
+                            "min_age": "30s",
+                            "actions": {
+                            }
+                        }
+                    }
+                }
+            }""";
+
+        prevState = updatedState;
+        updatedState = processJSON(action, prevState, twoPoliciesJSON);
+        assertThat(updatedState.keys(), containsInAnyOrder("my_timeseries_lifecycle", "my_timeseries_lifecycle1"));
+        IndexLifecycleMetadata ilmMetadata = updatedState.state()
+            .metadata()
+            .custom(IndexLifecycleMetadata.TYPE, IndexLifecycleMetadata.EMPTY);
+        assertThat(ilmMetadata.getPolicyMetadatas().keySet(), containsInAnyOrder("my_timeseries_lifecycle", "my_timeseries_lifecycle1"));
+
+        String onePolicyRemovedJSON = """
+            {
+                "my_timeseries_lifecycle": {
+                    "phases": {
+                        "warm": {
+                            "min_age": "10s",
+                            "actions": {
+                            }
+                        }
+                    }
+                }
+            }""";
+
+        prevState = updatedState;
+        updatedState = processJSON(action, prevState, onePolicyRemovedJSON);
+        assertThat(updatedState.keys(), containsInAnyOrder("my_timeseries_lifecycle"));
+        ilmMetadata = updatedState.state().metadata().custom(IndexLifecycleMetadata.TYPE, IndexLifecycleMetadata.EMPTY);
+        assertThat(ilmMetadata.getPolicyMetadatas().keySet(), containsInAnyOrder("my_timeseries_lifecycle"));
+
+        String onePolicyRenamedJSON = """
+            {
+                "my_timeseries_lifecycle2": {
+                    "phases": {
+                        "warm": {
+                            "min_age": "10s",
+                            "actions": {
+                            }
+                        }
+                    }
+                }
+            }""";
+
+        prevState = updatedState;
+        updatedState = processJSON(action, prevState, onePolicyRenamedJSON);
+        assertThat(updatedState.keys(), containsInAnyOrder("my_timeseries_lifecycle2"));
+        ilmMetadata = updatedState.state().metadata().custom(IndexLifecycleMetadata.TYPE, IndexLifecycleMetadata.EMPTY);
+        assertThat(ilmMetadata.getPolicyMetadatas().keySet(), containsInAnyOrder("my_timeseries_lifecycle2"));
+    }
+}

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

@@ -0,0 +1,35 @@
+/*
+ * 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.action;
+
+import org.elasticsearch.action.support.ActionFilters;
+import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
+import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.test.ESTestCase;
+import org.elasticsearch.threadpool.ThreadPool;
+import org.elasticsearch.transport.TransportService;
+import org.elasticsearch.xpack.core.ilm.action.DeleteLifecycleAction;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.mockito.Mockito.mock;
+
+public class TransportDeleteLifecycleActionTests extends ESTestCase {
+    public void testImmutableHandler() {
+        TransportDeleteLifecycleAction putAction = new TransportDeleteLifecycleAction(
+            mock(TransportService.class),
+            mock(ClusterService.class),
+            mock(ThreadPool.class),
+            mock(ActionFilters.class),
+            mock(IndexNameExpressionResolver.class)
+        );
+        assertEquals(ImmutableLifecycleAction.NAME, putAction.immutableStateHandlerName().get());
+
+        DeleteLifecycleAction.Request request = new DeleteLifecycleAction.Request("my_timeseries_lifecycle2");
+        assertThat(putAction.modifiedKeys(request), containsInAnyOrder("my_timeseries_lifecycle2"));
+    }
+}

+ 48 - 0
x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/action/TransportPutLifecycleActionTests.java

@@ -7,13 +7,28 @@
 
 package org.elasticsearch.xpack.ilm.action;
 
+import org.elasticsearch.action.support.ActionFilters;
+import org.elasticsearch.client.internal.Client;
+import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
+import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.license.XPackLicenseState;
 import org.elasticsearch.test.ESTestCase;
+import org.elasticsearch.threadpool.ThreadPool;
+import org.elasticsearch.transport.TransportService;
+import org.elasticsearch.xcontent.NamedXContentRegistry;
+import org.elasticsearch.xcontent.XContentParser;
+import org.elasticsearch.xcontent.XContentParserConfiguration;
+import org.elasticsearch.xcontent.XContentType;
 import org.elasticsearch.xpack.core.ilm.LifecyclePolicy;
 import org.elasticsearch.xpack.core.ilm.LifecyclePolicyMetadata;
+import org.elasticsearch.xpack.core.ilm.action.PutLifecycleAction;
 import org.elasticsearch.xpack.ilm.LifecyclePolicyTestsUtils;
 
 import java.util.Map;
 
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.mockito.Mockito.mock;
+
 public class TransportPutLifecycleActionTests extends ESTestCase {
     public void testIsNoop() {
         LifecyclePolicy policy1 = LifecyclePolicyTestsUtils.randomTimeseriesLifecyclePolicy("policy");
@@ -29,4 +44,37 @@ public class TransportPutLifecycleActionTests extends ESTestCase {
         assertFalse(TransportPutLifecycleAction.isNoopUpdate(existing, policy1, headers2));
         assertFalse(TransportPutLifecycleAction.isNoopUpdate(null, policy1, headers1));
     }
+
+    public void testImmutableStateHandler() throws Exception {
+        TransportPutLifecycleAction putAction = new TransportPutLifecycleAction(
+            mock(TransportService.class),
+            mock(ClusterService.class),
+            mock(ThreadPool.class),
+            mock(ActionFilters.class),
+            mock(IndexNameExpressionResolver.class),
+            mock(NamedXContentRegistry.class),
+            mock(XPackLicenseState.class),
+            mock(Client.class)
+        );
+        assertEquals(ImmutableLifecycleAction.NAME, putAction.immutableStateHandlerName().get());
+
+        String json = """
+            {
+                "policy": {
+                    "phases": {
+                        "warm": {
+                            "min_age": "10s",
+                            "actions": {
+                            }
+                        }
+                    }
+                }
+            }""";
+
+        try (XContentParser parser = XContentType.JSON.xContent().createParser(XContentParserConfiguration.EMPTY, json)) {
+            PutLifecycleAction.Request request = PutLifecycleAction.Request.parseRequest("my_timeseries_lifecycle2", parser);
+
+            assertThat(putAction.modifiedKeys(request), containsInAnyOrder("my_timeseries_lifecycle2"));
+        }
+    }
 }