diff --git a/docs/reference/migration/migrate_7_4.asciidoc b/docs/reference/migration/migrate_7_4.asciidoc index ebfca7d25c111..2608c3bcccab4 100644 --- a/docs/reference/migration/migrate_7_4.asciidoc +++ b/docs/reference/migration/migrate_7_4.asciidoc @@ -67,4 +67,27 @@ unsupported on buckets created after September 30th 2020. Starting in version 7.4, a `+` in a URL will be encoded as `%2B` by all REST API functionality. Prior versions handled a `+` as a single space. If your application requires handling `+` as a single space you can return to the old behaviour by setting the system property `es.rest.url_plus_as_space` to `true`. Note that this behaviour is deprecated and setting this system property to `true` will cease -to be supported in version 8. \ No newline at end of file +to be supported in version 8. + +[float] +[[breaking_74_cluster_changes]] +=== Cluster changes + +[float] +==== Rerouting after starting a shard runs at lower priority + +After starting each shard the elected master node must perform a reroute to +search for other shards that could be allocated. In particular, when creating +an index it is this task that allocates the replicas once the primaries have +started. In versions prior to 7.4 this task runs at priority `URGENT`, but +starting in version 7.4 its priority is reduced to `NORMAL`. In a +well-configured cluster this reduces the amount of work the master must do, but +means that a cluster with a master that is overloaded with other tasks at +`HIGH` or `URGENT` priority may take longer to allocate all replicas. + +Additionally, before 7.4 the `GET +_cluster_health?wait_for_no_initializing_shards` and `GET +_cluster/health?wait_for_no_relocating_shards` APIs would return only once all +pending reroutes have completed too, but starting in version 7.4 if you want to +wait for the rerouting process to completely finish you should add the +`wait_for_events=languid` query parameter when calling these APIs. diff --git a/server/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java b/server/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java index 75e5445ae5ef7..af72dc9f00d21 100644 --- a/server/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java +++ b/server/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java @@ -48,6 +48,7 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.node.NodeClosedException; @@ -72,6 +73,7 @@ import java.util.Objects; import java.util.Set; import java.util.function.Predicate; +import java.util.function.Supplier; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_PRIMARY_TERM; @@ -82,10 +84,34 @@ public class ShardStateAction { public static final String SHARD_STARTED_ACTION_NAME = "internal:cluster/shard/started"; public static final String SHARD_FAILED_ACTION_NAME = "internal:cluster/shard/failure"; + /** + * Adjusts the priority of the followup reroute task. NORMAL is right for reasonable clusters, but in a badly configured cluster it may + * be necessary to raise this higher to recover the older behaviour of rerouting after processing every shard-started task. Deliberately + * undocumented, since this is a last-resort escape hatch for experts rather than something we want to expose to anyone, and deprecated + * since we will remove it once we have confirmed from experience that this priority is appropriate in all cases. + */ + public static final Setting FOLLOW_UP_REROUTE_PRIORITY_SETTING + = new Setting<>("cluster.routing.allocation.shard_state.reroute.priority", Priority.NORMAL.toString(), + ShardStateAction::parseReroutePriority, Setting.Property.NodeScope, Setting.Property.Dynamic, Setting.Property.Deprecated); + + private static Priority parseReroutePriority(String priorityString) { + final Priority priority = Priority.valueOf(priorityString.toUpperCase(Locale.ROOT)); + switch (priority) { + case NORMAL: + case HIGH: + case URGENT: + return priority; + } + throw new IllegalArgumentException( + "priority [" + priority + "] not supported for [" + FOLLOW_UP_REROUTE_PRIORITY_SETTING.getKey() + "]"); + } + private final TransportService transportService; private final ClusterService clusterService; private final ThreadPool threadPool; + private volatile Priority followUpRerouteTaskPriority; + // a list of shards that failed during replication // we keep track of these shards in order to avoid sending duplicate failed shard requests for a single failing shard. private final TransportRequestDeduplicator remoteFailedShardsDeduplicator = new TransportRequestDeduplicator<>(); @@ -97,11 +123,18 @@ public ShardStateAction(ClusterService clusterService, TransportService transpor this.clusterService = clusterService; this.threadPool = threadPool; + followUpRerouteTaskPriority = FOLLOW_UP_REROUTE_PRIORITY_SETTING.get(clusterService.getSettings()); + clusterService.getClusterSettings().addSettingsUpdateConsumer(FOLLOW_UP_REROUTE_PRIORITY_SETTING, + this::setFollowUpRerouteTaskPriority); + transportService.registerRequestHandler(SHARD_STARTED_ACTION_NAME, ThreadPool.Names.SAME, StartedShardEntry::new, - new ShardStartedTransportHandler(clusterService, new ShardStartedClusterStateTaskExecutor(allocationService, logger), logger)); + new ShardStartedTransportHandler(clusterService, + new ShardStartedClusterStateTaskExecutor(allocationService, rerouteService, () -> followUpRerouteTaskPriority, logger), + logger)); transportService.registerRequestHandler(SHARD_FAILED_ACTION_NAME, ThreadPool.Names.SAME, FailedShardEntry::new, new ShardFailedTransportHandler(clusterService, - new ShardFailedClusterStateTaskExecutor(allocationService, rerouteService, logger), logger)); + new ShardFailedClusterStateTaskExecutor(allocationService, rerouteService, () -> followUpRerouteTaskPriority, logger), + logger)); } private void sendShardAction(final String actionName, final ClusterState currentState, @@ -218,6 +251,10 @@ public void onTimeout(TimeValue timeout) { }, changePredicate); } + private void setFollowUpRerouteTaskPriority(Priority followUpRerouteTaskPriority) { + this.followUpRerouteTaskPriority = followUpRerouteTaskPriority; + } + private static class ShardFailedTransportHandler implements TransportRequestHandler { private final ClusterService clusterService; private final ShardFailedClusterStateTaskExecutor shardFailedClusterStateTaskExecutor; @@ -285,11 +322,14 @@ public static class ShardFailedClusterStateTaskExecutor implements ClusterStateT private final AllocationService allocationService; private final RerouteService rerouteService; private final Logger logger; + private final Supplier prioritySupplier; - public ShardFailedClusterStateTaskExecutor(AllocationService allocationService, RerouteService rerouteService, Logger logger) { + public ShardFailedClusterStateTaskExecutor(AllocationService allocationService, RerouteService rerouteService, + Supplier prioritySupplier, Logger logger) { this.allocationService = allocationService; this.rerouteService = rerouteService; this.logger = logger; + this.prioritySupplier = prioritySupplier; } @Override @@ -383,7 +423,7 @@ public void clusterStatePublished(ClusterChangedEvent clusterChangedEvent) { // assign it again, even if that means putting it back on the node on which it previously failed: final String reason = String.format(Locale.ROOT, "[%d] unassigned shards after failing shards", numberOfUnassignedShards); logger.trace("{}, scheduling a reroute", reason); - rerouteService.reroute(reason, Priority.HIGH, ActionListener.wrap( + rerouteService.reroute(reason, prioritySupplier.get(), ActionListener.wrap( r -> logger.trace("{}, reroute completed", reason), e -> logger.debug(new ParameterizedMessage("{}, reroute failed", reason), e))); } @@ -520,10 +560,15 @@ public static class ShardStartedClusterStateTaskExecutor implements ClusterStateTaskExecutor, ClusterStateTaskListener { private final AllocationService allocationService; private final Logger logger; + private final RerouteService rerouteService; + private final Supplier prioritySupplier; - public ShardStartedClusterStateTaskExecutor(AllocationService allocationService, Logger logger) { + public ShardStartedClusterStateTaskExecutor(AllocationService allocationService, RerouteService rerouteService, + Supplier prioritySupplier, Logger logger) { this.allocationService = allocationService; this.logger = logger; + this.rerouteService = rerouteService; + this.prioritySupplier = prioritySupplier; } @Override @@ -598,6 +643,13 @@ public void onFailure(String source, Exception e) { logger.error(() -> new ParameterizedMessage("unexpected failure during [{}]", source), e); } } + + @Override + public void clusterStatePublished(ClusterChangedEvent clusterChangedEvent) { + rerouteService.reroute("reroute after starting shards", prioritySupplier.get(), ActionListener.wrap( + r -> logger.trace("reroute after starting shards succeeded"), + e -> logger.debug("reroute after starting shards failed", e))); + } } public static class StartedShardEntry extends TransportRequest { diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java index 7ba7f0bb578fd..26aeff6d2da16 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java @@ -109,7 +109,7 @@ public ClusterState applyStartedShards(ClusterState clusterState, List s.shardId().toString(), logger.isDebugEnabled()); return buildResultAndLogHealthChange(clusterState, allocation, "shards started [" + startedShardsAsString + "]"); diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 707476bb55c34..f62c840e11b88 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -32,6 +32,7 @@ import org.elasticsearch.cluster.InternalClusterInfoService; import org.elasticsearch.cluster.NodeConnectionsService; import org.elasticsearch.cluster.action.index.MappingUpdatedAction; +import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.coordination.ClusterBootstrapService; import org.elasticsearch.cluster.coordination.ClusterFormationFailureHelper; import org.elasticsearch.cluster.coordination.Coordinator; @@ -226,6 +227,7 @@ public void apply(Settings value, Settings current, Settings previous) { DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING, DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING, SameShardAllocationDecider.CLUSTER_ROUTING_ALLOCATION_SAME_HOST_SETTING, + ShardStateAction.FOLLOW_UP_REROUTE_PRIORITY_SETTING, InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING, InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING, DestructiveOperations.REQUIRES_NAME_SETTING, diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/shrink/TransportResizeActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/shrink/TransportResizeActionTests.java index 436cc597aa0f7..10da68760591d 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/shrink/TransportResizeActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/shrink/TransportResizeActionTests.java @@ -25,6 +25,7 @@ import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ESAllocationTestCase; import org.elasticsearch.cluster.EmptyClusterInfoService; import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -33,7 +34,6 @@ import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RoutingTable; -import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; @@ -116,8 +116,7 @@ public void testErrorCondition() { RoutingTable routingTable = service.reroute(clusterState, "reroute").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); // now we start the shard - routingTable = service.applyStartedShards(clusterState, - routingTable.index("source").shardsWithState(ShardRoutingState.INITIALIZING)).routingTable(); + routingTable = ESAllocationTestCase.startInitializingShardsAndReroute(service, clusterState, "source").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); TransportResizeAction.prepareCreateIndexRequest(new ResizeRequest("target", "source"), clusterState, @@ -135,8 +134,7 @@ public void testPassNumRoutingShards() { RoutingTable routingTable = service.reroute(clusterState, "reroute").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); // now we start the shard - routingTable = service.applyStartedShards(clusterState, - routingTable.index("source").shardsWithState(ShardRoutingState.INITIALIZING)).routingTable(); + routingTable = ESAllocationTestCase.startInitializingShardsAndReroute(service, clusterState, "source").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); ResizeRequest resizeRequest = new ResizeRequest("target", "source"); @@ -165,8 +163,7 @@ public void testPassNumRoutingShardsAndFail() { RoutingTable routingTable = service.reroute(clusterState, "reroute").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); // now we start the shard - routingTable = service.applyStartedShards(clusterState, - routingTable.index("source").shardsWithState(ShardRoutingState.INITIALIZING)).routingTable(); + routingTable = ESAllocationTestCase.startInitializingShardsAndReroute(service, clusterState, "source").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); ResizeRequest resizeRequest = new ResizeRequest("target", "source"); @@ -200,8 +197,7 @@ public void testShrinkIndexSettings() { RoutingTable routingTable = service.reroute(clusterState, "reroute").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); // now we start the shard - routingTable = service.applyStartedShards(clusterState, - routingTable.index(indexName).shardsWithState(ShardRoutingState.INITIALIZING)).routingTable(); + routingTable = ESAllocationTestCase.startInitializingShardsAndReroute(service, clusterState, indexName).routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); int numSourceShards = clusterState.metaData().index(indexName).getNumberOfShards(); DocsStats stats = new DocsStats(between(0, (IndexWriter.MAX_DOCS) / numSourceShards), between(1, 1000), between(1, 10000)); diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java index e332aa9623e60..462bec274d78c 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java @@ -466,7 +466,7 @@ public void testNoRerouteOnStaleClusterState() { ShardRouting relocationTarget = clusterService.state().getRoutingTable().shardRoutingTable(shardId) .shardsWithState(ShardRoutingState.INITIALIZING).get(0); AllocationService allocationService = ESAllocationTestCase.createAllocationService(); - ClusterState updatedState = allocationService.applyStartedShards(state, Collections.singletonList(relocationTarget)); + ClusterState updatedState = ESAllocationTestCase.startShardsAndReroute(allocationService, state, relocationTarget); setState(clusterService, updatedState); logger.debug("--> relocation complete state:\n{}", clusterService.state()); diff --git a/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java b/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java index 4dbe62cf5cebb..da31791413b57 100644 --- a/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java @@ -32,7 +32,6 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; -import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.routing.ShardRouting; @@ -42,6 +41,7 @@ import org.elasticsearch.cluster.routing.allocation.FailedShard; import org.elasticsearch.cluster.routing.allocation.StaleShard; import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider; +import org.elasticsearch.common.Priority; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.Settings; @@ -50,14 +50,12 @@ import org.elasticsearch.index.shard.ShardId; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.IntStream; -import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.Matchers.contains; @@ -89,7 +87,7 @@ public void setUp() throws Exception { .build(); clusterState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) .metaData(metaData).routingTable(routingTable).build(); - executor = new ShardStateAction.ShardFailedClusterStateTaskExecutor(allocationService, null, logger); + executor = new ShardStateAction.ShardFailedClusterStateTaskExecutor(allocationService, null, () -> Priority.NORMAL, logger); } public void testEmptyTaskListProducesSameClusterState() throws Exception { @@ -121,7 +119,7 @@ public void testTriviallySuccessfulTasksBatchedWithFailingTasks() throws Excepti List failingTasks = createExistingShards(currentState, reason); List nonExistentTasks = createNonExistentShards(currentState, reason); ShardStateAction.ShardFailedClusterStateTaskExecutor failingExecutor = - new ShardStateAction.ShardFailedClusterStateTaskExecutor(allocationService, null, logger) { + new ShardStateAction.ShardFailedClusterStateTaskExecutor(allocationService, null, () -> Priority.NORMAL, logger) { @Override ClusterState applyFailedShards(ClusterState currentState, List failedShards, List staleShards) { throw new RuntimeException("simulated applyFailedShards failure"); @@ -165,14 +163,14 @@ public void testIllegalShardFailureRequests() throws Exception { public void testMarkAsStaleWhenFailingShard() throws Exception { final MockAllocationService allocation = createAllocationService(); ClusterState clusterState = createClusterStateWithStartedShards("test markAsStale"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); IndexShardRoutingTable shardRoutingTable = clusterState.routingTable().index(INDEX).shard(0); long primaryTerm = clusterState.metaData().index(INDEX).primaryTerm(0); final Set oldInSync = clusterState.metaData().index(INDEX).inSyncAllocationIds(0); { ShardStateAction.FailedShardEntry failShardOnly = new ShardStateAction.FailedShardEntry(shardRoutingTable.shardId(), randomFrom(oldInSync), primaryTerm, "dummy", null, false); - ClusterState appliedState = executor.execute(clusterState, Arrays.asList(failShardOnly)).resultingState; + ClusterState appliedState = executor.execute(clusterState, Collections.singletonList(failShardOnly)).resultingState; Set newInSync = appliedState.metaData().index(INDEX).inSyncAllocationIds(0); assertThat(newInSync, equalTo(oldInSync)); } @@ -180,7 +178,7 @@ public void testMarkAsStaleWhenFailingShard() throws Exception { final String failedAllocationId = randomFrom(oldInSync); ShardStateAction.FailedShardEntry failAndMarkAsStale = new ShardStateAction.FailedShardEntry(shardRoutingTable.shardId(), failedAllocationId, primaryTerm, "dummy", null, true); - ClusterState appliedState = executor.execute(clusterState, Arrays.asList(failAndMarkAsStale)).resultingState; + ClusterState appliedState = executor.execute(clusterState, Collections.singletonList(failAndMarkAsStale)).resultingState; Set newInSync = appliedState.metaData().index(INDEX).inSyncAllocationIds(0); assertThat(Sets.difference(oldInSync, newInSync), contains(failedAllocationId)); } @@ -192,11 +190,9 @@ private ClusterState createClusterStateWithStartedShards(String reason) { IntStream.rangeClosed(1, numberOfNodes).mapToObj(node -> newNode("node" + node)).forEach(nodes::add); ClusterState stateAfterAddingNode = ClusterState.builder(clusterState).nodes(nodes).build(); - RoutingTable afterReroute = - allocationService.reroute(stateAfterAddingNode, reason).routingTable(); + RoutingTable afterReroute = allocationService.reroute(stateAfterAddingNode, reason).routingTable(); ClusterState stateAfterReroute = ClusterState.builder(stateAfterAddingNode).routingTable(afterReroute).build(); - RoutingNodes routingNodes = stateAfterReroute.getRoutingNodes(); - return allocationService.applyStartedShards(stateAfterReroute, routingNodes.shardsWithState(ShardRoutingState.INITIALIZING)); + return ESAllocationTestCase.startInitializingShardsAndReroute(allocationService, stateAfterReroute); } private List createExistingShards(ClusterState currentState, String reason) { diff --git a/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardStartedClusterStateTaskExecutorTests.java b/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardStartedClusterStateTaskExecutorTests.java index 20b7548004f4a..8b68516f61a96 100644 --- a/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardStartedClusterStateTaskExecutorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardStartedClusterStateTaskExecutorTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.cluster.action.shard; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ESAllocationTestCase; @@ -29,6 +30,7 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.allocation.AllocationService; +import org.elasticsearch.common.Priority; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.shard.ShardId; @@ -53,13 +55,18 @@ public class ShardStartedClusterStateTaskExecutorTests extends ESAllocationTestC private ShardStateAction.ShardStartedClusterStateTaskExecutor executor; + private static void neverReroutes(String reason, Priority priority, ActionListener listener) { + fail("unexpectedly ran a deferred reroute"); + } + @Override public void setUp() throws Exception { super.setUp(); AllocationService allocationService = createAllocationService(Settings.builder() .put(CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES_SETTING.getKey(), Integer.MAX_VALUE) .build()); - executor = new ShardStateAction.ShardStartedClusterStateTaskExecutor(allocationService, logger); + executor = new ShardStateAction.ShardStartedClusterStateTaskExecutor(allocationService, + ShardStartedClusterStateTaskExecutorTests::neverReroutes, () -> Priority.NORMAL, logger); } public void testEmptyTaskListProducesSameClusterState() throws Exception { diff --git a/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardStateActionIT.java b/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardStateActionIT.java new file mode 100644 index 0000000000000..02bea81d484e1 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardStateActionIT.java @@ -0,0 +1,139 @@ +/* + * 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.action.shard; + +import org.elasticsearch.action.ActionFuture; +import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; +import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateUpdateTask; +import org.elasticsearch.cluster.health.ClusterHealthStatus; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Priority; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.transport.MockTransportService; + +import java.util.Collection; +import java.util.Collections; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; + +public class ShardStateActionIT extends ESIntegTestCase { + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + final Settings.Builder builder = Settings.builder().put(super.nodeSettings(nodeOrdinal)); + if (randomBoolean()) { + builder.put(ShardStateAction.FOLLOW_UP_REROUTE_PRIORITY_SETTING.getKey(), randomPriority()); + } + return builder.build(); + } + + @Override + protected Collection> nodePlugins() { + return Collections.singletonList(MockTransportService.TestPlugin.class); + } + + public void testFollowupRerouteAlwaysOccursEventually() { + // Shows that no matter how cluster.routing.allocation.shard_state.reroute.priority is set, a follow-up reroute eventually occurs. + // Can be removed when this setting is removed, as we copiously test the default case. + + internalCluster().ensureAtLeastNumDataNodes(2); + + if (randomBoolean()) { + assertAcked(client().admin().cluster().prepareUpdateSettings().setPersistentSettings(Settings.builder() + .put(ShardStateAction.FOLLOW_UP_REROUTE_PRIORITY_SETTING.getKey(), randomPriority()))); + } + + createIndex("test"); + final ClusterHealthResponse clusterHealthResponse + = client().admin().cluster().prepareHealth().setWaitForNoInitializingShards(true).setWaitForEvents(Priority.LANGUID).get(); + assertFalse(clusterHealthResponse.isTimedOut()); + assertThat(clusterHealthResponse.getStatus(), equalTo(ClusterHealthStatus.GREEN)); + + assertAcked(client().admin().cluster().prepareUpdateSettings().setPersistentSettings(Settings.builder() + .putNull(ShardStateAction.FOLLOW_UP_REROUTE_PRIORITY_SETTING.getKey()))); + } + + public void testFollowupRerouteCanBeSetToHigherPriority() { + // Shows that in a cluster under unbearable pressure we can still assign replicas (for now at least) by setting + // cluster.routing.allocation.shard_state.reroute.priority to a higher priority. Can be removed when this setting is removed, as + // we should at that point be confident that the default priority is appropriate for all clusters. + + internalCluster().ensureAtLeastNumDataNodes(2); + + assertAcked(client().admin().cluster().prepareUpdateSettings().setPersistentSettings(Settings.builder() + .put(ShardStateAction.FOLLOW_UP_REROUTE_PRIORITY_SETTING.getKey(), "urgent"))); + + // ensure that the master always has a HIGH priority pending task + final AtomicBoolean stopSpammingMaster = new AtomicBoolean(); + final ClusterService masterClusterService = internalCluster().getInstance(ClusterService.class, internalCluster().getMasterName()); + masterClusterService.submitStateUpdateTask("spam", + new ClusterStateUpdateTask(Priority.HIGH) { + @Override + public ClusterState execute(ClusterState currentState) { + return currentState; + } + + @Override + public void onFailure(String source, Exception e) { + throw new AssertionError(source, e); + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + if (stopSpammingMaster.get() == false) { + masterClusterService.submitStateUpdateTask("spam", this); + } + } + }); + + // even with the master under such pressure, all shards of the index can be assigned; in particular, after the primaries have + // started there's a follow-up reroute at a higher priority than the spam + createIndex("test"); + assertFalse(client().admin().cluster().prepareHealth().setWaitForGreenStatus().get().isTimedOut()); + + stopSpammingMaster.set(true); + assertFalse(client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).get().isTimedOut()); + + assertAcked(client().admin().cluster().prepareUpdateSettings().setPersistentSettings(Settings.builder() + .putNull(ShardStateAction.FOLLOW_UP_REROUTE_PRIORITY_SETTING.getKey()))); + } + + public void testFollowupRerouteRejectsInvalidPriorities() { + final String invalidPriority = randomFrom("IMMEDIATE", "LOW", "LANGUID"); + final ActionFuture responseFuture = client().admin().cluster().prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(ShardStateAction.FOLLOW_UP_REROUTE_PRIORITY_SETTING.getKey(), invalidPriority)) + .execute(); + assertThat(expectThrows(IllegalArgumentException.class, responseFuture::actionGet).getMessage(), + allOf(containsString(invalidPriority), containsString(ShardStateAction.FOLLOW_UP_REROUTE_PRIORITY_SETTING.getKey()))); + } + + private String randomPriority() { + return randomFrom("normal", "high", "urgent", "NORMAL", "HIGH", "URGENT"); + // not "languid" (because we use that to wait for no pending tasks) nor "low" or "immediate" (because these are unreasonable) + } + +} diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java index 99dd8a7ecb6e8..b4d913b2ef48d 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java @@ -24,13 +24,13 @@ import org.elasticsearch.action.admin.indices.shrink.ResizeType; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ESAllocationTestCase; import org.elasticsearch.cluster.EmptyClusterInfoService; import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RoutingTable; -import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; @@ -180,8 +180,7 @@ public void testValidateShrinkIndex() { RoutingTable routingTable = service.reroute(clusterState, "reroute").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); // now we start the shard - routingTable = service.applyStartedShards(clusterState, - routingTable.index("source").shardsWithState(ShardRoutingState.INITIALIZING)).routingTable(); + routingTable = ESAllocationTestCase.startInitializingShardsAndReroute(service, clusterState, "source").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); int targetShards; do { @@ -250,8 +249,7 @@ public void testValidateSplitIndex() { RoutingTable routingTable = service.reroute(clusterState, "reroute").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); // now we start the shard - routingTable = service.applyStartedShards(clusterState, - routingTable.index("source").shardsWithState(ShardRoutingState.INITIALIZING)).routingTable(); + routingTable = ESAllocationTestCase.startInitializingShardsAndReroute(service, clusterState, "source").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); MetaDataCreateIndexService.validateSplitIndex(clusterState, "source", Collections.emptySet(), "target", @@ -386,9 +384,8 @@ private void runPrepareResizeIndexSettingsTest( final ClusterState routingTableClusterState = ClusterState.builder(initialClusterState).routingTable(initialRoutingTable).build(); // now we start the shard - final RoutingTable routingTable = service.applyStartedShards( - routingTableClusterState, - initialRoutingTable.index(indexName).shardsWithState(ShardRoutingState.INITIALIZING)).routingTable(); + final RoutingTable routingTable + = ESAllocationTestCase.startInitializingShardsAndReroute(service, routingTableClusterState, indexName).routingTable(); final ClusterState clusterState = ClusterState.builder(routingTableClusterState).routingTable(routingTable).build(); final Settings.Builder indexSettingsBuilder = Settings.builder().put("index.number_of_shards", 1).put(requestSettings); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/DelayedAllocationServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/DelayedAllocationServiceTests.java index 098803e3cfd53..4ebd5eb59ad45 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/DelayedAllocationServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/DelayedAllocationServiceTests.java @@ -45,7 +45,6 @@ import static java.util.Collections.singleton; import static org.elasticsearch.cluster.routing.DelayedAllocationService.CLUSTER_UPDATE_TASK_SOURCE; -import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED; import static org.elasticsearch.common.unit.TimeValue.timeValueMillis; import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds; @@ -92,9 +91,9 @@ public void testNoDelayedUnassigned() throws Exception { .build(); clusterState = allocationService.reroute(clusterState, "reroute"); // starting primaries - clusterState = allocationService.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocationService, clusterState); // starting replicas - clusterState = allocationService.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocationService, clusterState); assertThat(clusterState.getRoutingNodes().unassigned().size() > 0, equalTo(false)); ClusterState prevState = clusterState; // remove node2 and reroute @@ -136,9 +135,9 @@ public void testDelayedUnassignedScheduleReroute() throws Exception { allocationService.setNanoTimeOverride(baseTimestampNanos); clusterState = allocationService.reroute(clusterState, "reroute"); // starting primaries - clusterState = allocationService.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocationService, clusterState); // starting replicas - clusterState = allocationService.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocationService, clusterState); assertFalse("no shards should be unassigned", clusterState.getRoutingNodes().unassigned().size() > 0); String nodeId = null; final List allShards = clusterState.getRoutingTable().allShards("test"); @@ -228,9 +227,9 @@ public void testDelayedUnassignedScheduleRerouteAfterDelayedReroute() throws Exc // allocate shards clusterState = allocationService.reroute(clusterState, "reroute"); // start primaries - clusterState = allocationService.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocationService, clusterState); // start replicas - clusterState = allocationService.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocationService, clusterState); assertThat("all shards should be started", clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(4)); // find replica of short_delay @@ -385,9 +384,9 @@ public void testDelayedUnassignedScheduleRerouteRescheduledOnShorterDelay() thro allocationService.setNanoTimeOverride(nodeLeftTimestampNanos); clusterState = allocationService.reroute(clusterState, "reroute"); // starting primaries - clusterState = allocationService.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocationService, clusterState); // starting replicas - clusterState = allocationService.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocationService, clusterState); assertFalse("no shards should be unassigned", clusterState.getRoutingNodes().unassigned().size() > 0); String nodeIdOfFooReplica = null; for (ShardRouting shardRouting : clusterState.getRoutingTable().allShards("foo")) { diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryTermsTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryTermsTests.java index 8a9b00a8d4ff7..34dedc823fc67 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryTermsTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryTermsTests.java @@ -110,10 +110,10 @@ private void incrementPrimaryTerm(String index, int shard) { } private boolean startInitializingShards(String index) { - final List startedShards = this.clusterState.getRoutingNodes().shardsWithState(index, INITIALIZING); + final List startedShards = clusterState.getRoutingNodes().shardsWithState(index, INITIALIZING); logger.info("start primary shards for index [{}]: {} ", index, startedShards); - ClusterState rerouteResult = allocationService.applyStartedShards(this.clusterState, startedShards); - boolean changed = rerouteResult.equals(this.clusterState) == false; + ClusterState rerouteResult = startShardsAndReroute(allocationService, clusterState, startedShards); + boolean changed = rerouteResult.equals(clusterState) == false; applyRerouteResult(rerouteResult); return changed; } diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTests.java index 851fe9c550270..c573206d848bf 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTests.java @@ -38,7 +38,6 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.elasticsearch.cluster.routing.ShardRoutingState.UNASSIGNED; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -104,8 +103,7 @@ private void initPrimaries() { private void startInitializingShards(String index) { logger.info("start primary shards for index {}", index); - this.clusterState = ALLOCATION_SERVICE.applyStartedShards(this.clusterState, - this.clusterState.getRoutingNodes().shardsWithState(index, INITIALIZING)); + clusterState = startInitializingShardsAndReroute(ALLOCATION_SERVICE, clusterState, index); } private IndexMetaData.Builder createIndexMetaData(String indexName) { diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index bc3191c14dfba..7c43729a7cd25 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -46,7 +46,6 @@ import java.nio.ByteBuffer; import java.util.Collections; -import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED; import static org.elasticsearch.cluster.routing.ShardRoutingState.UNASSIGNED; import static org.hamcrest.Matchers.equalTo; @@ -209,7 +208,7 @@ public void testReplicaAdded() { clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build(); clusterState = allocation.reroute(clusterState, "reroute"); // starting primaries - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); IndexRoutingTable.Builder builder = IndexRoutingTable.builder(index); for (IndexShardRoutingTable indexShardRoutingTable : clusterState.routingTable().index(index)) { builder.addIndexShard(indexShardRoutingTable); @@ -253,9 +252,9 @@ public void testNodeLeave() { .nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build(); clusterState = allocation.reroute(clusterState, "reroute"); // starting primaries - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); // starting replicas - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertThat(clusterState.getRoutingNodes().unassigned().size() > 0, equalTo(false)); // remove node2 and reroute clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove("node2")).build(); @@ -285,9 +284,9 @@ public void testFailedShard() { .nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build(); clusterState = allocation.reroute(clusterState, "reroute"); // starting primaries - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); // starting replicas - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertThat(clusterState.getRoutingNodes().unassigned().size() > 0, equalTo(false)); // fail shard ShardRouting shardToFail = clusterState.getRoutingNodes().shardsWithState(STARTED).get(0); @@ -343,9 +342,9 @@ public void testNumberOfDelayedUnassigned() throws Exception { clusterState = allocation.reroute(clusterState, "reroute"); assertThat(UnassignedInfo.getNumberOfDelayedUnassigned(clusterState), equalTo(0)); // starting primaries - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); // starting replicas - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertThat(clusterState.getRoutingNodes().unassigned().size() > 0, equalTo(false)); // remove node2 and reroute clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove("node2")).build(); @@ -374,9 +373,9 @@ public void testFindNextDelayedAllocation() { clusterState = allocation.reroute(clusterState, "reroute"); assertThat(UnassignedInfo.getNumberOfDelayedUnassigned(clusterState), equalTo(0)); // starting primaries - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); // starting replicas - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertThat(clusterState.getRoutingNodes().unassigned().size() > 0, equalTo(false)); // remove node2 and reroute final long baseTime = System.nanoTime(); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AddIncrementallyTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AddIncrementallyTests.java index d2ef2de5458cb..31f0bfa01260c 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AddIncrementallyTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AddIncrementallyTests.java @@ -123,7 +123,7 @@ public void testMinimalRelocations() { assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(INITIALIZING).size(), equalTo(0)); assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(INITIALIZING).size(), equalTo(0)); - ClusterState newState = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + ClusterState newState = startInitializingShardsAndReroute(service, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; routingNodes = clusterState.getRoutingNodes(); @@ -132,7 +132,7 @@ public void testMinimalRelocations() { assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(INITIALIZING).size(), equalTo(0)); assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(INITIALIZING).size(), equalTo(0)); - newState = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(service, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; routingNodes = clusterState.getRoutingNodes(); @@ -141,7 +141,7 @@ public void testMinimalRelocations() { assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(INITIALIZING).size(), equalTo(0)); assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(INITIALIZING).size(), equalTo(0)); - newState = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(service, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; routingNodes = clusterState.getRoutingNodes(); @@ -150,7 +150,7 @@ public void testMinimalRelocations() { assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(INITIALIZING).size(), equalTo(0)); assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(INITIALIZING).size(), equalTo(0)); - newState = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(service, clusterState); assertThat(newState, equalTo(clusterState)); assertNumIndexShardsPerNode(clusterState, equalTo(2)); logger.debug("ClusterState: {}", clusterState.getRoutingNodes()); @@ -186,7 +186,7 @@ public void testMinimalRelocationsNoLimit() { assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(INITIALIZING).size(), equalTo(0)); assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(INITIALIZING).size(), equalTo(0)); - ClusterState newState = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + ClusterState newState = startInitializingShardsAndReroute(service, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; routingNodes = clusterState.getRoutingNodes(); @@ -195,7 +195,7 @@ public void testMinimalRelocationsNoLimit() { assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(INITIALIZING).size(), equalTo(0)); assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(INITIALIZING).size(), equalTo(0)); - newState = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(service, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; routingNodes = clusterState.getRoutingNodes(); @@ -204,7 +204,7 @@ public void testMinimalRelocationsNoLimit() { assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(INITIALIZING).size(), equalTo(0)); assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(INITIALIZING).size(), equalTo(0)); - newState = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(service, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; routingNodes = clusterState.getRoutingNodes(); @@ -213,7 +213,7 @@ public void testMinimalRelocationsNoLimit() { assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(INITIALIZING).size(), equalTo(0)); assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(INITIALIZING).size(), equalTo(0)); - newState = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(service, clusterState); assertThat(newState, equalTo(clusterState)); assertNumIndexShardsPerNode(clusterState, equalTo(2)); logger.debug("ClusterState: {}", clusterState.getRoutingNodes()); @@ -287,13 +287,10 @@ private ClusterState initCluster(AllocationService service, int numberOfNodes, i clusterState = service.reroute(clusterState, "reroute"); logger.info("restart all the primary shards, replicas will start initializing"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(service, clusterState); logger.info("start the replica shards"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(service, clusterState); logger.info("complete rebalancing"); return applyStartedShardsUntilNoChange(clusterState, service); @@ -316,12 +313,10 @@ private ClusterState addIndex(ClusterState clusterState, AllocationService servi clusterState = service.reroute(clusterState, "reroute"); logger.info("restart all the primary shards, replicas will start initializing"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(service, clusterState); logger.info("start the replica shards"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(service, clusterState); logger.info("complete rebalancing"); return applyStartedShardsUntilNoChange(clusterState, service); @@ -344,12 +339,10 @@ private ClusterState removeNodes(ClusterState clusterState, AllocationService se clusterState = service.disassociateDeadNodes(clusterState, true, "reroute"); logger.info("start all the primary shards, replicas will start initializing"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(service, clusterState); logger.info("start the replica shards"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(service, clusterState); logger.info("rebalancing"); clusterState = service.reroute(clusterState, "reroute"); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationCommandsTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationCommandsTests.java index e7c7d1c0a38e3..7bf14cc997ba3 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationCommandsTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationCommandsTests.java @@ -101,7 +101,7 @@ public void testMoveShardCommand() { clusterState = allocation.reroute(clusterState, "reroute"); logger.info("start primary shard"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); logger.info("move the shard"); String existingNodeId = clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId(); @@ -120,7 +120,7 @@ public void testMoveShardCommand() { assertThat(clusterState.getRoutingNodes().node(toNodeId).iterator().next().state(), equalTo(ShardRoutingState.INITIALIZING)); logger.info("finish moving the shard"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertThat(clusterState.getRoutingNodes().node(existingNodeId).isEmpty(), equalTo(true)); assertThat(clusterState.getRoutingNodes().node(toNodeId).iterator().next().state(), equalTo(ShardRoutingState.STARTED)); @@ -229,7 +229,7 @@ public void testAllocateCommand() { assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(0)); logger.info("--> start the primary shard"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(0)); @@ -254,7 +254,7 @@ public void testAllocateCommand() { logger.info("--> start the replica shard"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1)); @@ -309,7 +309,7 @@ public void testAllocateStalePrimaryCommand() { Set inSyncAllocationIds = clusterState.metaData().index(index).inSyncAllocationIds(0); assertThat(inSyncAllocationIds, equalTo(Collections.singleton(RecoverySource.ExistingStoreRecoverySource.FORCED_ALLOCATION_ID))); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); routingNode1 = clusterState.getRoutingNodes().node(node1); assertThat(routingNode1.size(), equalTo(1)); assertThat(routingNode1.shardsWithState(STARTED).size(), equalTo(1)); @@ -360,7 +360,7 @@ public void testCancelCommand() { } logger.info("--> start the primary shard"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(0)); @@ -410,7 +410,7 @@ public void testCancelCommand() { } logger.info("--> start the replica shard"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1)); @@ -436,7 +436,7 @@ public void testCancelCommand() { assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(INITIALIZING).size(), equalTo(1)); logger.info("--> start the replica shard"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1)); @@ -491,8 +491,8 @@ public void testCancelCommand() { assertThat(clusterState.getRoutingNodes().node("node3").shardsWithState(INITIALIZING).get(0).relocatingNodeId(), nullValue()); logger.info("--> start the former target replica shard"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); - assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); + assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(0)); assertThat(clusterState.getRoutingNodes().node("node3").shardsWithState(STARTED).size(), equalTo(1)); @@ -630,7 +630,7 @@ public void testMoveShardToNonDataNode() { .add(node2)).build(); logger.info("start primary shard"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); Index index = clusterState.getMetaData().index("test").getIndex(); MoveAllocationCommand command = new MoveAllocationCommand(index.getName(), 0, "node1", "node2"); @@ -669,7 +669,7 @@ public void testMoveShardFromNonDataNode() { .add(node1) .add(node2)).build(); logger.info("start primary shard"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); Index index = clusterState.getMetaData().index("test").getIndex(); MoveAllocationCommand command = new MoveAllocationCommand(index.getName(), 0, "node2", "node1"); @@ -738,7 +738,7 @@ public void testConflictingCommandsInSingleRequest() { clusterState = allocation.reroute(clusterState, new AllocationCommands(new AllocateEmptyPrimaryAllocationCommand(index3, 0, node1, true)), false, false).getClusterState(); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); final ClusterState updatedClusterState = clusterState; assertThat(updatedClusterState.getRoutingNodes().node(node1).shardsWithState(STARTED).size(), equalTo(1)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationPriorityTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationPriorityTests.java index fdb9fdb46a85c..02398e5b80208 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationPriorityTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationPriorityTests.java @@ -80,18 +80,18 @@ public void testPrioritizedIndicesAllocatedFirst() { assertEquals(highPriorityName, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(0).getIndexName()); assertEquals(highPriorityName, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(1).getIndexName()); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertEquals(2, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size()); assertEquals(lowPriorityName, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(0).getIndexName()); assertEquals(lowPriorityName, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(1).getIndexName()); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertEquals(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).toString(),2, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size()); assertEquals(highPriorityName, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(0).getIndexName()); assertEquals(highPriorityName, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(1).getIndexName()); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertEquals(2, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size()); assertEquals(lowPriorityName, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(0).getIndexName()); assertEquals(lowPriorityName, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(1).getIndexName()); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AwarenessAllocationTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AwarenessAllocationTests.java index b0ce9ad320a9e..6174323ca69e8 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AwarenessAllocationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AwarenessAllocationTests.java @@ -80,10 +80,10 @@ public void testMoveShardOnceNewNodeWithAttributeAdded1() { assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); logger.info("--> start the shards (primaries)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("--> start the shards (replicas)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2)); @@ -99,7 +99,7 @@ public void testMoveShardOnceNewNodeWithAttributeAdded1() { equalTo("node3")); logger.info("--> complete relocation"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2)); @@ -143,10 +143,10 @@ public void testMoveShardOnceNewNodeWithAttributeAdded2() { assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); logger.info("--> start the shards (primaries)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("--> start the shards (replicas)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2)); @@ -162,7 +162,7 @@ public void testMoveShardOnceNewNodeWithAttributeAdded2() { equalTo("node4")); logger.info("--> complete relocation"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2)); @@ -218,10 +218,10 @@ public void testMoveShardOnceNewNodeWithAttributeAdded3() { assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(5)); logger.info("--> start the shards (primaries)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("--> start the shards (replicas)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10)); @@ -238,10 +238,10 @@ public void testMoveShardOnceNewNodeWithAttributeAdded3() { equalTo("node3")); logger.info("--> complete initializing"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("--> run it again, since we still might have relocation"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10)); @@ -256,7 +256,7 @@ public void testMoveShardOnceNewNodeWithAttributeAdded3() { assertThat(clusterState.getRoutingNodes().shardsWithState(RELOCATING).size(), greaterThan(0)); logger.info("--> complete relocation"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10)); @@ -297,10 +297,10 @@ public void testMoveShardOnceNewNodeWithAttributeAdded4() { assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(10)); logger.info("--> start the shards (primaries)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("--> start the shards (replicas)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(20)); @@ -319,9 +319,9 @@ public void testMoveShardOnceNewNodeWithAttributeAdded4() { logger.info("--> complete initializing"); for (int i = 0; i < 2; i++) { logger.info("--> complete initializing round: [{}]", i); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); - } - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); + } + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(20)); assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(10)); @@ -341,8 +341,8 @@ public void testMoveShardOnceNewNodeWithAttributeAdded4() { logger.info("--> complete relocation"); for (int i = 0; i < 2; i++) { logger.info("--> complete initializing round: [{}]", i); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); - } + clusterState = startInitializingShardsAndReroute(strategy, clusterState); + } assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(20)); assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(5)); assertThat(clusterState.getRoutingNodes().node("node4").size(), equalTo(5)); @@ -382,10 +382,10 @@ public void testMoveShardOnceNewNodeWithAttributeAdded5() { assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); logger.info("--> start the shards (primaries)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("--> start the shards (replicas)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2)); @@ -401,7 +401,7 @@ public void testMoveShardOnceNewNodeWithAttributeAdded5() { equalTo("node3")); logger.info("--> complete relocation"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(3)); @@ -419,7 +419,7 @@ public void testMoveShardOnceNewNodeWithAttributeAdded5() { equalTo("node4")); logger.info("--> complete relocation"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(3)); @@ -458,10 +458,10 @@ public void testMoveShardOnceNewNodeWithAttributeAdded6() { assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); logger.info("--> start the shards (primaries)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("--> start the shards (replicas)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(4)); @@ -477,7 +477,7 @@ public void testMoveShardOnceNewNodeWithAttributeAdded6() { equalTo("node5")); logger.info("--> complete relocation"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(4)); @@ -495,7 +495,7 @@ public void testMoveShardOnceNewNodeWithAttributeAdded6() { equalTo("node6")); logger.info("--> complete relocation"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(4)); @@ -533,7 +533,7 @@ public void testFullAwareness1() { assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); logger.info("--> start the shards (primaries)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("--> replica will not start because we have only one rack value"); assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1)); @@ -551,7 +551,7 @@ public void testFullAwareness1() { equalTo("node3")); logger.info("--> complete relocation"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2)); @@ -598,7 +598,7 @@ public void testFullAwareness2() { assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); logger.info("--> start the shards (primaries)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("--> replica will not start because we have only one rack value"); assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1)); @@ -616,7 +616,7 @@ public void testFullAwareness2() { equalTo("node4")); logger.info("--> complete relocation"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2)); @@ -669,7 +669,7 @@ public void testFullAwareness3() { assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(10)); logger.info("--> start the shards (primaries)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10)); @@ -685,10 +685,10 @@ public void testFullAwareness3() { equalTo("node3")); logger.info("--> complete initializing"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("--> run it again, since we still might have relocation"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(20)); @@ -703,7 +703,7 @@ public void testFullAwareness3() { assertThat(clusterState.getRoutingNodes().shardsWithState(RELOCATING).size(), greaterThan(0)); logger.info("--> complete relocation"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(20)); @@ -744,11 +744,11 @@ public void testUnbalancedZones() { assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(5)); logger.info("--> start the shards (primaries)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(5)); assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(5)); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("--> all replicas are allocated and started since we have on node in each zone"); assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(10)); assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0)); @@ -764,7 +764,7 @@ public void testUnbalancedZones() { equalTo("A-1")); logger.info("--> starting initializing shards on the new node"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10)); assertThat(clusterState.getRoutingNodes().node("A-1").size(), equalTo(2)); @@ -806,7 +806,7 @@ public void testUnassignedShardsWithUnbalancedZones() { assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); logger.info("--> start the shard (primary)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(3)); assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).size(), equalTo(1)); // Unassigned shard is expected. @@ -867,11 +867,11 @@ public void testMultipleAwarenessAttributes() { assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); logger.info("--> start the shards (primaries)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("--> all replicas are allocated and started since we have one node in each zone and rack"); assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(2)); assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java index ba18bc0f6d42d..81ccdc35b248e 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java @@ -20,7 +20,6 @@ package org.elasticsearch.cluster.routing.allocation; import com.carrotsearch.hppc.cursors.ObjectCursor; - import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.util.ArrayUtil; @@ -47,7 +46,6 @@ import java.util.stream.Collectors; -import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED; import static org.elasticsearch.cluster.routing.ShardRoutingState.UNASSIGNED; @@ -145,12 +143,10 @@ private ClusterState initCluster(AllocationService strategy) { clusterState = strategy.reroute(clusterState, "reroute"); logger.info("restart all the primary shards, replicas will start initializing"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("start the replica shards"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("complete rebalancing"); return applyStartedShardsUntilNoChange(clusterState, strategy); @@ -185,12 +181,10 @@ private ClusterState removeNodes(ClusterState clusterState, AllocationService st } logger.info("start all the primary shards, replicas will start initializing"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("start the replica shards"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("rebalancing"); clusterState = strategy.reroute(clusterState, "reroute"); @@ -384,8 +378,7 @@ public ShardAllocationDecision decideShardAllocation(ShardRouting shard, Routing strategy = createAllocationService(settings.build(), new TestGatewayAllocator()); logger.info("use the new allocator and check if it moves shards"); - routingNodes = clusterState.getRoutingNodes(); - routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable(); + routingTable = startInitializingShardsAndReroute(strategy, clusterState).routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); routingNodes = clusterState.getRoutingNodes(); for (RoutingNode routingNode : routingNodes) { @@ -395,7 +388,7 @@ public ShardAllocationDecision decideShardAllocation(ShardRouting shard, Routing } logger.info("start the replica shards"); - routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable(); + routingTable = startInitializingShardsAndReroute(strategy, clusterState).routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); routingNodes = clusterState.getRoutingNodes(); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceUnbalancedClusterTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceUnbalancedClusterTests.java index 254ba81f93ce6..3f7321d2723a0 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceUnbalancedClusterTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceUnbalancedClusterTests.java @@ -21,6 +21,7 @@ import org.apache.lucene.util.TestUtil; import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ESAllocationTestCase; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; @@ -66,11 +67,8 @@ protected ClusterState allocateNew(ClusterState state) { ClusterState clusterState = ClusterState.builder(state).metaData(metaData).routingTable(initialRoutingTable).build(); clusterState = strategy.reroute(clusterState, "reroute"); - while (true) { - if (clusterState.routingTable().shardsWithState(INITIALIZING).isEmpty()) { - break; - } - clusterState = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING)); + while (clusterState.routingTable().shardsWithState(INITIALIZING).isEmpty() == false) { + clusterState = ESAllocationTestCase.startInitializingShardsAndReroute(strategy, clusterState); } Map counts = new HashMap<>(); for (IndexShardRoutingTable table : clusterState.routingTable().index(index)) { diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/CatAllocationTestCase.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/CatAllocationTestCase.java index db707c5478eff..0f28420e9325f 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/CatAllocationTestCase.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/CatAllocationTestCase.java @@ -160,7 +160,7 @@ private ClusterState rebalance(ClusterState clusterState) { } logger.debug("Initializing shards: {}", initializing); numRelocations += initializing.size(); - clusterState = strategy.applyStartedShards(clusterState, initializing); + clusterState = ESAllocationTestCase.startShardsAndReroute(strategy, clusterState, initializing); } logger.debug("--> num relocations to get balance: {}", numRelocations); return clusterState; diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ClusterRebalanceRoutingTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ClusterRebalanceRoutingTests.java index bca086b8fc92b..8eb056c77825c 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ClusterRebalanceRoutingTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ClusterRebalanceRoutingTests.java @@ -84,8 +84,7 @@ public void testAlways() { } logger.info("start all the primary shards for test1, replicas will start initializing"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, "test1"); for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) { assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2)); @@ -100,9 +99,7 @@ public void testAlways() { } logger.info("start the test1 replica shards"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, "test1"); for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) { assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2)); @@ -121,7 +118,7 @@ public void testAlways() { .add(newNode("node3"))) .build(); clusterState = strategy.reroute(clusterState, "reroute"); - routingNodes = clusterState.getRoutingNodes(); + RoutingNodes routingNodes = clusterState.getRoutingNodes(); assertThat(routingNodes.node("node3").size(), equalTo(1)); assertThat(routingNodes.node("node3").iterator().next().shardId().getIndex().getName(), equalTo("test1")); @@ -163,9 +160,7 @@ public void testClusterPrimariesActive1() { } logger.info("start all the primary shards for test1, replicas will start initializing"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, "test1"); for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) { assertThat(clusterState.routingTable().index("test1").shard(i).size(), equalTo(2)); @@ -180,9 +175,7 @@ public void testClusterPrimariesActive1() { } logger.info("start the test1 replica shards"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, "test1"); for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) { assertThat(clusterState.routingTable().index("test1").shard(i).size(), equalTo(2)); @@ -197,9 +190,7 @@ public void testClusterPrimariesActive1() { } logger.info("start all the primary shards for test2, replicas will start initializing"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test2", INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, "test2"); for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) { assertThat(clusterState.routingTable().index("test1").shard(i).size(), equalTo(2)); @@ -218,8 +209,8 @@ public void testClusterPrimariesActive1() { .add(newNode("node3"))) .build(); clusterState = strategy.reroute(clusterState, "reroute"); - routingNodes = clusterState.getRoutingNodes(); + RoutingNodes routingNodes = clusterState.getRoutingNodes(); assertThat(routingNodes.node("node3").size(), equalTo(1)); assertThat(routingNodes.node("node3").iterator().next().shardId().getIndex().getName(), equalTo("test1")); } @@ -260,9 +251,7 @@ public void testClusterPrimariesActive2() { } logger.info("start all the primary shards for test1, replicas will start initializing"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, "test1"); for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) { assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2)); @@ -277,9 +266,7 @@ public void testClusterPrimariesActive2() { } logger.info("start the test1 replica shards"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, "test1"); for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) { assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2)); @@ -298,8 +285,7 @@ public void testClusterPrimariesActive2() { .add(newNode("node3"))) .build(); clusterState = strategy.reroute(clusterState, "reroute"); - routingNodes = clusterState.getRoutingNodes(); - + RoutingNodes routingNodes = clusterState.getRoutingNodes(); assertThat(routingNodes.node("node3").isEmpty(), equalTo(true)); } @@ -339,9 +325,7 @@ public void testClusterAllActive1() { } logger.info("start all the primary shards for test1, replicas will start initializing"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, "test1"); for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) { assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2)); @@ -356,9 +340,7 @@ public void testClusterAllActive1() { } logger.info("start the test1 replica shards"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, "test1"); for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) { assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2)); @@ -373,9 +355,7 @@ public void testClusterAllActive1() { } logger.info("start all the primary shards for test2, replicas will start initializing"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test2", INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, "test2"); for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) { assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2)); @@ -390,9 +370,7 @@ public void testClusterAllActive1() { } logger.info("start the test2 replica shards"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test2", INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, "test2"); for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) { assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2)); @@ -411,7 +389,7 @@ public void testClusterAllActive1() { .add(newNode("node3"))) .build(); clusterState = strategy.reroute(clusterState, "reroute"); - routingNodes = clusterState.getRoutingNodes(); + RoutingNodes routingNodes = clusterState.getRoutingNodes(); assertThat(routingNodes.node("node3").size(), equalTo(1)); assertThat(routingNodes.node("node3").iterator().next().shardId().getIndex().getName(), anyOf(equalTo("test1"), @@ -454,9 +432,7 @@ public void testClusterAllActive2() { } logger.info("start all the primary shards for test1, replicas will start initializing"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, "test1"); for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) { assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2)); @@ -471,9 +447,7 @@ public void testClusterAllActive2() { } logger.info("start the test1 replica shards"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, "test1"); for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) { assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2)); @@ -492,7 +466,7 @@ public void testClusterAllActive2() { .add(newNode("node3"))) .build(); clusterState = strategy.reroute(clusterState, "reroute"); - routingNodes = clusterState.getRoutingNodes(); + RoutingNodes routingNodes = clusterState.getRoutingNodes(); assertThat(routingNodes.node("node3").isEmpty(), equalTo(true)); } @@ -533,9 +507,7 @@ public void testClusterAllActive3() { } logger.info("start all the primary shards for test1, replicas will start initializing"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, "test1"); for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) { assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2)); @@ -550,9 +522,7 @@ public void testClusterAllActive3() { } logger.info("start the test1 replica shards"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, "test1"); for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) { assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2)); @@ -567,9 +537,7 @@ public void testClusterAllActive3() { } logger.info("start all the primary shards for test2, replicas will start initializing"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test2", INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, "test2"); for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) { assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2)); @@ -588,7 +556,7 @@ public void testClusterAllActive3() { .add(newNode("node3"))) .build(); clusterState = strategy.reroute(clusterState, "reroute"); - routingNodes = clusterState.getRoutingNodes(); + RoutingNodes routingNodes = clusterState.getRoutingNodes(); assertThat(routingNodes.node("node3").isEmpty(), equalTo(true)); } @@ -637,8 +605,7 @@ public void allocateUnassigned(RoutingAllocation allocation) { } logger.debug("start all the primary shards for test"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test", INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, "test"); for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) { assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(1)); assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED)); @@ -673,8 +640,7 @@ public void allocateUnassigned(RoutingAllocation allocation) { logger.debug("now start initializing shards and expect exactly one rebalance" + " from node1 to node 2 since index [test] is all on node1"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, "test1"); for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) { assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(1)); @@ -736,8 +702,7 @@ public void allocateUnassigned(RoutingAllocation allocation) { } logger.debug("start all the primary shards for test"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test", INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, "test"); for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) { assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(1)); assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ConcurrentRebalanceRoutingTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ConcurrentRebalanceRoutingTests.java index d2e36b7fac9b0..22ef1d03d7190 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ConcurrentRebalanceRoutingTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ConcurrentRebalanceRoutingTests.java @@ -27,7 +27,6 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.common.settings.Settings; @@ -81,8 +80,7 @@ public void testClusterConcurrentRebalance() { } logger.info("start all the primary shards, replicas will start initializing"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) { assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2)); @@ -105,32 +103,28 @@ public void testClusterConcurrentRebalance() { } logger.info("start the replica shards, rebalancing should start, but, only 3 should be rebalancing"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); // we only allow one relocation at a time assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(7)); assertThat(clusterState.routingTable().shardsWithState(RELOCATING).size(), equalTo(3)); logger.info("finalize this session relocation, 3 more should relocate now"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); // we only allow one relocation at a time assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(7)); assertThat(clusterState.routingTable().shardsWithState(RELOCATING).size(), equalTo(3)); logger.info("finalize this session relocation, 2 more should relocate now"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); // we only allow one relocation at a time assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(8)); assertThat(clusterState.routingTable().shardsWithState(RELOCATING).size(), equalTo(2)); logger.info("finalize this session relocation, no more relocation"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); // we only allow one relocation at a time assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(10)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/DeadNodesAllocationTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/DeadNodesAllocationTests.java index f9dee9807b400..7d9aeeb606111 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/DeadNodesAllocationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/DeadNodesAllocationTests.java @@ -67,9 +67,9 @@ public void testSimpleDeadNodeOnStartedPrimaryShard() { clusterState = allocation.reroute(clusterState, "reroute"); // starting primaries - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); // starting replicas - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); logger.info("--> verifying all is allocated"); assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1)); @@ -115,9 +115,9 @@ public void testDeadNodeWhileRelocatingOnToNode() { clusterState = allocation.reroute(clusterState, "reroute"); // starting primaries - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); // starting replicas - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); logger.info("--> verifying all is allocated"); assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1)); @@ -185,9 +185,9 @@ public void testDeadNodeWhileRelocatingOnFromNode() { clusterState = allocation.reroute(clusterState, "reroute"); // starting primaries - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); // starting replicas - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); logger.info("--> verifying all is allocated"); assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitorTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitorTests.java index 213666b7011a8..1b4375364305b 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitorTests.java @@ -44,7 +44,6 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.hamcrest.Matchers.equalTo; public class DiskThresholdMonitorTests extends ESAllocationTestCase { @@ -73,7 +72,7 @@ public void testMarkFloodStageIndicesReadOnly() { .add(newNode("node2"))).build(); clusterState = allocation.reroute(clusterState, "reroute"); logger.info("start primary shard"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); ClusterState finalState = clusterState; AtomicBoolean reroute = new AtomicBoolean(false); AtomicReference> indices = new AtomicReference<>(); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ElectReplicaAsPrimaryDuringRelocationTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ElectReplicaAsPrimaryDuringRelocationTests.java index 74e7ac3273634..23c8d341c13e9 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ElectReplicaAsPrimaryDuringRelocationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ElectReplicaAsPrimaryDuringRelocationTests.java @@ -32,7 +32,6 @@ import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.common.settings.Settings; -import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.not; @@ -63,16 +62,14 @@ public void testElectReplicaAsPrimaryDuringRelocation() { clusterState = strategy.reroute(clusterState, "reroute"); logger.info("Start the primary shards"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("Start the replica shards"); - routingNodes = clusterState.getRoutingNodes(); - ClusterState resultingState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + ClusterState resultingState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(resultingState, not(equalTo(clusterState))); clusterState = resultingState; - routingNodes = clusterState.getRoutingNodes(); + RoutingNodes routingNodes = clusterState.getRoutingNodes(); assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(2)); assertThat(routingNodes.node("node1").numberOfShardsWithState(STARTED), equalTo(2)); assertThat(routingNodes.node("node2").numberOfShardsWithState(STARTED), equalTo(2)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ExpectedShardSizeAllocationTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ExpectedShardSizeAllocationTests.java index e649b8f6c180b..eb1385ec35210 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ExpectedShardSizeAllocationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ExpectedShardSizeAllocationTests.java @@ -28,7 +28,6 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; @@ -36,7 +35,6 @@ import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand; import org.elasticsearch.common.settings.Settings; -import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.not; @@ -78,8 +76,7 @@ public Long getShardSize(ShardRouting shardRouting) { assertEquals(byteSize, clusterState.getRoutingTable() .shardsWithState(ShardRoutingState.INITIALIZING).get(0).getExpectedShardSize()); logger.info("Start the primary shard"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertEquals(1, clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(ShardRoutingState.STARTED)); assertEquals(1, clusterState.getRoutingNodes().unassigned().size()); @@ -122,7 +119,7 @@ public Long getShardSize(ShardRouting shardRouting) { clusterState = allocation.reroute(clusterState, "reroute"); logger.info("start primary shard"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); logger.info("move the shard"); String existingNodeId = clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId(); @@ -144,7 +141,7 @@ public Long getShardSize(ShardRouting shardRouting) { assertEquals(clusterState.getRoutingNodes().node(toNodeId).iterator().next().getExpectedShardSize(), byteSize); logger.info("finish moving the shard"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertThat(clusterState.getRoutingNodes().node(existingNodeId).isEmpty(), equalTo(true)); assertThat(clusterState.getRoutingNodes().node(toNodeId).iterator().next().state(), equalTo(ShardRoutingState.STARTED)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/FailedNodeRoutingTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/FailedNodeRoutingTests.java index 5ec47d525dc19..2120cea6c3178 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/FailedNodeRoutingTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/FailedNodeRoutingTests.java @@ -88,14 +88,11 @@ public void testSimpleFailedNodeTest() { clusterState = strategy.reroute(clusterState, "reroute"); logger.info("start all the primary shards, replicas will start initializing"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("start the replica shards"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); + RoutingNodes routingNodes = clusterState.getRoutingNodes(); assertThat(routingNodes.node("node1").numberOfShardsWithState(STARTED), equalTo(1)); assertThat(routingNodes.node("node2").numberOfShardsWithState(STARTED), equalTo(1)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/FailedShardsRoutingTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/FailedShardsRoutingTests.java index c2d6a67468f3f..d9b6275b5c749 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/FailedShardsRoutingTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/FailedShardsRoutingTests.java @@ -20,7 +20,6 @@ package org.elasticsearch.cluster.routing.allocation; import com.carrotsearch.hppc.cursors.ObjectCursor; - import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.Version; @@ -41,7 +40,6 @@ import org.elasticsearch.test.VersionUtils; import java.util.ArrayList; -import java.util.Collections; import java.util.HashSet; import java.util.Set; @@ -84,9 +82,9 @@ public void testFailedShardPrimaryRelocatingToAndFrom() { clusterState = allocation.reroute(clusterState, "reroute"); // starting primaries - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); // starting replicas - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); logger.info("--> verifying all is allocated"); assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1)); @@ -170,10 +168,7 @@ public void testFailPrimaryStartedCheckReplicaElected() { clusterState = strategy.reroute(clusterState, "reroute"); logger.info("Start the shards (primaries)"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - - - ClusterState newState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + ClusterState newState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; @@ -191,8 +186,7 @@ public void testFailPrimaryStartedCheckReplicaElected() { } logger.info("Start the shards (backups)"); - routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; @@ -305,10 +299,7 @@ public void testSingleShardMultipleAllocationFailures() { } clusterState = ClusterState.builder(clusterState).nodes(nodeBuilder).build(); while (!clusterState.routingTable().shardsWithState(UNASSIGNED).isEmpty()) { - // start all initializing - clusterState = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING)); - // and assign more unassigned - clusterState = strategy.reroute(clusterState, "reroute"); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); } int shardsToFail = randomIntBetween(1, numberOfReplicas); @@ -425,8 +416,7 @@ public void testRebalanceFailure() { clusterState = strategy.reroute(clusterState, "reroute"); logger.info("Start the shards (primaries)"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - ClusterState newState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + ClusterState newState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; @@ -444,8 +434,7 @@ public void testRebalanceFailure() { } logger.info("Start the shards (backups)"); - routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; @@ -468,7 +457,7 @@ public void testRebalanceFailure() { newState = strategy.reroute(clusterState, "reroute"); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; - routingNodes = clusterState.getRoutingNodes(); + RoutingNodes routingNodes = clusterState.getRoutingNodes(); assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(2)); assertThat(routingNodes.node("node1").numberOfShardsWithState(STARTED, RELOCATING), equalTo(2)); @@ -519,13 +508,12 @@ public void testFailAllReplicasInitializingOnPrimaryFail() { assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).size(), equalTo(2)); // start primary shards - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2)); // start one replica so it can take over. - clusterState = allocation.applyStartedShards(clusterState, - Collections.singletonList(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(0))); + clusterState = startShardsAndReroute(allocation, clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(0)); assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(2)); assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); ShardRouting startedReplica = clusterState.getRoutingNodes().activeReplicaWithHighestVersion(shardId); @@ -567,13 +555,12 @@ public void testFailAllReplicasInitializingOnPrimaryFailWhileHavingAReplicaToEle assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).size(), equalTo(2)); // start primary shards - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2)); // start another replica shard, while keep one initializing - clusterState = allocation.applyStartedShards(clusterState, - Collections.singletonList(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(0))); + clusterState = startShardsAndReroute(allocation, clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(0)); assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(2)); assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); @@ -613,7 +600,7 @@ public void testReplicaOnNewestVersionIsPromoted() { assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).size(), equalTo(3)); // start primary shard - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).size(), equalTo(3)); @@ -627,7 +614,7 @@ public void testReplicaOnNewestVersionIsPromoted() { clusterState = allocation.reroute(clusterState, "reroute"); assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1)); assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(2)); assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).size(), equalTo(2)); @@ -641,7 +628,7 @@ public void testReplicaOnNewestVersionIsPromoted() { clusterState = allocation.reroute(clusterState, "reroute"); assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(2)); assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2)); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(4)); assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).size(), equalTo(0)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/FilterRoutingTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/FilterRoutingTests.java index 86e8887688ff2..4d0fb77d90451 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/FilterRoutingTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/FilterRoutingTests.java @@ -194,10 +194,10 @@ private void testClusterFilters(Settings.Builder allocationServiceSettings, Disc assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2)); logger.info("--> start the shards (primaries)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("--> start the shards (replicas)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("--> make sure shards are only allocated on tag1 with value1 and value2"); final List startedShards = clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED); @@ -297,10 +297,10 @@ private void testIndexFilters(Settings.Builder initialIndexSettings, Settings.Bu assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2)); logger.info("--> start the shards (primaries)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("--> start the shards (replicas)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("--> make sure shards are only allocated on tag1 with value1 and value2"); List startedShards = clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED); @@ -323,7 +323,7 @@ private void testIndexFilters(Settings.Builder initialIndexSettings, Settings.Bu assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(2)); logger.info("--> finish relocation"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); startedShards = clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED); assertThat(startedShards.size(), equalTo(4)); @@ -358,7 +358,7 @@ public void testConcurrentRecoveriesAfterShardsCannotRemainOnNode() { assertThat(clusterState.getRoutingNodes().node(node2.getId()).numberOfShardsWithState(INITIALIZING), equalTo(2)); logger.info("--> start the shards (only primaries)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("--> make sure all shards are started"); assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(4)); @@ -377,7 +377,7 @@ public void testConcurrentRecoveriesAfterShardsCannotRemainOnNode() { assertThat(clusterState.getRoutingNodes().node(node2.getId()).numberOfShardsWithState(STARTED), equalTo(2)); logger.info("--> start the shards (only primaries)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("--> move second shard from node1 to node2"); clusterState = strategy.reroute(clusterState, "reroute"); @@ -385,7 +385,7 @@ public void testConcurrentRecoveriesAfterShardsCannotRemainOnNode() { assertThat(clusterState.getRoutingNodes().node(node2.getId()).numberOfShardsWithState(STARTED), equalTo(3)); logger.info("--> start the shards (only primaries)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); clusterState = strategy.reroute(clusterState, "reroute"); assertThat(clusterState.getRoutingNodes().node(node2.getId()).numberOfShardsWithState(STARTED), equalTo(4)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/InSyncAllocationIdTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/InSyncAllocationIdTests.java index 5e1ff70d9d68e..e5af47119cc87 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/InSyncAllocationIdTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/InSyncAllocationIdTests.java @@ -32,6 +32,7 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.allocation.command.AllocateEmptyPrimaryAllocationCommand; import org.elasticsearch.cluster.routing.allocation.command.AllocationCommands; +import org.elasticsearch.common.Priority; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.set.Sets; import org.junit.Before; @@ -43,7 +44,6 @@ import java.util.List; import java.util.Set; -import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED; import static org.elasticsearch.cluster.routing.ShardRoutingState.UNASSIGNED; import static org.hamcrest.Matchers.equalTo; @@ -58,7 +58,8 @@ public class InSyncAllocationIdTests extends ESAllocationTestCase { @Before public void setupAllocationService() { allocation = createAllocationService(); - failedClusterStateTaskExecutor = new ShardStateAction.ShardFailedClusterStateTaskExecutor(allocation, null, logger); + failedClusterStateTaskExecutor + = new ShardStateAction.ShardFailedClusterStateTaskExecutor(allocation, null, () -> Priority.NORMAL, logger); } public void testInSyncAllocationIdsUpdated() { @@ -84,7 +85,7 @@ public void testInSyncAllocationIdsUpdated() { assertThat(clusterState.metaData().index("test-old").inSyncAllocationIds(0), equalTo(new HashSet<>(Arrays.asList("x", "y")))); logger.info("start primary shard"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertThat(clusterState.getRoutingTable().shardsWithState(STARTED).size(), equalTo(1)); assertThat(clusterState.metaData().index("test").inSyncAllocationIds(0).size(), equalTo(1)); @@ -93,7 +94,7 @@ public void testInSyncAllocationIdsUpdated() { assertThat(clusterState.metaData().index("test-old").inSyncAllocationIds(0), equalTo(new HashSet<>(Arrays.asList("x", "y")))); logger.info("start replica shards"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertThat(clusterState.metaData().index("test").inSyncAllocationIds(0).size(), equalTo(3)); @@ -128,7 +129,7 @@ public void testInSyncAllocationIdsUpdated() { assertThat(clusterState.metaData().index("test").inSyncAllocationIds(0).size(), equalTo(0)); logger.info("start primary shard"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertThat(clusterState.getRoutingTable().shardsWithState(STARTED).size(), equalTo(1)); assertThat(clusterState.metaData().index("test").inSyncAllocationIds(0).size(), equalTo(1)); @@ -164,7 +165,7 @@ public void testDeadNodesBeforeReplicaFailed() throws Exception { logger.info("fail replica (for which there is no shard routing in the CS anymore)"); assertNull(clusterState.getRoutingNodes().getByAllocationId(replicaShard.shardId(), replicaShard.allocationId().getId())); ShardStateAction.ShardFailedClusterStateTaskExecutor failedClusterStateTaskExecutor = - new ShardStateAction.ShardFailedClusterStateTaskExecutor(allocation, null, logger); + new ShardStateAction.ShardFailedClusterStateTaskExecutor(allocation, null, () -> Priority.NORMAL, logger); long primaryTerm = clusterState.metaData().index("test").primaryTerm(0); clusterState = failedClusterStateTaskExecutor.execute(clusterState, Arrays.asList( new FailedShardEntry(shardRoutingTable.shardId(), replicaShard.allocationId().getId(), primaryTerm, "dummy", null, true)) @@ -242,7 +243,7 @@ public void testInSyncIdsNotGrowingWithoutBounds() throws Exception { clusterState = allocation.reroute(clusterState, "reroute"); logger.info("start replica shards"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); logger.info("remove the node"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) @@ -305,7 +306,7 @@ public void testInSyncIdsNotTrimmedWhenNotGrowing() throws Exception { assertEquals(inSyncSet, clusterState.metaData().index("test").inSyncAllocationIds(0)); logger.info("start primary shard"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); // in-sync allocation ids should not be updated assertEquals(inSyncSet, clusterState.metaData().index("test").inSyncAllocationIds(0)); } @@ -360,7 +361,7 @@ private ClusterState createOnePrimaryOneReplicaClusterState(AllocationService al assertThat(clusterState.metaData().index("test").inSyncAllocationIds(0).size(), equalTo(0)); logger.info("start primary shard"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertThat(clusterState.getRoutingTable().shardsWithState(STARTED).size(), equalTo(1)); assertThat(clusterState.metaData().index("test").inSyncAllocationIds(0).size(), equalTo(1)); @@ -368,7 +369,7 @@ private ClusterState createOnePrimaryOneReplicaClusterState(AllocationService al equalTo(clusterState.metaData().index("test").inSyncAllocationIds(0).iterator().next())); logger.info("start replica shard"); - clusterState = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); assertThat(clusterState.metaData().index("test").inSyncAllocationIds(0).size(), equalTo(2)); return clusterState; } diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/IndexBalanceTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/IndexBalanceTests.java index a44f4c4f913cf..686dcc0643ae5 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/IndexBalanceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/IndexBalanceTests.java @@ -105,8 +105,7 @@ public void testBalanceAllNodesStarted() { newState = strategy.reroute(clusterState, "reroute"); assertThat(newState, equalTo(clusterState)); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3)); @@ -126,11 +125,10 @@ public void testBalanceAllNodesStarted() { assertThat(newState, equalTo(clusterState)); logger.info("Start the more shards"); - routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; - routingNodes = clusterState.getRoutingNodes(); + RoutingNodes routingNodes = clusterState.getRoutingNodes(); assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3)); for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) { @@ -226,8 +224,7 @@ public void testBalanceIncrementallyStartNodes() { logger.info("Start the primary shard"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3)); @@ -247,13 +244,11 @@ public void testBalanceIncrementallyStartNodes() { assertThat(newState, equalTo(clusterState)); logger.info("Start the backup shard"); - routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; - routingNodes = clusterState.getRoutingNodes(); - assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3)); + assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3)); for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) { assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2)); assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2)); @@ -282,20 +277,18 @@ public void testBalanceIncrementallyStartNodes() { assertThat(newState, equalTo(clusterState)); logger.info("Start the backup shard"); - routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; - routingNodes = clusterState.getRoutingNodes(); - assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3)); + assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3)); - newState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; - routingNodes = clusterState.getRoutingNodes(); + RoutingNodes routingNodes = clusterState.getRoutingNodes(); - assertThat(clusterState.routingTable().index("test1").shards().size(), equalTo(3)); + assertThat(clusterState.routingTable().index("test1").shards().size(), equalTo(3)); assertThat(routingNodes.node("node1").numberOfShardsWithState(STARTED), equalTo(4)); assertThat(routingNodes.node("node2").numberOfShardsWithState(STARTED), equalTo(4)); @@ -360,8 +353,7 @@ public void testBalanceAllNodesStartedAddIndex() { assertThat(newState, equalTo(clusterState)); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3)); @@ -381,11 +373,10 @@ public void testBalanceAllNodesStartedAddIndex() { assertThat(newState, equalTo(clusterState)); logger.info("Start the more shards"); - routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; - routingNodes = clusterState.getRoutingNodes(); + RoutingNodes routingNodes = clusterState.getRoutingNodes(); assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3)); for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) { assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2)); @@ -436,8 +427,7 @@ public void testBalanceAllNodesStartedAddIndex() { assertThat(newState, equalTo(clusterState)); - routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; assertThat(clusterState.routingTable().index("test1").shards().size(), equalTo(3)); @@ -457,8 +447,7 @@ public void testBalanceAllNodesStartedAddIndex() { assertThat(newState, equalTo(clusterState)); logger.info("Start the more shards"); - routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; routingNodes = clusterState.getRoutingNodes(); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/MaxRetryAllocationDeciderTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/MaxRetryAllocationDeciderTests.java index 2ce0b7b89bec2..f465d401ff206 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/MaxRetryAllocationDeciderTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/MaxRetryAllocationDeciderTests.java @@ -219,8 +219,7 @@ public void testFailedAllocation() { routingTable.index("idx").shard(0).shards().get(0), null, new RoutingAllocation(null, null, clusterState, null, 0))); // now we start the shard - clusterState = strategy.applyStartedShards(clusterState, Collections.singletonList( - routingTable.index("idx").shard(0).shards().get(0))); + clusterState = startShardsAndReroute(strategy, clusterState, routingTable.index("idx").shard(0).shards().get(0)); routingTable = clusterState.routingTable(); // all counters have been reset to 0 ie. no unassigned info diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java index f0d25175f6e43..e4b215cf2e25c 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java @@ -122,8 +122,7 @@ public void testDoNotAllocateFromPrimary() { } logger.info("start all the primary shards, replicas will start initializing"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) { assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(3)); @@ -132,8 +131,7 @@ public void testDoNotAllocateFromPrimary() { assertThat(clusterState.routingTable().index("test").shard(i).replicaShardsWithState(UNASSIGNED).size(), equalTo(1)); } - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) { assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(3)); @@ -167,8 +165,7 @@ public void testDoNotAllocateFromPrimary() { assertThat(clusterState.routingTable().index("test").shard(i).replicaShardsWithState(INITIALIZING).size(), equalTo(1)); } - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) { assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(3)); @@ -396,7 +393,7 @@ private ClusterState stabilize(ClusterState clusterState, AllocationService serv boolean changed; do { logger.trace("RoutingNodes: {}", clusterState.getRoutingNodes()); - ClusterState newState = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + ClusterState newState = startInitializingShardsAndReroute(service, clusterState); changed = newState.equals(clusterState) == false; clusterState = newState; routingNodes = clusterState.getRoutingNodes(); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PreferLocalPrimariesToRelocatingPrimariesTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PreferLocalPrimariesToRelocatingPrimariesTests.java index d80feedabf0e6..34c925bc0bcac 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PreferLocalPrimariesToRelocatingPrimariesTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PreferLocalPrimariesToRelocatingPrimariesTests.java @@ -74,8 +74,8 @@ public void testPreferLocalPrimaryAllocationOverFiltered() { clusterState = strategy.reroute(clusterState, "reroute"); - while (!clusterState.getRoutingNodes().shardsWithState(INITIALIZING).isEmpty()) { - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + while (clusterState.getRoutingNodes().shardsWithState(INITIALIZING).isEmpty() == false) { + clusterState = startInitializingShardsAndReroute(strategy, clusterState); } logger.info("remove one of the nodes and apply filter to move everything from another node"); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PreferPrimaryAllocationTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PreferPrimaryAllocationTests.java index d54d798544c84..34052b4c79132 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PreferPrimaryAllocationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PreferPrimaryAllocationTests.java @@ -65,8 +65,8 @@ public void testPreferPrimaryAllocationOverReplicas() { .add(newNode("node1")).add(newNode("node2"))).build(); clusterState = strategy.reroute(clusterState, "reroute"); - while (!clusterState.getRoutingNodes().shardsWithState(INITIALIZING).isEmpty()) { - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + while (clusterState.getRoutingNodes().shardsWithState(INITIALIZING).isEmpty() == false) { + clusterState = startInitializingShardsAndReroute(strategy, clusterState); } logger.info("increasing the number of replicas to 1, and perform a reroute (to get the replicas allocation going)"); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryElectionRoutingTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryElectionRoutingTests.java index f06a38a2ba462..e9a7afe6be3c6 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryElectionRoutingTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryElectionRoutingTests.java @@ -67,11 +67,11 @@ public void testBackupElectionToPrimaryWhenPrimaryCanBeAllocatedToAnotherNode() logger.info("Start the primary shard (on node1)"); RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, routingNodes.node("node1")); logger.info("Start the backup shard (on node2)"); routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.node("node2").shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, routingNodes.node("node2")); logger.info("Adding third node and reroute and kill first node"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) @@ -115,9 +115,8 @@ public void testRemovingInitializingReplicasIfPrimariesFails() { clusterState = allocation.reroute(clusterState, "reroute"); logger.info("Start the primary shards"); + clusterState = startInitializingShardsAndReroute(allocation, clusterState); RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = allocation.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); assertThat(routingNodes.shardsWithState(STARTED).size(), equalTo(2)); assertThat(routingNodes.shardsWithState(INITIALIZING).size(), equalTo(2)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryNotRelocatedWhileBeingRecoveredTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryNotRelocatedWhileBeingRecoveredTests.java index a3fddda7b6e33..e53638bdca485 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryNotRelocatedWhileBeingRecoveredTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryNotRelocatedWhileBeingRecoveredTests.java @@ -64,7 +64,7 @@ public void testPrimaryNotRelocatedWhileBeingRecoveredFrom() { logger.info("Start the primary shard (on node1)"); RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState, routingNodes.node("node1")); assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(5)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java index 0c8ebff594598..e7f46871b60e0 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java @@ -135,7 +135,7 @@ public void testRandomDecisions() { clusterState = strategy.reroute(clusterState, "reroute"); } if (clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size() > 0) { - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); } } logger.info("Fill up nodes such that every shard can be allocated"); @@ -158,7 +158,7 @@ public void testRandomDecisions() { iterations++; clusterState = strategy.reroute(clusterState, "reroute"); if (clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size() > 0) { - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); } } while (clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size() != 0 || diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RebalanceAfterActiveTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RebalanceAfterActiveTests.java index ff54fa06095f2..33c7b9afce6ea 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RebalanceAfterActiveTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RebalanceAfterActiveTests.java @@ -99,9 +99,7 @@ public Long getShardSize(ShardRouting shardRouting) { } logger.info("start all the primary shards, replicas will start initializing"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) { assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2)); @@ -116,7 +114,6 @@ public Long getShardSize(ShardRouting shardRouting) { .add(newNode("node7")).add(newNode("node8")).add(newNode("node9")).add(newNode("node10"))) .build(); clusterState = strategy.reroute(clusterState, "reroute"); - routingNodes = clusterState.getRoutingNodes(); for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) { assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2)); @@ -127,9 +124,7 @@ public Long getShardSize(ShardRouting shardRouting) { } logger.info("start the replica shards, rebalancing should start"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); // we only allow one relocation at a time assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(5)); @@ -146,9 +141,7 @@ public Long getShardSize(ShardRouting shardRouting) { } logger.info("complete relocation, other half of relocation should happen"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); // we now only relocate 3, since 2 remain where they are! assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(7)); @@ -163,9 +156,8 @@ public Long getShardSize(ShardRouting shardRouting) { logger.info("complete relocation, that's it!"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); + RoutingNodes routingNodes = clusterState.getRoutingNodes(); assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(10)); // make sure we have an even relocation diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ReplicaAllocatedAfterPrimaryTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ReplicaAllocatedAfterPrimaryTests.java index 5e61b35b5ec48..3625558178d29 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ReplicaAllocatedAfterPrimaryTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ReplicaAllocatedAfterPrimaryTests.java @@ -89,9 +89,7 @@ public void testBackupIsAllocatedAfterPrimary() { logger.info("Start all the primary shards"); RoutingNodes routingNodes = clusterState.getRoutingNodes(); prevRoutingTable = routingTable; - routingTable = strategy.applyStartedShards(clusterState, - routingNodes.node(nodeHoldingPrimary).shardsWithState(INITIALIZING)).routingTable(); - clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); + routingTable = startInitializingShardsAndReroute(strategy, clusterState, routingNodes.node(nodeHoldingPrimary)).routingTable(); final String nodeHoldingReplica = routingTable.index("test").shard(0).replicaShards().get(0).currentNodeId(); assertThat(nodeHoldingPrimary, not(equalTo(nodeHoldingReplica))); assertThat(prevRoutingTable != routingTable, equalTo(true)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ResizeAllocationDeciderTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ResizeAllocationDeciderTests.java index c64bc51bd5b7c..aa2fb0a3831ab 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ResizeAllocationDeciderTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ResizeAllocationDeciderTests.java @@ -40,7 +40,6 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.gateway.TestGatewayAllocator; -import java.util.Arrays; import java.util.Collections; import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; @@ -61,10 +60,6 @@ public void setUp() throws Exception { } private ClusterState createInitialClusterState(boolean startShards) { - return createInitialClusterState(startShards, Version.CURRENT); - } - - private ClusterState createInitialClusterState(boolean startShards, Version nodeVersion) { MetaData.Builder metaBuilder = MetaData.builder(); metaBuilder.put(IndexMetaData.builder("source").settings(settings(Version.CURRENT)) .numberOfShards(2).numberOfReplicas(0).setRoutingNumShards(16)); @@ -75,8 +70,8 @@ private ClusterState createInitialClusterState(boolean startShards, Version node RoutingTable routingTable = routingTableBuilder.build(); ClusterState clusterState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) .metaData(metaData).routingTable(routingTable).build(); - clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1", nodeVersion)).add(newNode - ("node2", nodeVersion))) + clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1", Version.CURRENT)).add(newNode + ("node2", Version.CURRENT))) .build(); RoutingTable prevRoutingTable = routingTable; routingTable = strategy.reroute(clusterState, "reroute", false).routingTable(); @@ -94,9 +89,9 @@ private ClusterState createInitialClusterState(boolean startShards, Version node if (startShards) { - clusterState = strategy.applyStartedShards(clusterState, - Arrays.asList(routingTable.index("source").shard(0).shards().get(0), - routingTable.index("source").shard(1).shards().get(0))); + clusterState = startShardsAndReroute(strategy, clusterState, + routingTable.index("source").shard(0).shards().get(0), + routingTable.index("source").shard(1).shards().get(0)); routingTable = clusterState.routingTable(); assertEquals(routingTable.index("source").shards().size(), 2); assertEquals(routingTable.index("source").shard(0).shards().get(0).state(), STARTED); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RetryFailedAllocationTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RetryFailedAllocationTests.java index 1ca516da26286..59ee72ddbd6e5 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RetryFailedAllocationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RetryFailedAllocationTests.java @@ -69,7 +69,7 @@ private ShardRouting getReplica() { public void testRetryFailedResetForAllocationCommands() { final int retries = MaxRetryAllocationDecider.SETTING_ALLOCATION_MAX_RETRY.get(Settings.EMPTY); clusterState = strategy.reroute(clusterState, "initial allocation"); - clusterState = strategy.applyStartedShards(clusterState, Collections.singletonList(getPrimary())); + clusterState = startShardsAndReroute(strategy, clusterState, getPrimary()); // Exhaust all replica allocation attempts with shard failures for (int i = 0; i < retries; i++) { @@ -90,7 +90,7 @@ public void testRetryFailedResetForAllocationCommands() { clusterState = result.getClusterState(); assertEquals(ShardRoutingState.INITIALIZING, getReplica().state()); - clusterState = strategy.applyStartedShards(clusterState, Collections.singletonList(getReplica())); + clusterState = startShardsAndReroute(strategy, clusterState, getReplica()); assertEquals(ShardRoutingState.STARTED, getReplica().state()); assertFalse(clusterState.getRoutingNodes().hasUnassignedShards()); } diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RoutingNodesIntegrityTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RoutingNodesIntegrityTests.java index 72364e3fbc925..3d52ae1087218 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RoutingNodesIntegrityTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/RoutingNodesIntegrityTests.java @@ -84,16 +84,14 @@ public void testBalanceAllNodesStarted() { clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())).build(); clusterState = strategy.reroute(clusterState, "reroute"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("Reroute, nothing should change"); ClusterState newState = strategy.reroute(clusterState, "reroute"); assertThat(newState, equalTo(clusterState)); logger.info("Start the more shards"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); routingNodes = clusterState.getRoutingNodes(); assertThat(assertShardStats(routingNodes), equalTo(true)); @@ -101,8 +99,7 @@ public void testBalanceAllNodesStarted() { assertThat(routingNodes.hasInactivePrimaries(), equalTo(false)); assertThat(routingNodes.hasUnassignedPrimaries(), equalTo(false)); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); - + startInitializingShardsAndReroute(strategy, clusterState); } public void testBalanceIncrementallyStartNodes() { @@ -135,19 +132,15 @@ public void testBalanceIncrementallyStartNodes() { clusterState = strategy.reroute(clusterState, "reroute"); logger.info("Start the primary shard"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("Reroute, nothing should change"); clusterState = strategy.reroute(clusterState, "reroute"); logger.info("Start the backup shard"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); - routingNodes = clusterState.getRoutingNodes(); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("Add another node and perform rerouting, nothing will happen since primary not started"); clusterState = ClusterState.builder(clusterState) @@ -159,18 +152,16 @@ public void testBalanceIncrementallyStartNodes() { assertThat(newState, equalTo(clusterState)); logger.info("Start the backup shard"); - routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; - routingNodes = clusterState.getRoutingNodes(); assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3)); - newState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; - routingNodes = clusterState.getRoutingNodes(); + RoutingNodes routingNodes = clusterState.getRoutingNodes(); assertThat(clusterState.routingTable().index("test1").shards().size(), equalTo(3)); @@ -233,7 +224,7 @@ public void testBalanceAllNodesStartedAddIndex() { assertThat(routingNodes.node("node2").numberOfShardsWithState(INITIALIZING), equalTo(1)); assertThat(routingNodes.node("node3").numberOfShardsWithState(INITIALIZING), equalTo(1)); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); routingNodes = clusterState.getRoutingNodes(); assertThat(assertShardStats(routingNodes), equalTo(true)); @@ -249,8 +240,7 @@ public void testBalanceAllNodesStartedAddIndex() { assertThat(newState, equalTo(clusterState)); logger.info("Start the more shards"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); routingNodes = clusterState.getRoutingNodes(); assertThat(assertShardStats(routingNodes), equalTo(true)); @@ -302,8 +292,7 @@ public void testBalanceAllNodesStartedAddIndex() { logger.info("Reroute, start the primaries"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); routingNodes = clusterState.getRoutingNodes(); assertThat(assertShardStats(routingNodes), equalTo(true)); @@ -312,8 +301,7 @@ public void testBalanceAllNodesStartedAddIndex() { assertThat(routingNodes.hasUnassignedPrimaries(), equalTo(false)); logger.info("Reroute, start the replicas"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); routingNodes = clusterState.getRoutingNodes(); assertThat(assertShardStats(routingNodes), equalTo(true)); @@ -344,8 +332,7 @@ public void testBalanceAllNodesStartedAddIndex() { assertThat(routingNodes.hasUnassignedPrimaries(), equalTo(false)); logger.info("Start Recovering shards round 1"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); routingNodes = clusterState.getRoutingNodes(); assertThat(assertShardStats(routingNodes), equalTo(true)); @@ -354,8 +341,7 @@ public void testBalanceAllNodesStartedAddIndex() { assertThat(routingNodes.hasUnassignedPrimaries(), equalTo(false)); logger.info("Start Recovering shards round 2"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); routingNodes = clusterState.getRoutingNodes(); assertThat(assertShardStats(routingNodes), equalTo(true)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/SameShardRoutingTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/SameShardRoutingTests.java index 9856bd064ca72..254d65a0e5b46 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/SameShardRoutingTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/SameShardRoutingTests.java @@ -80,7 +80,7 @@ public void testSameHost() { assertThat(numberOfShardsOfType(clusterState.getRoutingNodes(), ShardRoutingState.INITIALIZING), equalTo(2)); logger.info("--> start all primary shards, no replica will be started since its on the same host"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(numberOfShardsOfType(clusterState.getRoutingNodes(), ShardRoutingState.STARTED), equalTo(2)); assertThat(numberOfShardsOfType(clusterState.getRoutingNodes(), ShardRoutingState.INITIALIZING), equalTo(0)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ShardVersioningTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ShardVersioningTests.java index fb3f75aad5f21..323483f45d233 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ShardVersioningTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ShardVersioningTests.java @@ -23,14 +23,13 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ESAllocationTestCase; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.cluster.ESAllocationTestCase; import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED; @@ -77,10 +76,7 @@ public void testSimple() { } logger.info("start all the primary shards for test1, replicas will start initializing"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable(); - clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); - routingNodes = clusterState.getRoutingNodes(); + routingTable = startInitializingShardsAndReroute(strategy, clusterState, "test1").routingTable(); for (int i = 0; i < routingTable.index("test1").shards().size(); i++) { assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ShardsLimitAllocationTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ShardsLimitAllocationTests.java index 87339868e4c2c..b13d03cf9f4cc 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ShardsLimitAllocationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ShardsLimitAllocationTests.java @@ -27,14 +27,12 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocationDecider; import org.elasticsearch.common.settings.Settings; -import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.elasticsearch.cluster.routing.ShardRoutingState.RELOCATING; import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED; import static org.elasticsearch.cluster.routing.allocation.RoutingNodesUtils.numberOfShardsOfType; @@ -71,8 +69,7 @@ public void testIndexLevelShardsLimitAllocate() { assertThat(clusterState.getRoutingNodes().node("node2").numberOfShardsWithState(ShardRoutingState.INITIALIZING), equalTo(2)); logger.info("Start the primary shards"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(ShardRoutingState.STARTED), equalTo(2)); assertThat(clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(ShardRoutingState.INITIALIZING), equalTo(0)); @@ -81,8 +78,7 @@ public void testIndexLevelShardsLimitAllocate() { assertThat(clusterState.getRoutingNodes().unassigned().size(), equalTo(4)); logger.info("Do another reroute, make sure its still not allocated"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + startInitializingShardsAndReroute(strategy, clusterState); } public void testClusterLevelShardsLimitAllocate() { @@ -114,8 +110,7 @@ public void testClusterLevelShardsLimitAllocate() { assertThat(clusterState.getRoutingNodes().node("node2").numberOfShardsWithState(ShardRoutingState.INITIALIZING), equalTo(1)); logger.info("Start the primary shards"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(ShardRoutingState.STARTED), equalTo(1)); assertThat(clusterState.getRoutingNodes().node("node2").numberOfShardsWithState(ShardRoutingState.STARTED), equalTo(1)); @@ -133,8 +128,7 @@ public void testClusterLevelShardsLimitAllocate() { assertThat(clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(ShardRoutingState.INITIALIZING), equalTo(1)); assertThat(clusterState.getRoutingNodes().node("node2").numberOfShardsWithState(ShardRoutingState.INITIALIZING), equalTo(1)); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(ShardRoutingState.STARTED), equalTo(2)); assertThat(clusterState.getRoutingNodes().node("node2").numberOfShardsWithState(ShardRoutingState.STARTED), equalTo(2)); @@ -171,8 +165,7 @@ public void testIndexLevelShardsLimitRemain() { clusterState = strategy.reroute(clusterState, "reroute"); logger.info("Start the primary shards"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(numberOfShardsOfType(clusterState.getRoutingNodes(), STARTED), equalTo(5)); @@ -194,8 +187,7 @@ public void testIndexLevelShardsLimitRemain() { .nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node2"))).build(); clusterState = strategy.reroute(clusterState, "reroute"); - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(numberOfShardsOfType(clusterState.getRoutingNodes(), STARTED), equalTo(10)); @@ -228,8 +220,7 @@ public void testIndexLevelShardsLimitRemain() { assertThat(clusterState.getRoutingNodes().node("node2").numberOfShardsWithState(STARTED), equalTo(3)); // the first move will destroy the balance and the balancer will move 2 shards from node2 to node one right after // moving the nodes to node2 since we consider INITIALIZING nodes during rebalance - routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); // now we are done compared to EvenShardCountAllocator since the Balancer is not soely based on the average assertThat(clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(STARTED), equalTo(5)); assertThat(clusterState.getRoutingNodes().node("node2").numberOfShardsWithState(STARTED), equalTo(5)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/SingleShardNoReplicasRoutingTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/SingleShardNoReplicasRoutingTests.java index 8aab17160fd65..8c21aa0334189 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/SingleShardNoReplicasRoutingTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/SingleShardNoReplicasRoutingTests.java @@ -91,7 +91,7 @@ public void testSingleIndexStartedShard() { logger.info("Marking the shard as started"); RoutingNodes routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState, routingNodes.node("node1")); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; @@ -150,7 +150,7 @@ public void testSingleIndexStartedShard() { logger.info("Start the shard on node 1"); routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState, routingNodes.node("node1")); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; @@ -290,7 +290,7 @@ public void testMultiIndexEvenDistribution() { assertThat(newState, equalTo(clusterState)); logger.info("Marking the shard as started"); - newState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startShardsAndReroute(strategy, clusterState, routingNodes.shardsWithState(INITIALIZING)); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; @@ -373,8 +373,7 @@ public void testMultiIndexUnevenNodes() { assertThat(newState, equalTo(clusterState)); clusterState = newState; - routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; @@ -390,7 +389,7 @@ public void testMultiIndexUnevenNodes() { assertThat("4 target shard routing are initializing", numberOfShardsOfType(routingNodes, INITIALIZING), equalTo(4)); logger.info("Now, mark the relocated as started"); - newState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState); // routingTable = strategy.reroute(new RoutingStrategyInfo(metaData, routingTable), nodes); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/SingleShardOneReplicaRoutingTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/SingleShardOneReplicaRoutingTests.java index a7980b4d0f9e5..17f5f74547504 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/SingleShardOneReplicaRoutingTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/SingleShardOneReplicaRoutingTests.java @@ -90,7 +90,7 @@ public void testSingleIndexFirstStartPrimaryThenBackups() { logger.info("Start the primary shard (on node1)"); RoutingNodes routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState, routingNodes.node("node1")); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; @@ -113,7 +113,7 @@ public void testSingleIndexFirstStartPrimaryThenBackups() { logger.info("Start the backup shard"); routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.node("node2").shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState, routingNodes.node("node2")); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/StartedShardsRoutingTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/StartedShardsRoutingTests.java index d39912f0b1e6a..50b1d06273040 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/StartedShardsRoutingTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/StartedShardsRoutingTests.java @@ -38,7 +38,6 @@ import org.elasticsearch.index.shard.ShardId; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -74,7 +73,7 @@ public void testStartedShardsMatching() { logger.info("--> test starting of shard"); - ClusterState newState = allocation.applyStartedShards(state, Arrays.asList(initShard)); + ClusterState newState = startShardsAndReroute(allocation, state, initShard); assertThat("failed to start " + initShard + "\ncurrent routing table:" + newState.routingTable(), newState, not(equalTo(state))); assertTrue(initShard + "isn't started \ncurrent routing table:" + newState.routingTable(), @@ -82,7 +81,7 @@ public void testStartedShardsMatching() { state = newState; logger.info("--> testing starting of relocating shards"); - newState = allocation.applyStartedShards(state, Arrays.asList(relocatingShard.getTargetRelocatingShard())); + newState = startShardsAndReroute(allocation, state, relocatingShard.getTargetRelocatingShard()); assertThat("failed to start " + relocatingShard + "\ncurrent routing table:" + newState.routingTable(), newState, not(equalTo(state))); ShardRouting shardRouting = newState.routingTable().index("test").shard(relocatingShard.id()).getShards().get(0); @@ -131,7 +130,7 @@ public void testRelocatingPrimariesWithInitializingReplicas() { ClusterState state = stateBuilder.build(); logger.info("--> test starting of relocating primary shard with initializing / relocating replica"); - ClusterState newState = allocation.applyStartedShards(state, Arrays.asList(relocatingPrimary.getTargetRelocatingShard())); + ClusterState newState = startShardsAndReroute(allocation, state, relocatingPrimary.getTargetRelocatingShard()); assertNotEquals(newState, state); assertTrue(newState.routingTable().index("test").allPrimaryShardsActive()); ShardRouting startedReplica = newState.routingTable().index("test").shard(0).replicaShards().get(0); @@ -152,7 +151,7 @@ public void testRelocatingPrimariesWithInitializingReplicas() { startedShards.add(relocatingPrimary.getTargetRelocatingShard()); startedShards.add(relocatingReplica ? replica.getTargetRelocatingShard() : replica); Collections.shuffle(startedShards, random()); - newState = allocation.applyStartedShards(state, startedShards); + newState = startShardsAndReroute(allocation, state, startedShards); assertNotEquals(newState, state); assertTrue(newState.routingTable().index("test").shard(0).allShardsStarted()); } diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/TenShardsOneReplicaRoutingTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/TenShardsOneReplicaRoutingTests.java index 1a5127bcda501..c11cc99026f55 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/TenShardsOneReplicaRoutingTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/TenShardsOneReplicaRoutingTests.java @@ -104,7 +104,7 @@ public void testSingleIndexFirstStartPrimaryThenBackups() { logger.info("Start the primary shard (on node1)"); RoutingNodes routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState, routingNodes.node("node1")); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; @@ -127,7 +127,7 @@ public void testSingleIndexFirstStartPrimaryThenBackups() { logger.info("Start the backup shard"); routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.node("node2").shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState, routingNodes.node("node2")); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; routingNodes = clusterState.getRoutingNodes(); @@ -162,7 +162,7 @@ public void testSingleIndexFirstStartPrimaryThenBackups() { logger.info("Start the shards on node 3"); routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.node("node3").shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState, routingNodes.node("node3")); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; routingNodes = clusterState.getRoutingNodes(); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ThrottlingAllocationTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ThrottlingAllocationTests.java index 3c88de4b639ca..6a4b68c16f6ba 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ThrottlingAllocationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ThrottlingAllocationTests.java @@ -91,28 +91,28 @@ public void testPrimaryRecoveryThrottling() { assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(17)); logger.info("start initializing, another 3 should initialize"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(3)); assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(3)); assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(14)); logger.info("start initializing, another 3 should initialize"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(6)); assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(3)); assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(11)); logger.info("start initializing, another 1 should initialize"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(9)); assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(1)); assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(10)); logger.info("start initializing, all primaries should be started"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(10)); assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(0)); @@ -144,14 +144,14 @@ public void testReplicaAndPrimaryRecoveryThrottling() { assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(7)); logger.info("start initializing, another 2 should initialize"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(3)); assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(2)); assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(5)); logger.info("start initializing, all primaries should be started"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(5)); assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(0)); @@ -167,14 +167,14 @@ public void testReplicaAndPrimaryRecoveryThrottling() { assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(2)); logger.info("start initializing replicas"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(8)); assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(2)); assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(0)); logger.info("start initializing replicas, all should be started"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(10)); assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(0)); @@ -205,13 +205,13 @@ public void testThrottleIncomingAndOutgoing() { assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node1"), 5); logger.info("start initializing, all primaries should be started"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(5)); assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(4)); assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(0)); - clusterState = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("start another 2 nodes, 5 shards should be relocating - at most 5 are allowed per node"); clusterState = ClusterState.builder(clusterState) @@ -227,7 +227,7 @@ public void testThrottleIncomingAndOutgoing() { assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node1"), 0); assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node1"), 5); - clusterState = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("start the relocating shards, one more shard should relocate away from node1"); assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(8)); @@ -262,7 +262,7 @@ public void testOutgoingThrottlesAllocation() { assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(2)); logger.info("start initializing"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(1)); assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(0)); @@ -279,7 +279,7 @@ public void testOutgoingThrottlesAllocation() { assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node1"), 1); logger.info("start initializing non-primary"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(2)); assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(0)); assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(1)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/UpdateNumberOfReplicasTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/UpdateNumberOfReplicasTests.java index 0d063d7623410..851e2260313f9 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/UpdateNumberOfReplicasTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/UpdateNumberOfReplicasTests.java @@ -27,7 +27,6 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.common.settings.Settings; @@ -75,12 +74,10 @@ public void testUpdateNumberOfReplicas() { clusterState = strategy.reroute(clusterState, "reroute"); logger.info("Start all the primary shards"); - RoutingNodes routingNodes = clusterState.getRoutingNodes(); - clusterState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("Start all the replica shards"); - routingNodes = clusterState.getRoutingNodes(); - ClusterState newState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + ClusterState newState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; @@ -99,7 +96,6 @@ public void testUpdateNumberOfReplicas() { logger.info("add another replica"); - routingNodes = clusterState.getRoutingNodes(); final String[] indices = {"test"}; RoutingTable updatedRoutingTable = RoutingTable.builder(clusterState.routingTable()).updateNumberOfReplicas(2, indices).build(); @@ -137,8 +133,7 @@ public void testUpdateNumberOfReplicas() { assertThat(clusterState.routingTable().index("test").shard(0).replicaShardsWithState(INITIALIZING).get(0).currentNodeId(), equalTo("node3")); - routingNodes = clusterState.getRoutingNodes(); - newState = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); + newState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(newState, not(equalTo(clusterState))); clusterState = newState; @@ -154,7 +149,6 @@ public void testUpdateNumberOfReplicas() { anyOf(equalTo(nodeHoldingReplica), equalTo("node3"))); logger.info("now remove a replica"); - routingNodes = clusterState.getRoutingNodes(); updatedRoutingTable = RoutingTable.builder(clusterState.routingTable()).updateNumberOfReplicas(1, indices).build(); metaData = MetaData.builder(clusterState.metaData()).updateNumberOfReplicas(1, indices).build(); clusterState = ClusterState.builder(clusterState).routingTable(updatedRoutingTable).metaData(metaData).build(); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java index 651ed26e4cf5e..4c48ce7b36068 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java @@ -130,7 +130,7 @@ public void testDiskThreshold() { assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); logger.info("--> start the shards (primaries)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logShardStates(clusterState); // Assert that we're able to start the primary @@ -139,7 +139,7 @@ public void testDiskThreshold() { assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(0)); logger.info("--> start the shards (replicas)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logShardStates(clusterState); // Assert that the replica couldn't be started since node1 doesn't have enough space @@ -158,7 +158,7 @@ public void testDiskThreshold() { assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); logger.info("--> start the shards (replicas)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logShardStates(clusterState); // Assert that the replica couldn't be started since node1 doesn't have enough space @@ -237,7 +237,7 @@ public void testDiskThreshold() { assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); logger.info("--> apply INITIALIZING shards"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logShardStates(clusterState); assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(0)); @@ -336,7 +336,7 @@ public void testDiskThresholdWithAbsoluteSizes() { assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2)); logger.info("--> start the shards (primaries)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logShardStates(clusterState); // Assert that we're able to start the primary and replica, since they were both initializing @@ -362,7 +362,7 @@ public void testDiskThresholdWithAbsoluteSizes() { assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); logger.info("--> start the shards (replicas)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logShardStates(clusterState); // Assert that all replicas could be started @@ -445,7 +445,7 @@ public void testDiskThresholdWithAbsoluteSizes() { assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); logger.info("--> apply INITIALIZING shards"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logShardStates(clusterState); // primary shard already has been relocated away @@ -471,7 +471,7 @@ public void testDiskThresholdWithAbsoluteSizes() { assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); logger.info("--> apply INITIALIZING shards"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("--> final cluster state:"); logShardStates(clusterState); @@ -533,8 +533,7 @@ Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLU routingTable = strategy.reroute(clusterState, "reroute").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); logger.info("--> start the shards (primaries)"); - routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)) - .routingTable(); + routingTable = startInitializingShardsAndReroute(strategy, clusterState).routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); logShardStates(clusterState); @@ -599,8 +598,7 @@ Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLU assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); logger.info("--> start the shards (primaries)"); - routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)) - .routingTable(); + routingTable = startInitializingShardsAndReroute(strategy, clusterState).routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); logShardStates(clusterState); @@ -695,7 +693,7 @@ Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLU assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(4)); logger.info("--> start the shards"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logShardStates(clusterState); // Assert that we're able to start the primary and replicas diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java index 47c4cb60331c4..8f790b41dad04 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java @@ -381,7 +381,7 @@ public void testSizeShrinkIndex() { .build(); clusterState = allocationService.reroute(clusterState, "foo"); - clusterState = allocationService.applyStartedShards(clusterState, + clusterState = startShardsAndReroute(allocationService, clusterState, clusterState.getRoutingTable().index("test").shardsWithState(ShardRoutingState.UNASSIGNED)); RoutingAllocation allocation = new RoutingAllocation(null, clusterState.getRoutingNodes(), clusterState, info, 0); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationShortCircuitTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationShortCircuitTests.java index 9fcd3d97f1fbe..05f32f868a195 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationShortCircuitTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationShortCircuitTests.java @@ -75,9 +75,7 @@ private static ClusterState createClusterStateWithAllShardsAssigned() { while (clusterState.getRoutingNodes().hasUnassignedShards() || clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).isEmpty() == false) { - clusterState = allocationService.applyStartedShards(clusterState, - clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING)); - clusterState = allocationService.reroute(clusterState, "reroute"); + clusterState = startInitializingShardsAndReroute(allocationService, clusterState); } return clusterState; diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationTests.java index b5f68c3956f97..c342443ad707b 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationTests.java @@ -109,8 +109,7 @@ public void testClusterEnableOnlyPrimaries() { assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); logger.info("--> start the shards (primaries)"); - routingTable = strategy.applyStartedShards(clusterState, - clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable(); + routingTable = startInitializingShardsAndReroute(strategy, clusterState).routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0)); @@ -143,9 +142,9 @@ public void testIndexEnableNone() { clusterState = strategy.reroute(clusterState, "reroute"); assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1)); logger.info("--> start the shards (primaries)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("--> start the shards (replicas)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); logger.info("--> verify only enabled index has been routed"); assertThat(clusterState.getRoutingNodes().shardsWithState("enabled", STARTED).size(), equalTo(2)); @@ -192,11 +191,11 @@ public void testEnableClusterBalance() { clusterState = strategy.reroute(clusterState, "reroute"); assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(4)); logger.info("--> start the shards (primaries)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(4)); assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(4)); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(8)); assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0)); @@ -250,7 +249,7 @@ public void testEnableClusterBalance() { default: fail("only replicas, primaries or all are allowed"); } - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(8)); assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0)); @@ -288,7 +287,7 @@ public void testEnableClusterBalanceNoReplicas() { clusterState = strategy.reroute(clusterState, "reroute"); assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(6)); logger.info("--> start the shards (primaries)"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(6)); assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0)); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/FilterAllocationDeciderTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/FilterAllocationDeciderTests.java index 1bafb6b13755a..17d963caf85fd 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/FilterAllocationDeciderTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/FilterAllocationDeciderTests.java @@ -97,14 +97,14 @@ public void testFilterInitialRecovery() { assertEquals(routingTable.index("idx").shard(0).primaryShard().state(), INITIALIZING); assertEquals(routingTable.index("idx").shard(0).primaryShard().currentNodeId(), "node2"); - state = service.applyStartedShards(state, routingTable.index("idx").shard(0).shardsWithState(INITIALIZING)); + state = startShardsAndReroute(service, state, routingTable.index("idx").shard(0).shardsWithState(INITIALIZING)); routingTable = state.routingTable(); // ok now we are started and can be allocated anywhere!! lets see... // first create another copy assertEquals(routingTable.index("idx").shard(0).replicaShards().get(0).state(), INITIALIZING); assertEquals(routingTable.index("idx").shard(0).replicaShards().get(0).currentNodeId(), "node1"); - state = service.applyStartedShards(state, routingTable.index("idx").shard(0).replicaShardsWithState(INITIALIZING)); + state = startShardsAndReroute(service, state, routingTable.index("idx").shard(0).replicaShardsWithState(INITIALIZING)); routingTable = state.routingTable(); assertEquals(routingTable.index("idx").shard(0).replicaShards().get(0).state(), STARTED); assertEquals(routingTable.index("idx").shard(0).replicaShards().get(0).currentNodeId(), "node1"); diff --git a/server/src/test/java/org/elasticsearch/cluster/structure/RoutingIteratorTests.java b/server/src/test/java/org/elasticsearch/cluster/structure/RoutingIteratorTests.java index e9bae772f4ea0..5ed63a393d786 100644 --- a/server/src/test/java/org/elasticsearch/cluster/structure/RoutingIteratorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/structure/RoutingIteratorTests.java @@ -49,7 +49,6 @@ import static java.util.Collections.singletonMap; import static java.util.Collections.unmodifiableMap; -import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.not; @@ -263,9 +262,8 @@ public void testAttributePreferenceRouting() { ).build(); clusterState = strategy.reroute(clusterState, "reroute"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); - - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); // after all are started, check routing iteration ShardIterator shardIterator = clusterState.routingTable().index("test").shard(0) @@ -312,7 +310,7 @@ public void testNodeSelectorRouting(){ clusterState = strategy.reroute(clusterState, "reroute"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); ShardsIterator shardsIterator = clusterState.routingTable().index("test") .shard(0).onlyNodeSelectorActiveInitializingShardsIt("disk:ebs",clusterState.nodes()); @@ -390,9 +388,8 @@ public void testShardsAndPreferNodeRouting() { ).build(); clusterState = strategy.reroute(clusterState, "reroute"); - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); - - clusterState = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); + clusterState = startInitializingShardsAndReroute(strategy, clusterState); OperationRouting operationRouting = new OperationRouting(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); diff --git a/server/src/test/java/org/elasticsearch/indexlifecycle/IndexLifecycleActionIT.java b/server/src/test/java/org/elasticsearch/indexlifecycle/IndexLifecycleActionIT.java index fd96179a4c38b..f1b25e44c9f97 100644 --- a/server/src/test/java/org/elasticsearch/indexlifecycle/IndexLifecycleActionIT.java +++ b/server/src/test/java/org/elasticsearch/indexlifecycle/IndexLifecycleActionIT.java @@ -26,6 +26,7 @@ import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.RoutingNodes; +import org.elasticsearch.common.Priority; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; @@ -81,8 +82,8 @@ public void testIndexLifecycleActionsWith11Shards1Backup() throws Exception { // first wait for 2 nodes in the cluster logger.info("Waiting for replicas to be assigned"); - ClusterHealthResponse clusterHealth = - client().admin().cluster().health(clusterHealthRequest().waitForGreenStatus().waitForNodes("2")).actionGet(); + ClusterHealthResponse clusterHealth = client().admin().cluster().prepareHealth() + .setWaitForGreenStatus().setWaitForNodes("2").setWaitForNoRelocatingShards(true).setWaitForEvents(Priority.LANGUID).get(); logger.info("Done Cluster Health, status {}", clusterHealth.getStatus()); assertThat(clusterHealth.isTimedOut(), equalTo(false)); assertThat(clusterHealth.getStatus(), equalTo(ClusterHealthStatus.GREEN)); @@ -118,7 +119,8 @@ public void testIndexLifecycleActionsWith11Shards1Backup() throws Exception { // first wait for 3 nodes in the cluster logger.info("Waiting for replicas to be assigned"); - clusterHealth = client().admin().cluster().health(clusterHealthRequest().waitForGreenStatus().waitForNodes("3")).actionGet(); + clusterHealth = client().admin().cluster().prepareHealth() + .setWaitForGreenStatus().setWaitForNodes("3").setWaitForNoRelocatingShards(true).setWaitForEvents(Priority.LANGUID).get(); assertThat(clusterHealth.isTimedOut(), equalTo(false)); assertThat(clusterHealth.getStatus(), equalTo(ClusterHealthStatus.GREEN)); @@ -129,8 +131,8 @@ public void testIndexLifecycleActionsWith11Shards1Backup() throws Exception { // explicitly call reroute, so shards will get relocated to the new node (we delay it in ES in case other nodes join) client().admin().cluster().prepareReroute().execute().actionGet(); - clusterHealth = client().admin().cluster().health( - clusterHealthRequest().waitForGreenStatus().waitForNodes("3").waitForNoRelocatingShards(true)).actionGet(); + clusterHealth = client().admin().cluster().prepareHealth() + .setWaitForGreenStatus().setWaitForNodes("3").setWaitForNoRelocatingShards(true).setWaitForEvents(Priority.LANGUID).get(); assertThat(clusterHealth.isTimedOut(), equalTo(false)); assertThat(clusterHealth.getStatus(), equalTo(ClusterHealthStatus.GREEN)); assertThat(clusterHealth.getNumberOfDataNodes(), equalTo(3)); @@ -166,15 +168,16 @@ public void testIndexLifecycleActionsWith11Shards1Backup() throws Exception { internalCluster().stopRandomNode(InternalTestCluster.nameFilter(server_1)); // verify health logger.info("Running Cluster Health"); - clusterHealth = client().admin().cluster().health(clusterHealthRequest().waitForGreenStatus().waitForNodes("2")).actionGet(); + clusterHealth = client().admin().cluster().prepareHealth() + .setWaitForGreenStatus().setWaitForNodes("2").setWaitForNoRelocatingShards(true).setWaitForEvents(Priority.LANGUID).get(); logger.info("Done Cluster Health, status {}", clusterHealth.getStatus()); assertThat(clusterHealth.isTimedOut(), equalTo(false)); assertThat(clusterHealth.getStatus(), equalTo(ClusterHealthStatus.GREEN)); client().admin().cluster().prepareReroute().get(); - clusterHealth = client().admin().cluster().health( - clusterHealthRequest().waitForGreenStatus().waitForNoRelocatingShards(true).waitForNodes("2")).actionGet(); + clusterHealth = client().admin().cluster().prepareHealth() + .setWaitForGreenStatus().setWaitForNodes("2").setWaitForNoRelocatingShards(true).setWaitForEvents(Priority.LANGUID).get(); assertThat(clusterHealth.isTimedOut(), equalTo(false)); assertThat(clusterHealth.getStatus(), equalTo(ClusterHealthStatus.GREEN)); assertThat(clusterHealth.getRelocatingShards(), equalTo(0)); diff --git a/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java b/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java index dc14147330175..5b61573f3b6e6 100644 --- a/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java +++ b/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java @@ -74,6 +74,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.ReplicaAfterPrimaryActiveAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.SameShardAllocationDecider; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Priority; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.IndexScopedSettings; @@ -143,8 +144,10 @@ public ClusterStateChanges(NamedXContentRegistry xContentRegistry, ThreadPool th new RandomAllocationDeciderTests.RandomAllocationDecider(getRandom())))), new TestGatewayAllocator(), new BalancedShardsAllocator(SETTINGS), EmptyClusterInfoService.INSTANCE); - shardFailedClusterStateTaskExecutor = new ShardStateAction.ShardFailedClusterStateTaskExecutor(allocationService, null, logger); - shardStartedClusterStateTaskExecutor = new ShardStateAction.ShardStartedClusterStateTaskExecutor(allocationService, logger); + shardFailedClusterStateTaskExecutor + = new ShardStateAction.ShardFailedClusterStateTaskExecutor(allocationService, null, () -> Priority.NORMAL, logger); + shardStartedClusterStateTaskExecutor + = new ShardStateAction.ShardStartedClusterStateTaskExecutor(allocationService, null, () -> Priority.NORMAL, logger); ActionFilters actionFilters = new ActionFilters(Collections.emptySet()); IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); DestructiveOperations destructiveOperations = new DestructiveOperations(SETTINGS, clusterSettings); diff --git a/server/src/test/java/org/elasticsearch/search/basic/TransportSearchFailuresIT.java b/server/src/test/java/org/elasticsearch/search/basic/TransportSearchFailuresIT.java index 152a57a6c1387..03e856ba8b68b 100644 --- a/server/src/test/java/org/elasticsearch/search/basic/TransportSearchFailuresIT.java +++ b/server/src/test/java/org/elasticsearch/search/basic/TransportSearchFailuresIT.java @@ -86,7 +86,7 @@ public void testFailedSearchWithWrongQuery() throws Exception { ClusterHealthResponse clusterHealth = client() .admin() .cluster() - .health(clusterHealthRequest("test").waitForYellowStatus().waitForNoRelocatingShards(true) + .health(clusterHealthRequest("test").waitForYellowStatus().waitForNoRelocatingShards(true).waitForEvents(Priority.LANGUID) .waitForActiveShards(test.totalNumShards)).actionGet(); logger.info("Done Cluster Health, status {}", clusterHealth.getStatus()); assertThat(clusterHealth.isTimedOut(), equalTo(false)); diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java b/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java index 6e932dd5cdad7..4dd002a4602e1 100644 --- a/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java @@ -52,7 +52,6 @@ import static java.util.Collections.emptyMap; import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; -import static org.elasticsearch.common.util.CollectionUtils.arrayAsArrayList; public abstract class ESAllocationTestCase extends ESTestCase { private static final ClusterSettings EMPTY_CLUSTER_SETTINGS = @@ -123,8 +122,7 @@ protected static ClusterState startRandomInitializingShard(ClusterState cluster if (initializingShards.isEmpty()) { return clusterState; } - return strategy.applyStartedShards(clusterState, - arrayAsArrayList(initializingShards.get(randomInt(initializingShards.size() - 1)))); + return startShardsAndReroute(strategy, clusterState, randomFrom(initializingShards)); } protected static AllocationDeciders yesAllocationDeciders() { @@ -150,11 +148,65 @@ protected ClusterState applyStartedShardsUntilNoChange(ClusterState clusterState do { lastClusterState = clusterState; logger.debug("ClusterState: {}", clusterState.getRoutingNodes()); - clusterState = service.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); + clusterState = startInitializingShardsAndReroute(service, clusterState); } while (lastClusterState.equals(clusterState) == false); return clusterState; } + /** + * Mark all initializing shards as started, then perform a reroute (which may start some other shards initializing). + * + * @return the cluster state after completing the reroute. + */ + public static ClusterState startInitializingShardsAndReroute(AllocationService allocationService, ClusterState clusterState) { + return startShardsAndReroute(allocationService, clusterState, clusterState.routingTable().shardsWithState(INITIALIZING)); + } + + /** + * Mark all initializing shards on the given node as started, then perform a reroute (which may start some other shards initializing). + * + * @return the cluster state after completing the reroute. + */ + public static ClusterState startInitializingShardsAndReroute(AllocationService allocationService, + ClusterState clusterState, + RoutingNode routingNode) { + return startShardsAndReroute(allocationService, clusterState, routingNode.shardsWithState(INITIALIZING)); + } + + /** + * Mark all initializing shards for the given index as started, then perform a reroute (which may start some other shards initializing). + * + * @return the cluster state after completing the reroute. + */ + public static ClusterState startInitializingShardsAndReroute(AllocationService allocationService, + ClusterState clusterState, + String index) { + return startShardsAndReroute(allocationService, clusterState, + clusterState.routingTable().index(index).shardsWithState(INITIALIZING)); + } + + /** + * Mark the given shards as started, then perform a reroute (which may start some other shards initializing). + * + * @return the cluster state after completing the reroute. + */ + public static ClusterState startShardsAndReroute(AllocationService allocationService, + ClusterState clusterState, + ShardRouting... initializingShards) { + return startShardsAndReroute(allocationService, clusterState, Arrays.asList(initializingShards)); + } + + /** + * Mark the given shards as started, then perform a reroute (which may start some other shards initializing). + * + * @return the cluster state after completing the reroute. + */ + public static ClusterState startShardsAndReroute(AllocationService allocationService, + ClusterState clusterState, + List initializingShards) { + return allocationService.reroute(allocationService.applyStartedShards(clusterState, initializingShards), "reroute after starting"); + } + public static class TestAllocateDecision extends AllocationDecider { private final Decision decision;