|
@@ -19,20 +19,27 @@
|
|
|
|
|
|
package org.elasticsearch.cluster.routing.allocation.decider;
|
|
|
|
|
|
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
|
|
|
+import org.elasticsearch.cluster.ClusterChangedEvent;
|
|
|
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.MutableShardRouting;
|
|
|
import org.elasticsearch.cluster.routing.RoutingTable;
|
|
|
import org.elasticsearch.cluster.routing.allocation.AllocationService;
|
|
|
import org.elasticsearch.common.logging.ESLogger;
|
|
|
import org.elasticsearch.common.logging.Loggers;
|
|
|
import org.elasticsearch.common.settings.ImmutableSettings;
|
|
|
+import org.elasticsearch.common.settings.Settings;
|
|
|
+import org.elasticsearch.node.settings.NodeSettingsService;
|
|
|
import org.elasticsearch.test.ElasticsearchAllocationTestCase;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
-import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING;
|
|
|
-import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED;
|
|
|
+import java.util.EnumSet;
|
|
|
+import java.util.List;
|
|
|
+
|
|
|
+import static org.elasticsearch.cluster.routing.ShardRoutingState.*;
|
|
|
import static org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider.*;
|
|
|
import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
|
|
|
import static org.hamcrest.Matchers.equalTo;
|
|
@@ -145,4 +152,178 @@ public class EnableAllocationTests extends ElasticsearchAllocationTestCase {
|
|
|
assertThat(clusterState.readOnlyRoutingNodes().shardsWithState("disabled", STARTED).size(), equalTo(0));
|
|
|
}
|
|
|
|
|
|
+
|
|
|
+
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testEnableClusterBalance() {
|
|
|
+ final boolean useClusterSetting = randomBoolean();
|
|
|
+ final Rebalance allowedOnes = RandomPicks.randomFrom(getRandom(), EnumSet.of(Rebalance.PRIMARIES, Rebalance.REPLICAS, Rebalance.ALL));
|
|
|
+ Settings build = settingsBuilder()
|
|
|
+ .put(CLUSTER_ROUTING_REBALANCE_ENABLE, useClusterSetting ? Rebalance.NONE: RandomPicks.randomFrom(getRandom(), Rebalance.values())) // index settings override cluster settings
|
|
|
+ .put(ConcurrentRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE, 3)
|
|
|
+ .build();
|
|
|
+ NodeSettingsService nodeSettingsService = new NodeSettingsService(build);
|
|
|
+ AllocationService strategy = createAllocationService(build, nodeSettingsService, getRandom());
|
|
|
+ Settings indexSettings = useClusterSetting ? ImmutableSettings.EMPTY : settingsBuilder().put(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE, Rebalance.NONE).build();
|
|
|
+
|
|
|
+ logger.info("Building initial routing table");
|
|
|
+ MetaData metaData = MetaData.builder()
|
|
|
+ .put(IndexMetaData.builder("test").settings(indexSettings).numberOfShards(3).numberOfReplicas(1))
|
|
|
+
|
|
|
+ .put(IndexMetaData.builder("always_disabled").settings(settingsBuilder().put(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE, Rebalance.NONE)).numberOfShards(1).numberOfReplicas(1))
|
|
|
+ .build();
|
|
|
+
|
|
|
+ RoutingTable routingTable = RoutingTable.builder()
|
|
|
+ .addAsNew(metaData.index("test"))
|
|
|
+ .addAsNew(metaData.index("always_disabled"))
|
|
|
+ .build();
|
|
|
+
|
|
|
+ ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).metaData(metaData).routingTable(routingTable).build();
|
|
|
+
|
|
|
+ logger.info("--> adding one 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.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(4));
|
|
|
+ logger.info("--> start the shards (primaries)");
|
|
|
+ routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
|
|
|
+ clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
|
|
|
+ assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(4));
|
|
|
+ assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(4));
|
|
|
+
|
|
|
+ routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
|
|
|
+ clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
|
|
|
+ assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(8));
|
|
|
+ assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
|
|
|
+
|
|
|
+ logger.info("--> adding one nodes and do rerouting");
|
|
|
+ clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
|
|
|
+ .put(newNode("node1"))
|
|
|
+ .put(newNode("node2"))
|
|
|
+ .put(newNode("node3"))
|
|
|
+ ).build();
|
|
|
+ ClusterState prevState = clusterState;
|
|
|
+ routingTable = strategy.reroute(clusterState).routingTable();
|
|
|
+ clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
|
|
|
+ assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(8));
|
|
|
+ assertThat(clusterState.routingNodes().shardsWithState(RELOCATING).size(), equalTo(0));
|
|
|
+
|
|
|
+ if (useClusterSetting) {
|
|
|
+ prevState = clusterState;
|
|
|
+ clusterState = ClusterState.builder(clusterState).metaData(MetaData.builder(metaData).transientSettings(settingsBuilder()
|
|
|
+ .put(CLUSTER_ROUTING_REBALANCE_ENABLE, allowedOnes)
|
|
|
+ .build())).build();
|
|
|
+ } else {
|
|
|
+ prevState = clusterState;
|
|
|
+ IndexMetaData meta = clusterState.getMetaData().index("test");
|
|
|
+ IndexMetaData meta1 = clusterState.getMetaData().index("always_disabled");
|
|
|
+ clusterState = ClusterState.builder(clusterState).metaData(MetaData.builder(metaData).removeAllIndices().put(IndexMetaData.builder(meta1))
|
|
|
+ .put(IndexMetaData.builder(meta).settings(settingsBuilder().put(meta.getSettings()).put(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE, allowedOnes).build())))
|
|
|
+ .build();
|
|
|
+
|
|
|
+ }
|
|
|
+ nodeSettingsService.clusterChanged(new ClusterChangedEvent("foo", clusterState, prevState));
|
|
|
+ routingTable = strategy.reroute(clusterState).routingTable();
|
|
|
+ clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
|
|
|
+ assertThat("expected 6 shards to be started 2 to relocate useClusterSettings: " + useClusterSetting, clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(6));
|
|
|
+ assertThat("expected 2 shards to relocate useClusterSettings: " + useClusterSetting, clusterState.routingNodes().shardsWithState(RELOCATING).size(), equalTo(2));
|
|
|
+ List<MutableShardRouting> mutableShardRoutings = clusterState.routingNodes().shardsWithState(RELOCATING);
|
|
|
+ switch (allowedOnes) {
|
|
|
+ case PRIMARIES:
|
|
|
+ for (MutableShardRouting routing : mutableShardRoutings) {
|
|
|
+ assertTrue("only primaries are allowed to relocate", routing.primary());
|
|
|
+ assertThat("only test index can rebalance", routing.getIndex(), equalTo("test"));
|
|
|
+ }
|
|
|
+ break;
|
|
|
+ case REPLICAS:
|
|
|
+ for (MutableShardRouting routing : mutableShardRoutings) {
|
|
|
+ assertFalse("only replicas are allowed to relocate", routing.primary());
|
|
|
+ assertThat("only test index can rebalance", routing.getIndex(), equalTo("test"));
|
|
|
+ }
|
|
|
+ break;
|
|
|
+ case ALL:
|
|
|
+ for (MutableShardRouting routing : mutableShardRoutings) {
|
|
|
+ assertThat("only test index can rebalance", routing.getIndex(), equalTo("test"));
|
|
|
+ }
|
|
|
+ break;
|
|
|
+ default:
|
|
|
+ fail("only replicas, primaries or all are allowed");
|
|
|
+ }
|
|
|
+ routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
|
|
|
+ clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
|
|
|
+ assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(8));
|
|
|
+ assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testEnableClusterBalanceNoReplicas() {
|
|
|
+ final boolean useClusterSetting = randomBoolean();
|
|
|
+ Settings build = settingsBuilder()
|
|
|
+ .put(CLUSTER_ROUTING_REBALANCE_ENABLE, useClusterSetting ? Rebalance.NONE: RandomPicks.randomFrom(getRandom(), Rebalance.values())) // index settings override cluster settings
|
|
|
+ .put(ConcurrentRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE, 3)
|
|
|
+ .build();
|
|
|
+ NodeSettingsService nodeSettingsService = new NodeSettingsService(build);
|
|
|
+ AllocationService strategy = createAllocationService(build, nodeSettingsService, getRandom());
|
|
|
+ Settings indexSettings = useClusterSetting ? ImmutableSettings.EMPTY : settingsBuilder().put(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE, Rebalance.NONE).build();
|
|
|
+
|
|
|
+ logger.info("Building initial routing table");
|
|
|
+ MetaData metaData = MetaData.builder()
|
|
|
+ .put(IndexMetaData.builder("test").settings(indexSettings).numberOfShards(6).numberOfReplicas(0))
|
|
|
+ .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 one 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.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(6));
|
|
|
+ logger.info("--> start the shards (primaries)");
|
|
|
+ routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
|
|
|
+ clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
|
|
|
+ assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(6));
|
|
|
+ assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
|
|
|
+
|
|
|
+ logger.info("--> adding one nodes and do rerouting");
|
|
|
+ clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
|
|
|
+ .put(newNode("node1"))
|
|
|
+ .put(newNode("node2"))
|
|
|
+ .put(newNode("node3"))
|
|
|
+ ).build();
|
|
|
+ ClusterState prevState = clusterState;
|
|
|
+ routingTable = strategy.reroute(clusterState).routingTable();
|
|
|
+ clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
|
|
|
+ assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(6));
|
|
|
+ assertThat(clusterState.routingNodes().shardsWithState(RELOCATING).size(), equalTo(0));
|
|
|
+ if (useClusterSetting) {
|
|
|
+ prevState = clusterState;
|
|
|
+ clusterState = ClusterState.builder(clusterState).metaData(MetaData.builder(metaData).transientSettings(settingsBuilder()
|
|
|
+ .put(CLUSTER_ROUTING_REBALANCE_ENABLE, randomBoolean() ? Rebalance.PRIMARIES : Rebalance.ALL)
|
|
|
+ .build())).build();
|
|
|
+ } else {
|
|
|
+ prevState = clusterState;
|
|
|
+ IndexMetaData meta = clusterState.getMetaData().index("test");
|
|
|
+ clusterState = ClusterState.builder(clusterState).metaData(MetaData.builder(metaData).removeAllIndices()
|
|
|
+ .put(IndexMetaData.builder(meta).settings(settingsBuilder().put(meta.getSettings()).put(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE, randomBoolean() ? Rebalance.PRIMARIES : Rebalance.ALL).build()))).build();
|
|
|
+ }
|
|
|
+ nodeSettingsService.clusterChanged(new ClusterChangedEvent("foo", clusterState, prevState));
|
|
|
+ routingTable = strategy.reroute(clusterState).routingTable();
|
|
|
+ clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
|
|
|
+ assertThat("expected 4 primaries to be started and 2 to relocate useClusterSettings: " + useClusterSetting, clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(4));
|
|
|
+ assertThat("expected 2 primaries to relocate useClusterSettings: " + useClusterSetting, clusterState.routingNodes().shardsWithState(RELOCATING).size(), equalTo(2));
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
}
|