Browse Source

Remove dangling index auto import functionality (#59698)

Closes #48366. Remove all traces of automatically importing dangling indices. This functionality is
deprecated from 7.9.0.
Rory Hunter 5 years ago
parent
commit
4db094c008

+ 9 - 26
docs/reference/modules/gateway.asciidoc

@@ -56,29 +56,12 @@ NOTE: These settings only take effect on a full cluster restart.
 [[dangling-indices]]
 ==== Dangling indices
 
-When a node joins the cluster, if it finds any shards stored in its local data
-directory that do not already exist in the cluster, it will consider those
-shards to be "dangling". Importing dangling indices
-into the cluster using `gateway.auto_import_dangling_indices` is not safe.
-Instead, use the <<dangling-indices-api,Dangling indices API>>. Neither
-mechanism provides any guarantees as to whether the imported data truly
-represents the latest state of the data when the index was still part of
-the cluster.
-
-`gateway.auto_import_dangling_indices`::
-
-    deprecated:[7.9.0, This setting will be removed in 8.0. You should use the dedicated dangling indices API instead.]
-    Whether to automatically import dangling indices into the cluster
-    state, provided no indices already exist with the same name. Defaults
-    to `false`.
-
-WARNING: The auto-import functionality was intended as a best effort to help users
-who lose all master nodes. For example, if a new master node were to be
-started which was unaware of the other indices in the cluster, adding the
-old nodes would cause the old indices to be imported, instead of being
-deleted. However there are several issues with automatic importing, and
-its use is strongly discouraged in favour of the
-<<dangling-indices-api,dedicated API>.
-
-WARNING: Losing all master nodes is a situation that should be avoided at
-all costs, as it puts your cluster's metadata and data at risk.
+When a node joins the cluster, if it finds any shards stored in its local
+data directory that do not already exist in the cluster, it will consider
+those shards to belong to a "dangling" index. You can list, import or
+delete dangling indices using the <<dangling-indices-api,Dangling indices
+API>>.
+
+NOTE: The API cannot offer any guarantees as to whether the imported data
+truly represents the latest state of the data when the index was still part
+of the cluster.

+ 0 - 2
qa/smoke-test-http/src/test/java/org/elasticsearch/http/DanglingIndicesRestIT.java

@@ -39,7 +39,6 @@ import java.util.Map;
 import java.util.concurrent.atomic.AtomicReference;
 
 import static org.elasticsearch.cluster.metadata.IndexGraveyard.SETTING_MAX_TOMBSTONES;
-import static org.elasticsearch.gateway.DanglingIndicesState.AUTO_IMPORT_DANGLING_INDICES_SETTING;
 import static org.elasticsearch.indices.IndicesService.WRITE_DANGLING_INDICES_INFO_SETTING;
 import static org.elasticsearch.rest.RestStatus.ACCEPTED;
 import static org.elasticsearch.rest.RestStatus.OK;
@@ -70,7 +69,6 @@ public class DanglingIndicesRestIT extends HttpSmokeTestCase {
             // when we delete an index, it's definitely considered to be dangling.
             .put(SETTING_MAX_TOMBSTONES.getKey(), maxTombstones)
             .put(WRITE_DANGLING_INDICES_INFO_SETTING.getKey(), true)
-            .put(AUTO_IMPORT_DANGLING_INDICES_SETTING.getKey(), false)
             .build();
     }
 

+ 0 - 66
server/src/internalClusterTest/java/org/elasticsearch/cluster/coordination/UnsafeBootstrapAndDetachCommandIT.java

@@ -31,7 +31,6 @@ import org.elasticsearch.env.NodeEnvironment;
 import org.elasticsearch.env.TestEnvironment;
 import org.elasticsearch.gateway.GatewayMetaState;
 import org.elasticsearch.gateway.PersistedClusterStateService;
-import org.elasticsearch.indices.IndicesService;
 import org.elasticsearch.node.Node;
 import org.elasticsearch.test.ESIntegTestCase;
 import org.elasticsearch.test.InternalTestCluster;
@@ -41,12 +40,8 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Locale;
 
-import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE;
-import static org.elasticsearch.gateway.DanglingIndicesState.AUTO_IMPORT_DANGLING_INDICES_SETTING;
-import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
 import static org.elasticsearch.indices.recovery.RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING;
 import static org.elasticsearch.test.NodeRoles.nonMasterNode;
-import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.notNullValue;
@@ -312,67 +307,6 @@ public class UnsafeBootstrapAndDetachCommandIT extends ESIntegTestCase {
         ensureStableCluster(4);
     }
 
-    public void testAllMasterEligibleNodesFailedDanglingIndexImport() throws Exception {
-        internalCluster().setBootstrapMasterNodeIndex(0);
-
-        Settings settings = Settings.builder()
-            .put(AUTO_IMPORT_DANGLING_INDICES_SETTING.getKey(), true)
-            .build();
-
-        logger.info("--> start mixed data and master-eligible node and bootstrap cluster");
-        String masterNode = internalCluster().startNode(settings); // node ordinal 0
-
-        logger.info("--> start data-only node and ensure 2 nodes stable cluster");
-        String dataNode = internalCluster().startDataOnlyNode(settings); // node ordinal 1
-        ensureStableCluster(2);
-
-        logger.info("--> index 1 doc and ensure index is green");
-        client().prepareIndex("test").setId("1").setSource("field1", "value1").setRefreshPolicy(IMMEDIATE).get();
-        ensureGreen("test");
-        assertBusy(() -> internalCluster().getInstances(IndicesService.class).forEach(
-            indicesService -> assertTrue(indicesService.allPendingDanglingIndicesWritten())));
-
-        logger.info("--> verify 1 doc in the index");
-        assertHitCount(client().prepareSearch().setQuery(matchAllQuery()).get(), 1L);
-        assertThat(client().prepareGet("test", "1").execute().actionGet().isExists(), equalTo(true));
-
-        logger.info("--> stop data-only node and detach it from the old cluster");
-        Settings dataNodeDataPathSettings = Settings.builder()
-            .put(internalCluster().dataPathSettings(dataNode), true)
-            .put(AUTO_IMPORT_DANGLING_INDICES_SETTING.getKey(), true)
-            .build();
-        assertBusy(() -> internalCluster().getInstance(GatewayMetaState.class, dataNode).allPendingAsyncStatesWritten());
-        internalCluster().stopRandomNode(InternalTestCluster.nameFilter(dataNode));
-        final Environment environment = TestEnvironment.newEnvironment(
-            Settings.builder()
-                .put(internalCluster().getDefaultSettings())
-                .put(dataNodeDataPathSettings)
-                .put(AUTO_IMPORT_DANGLING_INDICES_SETTING.getKey(), true)
-                .build());
-        detachCluster(environment, false);
-
-        logger.info("--> stop master-eligible node, clear its data and start it again - new cluster should form");
-        internalCluster().restartNode(masterNode, new InternalTestCluster.RestartCallback(){
-            @Override
-            public boolean clearData(String nodeName) {
-                return true;
-            }
-        });
-
-        logger.info("--> start data-only only node and ensure 2 nodes stable cluster");
-        internalCluster().startDataOnlyNode(dataNodeDataPathSettings);
-        ensureStableCluster(2);
-
-        logger.info("--> verify that the dangling index exists and has green status");
-        assertBusy(() -> {
-            assertThat(indexExists("test"), equalTo(true));
-        });
-        ensureGreen("test");
-
-        logger.info("--> verify the doc is there");
-        assertThat(client().prepareGet("test", "1").execute().actionGet().isExists(), equalTo(true));
-    }
-
     public void testNoInitialBootstrapAfterDetach() throws Exception {
         internalCluster().setBootstrapMasterNodeIndex(0);
         String masterNode = internalCluster().startMasterOnlyNode();

+ 13 - 172
server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/DanglingIndicesIT.java

@@ -25,7 +25,6 @@ import org.elasticsearch.action.admin.indices.dangling.import_index.ImportDangli
 import org.elasticsearch.action.admin.indices.dangling.list.ListDanglingIndicesRequest;
 import org.elasticsearch.action.admin.indices.dangling.list.ListDanglingIndicesResponse;
 import org.elasticsearch.action.admin.indices.dangling.list.NodeListDanglingIndicesResponse;
-import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.indices.IndicesService;
 import org.elasticsearch.rest.RestStatus;
@@ -39,14 +38,12 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
 import static org.elasticsearch.cluster.metadata.IndexGraveyard.SETTING_MAX_TOMBSTONES;
-import static org.elasticsearch.gateway.DanglingIndicesState.AUTO_IMPORT_DANGLING_INDICES_SETTING;
 import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.empty;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.hasSize;
 import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.notNullValue;
 
 /**
  * This class tests how dangling indices are handled, in terms of how they
@@ -63,65 +60,15 @@ public class DanglingIndicesIT extends ESIntegTestCase {
     private static final String INDEX_NAME = "test-idx-1";
     private static final String OTHER_INDEX_NAME = INDEX_NAME + "-other";
 
-    private Settings buildSettings(int maxTombstones, boolean writeDanglingIndices, boolean importDanglingIndices) {
+    private Settings buildSettings(int maxTombstones, boolean writeDanglingIndices) {
         return Settings.builder()
             // Limit the indices kept in the graveyard. This can be set to zero, so that
             // when we delete an index, it's definitely considered to be dangling.
             .put(SETTING_MAX_TOMBSTONES.getKey(), maxTombstones)
             .put(IndicesService.WRITE_DANGLING_INDICES_INFO_SETTING.getKey(), writeDanglingIndices)
-            .put(AUTO_IMPORT_DANGLING_INDICES_SETTING.getKey(), importDanglingIndices)
             .build();
     }
 
-    /**
-     * Check that when the auto-recovery setting is enabled and a dangling index is
-     * discovered, then that index is recovered into the cluster.
-     */
-    public void testDanglingIndicesAreRecoveredWhenSettingIsEnabled() throws Exception {
-        final Settings settings = buildSettings(0, true, true);
-        internalCluster().startNodes(3, settings);
-
-        createIndices(INDEX_NAME);
-        ensurePendingDanglingIndicesWritten();
-
-        boolean refreshIntervalChanged = randomBoolean();
-        if (refreshIntervalChanged) {
-            client().admin()
-                .indices()
-                .prepareUpdateSettings(INDEX_NAME)
-                .setSettings(Settings.builder().put("index.refresh_interval", "42s").build())
-                .get();
-            ensurePendingDanglingIndicesWritten();
-        }
-
-        if (randomBoolean()) {
-            client().admin().indices().prepareClose(INDEX_NAME).get();
-        }
-        ensureGreen(INDEX_NAME);
-
-        // Restart node, deleting the index in its absence, so that there is a dangling index to recover
-        internalCluster().restartRandomDataNode(new InternalTestCluster.RestartCallback() {
-
-            @Override
-            public Settings onNodeStopped(String nodeName) throws Exception {
-                internalCluster().validateClusterFormed();
-                assertAcked(client().admin().indices().prepareDelete(INDEX_NAME));
-                return super.onNodeStopped(nodeName);
-            }
-        });
-
-        assertBusy(() -> assertTrue("Expected dangling index " + INDEX_NAME + " to be recovered", indexExists(INDEX_NAME)));
-        if (refreshIntervalChanged) {
-            assertThat(
-                client().admin().indices().prepareGetSettings(INDEX_NAME).get().getSetting(INDEX_NAME, "index.refresh_interval"),
-                equalTo("42s")
-            );
-        }
-        ensureGreen(INDEX_NAME);
-        final IndexMetadata indexMetadata = clusterService().state().metadata().index(INDEX_NAME);
-        assertThat(indexMetadata.getSettings().get(IndexMetadata.SETTING_HISTORY_UUID), notNullValue());
-    }
-
     private void ensurePendingDanglingIndicesWritten() throws Exception {
         assertBusy(
             () -> internalCluster().getInstances(IndicesService.class)
@@ -130,35 +77,18 @@ public class DanglingIndicesIT extends ESIntegTestCase {
     }
 
     /**
-     * Check that when dangling indices are discovered, then they are not recovered into
-     * the cluster when the recovery setting is disabled.
+     * Check that when dangling indices are not written, then they cannot be listed by the API
      */
-    public void testDanglingIndicesAreNotRecoveredWhenSettingIsDisabled() throws Exception {
-        internalCluster().startNodes(3, buildSettings(0, true, false));
+    public void testDanglingIndicesCannotBeListedWhenNotWritten() throws Exception {
+        internalCluster().startNodes(3, buildSettings(0, false));
 
         createDanglingIndices(INDEX_NAME);
 
         // Since index recovery is async, we can't prove index recovery will never occur, just that it doesn't occur within some reasonable
         // amount of time
         assertFalse(
-            "Did not expect dangling index " + INDEX_NAME + " to be recovered",
-            waitUntil(() -> indexExists(INDEX_NAME), 1, TimeUnit.SECONDS)
-        );
-    }
-
-    /**
-     * Check that when dangling indices are not written, then they cannot be recovered into the cluster.
-     */
-    public void testDanglingIndicesAreNotRecoveredWhenNotWritten() throws Exception {
-        internalCluster().startNodes(3, buildSettings(0, false, true));
-
-        createDanglingIndices(INDEX_NAME);
-
-        // Since index recovery is async, we can't prove index recovery will never occur, just that it doesn't occur within some reasonable
-        // amount of time
-        assertFalse(
-            "Did not expect dangling index " + INDEX_NAME + " to be recovered",
-            waitUntil(() -> indexExists(INDEX_NAME), 1, TimeUnit.SECONDS)
+            "Did not expect dangling index " + INDEX_NAME + " to be listed",
+            waitUntil(() -> listDanglingIndices().isEmpty() == false, 5, TimeUnit.SECONDS)
         );
     }
 
@@ -166,7 +96,7 @@ public class DanglingIndicesIT extends ESIntegTestCase {
      * Check that when dangling indices are discovered, then they can be listed.
      */
     public void testDanglingIndicesCanBeListed() throws Exception {
-        internalCluster().startNodes(3, buildSettings(0, true, false));
+        internalCluster().startNodes(3, buildSettings(0, true));
 
         final String stoppedNodeName = createDanglingIndices(INDEX_NAME);
 
@@ -191,49 +121,11 @@ public class DanglingIndicesIT extends ESIntegTestCase {
         }
     }
 
-    /**
-     * Check that when dangling index auto imports are enabled, and a dangling index is is discovered
-     * but cannot be imported due to a name clash with an existing index, then that dangling index can
-     * still be listed through the API.
-     */
-    public void testDanglingIndicesCanBeListedWhenAutoImportEnabled() throws Exception {
-        internalCluster().startNodes(3, buildSettings(0, true, true));
-
-        createIndices(INDEX_NAME);
-        ensurePendingDanglingIndicesWritten();
-
-        // Restart node, deleting the indices in its absence, so that there is a dangling index to recover
-        internalCluster().restartRandomDataNode(new InternalTestCluster.RestartCallback() {
-
-            @Override
-            public Settings onNodeStopped(String nodeName) throws Exception {
-                internalCluster().validateClusterFormed();
-                assertAcked(client().admin().indices().prepareDelete(INDEX_NAME));
-
-                // Create another index with the same name, which will prevent the dangling
-                // index from being restored, and also ensures that we are checking index
-                // existence by UUID, not name.
-                //
-                // Note: don't call `createIndices()` here as it calls `ensureGreen()`, which will
-                // fail while a node is offline
-                createIndex(INDEX_NAME);
-                ensurePendingDanglingIndicesWritten();
-
-                return super.onNodeStopped(nodeName);
-            }
-        });
-
-        final List<DanglingIndexInfo> danglingIndices = listDanglingIndices();
-
-        assertThat(danglingIndices, hasSize(1));
-        assertThat(danglingIndices.get(0).getIndexName(), equalTo(INDEX_NAME));
-    }
-
     /**
      * Check that dangling indices can be imported.
      */
     public void testDanglingIndicesCanBeImported() throws Exception {
-        internalCluster().startNodes(3, buildSettings(0, true, false));
+        internalCluster().startNodes(3, buildSettings(0, true));
 
         final String stoppedNodeName = createDanglingIndices(INDEX_NAME);
 
@@ -250,7 +142,7 @@ public class DanglingIndicesIT extends ESIntegTestCase {
      * being dangling.
      */
     public void testDanglingIndicesMustExistToBeImported() {
-        internalCluster().startNodes(1, buildSettings(0, true, false));
+        internalCluster().startNodes(1, buildSettings(0, true));
 
         final ImportDanglingIndexRequest request = new ImportDanglingIndexRequest("NonExistentUUID", true);
 
@@ -266,7 +158,7 @@ public class DanglingIndicesIT extends ESIntegTestCase {
      * Check that a dangling index can only be imported if "accept_data_loss" is set to true.
      */
     public void testMustAcceptDataLossToImportDanglingIndex() throws Exception {
-        internalCluster().startNodes(3, buildSettings(0, true, false));
+        internalCluster().startNodes(3, buildSettings(0, true));
 
         final String stoppedNodeName = createDanglingIndices(INDEX_NAME);
         final String danglingIndexUUID = findDanglingIndexForNode(stoppedNodeName, INDEX_NAME);
@@ -290,7 +182,7 @@ public class DanglingIndicesIT extends ESIntegTestCase {
      * deleted through the API
      */
     public void testDanglingIndexCanBeDeleted() throws Exception {
-        final Settings settings = buildSettings(1, true, false);
+        final Settings settings = buildSettings(1, true);
         internalCluster().startNodes(3, settings);
 
         final String stoppedNodeName = createDanglingIndices(INDEX_NAME, OTHER_INDEX_NAME);
@@ -307,62 +199,11 @@ public class DanglingIndicesIT extends ESIntegTestCase {
         assertThat(listDanglingIndices(), is(empty()));
     }
 
-    /**
-     * Check that when dangling index auto imports are enabled, and a dangling index is is discovered
-     * but cannot be imported due to a name clash with an existing index, then that dangling index can
-     * still be deleted through the API.
-     */
-    public void testDanglingIndexCanBeDeletedWhenAutoImportEnabled() throws Exception {
-        final Settings settings = buildSettings(1, true, true);
-        internalCluster().startNodes(3, settings);
-
-        createIndices(INDEX_NAME, OTHER_INDEX_NAME);
-        ensurePendingDanglingIndicesWritten();
-
-        AtomicReference<String> stoppedNodeName = new AtomicReference<>();
-
-        // Restart node, deleting the indices in its absence, so that there is a dangling index to recover
-        internalCluster().restartRandomDataNode(new InternalTestCluster.RestartCallback() {
-
-            @Override
-            public Settings onNodeStopped(String nodeName) throws Exception {
-                internalCluster().validateClusterFormed();
-                stoppedNodeName.set(nodeName);
-
-                assertAcked(client().admin().indices().prepareDelete(INDEX_NAME));
-                assertAcked(client().admin().indices().prepareDelete(OTHER_INDEX_NAME));
-
-                // Create another index with the same name, which will prevent the dangling
-                // index from being restored, and also ensures that we are checking index
-                // existence by UUID, not name.
-                //
-                // Note: don't call `createIndices()` here as it calls `ensureGreen()`, which will
-                // fail while a node is offline
-                createIndex(INDEX_NAME);
-                ensurePendingDanglingIndicesWritten();
-
-                return super.onNodeStopped(nodeName);
-            }
-        });
-
-        final String danglingIndexUUID = findDanglingIndexForNode(stoppedNodeName.get(), INDEX_NAME);
-
-        client().admin().cluster().deleteDanglingIndex(new DeleteDanglingIndexRequest(danglingIndexUUID, true)).actionGet();
-
-        // The dangling index that we deleted ought to have been removed from disk. Check by
-        // creating and deleting another index, which creates a new tombstone entry, which should
-        // not cause the deleted dangling index to be considered "live" again, just because its
-        // tombstone has been pushed out of the graveyard.
-        createIndex("additional");
-        assertAcked(client().admin().indices().prepareDelete("additional"));
-        assertThat(listDanglingIndices(), is(empty()));
-    }
-
     /**
      * Check that when a index is found to be dangling on more than one node, it can be deleted.
      */
     public void testDanglingIndexOverMultipleNodesCanBeDeleted() throws Exception {
-        final Settings settings = buildSettings(1, true, false);
+        final Settings settings = buildSettings(1, true);
         internalCluster().startNodes(3, settings);
 
         createIndices(INDEX_NAME, OTHER_INDEX_NAME);
@@ -416,7 +257,7 @@ public class DanglingIndicesIT extends ESIntegTestCase {
      * Check that when deleting a dangling index, it is required that the "accept_data_loss" flag is set.
      */
     public void testDeleteDanglingIndicesRequiresDataLossFlagToBeTrue() throws Exception {
-        final Settings settings = buildSettings(1, true, false);
+        final Settings settings = buildSettings(1, true);
         internalCluster().startNodes(3, settings);
 
         final String stoppedNodeName = createDanglingIndices(INDEX_NAME, OTHER_INDEX_NAME);

+ 0 - 2
server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java

@@ -74,7 +74,6 @@ import org.elasticsearch.discovery.SeedHostsResolver;
 import org.elasticsearch.discovery.SettingsBasedSeedHostsProvider;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.env.NodeEnvironment;
-import org.elasticsearch.gateway.DanglingIndicesState;
 import org.elasticsearch.gateway.GatewayService;
 import org.elasticsearch.gateway.PersistedClusterStateService;
 import org.elasticsearch.http.HttpTransportSettings;
@@ -199,7 +198,6 @@ public final class ClusterSettings extends AbstractScopedSettings {
             BreakerSettings.CIRCUIT_BREAKER_TYPE,
             ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING,
             ConcurrentRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE_SETTING,
-            DanglingIndicesState.AUTO_IMPORT_DANGLING_INDICES_SETTING,
             EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING,
             EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING,
             FilterAllocationDecider.CLUSTER_ROUTING_INCLUDE_GROUP_SETTING,

+ 17 - 188
server/src/main/java/org/elasticsearch/gateway/DanglingIndicesState.java

@@ -22,21 +22,14 @@ package org.elasticsearch.gateway;
 import com.carrotsearch.hppc.cursors.ObjectCursor;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
-import org.elasticsearch.action.ActionListener;
-import org.elasticsearch.cluster.ClusterChangedEvent;
-import org.elasticsearch.cluster.ClusterStateListener;
 import org.elasticsearch.cluster.metadata.IndexGraveyard;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.inject.Inject;
-import org.elasticsearch.common.settings.Setting;
-import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
-import org.elasticsearch.env.NodeEnvironment;
 import org.elasticsearch.index.Index;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -47,234 +40,70 @@ import static java.util.Collections.emptyMap;
 
 /**
  * The dangling indices state is responsible for finding new dangling indices (indices that have
- * their state written on disk, but don't exists in the metadata of the cluster), and importing
- * them into the cluster.
+ * their state written on disk, but don't exists in the metadata of the cluster).
  */
-public class DanglingIndicesState implements ClusterStateListener {
+public class DanglingIndicesState {
 
     private static final Logger logger = LogManager.getLogger(DanglingIndicesState.class);
 
-    /**
-     * Controls whether dangling indices should be automatically detected and imported into the cluster
-     * state upon discovery. This setting is deprecated - use the <code>_dangling</code> API instead.
-     * If disabled, dangling indices will not be automatically detected.
-     *
-     * @see org.elasticsearch.action.admin.indices.dangling
-     */
-    public static final Setting<Boolean> AUTO_IMPORT_DANGLING_INDICES_SETTING = Setting.boolSetting(
-        "gateway.auto_import_dangling_indices",
-        false,
-        Setting.Property.NodeScope,
-        Setting.Property.Deprecated
-    );
-
-    private final NodeEnvironment nodeEnv;
     private final MetaStateService metaStateService;
-    private final LocalAllocateDangledIndices danglingIndicesAllocator;
-    private final boolean isAutoImportDanglingIndicesEnabled;
     private final ClusterService clusterService;
 
-    private final Map<Index, IndexMetadata> danglingIndices = ConcurrentCollections.newConcurrentMap();
-
     @Inject
-    public DanglingIndicesState(NodeEnvironment nodeEnv, MetaStateService metaStateService,
-                                LocalAllocateDangledIndices danglingIndicesAllocator, ClusterService clusterService) {
-        this.nodeEnv = nodeEnv;
+    public DanglingIndicesState(MetaStateService metaStateService, ClusterService clusterService) {
         this.metaStateService = metaStateService;
-        this.danglingIndicesAllocator = danglingIndicesAllocator;
         this.clusterService = clusterService;
-
-        this.isAutoImportDanglingIndicesEnabled = AUTO_IMPORT_DANGLING_INDICES_SETTING.get(clusterService.getSettings());
-
-        if (this.isAutoImportDanglingIndicesEnabled) {
-            clusterService.addListener(this);
-        } else {
-            logger.warn(
-                AUTO_IMPORT_DANGLING_INDICES_SETTING.getKey()
-                    + " is disabled, dangling indices will not be automatically detected or imported and must be managed manually"
-            );
-        }
-    }
-
-    boolean isAutoImportDanglingIndicesEnabled() {
-        return this.isAutoImportDanglingIndicesEnabled;
-    }
-
-    /**
-     * Process dangling indices based on the provided meta data, handling cleanup, finding
-     * new dangling indices, and allocating outstanding ones.
-     */
-    public void processDanglingIndices(final Metadata metadata) {
-        assert this.isAutoImportDanglingIndicesEnabled;
-
-        if (nodeEnv.hasNodeFile() == false) {
-            return;
-        }
-        cleanupAllocatedDangledIndices(metadata);
-        findNewAndAddDanglingIndices(metadata);
-        allocateDanglingIndices(metadata);
     }
 
     /**
-     * Either return the current set of dangling indices, if auto-import is enabled, otherwise
-     * scan for dangling indices right away.
+     * Finds new dangling indices by iterating over the indices and trying to find indices
+     * that have state on disk, but are not part of the provided metadata.
      * @return a map of currently-known dangling indices
      */
     public Map<Index, IndexMetadata> getDanglingIndices() {
-        if (this.isAutoImportDanglingIndicesEnabled) {
-            // This might be a good use case for CopyOnWriteHashMap
-            return Map.copyOf(this.danglingIndices);
-        } else {
-            return findNewDanglingIndices(emptyMap(), this.clusterService.state().metadata());
-        }
-    }
-
-    /**
-     * Cleans dangling indices if they are already allocated on the provided meta data.
-     */
-    void cleanupAllocatedDangledIndices(Metadata metadata) {
-        for (Index index : danglingIndices.keySet()) {
-            final IndexMetadata indexMetadata = metadata.index(index);
-            if (indexMetadata != null && indexMetadata.getIndex().getName().equals(index.getName())) {
-                if (indexMetadata.getIndex().getUUID().equals(index.getUUID()) == false) {
-                    logger.warn("[{}] can not be imported as a dangling index, as there is already another index " +
-                        "with the same name but a different uuid. local index will be ignored (but not deleted)", index);
-                } else {
-                    logger.debug("[{}] no longer dangling (created), removing from dangling list", index);
-                }
-                danglingIndices.remove(index);
-            }
-        }
-    }
-
-    /**
-     * Finds (@{link #findNewAndAddDanglingIndices}) and adds the new dangling indices
-     * to the currently tracked dangling indices.
-     */
-    void findNewAndAddDanglingIndices(final Metadata metadata) {
-        final IndexGraveyard graveyard = metadata.indexGraveyard();
+        final Metadata metadata = this.clusterService.state().metadata();
 
-        // If a tombstone is created for a dangling index, we need to make sure that the
-        // index is no longer considered dangling.
-        danglingIndices.keySet().removeIf(graveyard::containsIndex);
+        final Set<String> excludeIndexPathIds = new HashSet<>(metadata.indices().size());
 
-        danglingIndices.putAll(findNewDanglingIndices(danglingIndices, metadata));
-    }
-
-    /**
-     * Finds new dangling indices by iterating over the indices and trying to find indices
-     * that have state on disk, but are not part of the provided metadata, or not detected
-     * as dangled already.
-     */
-    public Map<Index, IndexMetadata> findNewDanglingIndices(Map<Index, IndexMetadata> existingDanglingIndices, final Metadata metadata) {
-        final Set<String> excludeIndexPathIds = new HashSet<>(metadata.indices().size() + danglingIndices.size());
         for (ObjectCursor<IndexMetadata> cursor : metadata.indices().values()) {
             excludeIndexPathIds.add(cursor.value.getIndex().getUUID());
         }
-        for (Index index : existingDanglingIndices.keySet()) {
-            excludeIndexPathIds.add(index.getUUID());
-        }
+
         try {
             final List<IndexMetadata> indexMetadataList = metaStateService.loadIndicesStates(excludeIndexPathIds::contains);
-            Map<Index, IndexMetadata> newIndices = new HashMap<>(indexMetadataList.size());
+            final Map<Index, IndexMetadata> danglingIndices = new HashMap<>();
             final IndexGraveyard graveyard = metadata.indexGraveyard();
 
             for (IndexMetadata indexMetadata : indexMetadataList) {
                 Index index = indexMetadata.getIndex();
                 if (graveyard.containsIndex(index) == false) {
-                    newIndices.put(index, stripAliases(indexMetadata));
+                    danglingIndices.put(index, stripAliases(indexMetadata));
                 }
             }
 
-            return newIndices;
+            return danglingIndices;
         } catch (IOException e) {
             logger.warn("failed to list dangling indices", e);
             return emptyMap();
         }
     }
 
-    /**
-     * Filters out dangling indices that cannot be automatically imported into the cluster state.
-     * @param metadata the current cluster metadata
-     * @param allIndices all currently known dangling indices
-     * @return a filtered list of dangling index metadata
-     */
-    List<IndexMetadata> filterDanglingIndices(Metadata metadata, Map<Index, IndexMetadata> allIndices) {
-        List<IndexMetadata> filteredIndices = new ArrayList<>(allIndices.size());
-
-        allIndices.forEach((index, indexMetadata) -> {
-            if (metadata.hasIndex(indexMetadata.getIndex().getName())) {
-                logger.warn("[{}] can not be imported as a dangling index, as index with same name already exists in cluster metadata",
-                    indexMetadata.getIndex());
-            } else {
-                logger.info(
-                    "[{}] dangling index exists on local file system, but not in cluster metadata, auto import to cluster state",
-                    indexMetadata.getIndex()
-                );
-                filteredIndices.add(stripAliases(indexMetadata));
-            }
-        });
-
-        return filteredIndices;
-    }
-
     /**
      * Removes all aliases from the supplied index metadata.
      *
-     * Dangling importing indices with aliases is dangerous, it could for instance result in inability to write to an existing alias if it
+     * Importing dangling indices with aliases is dangerous, it could for instance result in inability to write to an existing alias if it
      * previously had only one index with any is_write_index indication.
      */
     private IndexMetadata stripAliases(IndexMetadata indexMetadata) {
         if (indexMetadata.getAliases().isEmpty()) {
             return indexMetadata;
         } else {
-            logger.info("[{}] stripping aliases: {} from index before importing",
-                indexMetadata.getIndex(), indexMetadata.getAliases().keys());
-            return IndexMetadata.builder(indexMetadata).removeAllAliases().build();
-        }
-    }
-
-    /**
-     * Allocates the detected list of dangling indices by sending them to the master node
-     * for allocation, provided auto-import is enabled via the
-     * {@link #AUTO_IMPORT_DANGLING_INDICES_SETTING} setting.
-     * @param metadata the current cluster metadata, used to filter out dangling indices that cannot be allocated
-     *                 for some reason.
-     */
-    void allocateDanglingIndices(Metadata metadata) {
-        if (danglingIndices.isEmpty()) {
-            return;
-        }
-
-        final List<IndexMetadata> filteredIndices = filterDanglingIndices(metadata, danglingIndices);
-
-        if (filteredIndices.isEmpty()) {
-            return;
-        }
-
-        try {
-            danglingIndicesAllocator.allocateDangled(filteredIndices,
-                new ActionListener<>() {
-                    @Override
-                    public void onResponse(LocalAllocateDangledIndices.AllocateDangledResponse response) {
-                        logger.trace("allocated dangled");
-                    }
-
-                    @Override
-                    public void onFailure(Exception e) {
-                        logger.info("failed to send allocated dangled", e);
-                    }
-                }
+            logger.info(
+                "[{}] stripping aliases: {} from index before importing",
+                indexMetadata.getIndex(),
+                indexMetadata.getAliases().keys()
             );
-        } catch (Exception e) {
-            logger.warn("failed to send allocate dangled", e);
-        }
-    }
-
-    @Override
-    public void clusterChanged(ClusterChangedEvent event) {
-        if (event.state().blocks().disableStatePersistence() == false) {
-            processDanglingIndices(event.state().metadata());
+            return IndexMetadata.builder(indexMetadata).removeAllAliases().build();
         }
     }
 }

+ 33 - 183
server/src/test/java/org/elasticsearch/gateway/DanglingIndicesStateTests.java

@@ -19,6 +19,7 @@
 package org.elasticsearch.gateway;
 
 import org.elasticsearch.Version;
+import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.AliasMetadata;
 import org.elasticsearch.cluster.metadata.IndexGraveyard;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
@@ -29,99 +30,52 @@ import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.env.NodeEnvironment;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.test.ESTestCase;
-import org.hamcrest.Matchers;
 
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.StandardCopyOption;
-import java.util.List;
 import java.util.Map;
 
 import static java.util.Collections.emptyMap;
-import static org.elasticsearch.gateway.DanglingIndicesState.AUTO_IMPORT_DANGLING_INDICES_SETTING;
 import static org.hamcrest.Matchers.aMapWithSize;
-import static org.hamcrest.Matchers.anEmptyMap;
-import static org.hamcrest.Matchers.empty;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.is;
-import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 public class DanglingIndicesStateTests extends ESTestCase {
 
-    private static Settings indexSettings = Settings.builder()
-            .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
-            .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0)
-            .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT)
-            .build();
-
-    // The setting AUTO_IMPORT_DANGLING_INDICES_SETTING is deprecated, so we must disable
-    // warning checks or all the tests will fail.
-    @Override
-    protected boolean enableWarningsCheck() {
-        return false;
-    }
+    private static final Settings indexSettings = Settings.builder()
+        .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
+        .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0)
+        .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT)
+        .build();
 
-    public void testCleanupWhenEmpty() throws Exception {
+    public void testDanglingIndicesAreDiscovered() throws Exception {
         try (NodeEnvironment env = newNodeEnvironment()) {
             MetaStateService metaStateService = new MetaStateService(env, xContentRegistry());
-            DanglingIndicesState danglingState = createDanglingIndicesState(env, metaStateService);
 
-            assertTrue(danglingState.getDanglingIndices().isEmpty());
             Metadata metadata = Metadata.builder().build();
-            danglingState.cleanupAllocatedDangledIndices(metadata);
-            assertTrue(danglingState.getDanglingIndices().isEmpty());
-        }
-    }
+            DanglingIndicesState danglingState = createDanglingIndicesState(metaStateService, metadata);
 
-    public void testDanglingIndicesDiscovery() throws Exception {
-        try (NodeEnvironment env = newNodeEnvironment()) {
-            MetaStateService metaStateService = new MetaStateService(env, xContentRegistry());
-            DanglingIndicesState danglingState = createDanglingIndicesState(env, metaStateService);
             assertTrue(danglingState.getDanglingIndices().isEmpty());
-            Metadata metadata = Metadata.builder().build();
-            final Settings.Builder settings = Settings.builder().put(indexSettings).put(IndexMetadata.SETTING_INDEX_UUID, "test1UUID");
-            IndexMetadata dangledIndex = IndexMetadata.builder("test1").settings(settings).build();
-            metaStateService.writeIndex("test_write", dangledIndex);
-            Map<Index, IndexMetadata> newDanglingIndices = danglingState.findNewDanglingIndices(emptyMap(), metadata);
-            assertTrue(newDanglingIndices.containsKey(dangledIndex.getIndex()));
-            metadata = Metadata.builder().put(dangledIndex, false).build();
-            newDanglingIndices = danglingState.findNewDanglingIndices(emptyMap(), metadata);
-            assertFalse(newDanglingIndices.containsKey(dangledIndex.getIndex()));
-        }
-    }
 
-    /**
-     * Check that a dangling index is not reported as newly discovered when we
-     * already known about it.
-     */
-    public void testDanglingIndicesNotDiscoveredWhenAlreadyKnown() throws Exception {
-        try (NodeEnvironment env = newNodeEnvironment()) {
-            MetaStateService metaStateService = new MetaStateService(env, xContentRegistry());
-            DanglingIndicesState danglingState = createDanglingIndicesState(env, metaStateService);
-
-            Metadata metadata = Metadata.builder().build();
             final Settings.Builder settings = Settings.builder().put(indexSettings).put(IndexMetadata.SETTING_INDEX_UUID, "test1UUID");
             IndexMetadata dangledIndex = IndexMetadata.builder("test1").settings(settings).build();
             metaStateService.writeIndex("test_write", dangledIndex);
 
-            Map<Index, IndexMetadata> newDanglingIndices = danglingState.findNewDanglingIndices(
-                Map.of(dangledIndex.getIndex(), dangledIndex),
-                metadata
-            );
-            assertThat(newDanglingIndices, is(anEmptyMap()));
+            Map<Index, IndexMetadata> newDanglingIndices = danglingState.getDanglingIndices();
+            assertTrue(newDanglingIndices.containsKey(dangledIndex.getIndex()));
         }
     }
 
     public void testInvalidIndexFolder() throws Exception {
         try (NodeEnvironment env = newNodeEnvironment()) {
             MetaStateService metaStateService = new MetaStateService(env, xContentRegistry());
-            DanglingIndicesState danglingState = createDanglingIndicesState(env, metaStateService);
-
             Metadata metadata = Metadata.builder().build();
+
+            DanglingIndicesState danglingState = createDanglingIndicesState(metaStateService, metadata);
+
             final String uuid = "test1UUID";
             final Settings.Builder settings = Settings.builder().put(indexSettings).put(IndexMetadata.SETTING_INDEX_UUID, uuid);
             IndexMetadata dangledIndex = IndexMetadata.builder("test1").settings(settings).build();
@@ -131,63 +85,15 @@ public class DanglingIndicesStateTests extends ESTestCase {
                     Files.move(path, path.resolveSibling("invalidUUID"), StandardCopyOption.ATOMIC_MOVE);
                 }
             }
-            try {
-                danglingState.findNewDanglingIndices(emptyMap(), metadata);
-                fail("no exception thrown for invalid folder name");
-            } catch (IllegalStateException e) {
-                assertThat(e.getMessage(), equalTo("[invalidUUID] invalid index folder name, rename to [test1UUID]"));
-            }
-        }
-    }
-
-    public void testDanglingProcessing() throws Exception {
-        try (NodeEnvironment env = newNodeEnvironment()) {
-            MetaStateService metaStateService = new MetaStateService(env, xContentRegistry());
-            DanglingIndicesState danglingState = createDanglingIndicesState(env, metaStateService);
-
-            Metadata metadata = Metadata.builder().build();
-
-            final Settings.Builder settings = Settings.builder().put(indexSettings).put(IndexMetadata.SETTING_INDEX_UUID, "test1UUID");
-            IndexMetadata dangledIndex = IndexMetadata.builder("test1").settings(settings).build();
-            metaStateService.writeIndex("test_write", dangledIndex);
-
-            // check that several runs when not in the metadata still keep the dangled index around
-            int numberOfChecks = randomIntBetween(1, 10);
-            for (int i = 0; i < numberOfChecks; i++) {
-                Map<Index, IndexMetadata> newDanglingIndices = danglingState.findNewDanglingIndices(emptyMap(), metadata);
-                assertThat(newDanglingIndices.size(), equalTo(1));
-                assertThat(newDanglingIndices.keySet(), Matchers.hasItems(dangledIndex.getIndex()));
-                assertTrue(danglingState.getDanglingIndices().isEmpty());
-            }
-
-            for (int i = 0; i < numberOfChecks; i++) {
-                danglingState.findNewAndAddDanglingIndices(metadata);
 
-                assertThat(danglingState.getDanglingIndices().size(), equalTo(1));
-                assertThat(danglingState.getDanglingIndices().keySet(), Matchers.hasItems(dangledIndex.getIndex()));
-            }
-
-            // simulate allocation to the metadata
-            metadata = Metadata.builder(metadata).put(dangledIndex, true).build();
-
-            // check that several runs when in the metadata, but not cleaned yet, still keeps dangled
-            for (int i = 0; i < numberOfChecks; i++) {
-                Map<Index, IndexMetadata> newDanglingIndices = danglingState.findNewDanglingIndices(emptyMap(), metadata);
-                assertTrue(newDanglingIndices.isEmpty());
-
-                assertThat(danglingState.getDanglingIndices().size(), equalTo(1));
-                assertThat(danglingState.getDanglingIndices().keySet(), Matchers.hasItems(dangledIndex.getIndex()));
-            }
-
-            danglingState.cleanupAllocatedDangledIndices(metadata);
-            assertTrue(danglingState.getDanglingIndices().isEmpty());
+            final IllegalStateException e = expectThrows(IllegalStateException.class, danglingState::getDanglingIndices);
+            assertThat(e.getMessage(), equalTo("[invalidUUID] invalid index folder name, rename to [test1UUID]"));
         }
     }
 
-    public void testDanglingIndicesNotImportedWhenTombstonePresent() throws Exception {
+    public void testDanglingIndicesNotReportedWhenTombstonePresent() throws Exception {
         try (NodeEnvironment env = newNodeEnvironment()) {
             MetaStateService metaStateService = new MetaStateService(env, xContentRegistry());
-            DanglingIndicesState danglingState = createDanglingIndicesState(env, metaStateService);
 
             final Settings.Builder settings = Settings.builder().put(indexSettings).put(IndexMetadata.SETTING_INDEX_UUID, "test1UUID");
             IndexMetadata dangledIndex = IndexMetadata.builder("test1").settings(settings).build();
@@ -196,15 +102,16 @@ public class DanglingIndicesStateTests extends ESTestCase {
             final IndexGraveyard graveyard = IndexGraveyard.builder().addTombstone(dangledIndex.getIndex()).build();
             final Metadata metadata = Metadata.builder().indexGraveyard(graveyard).build();
 
-            final Map<Index, IndexMetadata> newDanglingIndices = danglingState.findNewDanglingIndices(emptyMap(), metadata);
+            DanglingIndicesState danglingState = createDanglingIndicesState(metaStateService, metadata);
+
+            final Map<Index, IndexMetadata> newDanglingIndices = danglingState.getDanglingIndices();
             assertThat(newDanglingIndices, is(emptyMap()));
         }
     }
 
-    public void testDanglingIndicesNotImportedWhenIndexNameIsAlreadyUsed() throws Exception {
+    public void testDanglingIndicesReportedWhenIndexNameIsAlreadyUsed() throws Exception {
         try (NodeEnvironment env = newNodeEnvironment()) {
             MetaStateService metaStateService = new MetaStateService(env, xContentRegistry());
-            DanglingIndicesState danglingState = createDanglingIndicesState(env, metaStateService);
 
             final Settings.Builder danglingSettings = Settings.builder()
                 .put(indexSettings)
@@ -224,20 +131,17 @@ public class DanglingIndicesStateTests extends ESTestCase {
                 .build();
             final Metadata metadata = Metadata.builder().indices(indices).build();
 
-            // All dangling indices should be found...
-            final Map<Index, IndexMetadata> newDanglingIndices = danglingState.findNewDanglingIndices(emptyMap(), metadata);
-            assertThat(newDanglingIndices, is(aMapWithSize(1)));
+            DanglingIndicesState danglingState = createDanglingIndicesState(metaStateService, metadata);
 
-            // ...but the filter method should remove those where another index exists with the same name
-            final List<IndexMetadata> filteredIndices = danglingState.filterDanglingIndices(metadata, newDanglingIndices);
-            assertThat(filteredIndices, is(empty()));
+            // All dangling indices should be found
+            final Map<Index, IndexMetadata> newDanglingIndices = danglingState.getDanglingIndices();
+            assertThat(newDanglingIndices, is(aMapWithSize(1)));
         }
     }
 
     public void testDanglingIndicesStripAliases() throws Exception {
         try (NodeEnvironment env = newNodeEnvironment()) {
             MetaStateService metaStateService = new MetaStateService(env, xContentRegistry());
-            DanglingIndicesState danglingState = createDanglingIndicesState(env, metaStateService);
 
             final Settings.Builder settings = Settings.builder().put(indexSettings).put(IndexMetadata.SETTING_INDEX_UUID, "test1UUID");
             IndexMetadata dangledIndex = IndexMetadata.builder("test1")
@@ -248,7 +152,9 @@ public class DanglingIndicesStateTests extends ESTestCase {
             assertThat(dangledIndex.getAliases().size(), equalTo(1));
 
             final Metadata metadata = Metadata.builder().build();
-            Map<Index, IndexMetadata> newDanglingIndices = danglingState.findNewDanglingIndices(emptyMap(), metadata);
+            DanglingIndicesState danglingState = createDanglingIndicesState(metaStateService, metadata);
+
+            Map<Index, IndexMetadata> newDanglingIndices = danglingState.getDanglingIndices();
             assertThat(newDanglingIndices.size(), equalTo(1));
             Map.Entry<Index, IndexMetadata> entry = newDanglingIndices.entrySet().iterator().next();
             assertThat(entry.getKey().getName(), equalTo("test1"));
@@ -256,69 +162,13 @@ public class DanglingIndicesStateTests extends ESTestCase {
         }
     }
 
-    /**
-     * Check that when auto-imports are disabled, then no change listener is registered with the cluster state.
-     */
-    public void testClusterStateListenerNotRegisterWhenSettingDisabled() throws Exception {
-        try (NodeEnvironment env = newNodeEnvironment()) {
-            MetaStateService metaStateService = new MetaStateService(env, xContentRegistry());
-            LocalAllocateDangledIndices localAllocateDangledIndices = mock(LocalAllocateDangledIndices.class);
-
-            final Settings allocateSettings = Settings.builder().put(AUTO_IMPORT_DANGLING_INDICES_SETTING.getKey(), false).build();
-
-            final ClusterService clusterServiceMock = mock(ClusterService.class);
-            when(clusterServiceMock.getSettings()).thenReturn(allocateSettings);
-
-            new DanglingIndicesState(
-                env,
-                metaStateService,
-                localAllocateDangledIndices,
-                clusterServiceMock
-            );
-
-            verify(clusterServiceMock, never()).addListener(any());
-        }
-    }
-
-    /**
-     * Check that when auto-imports are enabled, then dangling indices are automatically imported.
-     */
-    public void testDanglingIndicesAreAllocatedWhenEnabled() throws Exception {
-        try (NodeEnvironment env = newNodeEnvironment()) {
-            MetaStateService metaStateService = new MetaStateService(env, xContentRegistry());
-            LocalAllocateDangledIndices localAllocateDangledIndices = mock(LocalAllocateDangledIndices.class);
-            final Settings allocateSettings = Settings.builder().put(AUTO_IMPORT_DANGLING_INDICES_SETTING.getKey(), true).build();
-
-            final ClusterService clusterServiceMock = mock(ClusterService.class);
-            when(clusterServiceMock.getSettings()).thenReturn(allocateSettings);
-
-            DanglingIndicesState danglingIndicesState = new DanglingIndicesState(
-                env,
-                metaStateService,
-                localAllocateDangledIndices, clusterServiceMock
-            );
-
-            assertTrue("Expected dangling imports to be enabled", danglingIndicesState.isAutoImportDanglingIndicesEnabled());
-
-            final Settings.Builder settings = Settings.builder().put(indexSettings).put(IndexMetadata.SETTING_INDEX_UUID, "test1UUID");
-            IndexMetadata dangledIndex = IndexMetadata.builder("test1").settings(settings).build();
-            metaStateService.writeIndex("test_write", dangledIndex);
-
-            final Metadata metadata = Metadata.builder().build();
-            danglingIndicesState.findNewAndAddDanglingIndices(metadata);
-
-            danglingIndicesState.allocateDanglingIndices(metadata);
-
-            verify(localAllocateDangledIndices).allocateDangled(any(), any());
-        }
-    }
-
-    private DanglingIndicesState createDanglingIndicesState(NodeEnvironment env, MetaStateService metaStateService) {
-        final Settings allocateSettings = Settings.builder().put(AUTO_IMPORT_DANGLING_INDICES_SETTING.getKey(), true).build();
+    private DanglingIndicesState createDanglingIndicesState(MetaStateService metaStateService, Metadata metadata) {
+        final ClusterState clusterState = mock(ClusterState.class);
+        when(clusterState.metadata()).thenReturn(metadata);
 
-        final ClusterService clusterServiceMock = mock(ClusterService.class);
-        when(clusterServiceMock.getSettings()).thenReturn(allocateSettings);
+        final ClusterService clusterService = mock(ClusterService.class);
+        when(clusterService.state()).thenReturn(clusterState);
 
-        return new DanglingIndicesState(env, metaStateService, null, clusterServiceMock);
+        return new DanglingIndicesState(metaStateService, clusterService);
     }
 }