1
0
Эх сурвалжийг харах

Remove deprecated ._tier allocation filtering settings (#73074)

These settings were deprecated in 7.13+ in #72835 and are now removed by this commit.

This commit also ensures that the settings are removed from index metadata when the metadata is
loaded. The reason for this is that if we allow the settings to remain (because they are not
technically "invalid"), then the index will not be able to be allocated, because the
FilterAllocationDecider will be looking for nodes with the _tier attribute.
Lee Hinman 4 жил өмнө
parent
commit
95bccda599
20 өөрчлөгдсөн 111 нэмэгдсэн , 784 устгасан
  1. 0 22
      docs/reference/index-modules/allocation/data_tier_allocation.asciidoc
  2. 24 1
      docs/reference/migration/migrate_8_0/settings.asciidoc
  3. 22 0
      server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetadataVerifier.java
  4. 2 2
      server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeFilters.java
  5. 0 40
      server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/FilterAllocationDeciderTests.java
  6. 1 2
      x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageIT.java
  7. 1 1
      x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/storage/ProactiveStorageDeciderService.java
  8. 1 1
      x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderService.java
  9. 2 3
      x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/storage/ProactiveStorageDeciderServiceTests.java
  10. 7 27
      x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderDecisionTests.java
  11. 1 5
      x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderServiceTests.java
  12. 11 28
      x-pack/plugin/core/src/internalClusterTest/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierIT.java
  13. 3 91
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierAllocationDecider.java
  14. 4 2
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/DataTier.java
  15. 1 7
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java
  16. 1 17
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/DataTierMigrationRoutedStep.java
  17. 1 14
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/SetSingleNodeAllocateStep.java
  18. 25 0
      x-pack/plugin/core/src/test/java/org/elasticsearch/cluster/metadata/IndexMetadataConversionTests.java
  19. 4 518
      x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierAllocationDeciderTests.java
  20. 0 3
      x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportMountSearchableSnapshotAction.java

+ 0 - 22
docs/reference/index-modules/allocation/data_tier_allocation.asciidoc

@@ -23,28 +23,6 @@ mounted indices>> exclusively.
 [[data-tier-allocation-filters]]
 ==== Data tier allocation settings
 
-
-`index.routing.allocation.include._tier`::
-
-    Assign the index to a node whose `node.roles` configuration has at
-    least one of to the comma-separated values.
-
-    deprecated::[7.13, Filtering based on `include._tier`, `require._tier` and `exclude._tier` is deprecated, use <<tier-preference-allocation-filter,_tier_preference>> instead]
-
-`index.routing.allocation.require._tier`::
-
-    Assign the index to a node whose `node.roles` configuration has _all_
-    of the comma-separated values.
-
-    deprecated::[7.13, Filtering based on `include._tier`, `require._tier` and `exclude._tier` is deprecated, use <<tier-preference-allocation-filter,_tier_preference>> instead]
-
-`index.routing.allocation.exclude._tier`::
-
-    Assign the index to a node whose `node.roles` configuration has _none_ of the
-    comma-separated values.
-
-    deprecated::[7.13, Filtering based on `include._tier`, `require._tier` and `exclude._tier` is deprecated, use <<tier-preference-allocation-filter,_tier_preference>> instead]
-
 [[tier-preference-allocation-filter]]
 `index.routing.allocation.include._tier_preference`::
 

+ 24 - 1
docs/reference/migration/migrate_8_0/settings.asciidoc

@@ -243,6 +243,30 @@ Discontinue use of the removed setting. Specifying this setting in Elasticsearch
 configuration will result in an error on startup.
 ====
 
+[[tier-filter-setting]]
+.Tier filtering settings removed
+[%collapsible]
+====
+*Details* +
+The cluster and index level settings ending in `._tier` used for filtering the allocation of a shard
+to a particular set of nodes have been removed. Instead, the <<tier-preference-allocation-filter,
+tier preference setting>>, `index.routing.allocation.include._tier_preference` should be used. The
+removed settings are:
+
+Cluster level settings:
+- `cluster.routing.allocation.include._tier`
+- `cluster.routing.allocation.exclude._tier`
+- `cluster.routing.allocation.require._tier`
+Index settings:
+- `index.routing.allocation.include._tier`
+- `index.routing.allocation.exclude._tier`
+- `index.routing.allocation.require._tier`
+
+*Impact* +
+Discontinue use of the removed settings. Specifying any of these cluster settings in Elasticsearch
+configuration will result in an error on startup. Any indices using these settings will have the
+settings archived (and they will have no effect) when the index metadata is loaded.
+
 [[shared-data-path-setting]]
 .Shared data path and per index data path settings deprecated
 [%collapsible]
@@ -256,4 +280,3 @@ per index data path settings.
 
 *Impact* +
 Discontinue use of the deprecated settings.
-====

+ 22 - 0
server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetadataVerifier.java

@@ -74,6 +74,10 @@ public class IndexMetadataVerifier {
 
         // First convert any shared_cache searchable snapshot indices to only use _tier_preference: data_frozen
         IndexMetadata newMetadata = convertSharedCacheTierPreference(indexMetadata);
+        // Remove _tier routing settings if available, because though these are technically not
+        // invalid settings, since they are now removed the FilterAllocationDecider treats them as
+        // regular attribute filters, and shards cannot be allocated.
+        newMetadata = removeTierFiltering(newMetadata);
         // Next we have to run this otherwise if we try to create IndexSettings
         // with broken settings it would fail in checkMappingsCompatibility
         newMetadata = archiveBrokenIndexSettings(newMetadata);
@@ -216,4 +220,22 @@ public class IndexMetadataVerifier {
             return indexMetadata;
         }
     }
+
+    /**
+     * Removes index level ._tier allocation filters, if they exist
+     */
+    IndexMetadata removeTierFiltering(IndexMetadata indexMetadata) {
+        final Settings settings = indexMetadata.getSettings();
+        final Settings.Builder settingsBuilder = Settings.builder().put(settings);
+        // Clear any allocation rules other than preference for tier
+        settingsBuilder.remove("index.routing.allocation.include._tier");
+        settingsBuilder.remove("index.routing.allocation.exclude._tier");
+        settingsBuilder.remove("index.routing.allocation.require._tier");
+        final Settings newSettings = settingsBuilder.build();
+        if (settings.equals(newSettings)) {
+            return indexMetadata;
+        } else {
+            return IndexMetadata.builder(indexMetadata).settings(newSettings).build();
+        }
+    }
 }

+ 2 - 2
server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeFilters.java

@@ -90,10 +90,10 @@ public class DiscoveryNodeFilters {
         }
 
         Map<String, String[]> newFilters = original.filters.entrySet().stream()
-            // Remove all entries that start with "_tier", as these will be handled elsewhere
+            // Remove all entries that use "_tier_preference", as these will be handled elsewhere
             .filter(entry -> {
                 String attr = entry.getKey();
-                return attr != null && attr.startsWith("_tier") == false;
+                return attr != null && attr.equals("_tier_preference") == false;
             })
             .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
 

+ 0 - 40
server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/FilterAllocationDeciderTests.java

@@ -129,46 +129,6 @@ public class FilterAllocationDeciderTests extends ESAllocationTestCase {
         assertEquals("node passes include/exclude/require filters", decision.getExplanation());
     }
 
-    public void testTierFilterIgnored() {
-        ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
-        FilterAllocationDecider filterAllocationDecider = new FilterAllocationDecider(Settings.EMPTY, clusterSettings);
-        AllocationDeciders allocationDeciders = new AllocationDeciders(
-            Arrays.asList(filterAllocationDecider,
-                new SameShardAllocationDecider(Settings.EMPTY, clusterSettings),
-                new ReplicaAfterPrimaryActiveAllocationDecider()));
-        AllocationService service = new AllocationService(allocationDeciders,
-            new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE,
-            EmptySnapshotsInfoService.INSTANCE);
-        ClusterState state = createInitialClusterState(service, Settings.builder()
-            .put("index.routing.allocation.require._tier", "data_cold")
-            .put("index.routing.allocation.include._tier", "data_cold")
-            .put("index.routing.allocation.include._tier_preference", "data_cold")
-            .put("index.routing.allocation.exclude._tier", "data_cold")
-            .build(),
-            Settings.builder()
-                .put("cluster.routing.allocation.require._tier", "data_cold")
-                .put("cluster.routing.allocation.include._tier", "data_cold")
-                .put("cluster.routing.allocation.exclude._tier", "data_cold")
-                .build());
-        RoutingTable routingTable = state.routingTable();
-        RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, state.getRoutingNodes(), state,
-            null, null, 0);
-        allocation.debugDecision(true);
-        allocation = new RoutingAllocation(allocationDeciders, state.getRoutingNodes(), state,
-            null, null, 0);
-        allocation.debugDecision(true);
-        Decision.Single decision = (Decision.Single) filterAllocationDecider.canAllocate(
-            routingTable.index("idx").shard(0).shards().get(0),
-            state.getRoutingNodes().node("node2"), allocation);
-        assertEquals(decision.toString(), Type.YES, decision.type());
-        assertEquals("node passes include/exclude/require filters", decision.getExplanation());
-        decision = (Decision.Single) filterAllocationDecider.canAllocate(
-            routingTable.index("idx").shard(0).shards().get(0),
-            state.getRoutingNodes().node("node1"), allocation);
-        assertEquals(Type.YES, decision.type());
-        assertEquals("node passes include/exclude/require filters", decision.getExplanation());
-    }
-
     private ClusterState createInitialClusterState(AllocationService service, Settings indexSettings) {
         return createInitialClusterState(service, indexSettings, Settings.EMPTY);
     }

+ 1 - 2
x-pack/plugin/autoscaling/src/internalClusterTest/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageIT.java

@@ -24,7 +24,6 @@ import org.elasticsearch.test.NodeRoles;
 import org.elasticsearch.xpack.autoscaling.action.GetAutoscalingCapacityAction;
 import org.elasticsearch.xpack.autoscaling.action.PutAutoscalingPolicyAction;
 import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider;
-import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDeciderTests;
 import org.elasticsearch.xpack.core.DataTier;
 import org.hamcrest.Matchers;
 
@@ -197,7 +196,7 @@ public class ReactiveStorageIT extends AutoscalingStorageIntegTestCase {
         internalCluster().startMasterOnlyNode();
         ReactiveStorageDeciderService service = new ReactiveStorageDeciderService(
             Settings.EMPTY,
-            new ClusterSettings(Settings.EMPTY, DataTierAllocationDeciderTests.ALL_SETTINGS),
+            new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS),
             null
         );
         assertThat(

+ 1 - 1
x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/storage/ProactiveStorageDeciderService.java

@@ -37,7 +37,7 @@ public class ProactiveStorageDeciderService implements AutoscalingDeciderService
 
     public ProactiveStorageDeciderService(Settings settings, ClusterSettings clusterSettings, AllocationDeciders allocationDeciders) {
         this.diskThresholdSettings = new DiskThresholdSettings(settings, clusterSettings);
-        this.dataTierAllocationDecider = new DataTierAllocationDecider(settings, clusterSettings);
+        this.dataTierAllocationDecider = new DataTierAllocationDecider();
         this.allocationDeciders = allocationDeciders;
     }
 

+ 1 - 1
x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderService.java

@@ -70,7 +70,7 @@ public class ReactiveStorageDeciderService implements AutoscalingDeciderService
 
     public ReactiveStorageDeciderService(Settings settings, ClusterSettings clusterSettings, AllocationDeciders allocationDeciders) {
         this.diskThresholdSettings = new DiskThresholdSettings(settings, clusterSettings);
-        this.dataTierAllocationDecider = new DataTierAllocationDecider(settings, clusterSettings);
+        this.dataTierAllocationDecider = new DataTierAllocationDecider();
         this.allocationDeciders = allocationDeciders;
     }
 

+ 2 - 3
x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/storage/ProactiveStorageDeciderServiceTests.java

@@ -11,9 +11,9 @@ import com.carrotsearch.hppc.cursors.ObjectCursor;
 import org.elasticsearch.cluster.ClusterInfo;
 import org.elasticsearch.cluster.ClusterModule;
 import org.elasticsearch.cluster.ClusterState;
-import org.elasticsearch.cluster.metadata.DataStreamTestHelper;
 import org.elasticsearch.cluster.DiskUsage;
 import org.elasticsearch.cluster.metadata.DataStream;
+import org.elasticsearch.cluster.metadata.DataStreamTestHelper;
 import org.elasticsearch.cluster.metadata.IndexAbstraction;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.Metadata;
@@ -40,7 +40,6 @@ import org.elasticsearch.xpack.autoscaling.AutoscalingTestCase;
 import org.elasticsearch.xpack.autoscaling.capacity.AutoscalingCapacity;
 import org.elasticsearch.xpack.autoscaling.capacity.AutoscalingDeciderContext;
 import org.elasticsearch.xpack.autoscaling.capacity.AutoscalingDeciderResult;
-import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDeciderTests;
 import org.hamcrest.Matchers;
 
 import java.util.ArrayList;
@@ -80,7 +79,7 @@ public class ProactiveStorageDeciderServiceTests extends AutoscalingTestCase {
         );
         ClusterState interimState = stateBuilder.build();
         final ClusterState state = startAll(interimState);
-        final ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, DataTierAllocationDeciderTests.ALL_SETTINGS);
+        final ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
         Collection<AllocationDecider> allocationDecidersList = new ArrayList<>(
             ClusterModule.createAllocationDeciders(Settings.EMPTY, clusterSettings, Collections.emptyList())
         );

+ 7 - 27
x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderDecisionTests.java

@@ -47,7 +47,6 @@ import org.elasticsearch.xpack.autoscaling.capacity.AutoscalingCapacity;
 import org.elasticsearch.xpack.autoscaling.capacity.AutoscalingDeciderContext;
 import org.elasticsearch.xpack.autoscaling.capacity.AutoscalingDeciderResult;
 import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider;
-import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDeciderTests;
 import org.elasticsearch.xpack.core.DataTier;
 import org.junit.Before;
 
@@ -96,10 +95,7 @@ public class ReactiveStorageDeciderDecisionTests extends AutoscalingTestCase {
         new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)
     );
 
-    private static final DataTierAllocationDecider DATA_TIER_ALLOCATION_DECIDER = new DataTierAllocationDecider(
-        Settings.EMPTY,
-        new ClusterSettings(Settings.EMPTY, DataTierAllocationDeciderTests.ALL_SETTINGS)
-    );
+    private static final DataTierAllocationDecider DATA_TIER_ALLOCATION_DECIDER = new DataTierAllocationDecider();
 
     private ClusterState state;
     private final int hotNodes = randomIntBetween(1, 8);
@@ -293,8 +289,6 @@ public class ReactiveStorageDeciderDecisionTests extends AutoscalingTestCase {
 
     private IndexMetadata moveToCold(IndexMetadata imd) {
         Settings.Builder builder = Settings.builder().put(imd.getSettings());
-        overrideSetting(imd, builder, DataTierAllocationDecider.INDEX_ROUTING_REQUIRE_SETTING, DataTier.DATA_COLD);
-        overrideSetting(imd, builder, DataTierAllocationDecider.INDEX_ROUTING_INCLUDE_SETTING, DataTier.DATA_COLD);
         overrideSetting(
             imd,
             builder,
@@ -391,7 +385,7 @@ public class ReactiveStorageDeciderDecisionTests extends AutoscalingTestCase {
     private static void verifyScale(ClusterState state, long expectedDifference, String reason, AllocationDecider... allocationDeciders) {
         ReactiveStorageDeciderService decider = new ReactiveStorageDeciderService(
             Settings.EMPTY,
-            new ClusterSettings(Settings.EMPTY, DataTierAllocationDeciderTests.ALL_SETTINGS),
+            new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS),
             createAllocationDeciders(allocationDeciders)
         );
         TestAutoscalingDeciderContext context = createContext(state, Set.of(DiscoveryNodeRole.DATA_HOT_NODE_ROLE));
@@ -430,16 +424,7 @@ public class ReactiveStorageDeciderDecisionTests extends AutoscalingTestCase {
     }
 
     private static AllocationDeciders createAllocationDeciders(AllocationDecider... extraDeciders) {
-        Set<Setting<?>> allSettings = Stream.concat(
-            ClusterSettings.BUILT_IN_CLUSTER_SETTINGS.stream(),
-            Stream.of(
-                DataTierAllocationDecider.CLUSTER_ROUTING_REQUIRE_SETTING,
-                DataTierAllocationDecider.CLUSTER_ROUTING_INCLUDE_SETTING,
-                DataTierAllocationDecider.CLUSTER_ROUTING_EXCLUDE_SETTING
-            )
-        ).collect(Collectors.toSet());
-
-        ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, allSettings);
+        ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
         Collection<AllocationDecider> systemAllocationDeciders = ClusterModule.createAllocationDeciders(
             Settings.builder()
                 .put(
@@ -451,11 +436,9 @@ public class ReactiveStorageDeciderDecisionTests extends AutoscalingTestCase {
             Collections.emptyList()
         );
         return new AllocationDeciders(
-            Stream.of(
-                Stream.of(extraDeciders),
-                Stream.of(new DataTierAllocationDecider(Settings.EMPTY, clusterSettings)),
-                systemAllocationDeciders.stream()
-            ).flatMap(s -> s).collect(Collectors.toList())
+            Stream.of(Stream.of(extraDeciders), Stream.of(new DataTierAllocationDecider()), systemAllocationDeciders.stream())
+                .flatMap(s -> s)
+                .collect(Collectors.toList())
         );
     }
 
@@ -610,10 +593,7 @@ public class ReactiveStorageDeciderDecisionTests extends AutoscalingTestCase {
     }
 
     private static ClusterState addRandomIndices(int minShards, int maxShardCopies, ClusterState state) {
-        String[] tierSettingNames = new String[] {
-            DataTierAllocationDecider.INDEX_ROUTING_REQUIRE,
-            DataTierAllocationDecider.INDEX_ROUTING_INCLUDE,
-            DataTierAllocationDecider.INDEX_ROUTING_PREFER };
+        String[] tierSettingNames = new String[] { DataTierAllocationDecider.INDEX_ROUTING_PREFER };
         int shards = randomIntBetween(minShards, 20);
         Metadata.Builder builder = Metadata.builder();
         RoutingTable.Builder routingTableBuilder = RoutingTable.builder();

+ 1 - 5
x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderServiceTests.java

@@ -50,7 +50,6 @@ import org.elasticsearch.snapshots.SnapshotId;
 import org.elasticsearch.snapshots.SnapshotShardSizeInfo;
 import org.elasticsearch.xpack.autoscaling.AutoscalingTestCase;
 import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider;
-import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDeciderTests;
 
 import java.util.Arrays;
 import java.util.Collection;
@@ -421,10 +420,7 @@ public class ReactiveStorageDeciderServiceTests extends AutoscalingTestCase {
 
     public boolean canRemainWithNoNodes(ClusterState clusterState, ShardRouting shardRouting, AllocationDecider... deciders) {
         AllocationDeciders allocationDeciders = new AllocationDeciders(Arrays.asList(deciders));
-        DataTierAllocationDecider dataTierAllocationDecider = new DataTierAllocationDecider(
-            Settings.EMPTY,
-            new ClusterSettings(Settings.EMPTY, DataTierAllocationDeciderTests.ALL_SETTINGS)
-        );
+        DataTierAllocationDecider dataTierAllocationDecider = new DataTierAllocationDecider();
         ReactiveStorageDeciderService.AllocationState allocationState = new ReactiveStorageDeciderService.AllocationState(
             clusterState,
             allocationDeciders,

+ 11 - 28
x-pack/plugin/core/src/internalClusterTest/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierIT.java

@@ -70,17 +70,14 @@ public class DataTierIT extends ESIntegTestCase {
         startColdOnlyNode();
         ensureGreen();
 
-        String setting = randomBoolean() ? DataTierAllocationDecider.INDEX_ROUTING_REQUIRE :
-            DataTierAllocationDecider.INDEX_ROUTING_PREFER;
-
         client().admin().indices().prepareCreate(index)
             .setWaitForActiveShards(0)
             .setSettings(Settings.builder()
-                .put(setting, DataTier.DATA_WARM))
+                .put(DataTierAllocationDecider.INDEX_ROUTING_PREFER, DataTier.DATA_WARM))
             .get();
 
         Settings idxSettings = client().admin().indices().prepareGetIndex().addIndices(index).get().getSettings().get(index);
-        assertThat(idxSettings.get(setting), equalTo(DataTier.DATA_WARM));
+        assertThat(idxSettings.get(DataTierAllocationDecider.INDEX_ROUTING_PREFER), equalTo(DataTier.DATA_WARM));
 
         // index should be yellow
         logger.info("--> waiting for {} to be yellow", index);
@@ -113,14 +110,14 @@ public class DataTierIT extends ESIntegTestCase {
         client().admin().indices().prepareCreate(index)
             .setWaitForActiveShards(0)
             .setSettings(Settings.builder()
-                .put(DataTierAllocationDecider.INDEX_ROUTING_REQUIRE, DataTier.DATA_COLD))
+                .put(IndexMetadata.INDEX_ROUTING_REQUIRE_GROUP_PREFIX + ".box", "cold"))
             .get();
 
         idxSettings = client().admin().indices().prepareGetIndex().addIndices(index).get().getSettings().get(index);
         assertThat(DataTierAllocationDecider.INDEX_ROUTING_PREFER_SETTING.get(idxSettings), equalTo(""));
         // The key should not be put in place since it was overridden
         assertFalse(idxSettings.keySet().contains(DataTierAllocationDecider.INDEX_ROUTING_PREFER));
-        assertThat(DataTierAllocationDecider.INDEX_ROUTING_REQUIRE_SETTING.get(idxSettings), equalTo(DataTier.DATA_COLD));
+        assertThat(idxSettings.get(IndexMetadata.INDEX_ROUTING_REQUIRE_GROUP_PREFIX + ".box"), equalTo("cold"));
 
         // index should be yellow
         logger.info("--> waiting for {} to be yellow", index);
@@ -169,7 +166,7 @@ public class DataTierIT extends ESIntegTestCase {
         startWarmOnlyNode();
 
         Template t = new Template(Settings.builder()
-            .put(DataTierAllocationDecider.INDEX_ROUTING_REQUIRE, DataTier.DATA_WARM)
+            .put(IndexMetadata.INDEX_ROUTING_REQUIRE_GROUP_PREFIX + ".box", "warm")
             .build(), null, null);
         ComposableIndexTemplate ct = new ComposableIndexTemplate.Builder()
             .indexPatterns(Collections.singletonList(index))
@@ -249,26 +246,6 @@ public class DataTierIT extends ESIntegTestCase {
         assertThat(usage.getTierStats().get(DataTier.DATA_HOT).primaryShardBytesMAD, greaterThanOrEqualTo(0L));
     }
 
-    public void testTierFilteringIgnoredByFilterAllocationDecider() {
-        startContentOnlyNode();
-        startHotOnlyNode();
-
-        // Exclude all data_cold nodes
-        client().admin().cluster().prepareUpdateSettings()
-            .setTransientSettings(Settings.builder()
-                .put(DataTierAllocationDecider.CLUSTER_ROUTING_EXCLUDE, "data_cold")
-                .build())
-            .get();
-
-        // Create an index, which should be excluded just fine, ignored by the FilterAllocationDecider
-        client().admin().indices().prepareCreate(index)
-            .setSettings(Settings.builder()
-                .put("index.number_of_shards", 2)
-                .put("index.number_of_replicas", 0))
-            .setWaitForActiveShards(0)
-            .get();
-    }
-
     public void testIllegalOnFrozen() {
         startDataNode();
 
@@ -310,6 +287,7 @@ public class DataTierIT extends ESIntegTestCase {
     public void startDataNode() {
         Settings nodeSettings = Settings.builder()
             .putList("node.roles", Arrays.asList("master", "data", "ingest"))
+            .put("node.attr.box", "all")
             .build();
         internalCluster().startNode(nodeSettings);
     }
@@ -317,6 +295,7 @@ public class DataTierIT extends ESIntegTestCase {
     public void startContentOnlyNode() {
         Settings nodeSettings = Settings.builder()
             .putList("node.roles", Arrays.asList("master", "data_content", "ingest"))
+            .put("node.attr.box", "content")
             .build();
         internalCluster().startNode(nodeSettings);
     }
@@ -324,6 +303,7 @@ public class DataTierIT extends ESIntegTestCase {
     public void startHotOnlyNode() {
         Settings nodeSettings = Settings.builder()
             .putList("node.roles", Arrays.asList("master", "data_hot", "ingest"))
+            .put("node.attr.box", "hot")
             .build();
         internalCluster().startNode(nodeSettings);
     }
@@ -331,6 +311,7 @@ public class DataTierIT extends ESIntegTestCase {
     public void startWarmOnlyNode() {
         Settings nodeSettings = Settings.builder()
             .putList("node.roles", Arrays.asList("master", "data_warm", "ingest"))
+            .put("node.attr.box", "warm")
             .build();
         internalCluster().startNode(nodeSettings);
     }
@@ -338,6 +319,7 @@ public class DataTierIT extends ESIntegTestCase {
     public void startColdOnlyNode() {
         Settings nodeSettings = Settings.builder()
             .putList("node.roles", Arrays.asList("master", "data_cold", "ingest"))
+            .put("node.attr.box", "cold")
             .build();
         internalCluster().startNode(nodeSettings);
     }
@@ -345,6 +327,7 @@ public class DataTierIT extends ESIntegTestCase {
     public void startFrozenOnlyNode() {
         Settings nodeSettings = Settings.builder()
             .putList("node.roles", Arrays.asList("master", "data_frozen", "ingest"))
+            .put("node.attr.box", "frozen")
             .build();
         internalCluster().startNode(nodeSettings);
     }

+ 3 - 91
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierAllocationDecider.java

@@ -18,7 +18,6 @@ import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
 import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider;
 import org.elasticsearch.cluster.routing.allocation.decider.Decision;
 import org.elasticsearch.common.Strings;
-import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.index.IndexModule;
@@ -46,27 +45,9 @@ public class DataTierAllocationDecider extends AllocationDecider {
 
     public static final String NAME = "data_tier";
 
-    public static final String CLUSTER_ROUTING_REQUIRE = "cluster.routing.allocation.require._tier";
-    public static final String CLUSTER_ROUTING_INCLUDE = "cluster.routing.allocation.include._tier";
-    public static final String CLUSTER_ROUTING_EXCLUDE = "cluster.routing.allocation.exclude._tier";
-    public static final String INDEX_ROUTING_REQUIRE = "index.routing.allocation.require._tier";
-    public static final String INDEX_ROUTING_INCLUDE = "index.routing.allocation.include._tier";
     public static final String INDEX_ROUTING_PREFER = "index.routing.allocation.include._tier_preference";
-    public static final String INDEX_ROUTING_EXCLUDE = "index.routing.allocation.exclude._tier";
 
     private static final DataTierValidator VALIDATOR = new DataTierValidator();
-    public static final Setting<String> CLUSTER_ROUTING_REQUIRE_SETTING = Setting.simpleString(CLUSTER_ROUTING_REQUIRE,
-        DataTierAllocationDecider::validateTierSetting, Setting.Property.Dynamic, Setting.Property.NodeScope, Setting.Property.Deprecated);
-    public static final Setting<String> CLUSTER_ROUTING_INCLUDE_SETTING = Setting.simpleString(CLUSTER_ROUTING_INCLUDE,
-        DataTierAllocationDecider::validateTierSetting, Setting.Property.Dynamic, Setting.Property.NodeScope, Setting.Property.Deprecated);
-    public static final Setting<String> CLUSTER_ROUTING_EXCLUDE_SETTING = Setting.simpleString(CLUSTER_ROUTING_EXCLUDE,
-        DataTierAllocationDecider::validateTierSetting, Setting.Property.Dynamic, Setting.Property.NodeScope, Setting.Property.Deprecated);
-    public static final Setting<String> INDEX_ROUTING_REQUIRE_SETTING = Setting.simpleString(INDEX_ROUTING_REQUIRE,
-        VALIDATOR, Setting.Property.Dynamic, Setting.Property.IndexScope, Setting.Property.Deprecated);
-    public static final Setting<String> INDEX_ROUTING_INCLUDE_SETTING = Setting.simpleString(INDEX_ROUTING_INCLUDE,
-        VALIDATOR, Setting.Property.Dynamic, Setting.Property.IndexScope, Setting.Property.Deprecated);
-    public static final Setting<String> INDEX_ROUTING_EXCLUDE_SETTING = Setting.simpleString(INDEX_ROUTING_EXCLUDE,
-        VALIDATOR, Setting.Property.Dynamic, Setting.Property.IndexScope, Setting.Property.Deprecated);
     public static final Setting<String> INDEX_ROUTING_PREFER_SETTING = new Setting<>(new Setting.SimpleKey(INDEX_ROUTING_PREFER),
         DataTierValidator::getDefaultTierPreference, Function.identity(), VALIDATOR, Setting.Property.Dynamic, Setting.Property.IndexScope);
 
@@ -121,17 +102,7 @@ public class DataTierAllocationDecider extends AllocationDecider {
         }
     }
 
-    private volatile String clusterRequire;
-    private volatile String clusterInclude;
-    private volatile String clusterExclude;
-
-    public DataTierAllocationDecider(Settings settings, ClusterSettings clusterSettings) {
-        clusterRequire = CLUSTER_ROUTING_REQUIRE_SETTING.get(settings);
-        clusterInclude = CLUSTER_ROUTING_INCLUDE_SETTING.get(settings);
-        clusterExclude = CLUSTER_ROUTING_EXCLUDE_SETTING.get(settings);
-        clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_REQUIRE_SETTING, s -> this.clusterRequire = s);
-        clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_INCLUDE_SETTING, s -> this.clusterInclude = s);
-        clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_EXCLUDE_SETTING, s -> this.clusterExclude = s);
+    public DataTierAllocationDecider() {
     }
 
     @Override
@@ -168,22 +139,12 @@ public class DataTierAllocationDecider extends AllocationDecider {
 
     public Decision shouldFilter(IndexMetadata indexMd, Set<DiscoveryNodeRole> roles,
                                  PreferredTierFunction preferredTierFunction, RoutingAllocation allocation) {
-        Decision decision = shouldClusterFilter(roles, allocation);
+        Decision decision = shouldIndexPreferTier(indexMd, roles, preferredTierFunction, allocation);
         if (decision != null) {
             return decision;
         }
 
-        decision = shouldIndexFilter(indexMd, roles, allocation);
-        if (decision != null) {
-            return decision;
-        }
-
-        decision = shouldIndexPreferTier(indexMd, roles, preferredTierFunction, allocation);
-        if (decision != null) {
-            return decision;
-        }
-
-        return allocation.decision(Decision.YES, NAME, "node passes include/exclude/require/prefer tier filters");
+        return allocation.decision(Decision.YES, NAME, "node passes tier preference filters");
     }
 
     private Decision shouldIndexPreferTier(IndexMetadata indexMetadata, Set<DiscoveryNodeRole> roles,
@@ -212,55 +173,6 @@ public class DataTierAllocationDecider extends AllocationDecider {
         return null;
     }
 
-    private Decision shouldIndexFilter(IndexMetadata indexMd, Set<DiscoveryNodeRole> roles, RoutingAllocation allocation) {
-        Settings indexSettings = indexMd.getSettings();
-        String indexRequire = INDEX_ROUTING_REQUIRE_SETTING.get(indexSettings);
-        String indexInclude = INDEX_ROUTING_INCLUDE_SETTING.get(indexSettings);
-        String indexExclude = INDEX_ROUTING_EXCLUDE_SETTING.get(indexSettings);
-
-        if (Strings.hasText(indexRequire)) {
-            if (allocationAllowed(OpType.AND, indexRequire, roles) == false) {
-                return allocation.decision(Decision.NO, NAME, "node does not match all index setting [%s] tier filters [%s]",
-                    INDEX_ROUTING_REQUIRE, indexRequire);
-            }
-        }
-        if (Strings.hasText(indexInclude)) {
-            if (allocationAllowed(OpType.OR, indexInclude, roles) == false) {
-                return allocation.decision(Decision.NO, NAME, "node does not match any index setting [%s] tier filters [%s]",
-                    INDEX_ROUTING_INCLUDE, indexInclude);
-            }
-        }
-        if (Strings.hasText(indexExclude)) {
-            if (allocationAllowed(OpType.OR, indexExclude, roles)) {
-                return allocation.decision(Decision.NO, NAME, "node matches any index setting [%s] tier filters [%s]",
-                    INDEX_ROUTING_EXCLUDE, indexExclude);
-            }
-        }
-        return null;
-    }
-
-    private Decision shouldClusterFilter(Set<DiscoveryNodeRole> roles, RoutingAllocation allocation) {
-        if (Strings.hasText(clusterRequire)) {
-            if (allocationAllowed(OpType.AND, clusterRequire, roles) == false) {
-                return allocation.decision(Decision.NO, NAME, "node does not match all cluster setting [%s] tier filters [%s]",
-                    CLUSTER_ROUTING_REQUIRE, clusterRequire);
-            }
-        }
-        if (Strings.hasText(clusterInclude)) {
-            if (allocationAllowed(OpType.OR, clusterInclude, roles) == false) {
-                return allocation.decision(Decision.NO, NAME, "node does not match any cluster setting [%s] tier filters [%s]",
-                    CLUSTER_ROUTING_INCLUDE, clusterInclude);
-            }
-        }
-        if (Strings.hasText(clusterExclude)) {
-            if (allocationAllowed(OpType.OR, clusterExclude, roles)) {
-                return allocation.decision(Decision.NO, NAME, "node matches any cluster setting [%s] tier filters [%s]",
-                    CLUSTER_ROUTING_EXCLUDE, clusterExclude);
-            }
-        }
-        return null;
-    }
-
     private enum OpType {
         AND,
         OR

+ 4 - 2
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/DataTier.java

@@ -98,8 +98,10 @@ public class DataTier {
 
     /**
      * This setting provider injects the setting allocating all newly created indices with
-     * {@code index.routing.allocation.include._tier: "data_hot"} unless the user overrides the
-     * setting while the index is being created (in a create index request for instance)
+     * {@code index.routing.allocation.include._tier_preference: "data_hot"} for a data stream index
+     * or {@code index.routing.allocation.include._tier_preference: "data_content"} for an index not part of
+     * a data stream unless the user overrides the setting while the index is being created
+     * (in a create index request for instance)
      */
     public static class DefaultHotAllocationSettingProvider implements IndexSettingProvider {
         private static final Logger logger = LogManager.getLogger(DefaultHotAllocationSettingProvider.class);

+ 1 - 7
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java

@@ -384,19 +384,13 @@ public class XPackPlugin extends XPackClientPlugin
     public List<Setting<?>> getSettings() {
         List<Setting<?>> settings = super.getSettings();
         settings.add(SourceOnlySnapshotRepository.SOURCE_ONLY);
-        settings.add(DataTierAllocationDecider.CLUSTER_ROUTING_REQUIRE_SETTING);
-        settings.add(DataTierAllocationDecider.CLUSTER_ROUTING_INCLUDE_SETTING);
-        settings.add(DataTierAllocationDecider.CLUSTER_ROUTING_EXCLUDE_SETTING);
-        settings.add(DataTierAllocationDecider.INDEX_ROUTING_REQUIRE_SETTING);
-        settings.add(DataTierAllocationDecider.INDEX_ROUTING_INCLUDE_SETTING);
-        settings.add(DataTierAllocationDecider.INDEX_ROUTING_EXCLUDE_SETTING);
         settings.add(DataTierAllocationDecider.INDEX_ROUTING_PREFER_SETTING);
         return settings;
     }
 
     @Override
     public Collection<AllocationDecider> createAllocationDeciders(Settings settings, ClusterSettings clusterSettings) {
-        return Collections.singleton(new DataTierAllocationDecider(settings, clusterSettings));
+        return Collections.singleton(new DataTierAllocationDecider());
     }
 
     @Override

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

@@ -13,18 +13,13 @@ import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
 import org.elasticsearch.common.Strings;
-import org.elasticsearch.common.settings.ClusterSettings;
-import org.elasticsearch.common.settings.Setting;
-import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider;
 import org.elasticsearch.xpack.core.ilm.step.info.AllocationInfo;
 
-import java.util.HashSet;
 import java.util.List;
 import java.util.Locale;
 import java.util.Optional;
-import java.util.Set;
 
 import static org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider.INDEX_ROUTING_PREFER_SETTING;
 import static org.elasticsearch.xpack.core.ilm.AllocationRoutedStep.getPendingAllocations;
@@ -39,16 +34,6 @@ public class DataTierMigrationRoutedStep extends ClusterStateWaitStep {
 
     private static final Logger logger = LogManager.getLogger(DataTierMigrationRoutedStep.class);
 
-    private static final Set<Setting<?>> ALL_CLUSTER_SETTINGS;
-
-    static {
-        Set<Setting<?>> allSettings = new HashSet<>(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
-        allSettings.add(DataTierAllocationDecider.CLUSTER_ROUTING_REQUIRE_SETTING);
-        allSettings.add(DataTierAllocationDecider.CLUSTER_ROUTING_INCLUDE_SETTING);
-        allSettings.add(DataTierAllocationDecider.CLUSTER_ROUTING_EXCLUDE_SETTING);
-        ALL_CLUSTER_SETTINGS = allSettings;
-    }
-
     DataTierMigrationRoutedStep(StepKey key, StepKey nextStepKey) {
         super(key, nextStepKey);
     }
@@ -62,8 +47,7 @@ public class DataTierMigrationRoutedStep extends ClusterStateWaitStep {
     public Result isConditionMet(Index index, ClusterState clusterState) {
         AllocationDeciders allocationDeciders = new AllocationDeciders(
             List.of(
-                new DataTierAllocationDecider(clusterState.getMetadata().settings(),
-                    new ClusterSettings(Settings.EMPTY, ALL_CLUSTER_SETTINGS))
+                new DataTierAllocationDecider()
             )
         );
         IndexMetadata idxMeta = clusterState.metadata().index(index);

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

@@ -25,18 +25,15 @@ import org.elasticsearch.cluster.routing.allocation.decider.FilterAllocationDeci
 import org.elasticsearch.cluster.routing.allocation.decider.NodeVersionAllocationDecider;
 import org.elasticsearch.common.Randomness;
 import org.elasticsearch.common.settings.ClusterSettings;
-import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.index.IndexNotFoundException;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider;
 
 import java.util.ArrayList;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
-import java.util.Set;
 import java.util.stream.Collectors;
 
 /**
@@ -47,16 +44,6 @@ public class SetSingleNodeAllocateStep extends AsyncActionStep {
     private static final Logger logger = LogManager.getLogger(SetSingleNodeAllocateStep.class);
     public static final String NAME = "set-single-node-allocation";
 
-    private static final Set<Setting<?>> ALL_CLUSTER_SETTINGS;
-
-    static {
-        Set<Setting<?>> allSettings = new HashSet<>(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
-        allSettings.add(DataTierAllocationDecider.CLUSTER_ROUTING_REQUIRE_SETTING);
-        allSettings.add(DataTierAllocationDecider.CLUSTER_ROUTING_INCLUDE_SETTING);
-        allSettings.add(DataTierAllocationDecider.CLUSTER_ROUTING_EXCLUDE_SETTING);
-        ALL_CLUSTER_SETTINGS = allSettings;
-    }
-
     public SetSingleNodeAllocateStep(StepKey key, StepKey nextStepKey, Client client) {
         super(key, nextStepKey, client);
     }
@@ -76,7 +63,7 @@ public class SetSingleNodeAllocateStep extends AsyncActionStep {
         AllocationDeciders allocationDeciders = new AllocationDeciders(List.of(
             new FilterAllocationDecider(clusterState.getMetadata().settings(),
                 new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)),
-            new DataTierAllocationDecider(clusterState.getMetadata().settings(), new ClusterSettings(Settings.EMPTY, ALL_CLUSTER_SETTINGS)),
+            new DataTierAllocationDecider(),
             new NodeVersionAllocationDecider()
         ));
         final RoutingNodes routingNodes = clusterState.getRoutingNodes();

+ 25 - 0
x-pack/plugin/core/src/test/java/org/elasticsearch/cluster/metadata/IndexMetadataConversionTests.java

@@ -68,6 +68,31 @@ public class IndexMetadataConversionTests extends ESTestCase {
         assertThat(newSettings.get("index.routing.allocation.include._tier_preference"), equalTo("data_frozen"));
     }
 
+    public void testRemoveSingleTierAllocationFilters() {
+        IndexMetadataVerifier service = getIndexMetadataVerifier();
+        IndexMetadata src = newIndexMeta("foo", Settings.builder()
+            .put("index.routing.allocation.include._tier", "data_hot")
+            .put("index.routing.allocation.exclude._tier", "data_warm")
+            .put("index.routing.allocation.require._tier", "data_hot")
+            .put("index.routing.allocation.include._tier_preference", "data_cold")
+            .build());
+        IndexMetadata indexMetadata = service.removeTierFiltering(src);
+        assertNotSame(indexMetadata, src);
+
+        Settings newSettings = indexMetadata.getSettings();
+        assertNull(newSettings.get("index.routing.allocation.include._tier"));
+        assertNull(newSettings.get("index.routing.allocation.exclude._tier"));
+        assertNull(newSettings.get("index.routing.allocation.require._tier"));
+        assertThat(newSettings.get("index.routing.allocation.include._tier_preference"), equalTo("data_cold"));
+
+        src = newIndexMeta("foo", Settings.builder()
+            .put("index.routing.allocation.include._tier_preference", "data_cold")
+            .put("index.number_of_shards", randomIntBetween(1, 10))
+            .build());
+        indexMetadata = service.removeTierFiltering(src);
+        assertSame(indexMetadata, src);
+    }
+
     private IndexMetadataVerifier getIndexMetadataVerifier() {
         return new IndexMetadataVerifier(
             Settings.EMPTY,

+ 4 - 518
x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierAllocationDeciderTests.java

@@ -42,10 +42,8 @@ import org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Optional;
-import java.util.Set;
 
 import static org.elasticsearch.xpack.core.DataTier.DATA_COLD;
 import static org.elasticsearch.xpack.core.DataTier.DATA_FROZEN;
@@ -54,7 +52,6 @@ import static org.hamcrest.Matchers.equalTo;
 
 public class DataTierAllocationDeciderTests extends ESAllocationTestCase {
 
-    public static final Set<Setting<?>> ALL_SETTINGS;
     private static final DiscoveryNode HOT_NODE = newNode("node-hot", Collections.singleton(DiscoveryNodeRole.DATA_HOT_NODE_ROLE));
     private static final DiscoveryNode WARM_NODE = newNode("node-warm", Collections.singleton(DiscoveryNodeRole.DATA_WARM_NODE_ROLE));
     private static final DiscoveryNode COLD_NODE = newNode("node-cold", Collections.singleton(DiscoveryNodeRole.DATA_COLD_NODE_ROLE));
@@ -62,8 +59,8 @@ public class DataTierAllocationDeciderTests extends ESAllocationTestCase {
         newNode("node-content", Collections.singleton(DiscoveryNodeRole.DATA_CONTENT_NODE_ROLE));
     private static final DiscoveryNode DATA_NODE = newNode("node-data", Collections.singleton(DiscoveryNodeRole.DATA_ROLE));
 
-    private final ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ALL_SETTINGS);
-    private final DataTierAllocationDecider decider = new DataTierAllocationDecider(Settings.EMPTY, clusterSettings);
+    private final ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
+    private final DataTierAllocationDecider decider = new DataTierAllocationDecider();
     private final AllocationDeciders allocationDeciders = new AllocationDeciders(
         Arrays.asList(decider,
             new SameShardAllocationDecider(Settings.EMPTY, clusterSettings),
@@ -75,219 +72,6 @@ public class DataTierAllocationDeciderTests extends ESAllocationTestCase {
     private final ShardRouting shard = ShardRouting.newUnassigned(new ShardId("myindex", "myindex", 0), true,
         RecoverySource.EmptyStoreRecoverySource.INSTANCE, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "index created"));
 
-    static {
-        Set<Setting<?>> allSettings = new HashSet<>(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
-        allSettings.add(DataTierAllocationDecider.CLUSTER_ROUTING_REQUIRE_SETTING);
-        allSettings.add(DataTierAllocationDecider.CLUSTER_ROUTING_INCLUDE_SETTING);
-        allSettings.add(DataTierAllocationDecider.CLUSTER_ROUTING_EXCLUDE_SETTING);
-        ALL_SETTINGS = allSettings;
-    }
-
-    public void testClusterRequires() {
-        ClusterState state = prepareState(service.reroute(ClusterState.EMPTY_STATE, "initial state"));
-        RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, state.getRoutingNodes(), state,
-            null, null, 0);
-        allocation.debugDecision(true);
-        clusterSettings.applySettings(Settings.builder()
-            .put(DataTierAllocationDecider.CLUSTER_ROUTING_REQUIRE, "data_hot")
-            .build());
-        Decision d;
-        RoutingNode node;
-
-        for (DiscoveryNode n : Arrays.asList(HOT_NODE, DATA_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.YES));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.YES));
-        }
-
-        for (DiscoveryNode n : Arrays.asList(WARM_NODE, COLD_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.NO));
-            assertThat(d.getExplanation(),
-                containsString("node does not match all cluster setting [cluster.routing.allocation.require._tier] " +
-                    "tier filters [data_hot]"));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.NO));
-            assertThat(d.getExplanation(),
-                containsString("node does not match all cluster setting [cluster.routing.allocation.require._tier] " +
-                    "tier filters [data_hot]"));
-        }
-    }
-
-    public void testClusterIncludes() {
-        ClusterState state = prepareState(service.reroute(ClusterState.EMPTY_STATE, "initial state"));
-        RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, state.getRoutingNodes(), state,
-            null, null, 0);
-        allocation.debugDecision(true);
-        clusterSettings.applySettings(Settings.builder()
-            .put(DataTierAllocationDecider.CLUSTER_ROUTING_INCLUDE, "data_warm,data_cold")
-            .build());
-        Decision d;
-        RoutingNode node;
-
-        for (DiscoveryNode n : Arrays.asList(WARM_NODE, DATA_NODE, COLD_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.YES));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.YES));
-        }
-
-        for (DiscoveryNode n : Arrays.asList(HOT_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.NO));
-            assertThat(d.getExplanation(),
-                containsString("node does not match any cluster setting [cluster.routing.allocation.include._tier] " +
-                    "tier filters [data_warm,data_cold]"));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.NO));
-            assertThat(d.getExplanation(),
-                containsString("node does not match any cluster setting [cluster.routing.allocation.include._tier] " +
-                    "tier filters [data_warm,data_cold]"));
-        }
-    }
-
-
-    public void testClusterExcludes() {
-        ClusterState state = prepareState(service.reroute(ClusterState.EMPTY_STATE, "initial state"));
-        RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, state.getRoutingNodes(), state,
-            null, null, 0);
-        allocation.debugDecision(true);
-        clusterSettings.applySettings(Settings.builder()
-            .put(DataTierAllocationDecider.CLUSTER_ROUTING_EXCLUDE, "data_warm")
-            .build());
-        Decision d;
-        RoutingNode node;
-
-        for (DiscoveryNode n : Arrays.asList(WARM_NODE, DATA_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.NO));
-            assertThat(d.getExplanation(),
-                containsString("node matches any cluster setting [cluster.routing.allocation.exclude._tier] " +
-                    "tier filters [data_warm]"));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.NO));
-            assertThat(d.getExplanation(),
-                containsString("node matches any cluster setting [cluster.routing.allocation.exclude._tier] " +
-                    "tier filters [data_warm]"));
-
-        }
-
-        for (DiscoveryNode n : Arrays.asList(HOT_NODE, COLD_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(n.toString(), d.type(), equalTo(Decision.Type.YES));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(n.toString(), d.type(), equalTo(Decision.Type.YES));
-        }
-    }
-
-    public void testIndexRequires() {
-        ClusterState state = prepareState(service.reroute(ClusterState.EMPTY_STATE, "initial state"),
-            Settings.builder()
-                .put(DataTierAllocationDecider.INDEX_ROUTING_REQUIRE, "data_hot")
-                .build());
-        RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, state.getRoutingNodes(), state,
-            null, null, 0);
-        allocation.debugDecision(true);
-        Decision d;
-        RoutingNode node;
-
-        for (DiscoveryNode n : Arrays.asList(HOT_NODE, DATA_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.YES));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.YES));
-        }
-
-        for (DiscoveryNode n : Arrays.asList(WARM_NODE, COLD_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.NO));
-            assertThat(d.getExplanation(),
-                containsString("node does not match all index setting [index.routing.allocation.require._tier] tier filters [data_hot]"));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.NO));
-            assertThat(d.getExplanation(),
-                containsString("node does not match all index setting [index.routing.allocation.require._tier] tier filters [data_hot]"));
-        }
-    }
-
-    public void testIndexIncludes() {
-        ClusterState state = prepareState(service.reroute(ClusterState.EMPTY_STATE, "initial state"),
-            Settings.builder()
-                .put(DataTierAllocationDecider.INDEX_ROUTING_INCLUDE, "data_warm,data_cold")
-                .build());
-        RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, state.getRoutingNodes(), state,
-            null, null, 0);
-        allocation.debugDecision(true);
-        Decision d;
-        RoutingNode node;
-
-        for (DiscoveryNode n : Arrays.asList(WARM_NODE, DATA_NODE, COLD_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.YES));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.YES));
-        }
-
-        for (DiscoveryNode n : Arrays.asList(HOT_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.NO));
-            assertThat(d.getExplanation(),
-                containsString("node does not match any index setting [index.routing.allocation.include._tier] " +
-                    "tier filters [data_warm,data_cold]"));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.NO));
-            assertThat(d.getExplanation(),
-                containsString("node does not match any index setting [index.routing.allocation.include._tier] " +
-                    "tier filters [data_warm,data_cold]"));
-        }
-    }
-
-    public void testIndexExcludes() {
-        ClusterState state = prepareState(service.reroute(ClusterState.EMPTY_STATE, "initial state"),
-            Settings.builder()
-                .put(DataTierAllocationDecider.INDEX_ROUTING_EXCLUDE, "data_warm,data_cold")
-                .build());
-        RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, state.getRoutingNodes(), state,
-            null, null,0);
-        allocation.debugDecision(true);
-        Decision d;
-        RoutingNode node;
-
-        for (DiscoveryNode n : Arrays.asList(WARM_NODE, DATA_NODE, COLD_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.NO));
-            assertThat(d.getExplanation(),
-                containsString("node matches any index setting [index.routing.allocation.exclude._tier] " +
-                    "tier filters [data_warm,data_cold]"));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.NO));
-            assertThat(d.getExplanation(),
-                containsString("node matches any index setting [index.routing.allocation.exclude._tier] " +
-                    "tier filters [data_warm,data_cold]"));
-
-        }
-
-        for (DiscoveryNode n : Arrays.asList(HOT_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(n.toString(), d.type(), equalTo(Decision.Type.YES));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(n.toString(), d.type(), equalTo(Decision.Type.YES));
-        }
-    }
-
     public void testIndexPrefer() {
         ClusterState state = ClusterState.builder(service.reroute(ClusterState.EMPTY_STATE, "initial state"))
             .nodes(DiscoveryNodes.builder()
@@ -369,241 +153,6 @@ public class DataTierAllocationDeciderTests extends ESAllocationTestCase {
         }
     }
 
-    public void testIndexPreferWithInclude() {
-        ClusterState state = ClusterState.builder(service.reroute(ClusterState.EMPTY_STATE, "initial state"))
-            .nodes(DiscoveryNodes.builder()
-                .add(WARM_NODE)
-                .add(COLD_NODE)
-                .build())
-            .metadata(Metadata.builder()
-                .put(IndexMetadata.builder("myindex")
-                    .settings(Settings.builder()
-                        .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT)
-                        .put(IndexMetadata.SETTING_INDEX_UUID, "myindex")
-                        .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
-                        .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0)
-                        .put(DataTierAllocationDecider.INDEX_ROUTING_INCLUDE, "data_cold")
-                        .put(DataTierAllocationDecider.INDEX_ROUTING_PREFER, "data_warm,data_cold")
-                        .build()))
-                .build())
-            .build();
-        RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, state.getRoutingNodes(), state, null, null, 0);
-        allocation.debugDecision(true);
-        Decision d;
-        RoutingNode node;
-
-        for (DiscoveryNode n : Arrays.asList(HOT_NODE, WARM_NODE, CONTENT_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.NO));
-            assertThat(node.toString(), d.getExplanation(),
-                containsString("node does not match any index setting [index.routing.allocation.include._tier] tier filters [data_cold]"));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.NO));
-            assertThat(node.toString(), d.getExplanation(),
-                containsString("node does not match any index setting [index.routing.allocation.include._tier] tier filters [data_cold]"));
-        }
-
-        for (DiscoveryNode n : Arrays.asList(COLD_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.NO));
-            assertThat(node.toString(), d.getExplanation(),
-                containsString("index has a preference for tiers [data_warm,data_cold] " +
-                    "and node does not meet the required [data_warm] tier"));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.NO));
-            assertThat(node.toString(), d.getExplanation(),
-                containsString("index has a preference for tiers [data_warm,data_cold] " +
-                    "and node does not meet the required [data_warm] tier"));
-        }
-
-        for (DiscoveryNode n : Arrays.asList(DATA_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.YES));
-            assertThat(node.toString(), d.getExplanation(),
-                containsString("index has a preference for tiers [data_warm,data_cold] and node has tier [data_warm]"));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.YES));
-            assertThat(node.toString(), d.getExplanation(),
-                containsString("index has a preference for tiers [data_warm,data_cold] and node has tier [data_warm]"));
-        }
-    }
-
-    public void testIndexPreferWithExclude() {
-        ClusterState state = ClusterState.builder(service.reroute(ClusterState.EMPTY_STATE, "initial state"))
-            .nodes(DiscoveryNodes.builder()
-                .add(WARM_NODE)
-                .add(COLD_NODE)
-                .build())
-            .metadata(Metadata.builder()
-                .put(IndexMetadata.builder("myindex")
-                    .settings(Settings.builder()
-                        .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT)
-                        .put(IndexMetadata.SETTING_INDEX_UUID, "myindex")
-                        .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
-                        .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0)
-                        .put(DataTierAllocationDecider.INDEX_ROUTING_EXCLUDE, "data_warm")
-                        .put(DataTierAllocationDecider.INDEX_ROUTING_PREFER, "data_warm,data_cold")
-                        .build()))
-                .build())
-            .build();
-        RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, state.getRoutingNodes(), state, null, null, 0);
-        allocation.debugDecision(true);
-        Decision d;
-        RoutingNode node;
-
-        for (DiscoveryNode n : Arrays.asList(HOT_NODE, COLD_NODE, CONTENT_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.NO));
-            assertThat(node.toString(), d.getExplanation(),
-                containsString("index has a preference for tiers [data_warm,data_cold] " +
-                    "and node does not meet the required [data_warm] tier"));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.NO));
-            assertThat(node.toString(), d.getExplanation(),
-                containsString("index has a preference for tiers [data_warm,data_cold] " +
-                    "and node does not meet the required [data_warm] tier"));
-        }
-
-        for (DiscoveryNode n : Arrays.asList(WARM_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.NO));
-            assertThat(node.toString(), d.getExplanation(),
-                containsString("node matches any index setting [index.routing.allocation.exclude._tier] tier filters [data_warm]"));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.NO));
-            assertThat(node.toString(), d.getExplanation(),
-                containsString("node matches any index setting [index.routing.allocation.exclude._tier] tier filters [data_warm]"));
-        }
-
-        for (DiscoveryNode n : Arrays.asList(DATA_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.NO));
-            assertThat(node.toString(), d.getExplanation(),
-                containsString("node matches any index setting [index.routing.allocation.exclude._tier] tier filters [data_warm]"));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.NO));
-            assertThat(node.toString(), d.getExplanation(),
-                containsString("node matches any index setting [index.routing.allocation.exclude._tier] tier filters [data_warm]"));
-        }
-    }
-
-    public void testIndexPreferWithRequire() {
-        ClusterState state = ClusterState.builder(service.reroute(ClusterState.EMPTY_STATE, "initial state"))
-            .nodes(DiscoveryNodes.builder()
-                .add(WARM_NODE)
-                .add(COLD_NODE)
-                .build())
-            .metadata(Metadata.builder()
-                .put(IndexMetadata.builder("myindex")
-                    .settings(Settings.builder()
-                        .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT)
-                        .put(IndexMetadata.SETTING_INDEX_UUID, "myindex")
-                        .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
-                        .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0)
-                        .put(DataTierAllocationDecider.INDEX_ROUTING_REQUIRE, "data_cold")
-                        .put(DataTierAllocationDecider.INDEX_ROUTING_PREFER, "data_warm,data_cold")
-                        .build()))
-                .build())
-            .build();
-        RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, state.getRoutingNodes(), state, null, null, 0);
-        allocation.debugDecision(true);
-        Decision d;
-        RoutingNode node;
-
-        for (DiscoveryNode n : Arrays.asList(HOT_NODE, WARM_NODE, CONTENT_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.NO));
-            assertThat(node.toString(), d.getExplanation(),
-                containsString("node does not match all index setting [index.routing.allocation.require._tier] tier filters [data_cold]"));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.NO));
-            assertThat(node.toString(), d.getExplanation(),
-                containsString("node does not match all index setting [index.routing.allocation.require._tier] tier filters [data_cold]"));
-        }
-
-        for (DiscoveryNode n : Arrays.asList(COLD_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.NO));
-            assertThat(node.toString(), d.getExplanation(),
-                containsString("index has a preference for tiers [data_warm,data_cold] " +
-                    "and node does not meet the required [data_warm] tier"));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.NO));
-            assertThat(node.toString(), d.getExplanation(),
-                containsString("index has a preference for tiers [data_warm,data_cold] " +
-                    "and node does not meet the required [data_warm] tier"));
-        }
-
-        for (DiscoveryNode n : Arrays.asList(DATA_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.YES));
-            assertThat(node.toString(), d.getExplanation(),
-                containsString("index has a preference for tiers [data_warm,data_cold] and node has tier [data_warm]"));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.YES));
-            assertThat(node.toString(), d.getExplanation(),
-                containsString("index has a preference for tiers [data_warm,data_cold] and node has tier [data_warm]"));
-        }
-    }
-
-    public void testClusterAndIndex() {
-        ClusterState state = prepareState(service.reroute(ClusterState.EMPTY_STATE, "initial state"),
-            Settings.builder()
-                .put(DataTierAllocationDecider.INDEX_ROUTING_INCLUDE, "data_warm,data_cold")
-                .build());
-        RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, state.getRoutingNodes(), state,
-            null, null,0);
-        clusterSettings.applySettings(Settings.builder()
-            .put(DataTierAllocationDecider.CLUSTER_ROUTING_EXCLUDE, "data_cold")
-            .build());
-        allocation.debugDecision(true);
-        Decision d;
-        RoutingNode node;
-
-        for (DiscoveryNode n : Arrays.asList(HOT_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.NO));
-            assertThat(node.toString(), d.getExplanation(),
-                containsString("node does not match any index setting [index.routing.allocation.include._tier] " +
-                    "tier filters [data_warm,data_cold]"));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.NO));
-            assertThat(node.toString(), d.getExplanation(),
-                containsString("node does not match any index setting [index.routing.allocation.include._tier] " +
-                    "tier filters [data_warm,data_cold]"));
-        }
-
-        for (DiscoveryNode n : Arrays.asList(DATA_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.NO));
-            assertThat(d.getExplanation(),
-                containsString("node matches any cluster setting [cluster.routing.allocation.exclude._tier] tier filters [data_cold]"));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(node.toString(), d.type(), equalTo(Decision.Type.NO));
-            assertThat(d.getExplanation(),
-                containsString("node matches any cluster setting [cluster.routing.allocation.exclude._tier] tier filters [data_cold]"));
-        }
-
-        for (DiscoveryNode n : Arrays.asList(WARM_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = decider.canAllocate(shard, node, allocation);
-            assertThat(n.toString(), d.type(), equalTo(Decision.Type.YES));
-            d = decider.canRemain(shard, node, allocation);
-            assertThat(n.toString(), d.type(), equalTo(Decision.Type.YES));
-        }
-    }
-
     public void testTierNodesPresent() {
         DiscoveryNodes nodes = DiscoveryNodes.builder().build();
 
@@ -658,60 +207,6 @@ public class DataTierAllocationDeciderTests extends ESAllocationTestCase {
             equalTo(Optional.of("data_warm")));
     }
 
-    public void testExistedClusterFilters() {
-        Settings existedSettings = Settings.builder()
-            .put("cluster.routing.allocation.include._tier", "data_hot,data_warm")
-            .put("cluster.routing.allocation.exclude._tier", "data_cold")
-            .build();
-        ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ALL_SETTINGS);
-        DataTierAllocationDecider dataTierAllocationDecider = new DataTierAllocationDecider(existedSettings, clusterSettings);
-        AllocationDeciders allocationDeciders = new AllocationDeciders(
-            List.of(dataTierAllocationDecider));
-        AllocationService service = new AllocationService(allocationDeciders,
-            new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE,
-            EmptySnapshotsInfoService.INSTANCE);
-
-        ClusterState clusterState = prepareState(service.reroute(ClusterState.EMPTY_STATE, "initial state"));
-
-        RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, clusterState.getRoutingNodes(), clusterState,
-            null, null, 0);
-        allocation.debugDecision(true);
-        Decision d;
-        RoutingNode node;
-
-        for (DiscoveryNode n : Arrays.asList(HOT_NODE, WARM_NODE)) {
-            node = new RoutingNode(n.getId(), n, shard);
-            d = dataTierAllocationDecider.canAllocate(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.YES));
-            d = dataTierAllocationDecider.canRemain(shard, node, allocation);
-            assertThat(d.type(), equalTo(Decision.Type.YES));
-        }
-
-        node = new RoutingNode(DATA_NODE.getId(), DATA_NODE, shard);
-        d = dataTierAllocationDecider.canAllocate(shard, node, allocation);
-        assertThat(d.type(), equalTo(Decision.Type.NO));
-        assertThat(d.getExplanation(),
-            containsString("node matches any cluster setting [cluster.routing.allocation.exclude._tier] " +
-                "tier filters [data_cold]"));
-        d = dataTierAllocationDecider.canRemain(shard, node, allocation);
-        assertThat(d.type(), equalTo(Decision.Type.NO));
-        assertThat(d.getExplanation(),
-            containsString("node matches any cluster setting [cluster.routing.allocation.exclude._tier] " +
-                "tier filters [data_cold]"));
-
-        node = new RoutingNode(COLD_NODE.getId(), COLD_NODE, shard);
-        d = dataTierAllocationDecider.canAllocate(shard, node, allocation);
-        assertThat(d.type(), equalTo(Decision.Type.NO));
-        assertThat(d.getExplanation(),
-            containsString("node does not match any cluster setting [cluster.routing.allocation.include._tier] " +
-                "tier filters [data_hot,data_warm]"));
-        d = dataTierAllocationDecider.canRemain(shard, node, allocation);
-        assertThat(d.type(), equalTo(Decision.Type.NO));
-        assertThat(d.getExplanation(),
-            containsString("node does not match any cluster setting [cluster.routing.allocation.include._tier] " +
-                "tier filters [data_hot,data_warm]"));
-    }
-
     public void testFrozenIllegalForRegularIndices() {
         List<String> tierList = new ArrayList<>(randomSubsetOf(DataTier.ALL_DATA_TIERS));
         if (tierList.contains(DATA_FROZEN) == false) {
@@ -720,7 +215,7 @@ public class DataTierAllocationDeciderTests extends ESAllocationTestCase {
         Randomness.shuffle(tierList);
 
         String value = Strings.join(tierList, ",");
-        Setting<String> setting = randomTierSetting();
+        Setting<String> setting = DataTierAllocationDecider.INDEX_ROUTING_PREFER_SETTING;
         Settings.Builder builder = Settings.builder().put(setting.getKey(), value);
         if (randomBoolean()) {
             builder.put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), SearchableSnapshotsConstants.SNAPSHOT_DIRECTORY_FACTORY_KEY);
@@ -732,7 +227,7 @@ public class DataTierAllocationDeciderTests extends ESAllocationTestCase {
     }
 
     public void testFrozenLegalForPartialSnapshot() {
-        Setting<String> setting = randomTierSetting();
+        Setting<String> setting = DataTierAllocationDecider.INDEX_ROUTING_PREFER_SETTING;
         Settings.Builder builder = Settings.builder().put(setting.getKey(), DATA_FROZEN);
         builder.put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), SearchableSnapshotsConstants.SNAPSHOT_DIRECTORY_FACTORY_KEY);
         builder.put(SearchableSnapshotsConstants.SNAPSHOT_PARTIAL_SETTING.getKey(), true);
@@ -803,15 +298,6 @@ public class DataTierAllocationDeciderTests extends ESAllocationTestCase {
         assertThat(DataTierAllocationDecider.INDEX_ROUTING_PREFER_SETTING.get(settings), equalTo(DATA_FROZEN));
     }
 
-    public Setting<String> randomTierSetting() {
-        //noinspection unchecked
-        return randomFrom(
-            DataTierAllocationDecider.INDEX_ROUTING_EXCLUDE_SETTING,
-            DataTierAllocationDecider.INDEX_ROUTING_INCLUDE_SETTING,
-            DataTierAllocationDecider.INDEX_ROUTING_REQUIRE_SETTING,
-            DataTierAllocationDecider.INDEX_ROUTING_PREFER_SETTING);
-    }
-
     private ClusterState prepareState(ClusterState initialState) {
         return prepareState(initialState, Settings.EMPTY);
     }

+ 0 - 3
x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportMountSearchableSnapshotAction.java

@@ -71,9 +71,6 @@ public class TransportMountSearchableSnapshotAction extends TransportMasterNodeA
     RestoreSnapshotResponse> {
 
     private static final Collection<Setting<String>> DATA_TIER_ALLOCATION_SETTINGS = List.of(
-        DataTierAllocationDecider.INDEX_ROUTING_EXCLUDE_SETTING,
-        DataTierAllocationDecider.INDEX_ROUTING_INCLUDE_SETTING,
-        DataTierAllocationDecider.INDEX_ROUTING_REQUIRE_SETTING,
         DataTierAllocationDecider.INDEX_ROUTING_PREFER_SETTING
     );