Browse Source

Remove DisableAllocationDecider

The `EnableAllocatiorDecider` has replaced the
`DisableAllocationDecider`, which has been deprecated. It can now be
removed.
Lee Hinman 10 years ago
parent
commit
ffb2f75305

+ 1 - 9
core/src/main/java/org/elasticsearch/cluster/ClusterModule.java

@@ -48,7 +48,6 @@ import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
 import org.elasticsearch.cluster.routing.allocation.decider.AwarenessAllocationDecider;
 import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider;
 import org.elasticsearch.cluster.routing.allocation.decider.ConcurrentRebalanceAllocationDecider;
-import org.elasticsearch.cluster.routing.allocation.decider.DisableAllocationDecider;
 import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider;
 import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
 import org.elasticsearch.cluster.routing.allocation.decider.FilterAllocationDecider;
@@ -114,8 +113,7 @@ public class ClusterModule extends AbstractModule {
             RebalanceOnlyWhenActiveAllocationDecider.class,
             ClusterRebalanceAllocationDecider.class,
             ConcurrentRebalanceAllocationDecider.class,
-            EnableAllocationDecider.class, // new enable allocation logic should proceed old disable allocation logic
-            DisableAllocationDecider.class,
+            EnableAllocationDecider.class,
             AwarenessAllocationDecider.class,
             ShardsLimitAllocationDecider.class,
             NodeVersionAllocationDecider.class,
@@ -155,9 +153,6 @@ public class ClusterModule extends AbstractModule {
         registerClusterDynamicSetting(ConcurrentRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE, Validator.INTEGER);
         registerClusterDynamicSetting(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, Validator.EMPTY);
         registerClusterDynamicSetting(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE, Validator.EMPTY);
-        registerClusterDynamicSetting(DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_NEW_ALLOCATION, Validator.EMPTY);
-        registerClusterDynamicSetting(DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_ALLOCATION, Validator.EMPTY);
-        registerClusterDynamicSetting(DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_REPLICA_ALLOCATION, Validator.EMPTY);
         registerClusterDynamicSetting(ZenDiscovery.SETTING_REJOIN_ON_MASTER_GONE, Validator.BOOLEAN);
         registerClusterDynamicSetting(DiscoverySettings.NO_MASTER_BLOCK, Validator.EMPTY);
         registerClusterDynamicSetting(FilterAllocationDecider.CLUSTER_ROUTING_INCLUDE_GROUP + "*", Validator.EMPTY);
@@ -220,9 +215,6 @@ public class ClusterModule extends AbstractModule {
         registerIndexDynamicSetting(FilterAllocationDecider.INDEX_ROUTING_EXCLUDE_GROUP + "*", Validator.EMPTY);
         registerIndexDynamicSetting(EnableAllocationDecider.INDEX_ROUTING_ALLOCATION_ENABLE, Validator.EMPTY);
         registerIndexDynamicSetting(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE, Validator.EMPTY);
-        registerIndexDynamicSetting(DisableAllocationDecider.INDEX_ROUTING_ALLOCATION_DISABLE_ALLOCATION, Validator.EMPTY);
-        registerIndexDynamicSetting(DisableAllocationDecider.INDEX_ROUTING_ALLOCATION_DISABLE_NEW_ALLOCATION, Validator.EMPTY);
-        registerIndexDynamicSetting(DisableAllocationDecider.INDEX_ROUTING_ALLOCATION_DISABLE_REPLICA_ALLOCATION, Validator.EMPTY);
         registerIndexDynamicSetting(TranslogConfig.INDEX_TRANSLOG_FS_TYPE, Validator.EMPTY);
         registerIndexDynamicSetting(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, Validator.NON_NEGATIVE_INTEGER);
         registerIndexDynamicSetting(IndexMetaData.SETTING_AUTO_EXPAND_REPLICAS, Validator.EMPTY);

+ 0 - 133
core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DisableAllocationDecider.java

@@ -1,133 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch licenses this file to you under
- * the Apache License, Version 2.0 (the "License"); you may
- * not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.elasticsearch.cluster.routing.allocation.decider;
-
-import org.elasticsearch.cluster.routing.RoutingNode;
-import org.elasticsearch.cluster.routing.ShardRouting;
-import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
-import org.elasticsearch.common.inject.Inject;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.node.settings.NodeSettingsService;
-
-/**
- * This {@link AllocationDecider} prevents cluster-wide shard allocations. The
- * behavior of this {@link AllocationDecider} can be changed in real-time via
- * the cluster settings API. It respects the following settings:
- * <ul>
- * <li><tt>cluster.routing.allocation.disable_new_allocation</tt> - if set to
- * <code>true</code> no new shard-allocation are allowed. Note: this setting is
- * only applied if the allocated shard is a primary and it has not been
- * allocated before the this setting was applied.</li>
- * <p/>
- * <li><tt>cluster.routing.allocation.disable_allocation</tt> - if set to
- * <code>true</code> cluster wide allocations are disabled</li>
- * <p/>
- * <li><tt>cluster.routing.allocation.disable_replica_allocation</tt> - if set
- * to <code>true</code> cluster wide replica allocations are disabled while
- * primary shards can still be allocated</li>
- * </ul>
- * <p/>
- * <p>
- * Note: all of the above settings might be ignored if the allocation happens on
- * a shard that explicitly ignores disabled allocations via
- * {@link RoutingAllocation#ignoreDisable()}. Which is set if allocation are
- * explicit.
- * </p>
- *
- * @deprecated In favour for {@link EnableAllocationDecider}.
- */
-@Deprecated
-public class DisableAllocationDecider extends AllocationDecider {
-
-    public static final String NAME = "disable";
-
-    public static final String CLUSTER_ROUTING_ALLOCATION_DISABLE_NEW_ALLOCATION = "cluster.routing.allocation.disable_new_allocation";
-    public static final String CLUSTER_ROUTING_ALLOCATION_DISABLE_ALLOCATION = "cluster.routing.allocation.disable_allocation";
-    public static final String CLUSTER_ROUTING_ALLOCATION_DISABLE_REPLICA_ALLOCATION = "cluster.routing.allocation.disable_replica_allocation";
-
-    public static final String INDEX_ROUTING_ALLOCATION_DISABLE_NEW_ALLOCATION = "index.routing.allocation.disable_new_allocation";
-    public static final String INDEX_ROUTING_ALLOCATION_DISABLE_ALLOCATION = "index.routing.allocation.disable_allocation";
-    public static final String INDEX_ROUTING_ALLOCATION_DISABLE_REPLICA_ALLOCATION = "index.routing.allocation.disable_replica_allocation";
-
-    class ApplySettings implements NodeSettingsService.Listener {
-        @Override
-        public void onRefreshSettings(Settings settings) {
-            boolean disableNewAllocation = settings.getAsBoolean(CLUSTER_ROUTING_ALLOCATION_DISABLE_NEW_ALLOCATION, DisableAllocationDecider.this.disableNewAllocation);
-            if (disableNewAllocation != DisableAllocationDecider.this.disableNewAllocation) {
-                logger.info("updating [cluster.routing.allocation.disable_new_allocation] from [{}] to [{}]", DisableAllocationDecider.this.disableNewAllocation, disableNewAllocation);
-                DisableAllocationDecider.this.disableNewAllocation = disableNewAllocation;
-            }
-
-            boolean disableAllocation = settings.getAsBoolean(CLUSTER_ROUTING_ALLOCATION_DISABLE_ALLOCATION, DisableAllocationDecider.this.disableAllocation);
-            if (disableAllocation != DisableAllocationDecider.this.disableAllocation) {
-                logger.info("updating [cluster.routing.allocation.disable_allocation] from [{}] to [{}]", DisableAllocationDecider.this.disableAllocation, disableAllocation);
-                DisableAllocationDecider.this.disableAllocation = disableAllocation;
-            }
-
-            boolean disableReplicaAllocation = settings.getAsBoolean(CLUSTER_ROUTING_ALLOCATION_DISABLE_REPLICA_ALLOCATION, DisableAllocationDecider.this.disableReplicaAllocation);
-            if (disableReplicaAllocation != DisableAllocationDecider.this.disableReplicaAllocation) {
-                logger.info("updating [cluster.routing.allocation.disable_replica_allocation] from [{}] to [{}]", DisableAllocationDecider.this.disableReplicaAllocation, disableReplicaAllocation);
-                DisableAllocationDecider.this.disableReplicaAllocation = disableReplicaAllocation;
-            }
-        }
-    }
-
-    private volatile boolean disableNewAllocation;
-    private volatile boolean disableAllocation;
-    private volatile boolean disableReplicaAllocation;
-
-    @Inject
-    public DisableAllocationDecider(Settings settings, NodeSettingsService nodeSettingsService) {
-        super(settings);
-        this.disableNewAllocation = settings.getAsBoolean(CLUSTER_ROUTING_ALLOCATION_DISABLE_NEW_ALLOCATION, false);
-        this.disableAllocation = settings.getAsBoolean(CLUSTER_ROUTING_ALLOCATION_DISABLE_ALLOCATION, false);
-        this.disableReplicaAllocation = settings.getAsBoolean(CLUSTER_ROUTING_ALLOCATION_DISABLE_REPLICA_ALLOCATION, false);
-
-        nodeSettingsService.addListener(new ApplySettings());
-    }
-
-    @Override
-    public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) {
-        if (allocation.ignoreDisable()) {
-            return allocation.decision(Decision.YES, NAME, "allocation disabling is ignored");
-        }
-        Settings indexSettings = allocation.routingNodes().metaData().index(shardRouting.index()).settings();
-        if (shardRouting.primary() && shardRouting.allocatedPostIndexCreate() == false) {
-            // if its primary, and it hasn't been allocated post API (meaning its a "fresh newly created shard"), only disable allocation
-            // on a special disable allocation flag
-            if (indexSettings.getAsBoolean(INDEX_ROUTING_ALLOCATION_DISABLE_NEW_ALLOCATION, disableNewAllocation)) {
-                return allocation.decision(Decision.NO, NAME, "new primary allocation is disabled");
-            } else {
-                return allocation.decision(Decision.YES, NAME, "new primary allocation is enabled");
-            }
-        }
-        if (indexSettings.getAsBoolean(INDEX_ROUTING_ALLOCATION_DISABLE_ALLOCATION, disableAllocation)) {
-            return allocation.decision(Decision.NO, NAME, "all allocation is disabled");
-        }
-        if (indexSettings.getAsBoolean(INDEX_ROUTING_ALLOCATION_DISABLE_REPLICA_ALLOCATION, disableReplicaAllocation)) {
-            if (shardRouting.primary()) {
-                return allocation.decision(Decision.YES, NAME, "primary allocation is enabled");
-            } else {
-                return allocation.decision(Decision.NO, NAME, "replica allocation is disabled");
-            }
-        }
-        return allocation.decision(Decision.YES, NAME, "all allocation is enabled");
-    }
-}

+ 5 - 5
core/src/test/java/org/elasticsearch/cluster/allocation/ClusterRerouteIT.java

@@ -32,7 +32,7 @@ import org.elasticsearch.cluster.routing.allocation.RoutingExplanations;
 import org.elasticsearch.cluster.routing.allocation.command.AllocateAllocationCommand;
 import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand;
 import org.elasticsearch.cluster.routing.allocation.decider.Decision;
-import org.elasticsearch.cluster.routing.allocation.decider.DisableAllocationDecider;
+import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
 import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider.Allocation;
 import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider;
 import org.elasticsearch.common.Priority;
@@ -71,8 +71,8 @@ public class ClusterRerouteIT extends ESIntegTestCase {
     @Test
     public void rerouteWithCommands_disableAllocationSettings() throws Exception {
         Settings commonSettings = settingsBuilder()
-                .put(DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_NEW_ALLOCATION, true)
-                .put(DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_ALLOCATION, true)
+                .put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, "none")
+                .put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE, "none")
                 .build();
         rerouteWithCommands(commonSettings);
     }
@@ -149,8 +149,8 @@ public class ClusterRerouteIT extends ESIntegTestCase {
     @Test
     public void rerouteWithAllocateLocalGateway_disableAllocationSettings() throws Exception {
         Settings commonSettings = settingsBuilder()
-                .put(DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_NEW_ALLOCATION, true)
-                .put(DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_ALLOCATION, true)
+                .put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, "none")
+                .put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE, "none")
                 .build();
         rerouteWithAllocateLocalGateway(commonSettings);
     }

+ 5 - 5
core/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationCommandsTests.java

@@ -31,7 +31,7 @@ import org.elasticsearch.cluster.routing.allocation.command.AllocateAllocationCo
 import org.elasticsearch.cluster.routing.allocation.command.AllocationCommands;
 import org.elasticsearch.cluster.routing.allocation.command.CancelAllocationCommand;
 import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand;
-import org.elasticsearch.cluster.routing.allocation.decider.DisableAllocationDecider;
+import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
 import org.elasticsearch.common.io.stream.BytesStreamOutput;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.logging.ESLogger;
@@ -100,8 +100,8 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
     @Test
     public void allocateCommand() {
         AllocationService allocation = createAllocationService(settingsBuilder()
-                .put(DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_NEW_ALLOCATION, true)
-                .put(DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_ALLOCATION, true)
+                .put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, "none")
+                .put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE, "none")
                 .build());
 
         logger.info("--> building initial routing table");
@@ -189,8 +189,8 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
     @Test
     public void cancelCommand() {
         AllocationService allocation = createAllocationService(settingsBuilder()
-                .put(DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_NEW_ALLOCATION, true)
-                .put(DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_ALLOCATION, true)
+                .put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, "none")
+                .put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE, "none")
                 .build());
 
         logger.info("--> building initial routing table");

+ 0 - 146
core/src/test/java/org/elasticsearch/cluster/routing/allocation/DisableAllocationTests.java

@@ -1,146 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch licenses this file to you under
- * the Apache License, Version 2.0 (the "License"); you may
- * not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.elasticsearch.cluster.routing.allocation;
-
-import org.elasticsearch.Version;
-import org.elasticsearch.cluster.ClusterState;
-import org.elasticsearch.cluster.metadata.IndexMetaData;
-import org.elasticsearch.cluster.metadata.MetaData;
-import org.elasticsearch.cluster.node.DiscoveryNodes;
-import org.elasticsearch.cluster.routing.RoutingTable;
-import org.elasticsearch.cluster.routing.allocation.decider.DisableAllocationDecider;
-import org.elasticsearch.common.logging.ESLogger;
-import org.elasticsearch.common.logging.Loggers;
-import org.elasticsearch.test.ESAllocationTestCase;
-import org.junit.Test;
-
-import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING;
-import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED;
-import static org.elasticsearch.common.settings.Settings.settingsBuilder;
-import static org.hamcrest.Matchers.equalTo;
-
-/**
- */
-public class DisableAllocationTests extends ESAllocationTestCase {
-
-    private final ESLogger logger = Loggers.getLogger(DisableAllocationTests.class);
-
-    @Test
-    public void testClusterDisableAllocation() {
-        AllocationService strategy = createAllocationService(settingsBuilder()
-                .put(DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_NEW_ALLOCATION, true)
-                .put(DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_ALLOCATION, true)
-                .build());
-
-        logger.info("Building initial routing table");
-
-        MetaData metaData = MetaData.builder()
-                .put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
-                .build();
-
-        RoutingTable routingTable = RoutingTable.builder()
-                .addAsNew(metaData.index("test"))
-                .build();
-
-        ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).metaData(metaData).routingTable(routingTable).build();
-
-        logger.info("--> adding two nodes and do rerouting");
-        clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
-                .put(newNode("node1"))
-                .put(newNode("node2"))
-        ).build();
-        routingTable = strategy.reroute(clusterState).routingTable();
-        clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
-        assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
-
-    }
-
-    @Test
-    public void testClusterDisableReplicaAllocation() {
-        AllocationService strategy = createAllocationService(settingsBuilder()
-                .put("cluster.routing.allocation.disable_replica_allocation", true)
-                .build());
-
-        logger.info("Building initial routing table");
-
-        MetaData metaData = MetaData.builder()
-                .put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
-                .build();
-
-        RoutingTable routingTable = RoutingTable.builder()
-                .addAsNew(metaData.index("test"))
-                .build();
-
-        ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).metaData(metaData).routingTable(routingTable).build();
-
-        logger.info("--> adding two nodes do rerouting");
-        clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
-                .put(newNode("node1"))
-                .put(newNode("node2"))
-        ).build();
-        routingTable = strategy.reroute(clusterState).routingTable();
-        clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
-        assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
-
-        logger.info("--> start the shards (primaries)");
-        routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
-        clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
-
-        assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
-    }
-
-    @Test
-    public void testIndexDisableAllocation() {
-        AllocationService strategy = createAllocationService(settingsBuilder()
-                .build());
-
-        MetaData metaData = MetaData.builder()
-                .put(IndexMetaData.builder("disabled").settings(settings(Version.CURRENT).put(DisableAllocationDecider.INDEX_ROUTING_ALLOCATION_DISABLE_ALLOCATION, true).put(DisableAllocationDecider.INDEX_ROUTING_ALLOCATION_DISABLE_NEW_ALLOCATION, true)).numberOfShards(1).numberOfReplicas(1))
-                .put(IndexMetaData.builder("enabled").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
-                .build();
-
-        RoutingTable routingTable = RoutingTable.builder()
-                .addAsNew(metaData.index("disabled"))
-                .addAsNew(metaData.index("enabled"))
-                .build();
-
-        ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).metaData(metaData).routingTable(routingTable).build();
-
-        logger.info("--> adding two nodes and do rerouting");
-        clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
-                .put(newNode("node1"))
-                .put(newNode("node2"))
-        ).build();
-        routingTable = strategy.reroute(clusterState).routingTable();
-        clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
-        assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
-        logger.info("--> start the shards (primaries)");
-        routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
-        clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
-        logger.info("--> start the shards (replicas)");
-        routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
-        clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
-
-        logger.info("--> verify only enabled index has been routed");
-        assertThat(clusterState.getRoutingNodes().shardsWithState("enabled", STARTED).size(), equalTo(2));
-        assertThat(clusterState.getRoutingNodes().shardsWithState("disabled", STARTED).size(), equalTo(0));
-    }
-
-}

+ 3 - 3
core/src/test/java/org/elasticsearch/cluster/settings/ClusterSettingsIT.java

@@ -24,7 +24,7 @@ import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResp
 import org.elasticsearch.cluster.ClusterName;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.cluster.metadata.MetaData;
-import org.elasticsearch.cluster.routing.allocation.decider.DisableAllocationDecider;
+import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.common.util.concurrent.EsExecutors;
@@ -63,8 +63,8 @@ public class ClusterSettingsIT extends ESIntegTestCase {
         String key1 = IndicesStore.INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC;
         int value1 = 10;
 
-        String key2 = DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_ALLOCATION;
-        boolean value2 = true;
+        String key2 = EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE;
+        boolean value2 = false;
 
         Settings transientSettings1 = Settings.builder().put(key1, value1, ByteSizeUnit.BYTES).build();
         Settings persistentSettings1 = Settings.builder().put(key2, value2).build();

+ 3 - 3
core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerIT.java

@@ -27,6 +27,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.cluster.routing.ShardRouting;
 import org.elasticsearch.cluster.routing.ShardRoutingState;
 import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand;
+import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
 import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.settings.Settings;
@@ -48,7 +49,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
 import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
-import static org.elasticsearch.cluster.routing.allocation.decider.DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_ALLOCATION;
 import static org.elasticsearch.common.settings.Settings.builder;
 import static org.elasticsearch.index.shard.IndexShardState.*;
 import static org.elasticsearch.test.ESIntegTestCase.ClusterScope;
@@ -163,14 +163,14 @@ public class IndicesLifecycleListenerIT extends ESIntegTestCase {
         //add a node: 3 out of the 6 shards will be relocated to it
         //disable allocation before starting a new node, as we need to register the listener first
         assertAcked(client().admin().cluster().prepareUpdateSettings()
-                .setPersistentSettings(builder().put(CLUSTER_ROUTING_ALLOCATION_DISABLE_ALLOCATION, true)));
+                .setPersistentSettings(builder().put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, "none")));
         String node2 = internalCluster().startNode();
         IndexShardStateChangeListener stateChangeListenerNode2 = new IndexShardStateChangeListener();
         //add a listener that keeps track of the shard state changes
         internalCluster().getInstance(IndicesLifecycle.class, node2).addListener(stateChangeListenerNode2);
         //re-enable allocation
         assertAcked(client().admin().cluster().prepareUpdateSettings()
-                .setPersistentSettings(builder().put(CLUSTER_ROUTING_ALLOCATION_DISABLE_ALLOCATION, false)));
+                .setPersistentSettings(builder().put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, "all")));
         ensureGreen();
 
         //the 3 relocated shards get closed on the first node

+ 2 - 2
docs/reference/setup/backup.asciidoc

@@ -50,7 +50,7 @@ while the backup is in process:
 PUT /_cluster/settings
 {
   "transient": {
-    "cluster.routing.allocation.disable_allocation": "true"
+    "cluster.routing.allocation.enable": "none"
   }
 }
 -----------------------------------
@@ -79,7 +79,7 @@ PUT /_all/_settings
 PUT /_cluster/settings
 {
   "transient": {
-    "cluster.routing.allocation.disable_allocation": "false"
+    "cluster.routing.allocation.enable": "all"
   }
 }
 -----------------------------------