Răsfoiți Sursa

Make the ILM and SLM history_index_enabled settings dynamic (#86493)

Joe Gallo 3 ani în urmă
părinte
comite
6aaf0972a3

+ 5 - 0
docs/changelog/86493.yaml

@@ -0,0 +1,5 @@
+pr: 86493
+summary: Make the ILM and SLM `history_index_enabled` settings dynamic
+area: ILM+SLM
+type: enhancement
+issues: []

+ 1 - 1
docs/reference/settings/ilm-settings.asciidoc

@@ -16,7 +16,7 @@ deprecated:[7.8.0,Basic License features are always enabled] +
 This deprecated setting has no effect and will be removed in Elasticsearch 8.0.
 
 `indices.lifecycle.history_index_enabled`::
-(<<static-cluster-setting,Static>>, Boolean)
+(<<dynamic-cluster-setting,Dynamic>>, Boolean)
 Whether ILM's history index is enabled. If enabled, ILM will record the
 history of actions taken as part of ILM policies to the `ilm-history-*`
 indices. Defaults to `true`.

+ 1 - 1
docs/reference/settings/snapshot-settings.asciidoc

@@ -20,7 +20,7 @@ The following cluster settings configure <<automate-snapshots-slm,{slm}
 
 [[slm-history-index-enabled]]
 `slm.history_index_enabled`::
-(<<static-cluster-setting,Static>>, Boolean)
+(<<dynamic-cluster-setting,Dynamic>>, Boolean)
 Controls whether {slm-init} records the history of actions taken as part of {slm-init} policies
 to the `slm-history-*` indices. Defaults to `true`.
 

+ 15 - 5
test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java

@@ -100,6 +100,14 @@ public class ClusterServiceUtils {
     }
 
     public static ClusterService createClusterService(ThreadPool threadPool) {
+        return createClusterService(threadPool, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS));
+    }
+
+    public static ClusterService createClusterService(ThreadPool threadPool, DiscoveryNode localNode) {
+        return createClusterService(threadPool, localNode, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS));
+    }
+
+    public static ClusterService createClusterService(ThreadPool threadPool, ClusterSettings clusterSettings) {
         DiscoveryNode discoveryNode = new DiscoveryNode(
             "node",
             "node",
@@ -108,11 +116,7 @@ public class ClusterServiceUtils {
             DiscoveryNodeRole.roles(),
             Version.CURRENT
         );
-        return createClusterService(threadPool, discoveryNode);
-    }
-
-    public static ClusterService createClusterService(ThreadPool threadPool, DiscoveryNode localNode) {
-        return createClusterService(threadPool, localNode, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS));
+        return createClusterService(threadPool, discoveryNode, clusterSettings);
     }
 
     public static ClusterService createClusterService(ThreadPool threadPool, DiscoveryNode localNode, ClusterSettings clusterSettings) {
@@ -162,6 +166,12 @@ public class ClusterServiceUtils {
         return clusterService;
     }
 
+    public static ClusterService createClusterService(ClusterState initialState, ThreadPool threadPool, ClusterSettings clusterSettings) {
+        ClusterService clusterService = createClusterService(threadPool, clusterSettings);
+        setState(clusterService, initialState);
+        return clusterService;
+    }
+
     public static void setState(ClusterService clusterService, ClusterState.Builder clusterStateBuilder) {
         setState(clusterService, clusterStateBuilder.build());
     }

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

@@ -68,6 +68,7 @@ public class LifecycleSettings {
     public static final Setting<Boolean> LIFECYCLE_HISTORY_INDEX_ENABLED_SETTING = Setting.boolSetting(
         LIFECYCLE_HISTORY_INDEX_ENABLED,
         true,
+        Setting.Property.Dynamic,
         Setting.Property.NodeScope
     );
     public static final Setting<TimeValue> LIFECYCLE_STEP_MASTER_TIMEOUT_SETTING = Setting.positiveTimeSetting(
@@ -91,6 +92,7 @@ public class LifecycleSettings {
     public static final Setting<Boolean> SLM_HISTORY_INDEX_ENABLED_SETTING = Setting.boolSetting(
         SLM_HISTORY_INDEX_ENABLED,
         true,
+        Setting.Property.Dynamic,
         Setting.Property.NodeScope
     );
     public static final Setting<String> SLM_RETENTION_SCHEDULE_SETTING = Setting.simpleString(

+ 2 - 6
x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycle.java

@@ -215,9 +215,7 @@ public class IndexLifecycle extends Plugin implements ActionPlugin, HealthPlugin
             xContentRegistry
         );
         ilmTemplateRegistry.initialize();
-        ilmHistoryStore.set(
-            new ILMHistoryStore(settings, new OriginSettingClient(client, INDEX_LIFECYCLE_ORIGIN), clusterService, threadPool)
-        );
+        ilmHistoryStore.set(new ILMHistoryStore(new OriginSettingClient(client, INDEX_LIFECYCLE_ORIGIN), clusterService, threadPool));
         /*
          * Here we use threadPool::absoluteTimeInMillis rather than System::currentTimeInMillis because snapshot start time is set using
          * ThreadPool.absoluteTimeInMillis(). ThreadPool.absoluteTimeInMillis() returns a cached time that can be several hundred
@@ -248,9 +246,7 @@ public class IndexLifecycle extends Plugin implements ActionPlugin, HealthPlugin
             xContentRegistry
         );
         templateRegistry.initialize();
-        snapshotHistoryStore.set(
-            new SnapshotHistoryStore(settings, new OriginSettingClient(client, INDEX_LIFECYCLE_ORIGIN), clusterService)
-        );
+        snapshotHistoryStore.set(new SnapshotHistoryStore(new OriginSettingClient(client, INDEX_LIFECYCLE_ORIGIN), clusterService));
         snapshotLifecycleService.set(
             new SnapshotLifecycleService(
                 settings,

+ 9 - 4
x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/history/ILMHistoryStore.java

@@ -21,7 +21,6 @@ import org.elasticsearch.action.index.IndexRequest;
 import org.elasticsearch.client.internal.Client;
 import org.elasticsearch.client.internal.OriginSettingClient;
 import org.elasticsearch.cluster.service.ClusterService;
-import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.ByteSizeValue;
 import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.threadpool.ThreadPool;
@@ -59,12 +58,14 @@ public class ILMHistoryStore implements Closeable {
         ).getBytes()
     );
 
-    private final boolean ilmHistoryEnabled;
+    private volatile boolean ilmHistoryEnabled = true;
     private final BulkProcessor processor;
     private final ThreadPool threadPool;
 
-    public ILMHistoryStore(Settings nodeSettings, Client client, ClusterService clusterService, ThreadPool threadPool) {
-        this.ilmHistoryEnabled = LIFECYCLE_HISTORY_INDEX_ENABLED_SETTING.get(nodeSettings);
+    public ILMHistoryStore(Client client, ClusterService clusterService, ThreadPool threadPool) {
+        this.setIlmHistoryEnabled(LIFECYCLE_HISTORY_INDEX_ENABLED_SETTING.get(clusterService.getSettings()));
+        clusterService.getClusterSettings().addSettingsUpdateConsumer(LIFECYCLE_HISTORY_INDEX_ENABLED_SETTING, this::setIlmHistoryEnabled);
+
         this.threadPool = threadPool;
 
         this.processor = BulkProcessor.builder(new OriginSettingClient(client, INDEX_LIFECYCLE_ORIGIN)::bulk, new BulkProcessor.Listener() {
@@ -186,4 +187,8 @@ public class ILMHistoryStore implements Closeable {
             logger.warn("failed to shut down ILM history bulk processor after 10 seconds", e);
         }
     }
+
+    public void setIlmHistoryEnabled(boolean ilmHistoryEnabled) {
+        this.ilmHistoryEnabled = ilmHistoryEnabled;
+    }
 }

+ 8 - 4
x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/slm/history/SnapshotHistoryStore.java

@@ -16,7 +16,6 @@ import org.elasticsearch.action.index.IndexRequest;
 import org.elasticsearch.client.internal.Client;
 import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.cluster.service.ClusterService;
-import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.xcontent.ToXContent;
 import org.elasticsearch.xcontent.XContentBuilder;
 import org.elasticsearch.xcontent.XContentFactory;
@@ -38,12 +37,13 @@ public class SnapshotHistoryStore {
 
     private final Client client;
     private final ClusterService clusterService;
-    private final boolean slmHistoryEnabled;
+    private volatile boolean slmHistoryEnabled = true;
 
-    public SnapshotHistoryStore(Settings nodeSettings, Client client, ClusterService clusterService) {
+    public SnapshotHistoryStore(Client client, ClusterService clusterService) {
         this.client = client;
         this.clusterService = clusterService;
-        slmHistoryEnabled = SLM_HISTORY_INDEX_ENABLED_SETTING.get(nodeSettings);
+        this.setSlmHistoryEnabled(SLM_HISTORY_INDEX_ENABLED_SETTING.get(clusterService.getSettings()));
+        clusterService.getClusterSettings().addSettingsUpdateConsumer(SLM_HISTORY_INDEX_ENABLED_SETTING, this::setSlmHistoryEnabled);
     }
 
     /**
@@ -100,4 +100,8 @@ public class SnapshotHistoryStore {
             );
         }
     }
+
+    public void setSlmHistoryEnabled(boolean slmHistoryEnabled) {
+        this.slmHistoryEnabled = slmHistoryEnabled;
+    }
 }

+ 11 - 3
x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/IndexLifecycleRunnerTests.java

@@ -22,7 +22,9 @@ import org.elasticsearch.cluster.node.DiscoveryNodes;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.bytes.BytesReference;
+import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.util.set.Sets;
 import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.test.ClusterServiceUtils;
@@ -74,6 +76,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.concurrent.CopyOnWriteArrayList;
@@ -86,6 +89,7 @@ import java.util.function.BiFunction;
 import static java.util.stream.Collectors.toList;
 import static org.elasticsearch.cluster.metadata.LifecycleExecutionState.ILM_CUSTOM_METADATA_KEY;
 import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.awaitLatch;
+import static org.elasticsearch.xpack.core.ilm.LifecycleSettings.LIFECYCLE_HISTORY_INDEX_ENABLED_SETTING;
 import static org.elasticsearch.xpack.ilm.LifecyclePolicyTestsUtils.newTestLifecyclePolicy;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.equalTo;
@@ -114,7 +118,11 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
     public void prepare() {
         threadPool = new TestThreadPool("test");
         noopClient = new NoOpClient(threadPool);
-        historyStore = new NoOpHistoryStore(noopClient);
+        ClusterSettings settings = new ClusterSettings(
+            Settings.EMPTY,
+            Sets.union(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS, Set.of(LIFECYCLE_HISTORY_INDEX_ENABLED_SETTING))
+        );
+        historyStore = new NoOpHistoryStore(noopClient, ClusterServiceUtils.createClusterService(threadPool, settings));
     }
 
     @After
@@ -1259,8 +1267,8 @@ public class IndexLifecycleRunnerTests extends ESTestCase {
 
         private final List<ILMHistoryItem> items = new CopyOnWriteArrayList<>();
 
-        NoOpHistoryStore(Client noopClient) {
-            super(Settings.EMPTY, noopClient, null, null);
+        NoOpHistoryStore(Client noopClient, ClusterService clusterService) {
+            super(noopClient, clusterService, null);
         }
 
         public List<ILMHistoryItem> getItems() {

+ 26 - 17
x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/history/ILMHistoryStoreTests.java

@@ -26,7 +26,9 @@ import org.elasticsearch.cluster.metadata.LifecycleExecutionState;
 import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.TriFunction;
+import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.util.set.Sets;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.test.ClusterServiceUtils;
 import org.elasticsearch.test.ESTestCase;
@@ -38,6 +40,7 @@ import org.hamcrest.Matchers;
 import org.junit.After;
 import org.junit.Before;
 
+import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -59,7 +62,11 @@ public class ILMHistoryStoreTests extends ESTestCase {
     public void setup() {
         threadPool = new TestThreadPool(this.getClass().getName());
         client = new VerifyingClient(threadPool);
-        clusterService = ClusterServiceUtils.createClusterService(threadPool);
+        ClusterSettings settings = new ClusterSettings(
+            Settings.EMPTY,
+            Sets.union(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS, Set.of(LIFECYCLE_HISTORY_INDEX_ENABLED_SETTING))
+        );
+        clusterService = ClusterServiceUtils.createClusterService(threadPool, settings);
         ILMHistoryTemplateRegistry registry = new ILMHistoryTemplateRegistry(
             clusterService.getSettings(),
             clusterService,
@@ -74,7 +81,7 @@ public class ILMHistoryStoreTests extends ESTestCase {
                 .metadata(Metadata.builder(state.metadata()).indexTemplates(registry.getComposableTemplateConfigs()))
                 .build()
         );
-        historyStore = new ILMHistoryStore(Settings.EMPTY, client, clusterService, threadPool);
+        historyStore = new ILMHistoryStore(client, clusterService, threadPool);
     }
 
     @After
@@ -86,21 +93,23 @@ public class ILMHistoryStoreTests extends ESTestCase {
     }
 
     public void testNoActionIfDisabled() throws Exception {
-        Settings settings = Settings.builder().put(LIFECYCLE_HISTORY_INDEX_ENABLED_SETTING.getKey(), false).build();
-        try (ILMHistoryStore disabledHistoryStore = new ILMHistoryStore(settings, client, null, threadPool)) {
-            String policyId = randomAlphaOfLength(5);
-            final long timestamp = randomNonNegativeLong();
-            ILMHistoryItem record = ILMHistoryItem.success("index", policyId, timestamp, null, null);
-
-            CountDownLatch latch = new CountDownLatch(1);
-            client.setVerifier((a, r, l) -> {
-                fail("the history store is disabled, no action should have been taken");
-                latch.countDown();
-                return null;
-            });
-            disabledHistoryStore.putAsync(record);
-            latch.await(10, TimeUnit.SECONDS);
-        }
+        ClusterState state = clusterService.state();
+        Metadata.Builder metadata = Metadata.builder(state.metadata())
+            .persistentSettings(Settings.builder().put(LIFECYCLE_HISTORY_INDEX_ENABLED_SETTING.getKey(), false).build());
+        ClusterServiceUtils.setState(clusterService, ClusterState.builder(state).metadata(metadata));
+
+        String policyId = randomAlphaOfLength(5);
+        final long timestamp = randomNonNegativeLong();
+        ILMHistoryItem record = ILMHistoryItem.success("index", policyId, timestamp, null, null);
+
+        CountDownLatch latch = new CountDownLatch(1);
+        client.setVerifier((a, r, l) -> {
+            fail("the history store is disabled, no action should have been taken");
+            latch.countDown();
+            return null;
+        });
+        historyStore.putAsync(record);
+        latch.await(10, TimeUnit.SECONDS);
     }
 
     public void testPut() throws Exception {

+ 25 - 11
x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleTaskTests.java

@@ -22,7 +22,9 @@ import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.TriFunction;
+import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.util.set.Sets;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.snapshots.Snapshot;
 import org.elasticsearch.snapshots.SnapshotId;
@@ -44,16 +46,16 @@ import org.elasticsearch.xpack.slm.history.SnapshotHistoryItem;
 import org.elasticsearch.xpack.slm.history.SnapshotHistoryStore;
 
 import java.io.IOException;
-import java.time.ZoneId;
-import java.time.ZoneOffset;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Optional;
+import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.function.Consumer;
 
+import static org.elasticsearch.xpack.core.ilm.LifecycleSettings.SLM_HISTORY_INDEX_ENABLED_SETTING;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.instanceOf;
@@ -99,8 +101,12 @@ public class SnapshotLifecycleTaskTests extends ESTestCase {
             .build();
 
         final ThreadPool threadPool = new TestThreadPool("test");
+        ClusterSettings settings = new ClusterSettings(
+            Settings.EMPTY,
+            Sets.union(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS, Set.of(SLM_HISTORY_INDEX_ENABLED_SETTING))
+        );
         try (
-            ClusterService clusterService = ClusterServiceUtils.createClusterService(state, threadPool);
+            ClusterService clusterService = ClusterServiceUtils.createClusterService(state, threadPool, settings);
             VerifyingClient client = new VerifyingClient(threadPool, (a, r, l) -> {
                 fail("should not have tried to take a snapshot");
                 return null;
@@ -108,7 +114,7 @@ public class SnapshotLifecycleTaskTests extends ESTestCase {
         ) {
             SnapshotHistoryStore historyStore = new VerifyingHistoryStore(
                 null,
-                ZoneOffset.UTC,
+                clusterService,
                 item -> fail("should not have tried to store an item")
             );
 
@@ -136,6 +142,10 @@ public class SnapshotLifecycleTaskTests extends ESTestCase {
             .build();
 
         final ThreadPool threadPool = new TestThreadPool("test");
+        ClusterSettings settings = new ClusterSettings(
+            Settings.EMPTY,
+            Sets.union(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS, Set.of(SLM_HISTORY_INDEX_ENABLED_SETTING))
+        );
         final String createSnapResponse = """
             {
                 "snapshot": {
@@ -163,7 +173,7 @@ public class SnapshotLifecycleTaskTests extends ESTestCase {
         final AtomicBoolean clientCalled = new AtomicBoolean(false);
         final SetOnce<String> snapshotName = new SetOnce<>();
         try (
-            ClusterService clusterService = ClusterServiceUtils.createClusterService(state, threadPool);
+            ClusterService clusterService = ClusterServiceUtils.createClusterService(state, threadPool, settings);
             // This verifying client will verify that we correctly invoked
             // client.admin().createSnapshot(...) with the appropriate
             // request. It also returns a mock real response
@@ -194,7 +204,7 @@ public class SnapshotLifecycleTaskTests extends ESTestCase {
             })
         ) {
             final AtomicBoolean historyStoreCalled = new AtomicBoolean(false);
-            SnapshotHistoryStore historyStore = new VerifyingHistoryStore(null, ZoneOffset.UTC, item -> {
+            SnapshotHistoryStore historyStore = new VerifyingHistoryStore(null, clusterService, item -> {
                 assertFalse(historyStoreCalled.getAndSet(true));
                 final SnapshotLifecyclePolicy policy = slpm.getPolicy();
                 assertEquals(policy.getId(), item.getPolicyId());
@@ -230,10 +240,14 @@ public class SnapshotLifecycleTaskTests extends ESTestCase {
             .build();
 
         final ThreadPool threadPool = new TestThreadPool("test");
+        ClusterSettings settings = new ClusterSettings(
+            Settings.EMPTY,
+            Sets.union(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS, Set.of(SLM_HISTORY_INDEX_ENABLED_SETTING))
+        );
         final AtomicBoolean clientCalled = new AtomicBoolean(false);
         final SetOnce<String> snapshotName = new SetOnce<>();
         try (
-            ClusterService clusterService = ClusterServiceUtils.createClusterService(state, threadPool);
+            ClusterService clusterService = ClusterServiceUtils.createClusterService(state, threadPool, settings);
             VerifyingClient client = new VerifyingClient(threadPool, (action, request, listener) -> {
                 assertFalse(clientCalled.getAndSet(true));
                 assertThat(action, instanceOf(CreateSnapshotAction.class));
@@ -273,7 +287,7 @@ public class SnapshotLifecycleTaskTests extends ESTestCase {
             })
         ) {
             final AtomicBoolean historyStoreCalled = new AtomicBoolean(false);
-            SnapshotHistoryStore historyStore = new VerifyingHistoryStore(null, ZoneOffset.UTC, item -> {
+            SnapshotHistoryStore historyStore = new VerifyingHistoryStore(null, clusterService, item -> {
                 assertFalse(historyStoreCalled.getAndSet(true));
                 final SnapshotLifecyclePolicy policy = slpm.getPolicy();
                 assertEquals(policy.getId(), item.getPolicyId());
@@ -332,10 +346,10 @@ public class SnapshotLifecycleTaskTests extends ESTestCase {
 
     public static class VerifyingHistoryStore extends SnapshotHistoryStore {
 
-        Consumer<SnapshotHistoryItem> verifier;
+        private final Consumer<SnapshotHistoryItem> verifier;
 
-        public VerifyingHistoryStore(Client client, ZoneId timeZone, Consumer<SnapshotHistoryItem> verifier) {
-            super(Settings.EMPTY, client, null);
+        public VerifyingHistoryStore(Client client, ClusterService clusterService, Consumer<SnapshotHistoryItem> verifier) {
+            super(client, clusterService);
             this.verifier = verifier;
         }
 

+ 2 - 20
x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionServiceTests.java

@@ -7,10 +7,7 @@
 
 package org.elasticsearch.xpack.slm;
 
-import org.elasticsearch.Version;
 import org.elasticsearch.client.internal.Client;
-import org.elasticsearch.cluster.node.DiscoveryNode;
-import org.elasticsearch.cluster.node.DiscoveryNodeRole;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Setting;
@@ -24,7 +21,6 @@ import org.elasticsearch.xpack.core.scheduler.SchedulerEngine;
 import org.elasticsearch.xpack.core.watcher.watch.ClockMock;
 import org.elasticsearch.xpack.slm.history.SnapshotHistoryStore;
 
-import java.util.Collections;
 import java.util.HashSet;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
@@ -45,18 +41,11 @@ public class SnapshotRetentionServiceTests extends ESTestCase {
     }
 
     public void testJobsAreScheduled() throws InterruptedException {
-        final DiscoveryNode discoveryNode = new DiscoveryNode(
-            "node",
-            ESTestCase.buildNewFakeTransportAddress(),
-            Collections.emptyMap(),
-            DiscoveryNodeRole.roles(),
-            Version.CURRENT
-        );
         ClockMock clock = new ClockMock();
 
         ThreadPool threadPool = new TestThreadPool("test");
         try (
-            ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool, discoveryNode, clusterSettings);
+            ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool, clusterSettings);
             SnapshotRetentionService service = new SnapshotRetentionService(Settings.EMPTY, FakeRetentionTask::new, clock)
         ) {
             service.init(clusterService);
@@ -85,19 +74,12 @@ public class SnapshotRetentionServiceTests extends ESTestCase {
     }
 
     public void testManualTriggering() throws InterruptedException {
-        final DiscoveryNode discoveryNode = new DiscoveryNode(
-            "node",
-            ESTestCase.buildNewFakeTransportAddress(),
-            Collections.emptyMap(),
-            DiscoveryNodeRole.roles(),
-            Version.CURRENT
-        );
         ClockMock clock = new ClockMock();
         AtomicInteger invoked = new AtomicInteger(0);
 
         ThreadPool threadPool = new TestThreadPool("test");
         try (
-            ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool, discoveryNode, clusterSettings);
+            ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool, clusterSettings);
             SnapshotRetentionService service = new SnapshotRetentionService(Settings.EMPTY, () -> new FakeRetentionTask(event -> {
                 assertThat(event.getJobName(), equalTo(SnapshotRetentionService.SLM_RETENTION_MANUAL_JOB_ID));
                 invoked.incrementAndGet();

+ 34 - 11
x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionTaskTests.java

@@ -21,6 +21,9 @@ import org.elasticsearch.cluster.ClusterName;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.common.settings.ClusterSettings;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.util.set.Sets;
 import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.snapshots.Snapshot;
 import org.elasticsearch.snapshots.SnapshotId;
@@ -39,7 +42,6 @@ import org.elasticsearch.xpack.core.slm.SnapshotLifecycleStats;
 import org.elasticsearch.xpack.core.slm.SnapshotRetentionConfiguration;
 import org.elasticsearch.xpack.slm.history.SnapshotHistoryStore;
 
-import java.time.ZoneOffset;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -58,6 +60,7 @@ import java.util.function.LongSupplier;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
 
+import static org.elasticsearch.xpack.core.ilm.LifecycleSettings.SLM_HISTORY_INDEX_ENABLED_SETTING;
 import static org.elasticsearch.xpack.slm.history.SnapshotHistoryItem.DELETE_OPERATION;
 import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.containsString;
@@ -198,8 +201,12 @@ public class SnapshotRetentionTaskTests extends ESTestCase {
 
     private void retentionTaskTest(final boolean deletionSuccess) throws Exception {
         ThreadPool threadPool = new TestThreadPool("slm-test");
+        ClusterSettings settings = new ClusterSettings(
+            Settings.EMPTY,
+            Sets.union(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS, Set.of(SLM_HISTORY_INDEX_ENABLED_SETTING))
+        );
         try (
-            ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool);
+            ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool, settings);
             Client noOpClient = new NoOpClient("slm-test")
         ) {
 
@@ -254,7 +261,7 @@ public class SnapshotRetentionTaskTests extends ESTestCase {
             MockSnapshotRetentionTask retentionTask = new MockSnapshotRetentionTask(
                 noOpClient,
                 clusterService,
-                new SnapshotLifecycleTaskTests.VerifyingHistoryStore(noOpClient, ZoneOffset.UTC, (historyItem) -> {
+                new SnapshotLifecycleTaskTests.VerifyingHistoryStore(noOpClient, clusterService, (historyItem) -> {
                     assertEquals(deletionSuccess, historyItem.isSuccess());
                     if (historyItem.isSuccess() == false) {
                         assertThat(historyItem.getErrorDetails(), containsString("deletion_failed"));
@@ -305,10 +312,14 @@ public class SnapshotRetentionTaskTests extends ESTestCase {
 
     public void testErrStillRunsFailureHandlerWhenRetrieving() throws Exception {
         ThreadPool threadPool = new TestThreadPool("slm-test");
+        ClusterSettings settings = new ClusterSettings(
+            Settings.EMPTY,
+            Sets.union(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS, Set.of(SLM_HISTORY_INDEX_ENABLED_SETTING))
+        );
         final String policyId = "policy";
         final String repoId = "repo";
         try (
-            ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool);
+            ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool, settings);
             Client noOpClient = new NoOpClient("slm-test") {
 
                 @Override
@@ -347,7 +358,7 @@ public class SnapshotRetentionTaskTests extends ESTestCase {
                 System::nanoTime,
                 new SnapshotLifecycleTaskTests.VerifyingHistoryStore(
                     noOpClient,
-                    ZoneOffset.UTC,
+                    clusterService,
                     (historyItem) -> fail("should never write history")
                 )
             );
@@ -376,8 +387,12 @@ public class SnapshotRetentionTaskTests extends ESTestCase {
 
     public void testErrStillRunsFailureHandlerWhenDeleting() throws Exception {
         ThreadPool threadPool = new TestThreadPool("slm-test");
+        ClusterSettings settings = new ClusterSettings(
+            Settings.EMPTY,
+            Sets.union(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS, Set.of(SLM_HISTORY_INDEX_ENABLED_SETTING))
+        );
         try (
-            ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool);
+            ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool, settings);
             Client noOpClient = new NoOpClient("slm-test") {
 
                 @Override
@@ -416,7 +431,7 @@ public class SnapshotRetentionTaskTests extends ESTestCase {
                 System::nanoTime,
                 new SnapshotLifecycleTaskTests.VerifyingHistoryStore(
                     noOpClient,
-                    ZoneOffset.UTC,
+                    clusterService,
                     (historyItem) -> fail("should never write history")
                 )
             );
@@ -458,8 +473,12 @@ public class SnapshotRetentionTaskTests extends ESTestCase {
 
     private void doTestSkipDuringMode(OperationMode mode) throws Exception {
         ThreadPool threadPool = new TestThreadPool("slm-test");
+        ClusterSettings settings = new ClusterSettings(
+            Settings.EMPTY,
+            Sets.union(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS, Set.of(SLM_HISTORY_INDEX_ENABLED_SETTING))
+        );
         try (
-            ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool);
+            ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool, settings);
             Client noOpClient = new NoOpClient("slm-test")
         ) {
             final String policyId = "policy";
@@ -481,7 +500,7 @@ public class SnapshotRetentionTaskTests extends ESTestCase {
                 clusterService,
                 new SnapshotLifecycleTaskTests.VerifyingHistoryStore(
                     noOpClient,
-                    ZoneOffset.UTC,
+                    clusterService,
                     (historyItem) -> fail("should never write history")
                 ),
                 () -> {
@@ -510,8 +529,12 @@ public class SnapshotRetentionTaskTests extends ESTestCase {
 
     private void doTestRunManuallyDuringMode(OperationMode mode) throws Exception {
         ThreadPool threadPool = new TestThreadPool("slm-test");
+        ClusterSettings settings = new ClusterSettings(
+            Settings.EMPTY,
+            Sets.union(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS, Set.of(SLM_HISTORY_INDEX_ENABLED_SETTING))
+        );
         try (
-            ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool);
+            ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool, settings);
             Client noOpClient = new NoOpClient("slm-test")
         ) {
             final String policyId = "policy";
@@ -532,7 +555,7 @@ public class SnapshotRetentionTaskTests extends ESTestCase {
             MockSnapshotRetentionTask task = new MockSnapshotRetentionTask(
                 noOpClient,
                 clusterService,
-                new SnapshotLifecycleTaskTests.VerifyingHistoryStore(noOpClient, ZoneOffset.UTC, (historyItem) -> {}),
+                new SnapshotLifecycleTaskTests.VerifyingHistoryStore(noOpClient, clusterService, (historyItem) -> {}),
                 () -> {
                     retentionWasRun.set(true);
                     return Collections.emptyMap();

+ 17 - 7
x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/history/SnapshotHistoryStoreTests.java

@@ -16,7 +16,9 @@ import org.elasticsearch.action.index.IndexResponse;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.util.set.Sets;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.test.ClusterServiceUtils;
 import org.elasticsearch.test.ESTestCase;
@@ -28,6 +30,7 @@ import org.junit.Before;
 
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.elasticsearch.xpack.core.ilm.GenerateSnapshotNameStep.generateSnapshotName;
@@ -48,13 +51,16 @@ public class SnapshotHistoryStoreTests extends ESTestCase {
     public void setup() {
         threadPool = new TestThreadPool(this.getClass().getName());
         client = new SnapshotLifecycleTemplateRegistryTests.VerifyingClient(threadPool);
-        clusterService = ClusterServiceUtils.createClusterService(threadPool);
+        ClusterSettings settings = new ClusterSettings(
+            Settings.EMPTY,
+            Sets.union(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS, Set.of(SLM_HISTORY_INDEX_ENABLED_SETTING))
+        );
+        clusterService = ClusterServiceUtils.createClusterService(threadPool, settings);
         ClusterState state = clusterService.state();
         Metadata.Builder metadataBuilder = Metadata.builder(state.getMetadata())
             .indexTemplates(SnapshotLifecycleTemplateRegistry.COMPOSABLE_INDEX_TEMPLATE_CONFIGS);
         ClusterServiceUtils.setState(clusterService, ClusterState.builder(state).metadata(metadataBuilder).build());
-        historyStore = new SnapshotHistoryStore(Settings.EMPTY, client, clusterService);
-        clusterService.stop();
+        historyStore = new SnapshotHistoryStore(client, clusterService);
     }
 
     @After
@@ -62,12 +68,16 @@ public class SnapshotHistoryStoreTests extends ESTestCase {
     public void tearDown() throws Exception {
         super.tearDown();
         clusterService.stop();
+        client.close();
         threadPool.shutdownNow();
     }
 
-    public void testNoActionIfDisabled() {
-        Settings settings = Settings.builder().put(SLM_HISTORY_INDEX_ENABLED_SETTING.getKey(), false).build();
-        SnapshotHistoryStore disabledHistoryStore = new SnapshotHistoryStore(settings, client, null);
+    public void testNoActionIfDisabled() throws Exception {
+        ClusterState state = clusterService.state();
+        Metadata.Builder metadata = Metadata.builder(state.metadata())
+            .persistentSettings(Settings.builder().put(SLM_HISTORY_INDEX_ENABLED_SETTING.getKey(), false).build());
+        ClusterServiceUtils.setState(clusterService, ClusterState.builder(state).metadata(metadata));
+
         String policyId = randomAlphaOfLength(5);
         SnapshotLifecyclePolicy policy = randomSnapshotLifecyclePolicy(policyId);
         final long timestamp = randomNonNegativeLong();
@@ -78,7 +88,7 @@ public class SnapshotHistoryStoreTests extends ESTestCase {
             fail("the history store is disabled, no action should have been taken");
             return null;
         });
-        disabledHistoryStore.putAsync(record);
+        historyStore.putAsync(record);
     }
 
     @SuppressWarnings("unchecked")

+ 1 - 10
x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/AbstractSearchableSnapshotsTestCase.java

@@ -16,7 +16,6 @@ import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.elasticsearch.Version;
 import org.elasticsearch.cluster.node.DiscoveryNode;
-import org.elasticsearch.cluster.node.DiscoveryNodeRole;
 import org.elasticsearch.cluster.routing.RecoverySource;
 import org.elasticsearch.cluster.routing.ShardRouting;
 import org.elasticsearch.cluster.routing.ShardRoutingState;
@@ -42,7 +41,6 @@ import org.elasticsearch.repositories.IndexId;
 import org.elasticsearch.snapshots.Snapshot;
 import org.elasticsearch.snapshots.SnapshotId;
 import org.elasticsearch.test.ClusterServiceUtils;
-import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.threadpool.TestThreadPool;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.threadpool.ThreadPoolStats;
@@ -96,15 +94,8 @@ public abstract class AbstractSearchableSnapshotsTestCase extends ESIndexInputTe
 
     @Before
     public void setUpTest() throws Exception {
-        final DiscoveryNode node = new DiscoveryNode(
-            "node",
-            ESTestCase.buildNewFakeTransportAddress(),
-            Collections.emptyMap(),
-            DiscoveryNodeRole.roles(),
-            Version.CURRENT
-        );
         threadPool = new TestThreadPool(getTestName(), SearchableSnapshots.executorBuilders(Settings.EMPTY));
-        clusterService = ClusterServiceUtils.createClusterService(threadPool, node, CLUSTER_SETTINGS);
+        clusterService = ClusterServiceUtils.createClusterService(threadPool, CLUSTER_SETTINGS);
         nodeEnvironment = newNodeEnvironment();
         singlePathNodeEnvironment = newSinglePathNodeEnvironment();
     }