Defer reroute when starting shards (#44539)

Today we reroute the cluster as part of the process of starting a shard, which
runs at `URGENT` priority. In large clusters, rerouting may take some time to
complete, and this means that a mere trickle of shard-started events can cause
starvation for other, lower-priority, tasks that are pending on the master.

However, it isn't really necessary to perform a reroute when starting a shard,
as long as one occurs eventually. This commit removes the inline reroute from
the process of starting a shard and replaces it with a deferred one that runs
at `NORMAL` priority, avoiding starvation of higher-priority tasks.

Backport of #44433 and #44543.
This commit is contained in:
David Turner 2019-07-18 14:10:40 +01:00 committed by GitHub
parent 4c95cc3223
commit 452f7f67a0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
63 changed files with 630 additions and 517 deletions

View File

@ -68,3 +68,26 @@ Starting in version 7.4, a `+` in a URL will be encoded as `%2B` by all REST API
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.
[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.

View File

@ -48,6 +48,7 @@ import org.elasticsearch.common.Priority;
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.Locale;
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<Priority> 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<FailedShardEntry> remoteFailedShardsDeduplicator = new TransportRequestDeduplicator<>();
@ -97,11 +123,18 @@ public class ShardStateAction {
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 class ShardStateAction {
}, changePredicate);
}
private void setFollowUpRerouteTaskPriority(Priority followUpRerouteTaskPriority) {
this.followUpRerouteTaskPriority = followUpRerouteTaskPriority;
}
private static class ShardFailedTransportHandler implements TransportRequestHandler<FailedShardEntry> {
private final ClusterService clusterService;
private final ShardFailedClusterStateTaskExecutor shardFailedClusterStateTaskExecutor;
@ -285,11 +322,14 @@ public class ShardStateAction {
private final AllocationService allocationService;
private final RerouteService rerouteService;
private final Logger logger;
private final Supplier<Priority> prioritySupplier;
public ShardFailedClusterStateTaskExecutor(AllocationService allocationService, RerouteService rerouteService, Logger logger) {
public ShardFailedClusterStateTaskExecutor(AllocationService allocationService, RerouteService rerouteService,
Supplier<Priority> prioritySupplier, Logger logger) {
this.allocationService = allocationService;
this.rerouteService = rerouteService;
this.logger = logger;
this.prioritySupplier = prioritySupplier;
}
@Override
@ -383,7 +423,7 @@ public class ShardStateAction {
// 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 class ShardStateAction {
implements ClusterStateTaskExecutor<StartedShardEntry>, ClusterStateTaskListener {
private final AllocationService allocationService;
private final Logger logger;
private final RerouteService rerouteService;
private final Supplier<Priority> prioritySupplier;
public ShardStartedClusterStateTaskExecutor(AllocationService allocationService, Logger logger) {
public ShardStartedClusterStateTaskExecutor(AllocationService allocationService, RerouteService rerouteService,
Supplier<Priority> prioritySupplier, Logger logger) {
this.allocationService = allocationService;
this.logger = logger;
this.rerouteService = rerouteService;
this.prioritySupplier = prioritySupplier;
}
@Override
@ -598,6 +643,13 @@ public class ShardStateAction {
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 {

View File

@ -109,7 +109,7 @@ public class AllocationService {
Collections.sort(startedShards, Comparator.comparing(ShardRouting::primary));
applyStartedShards(allocation, startedShards);
gatewayAllocator.applyStartedShards(allocation, startedShards);
reroute(allocation);
assert RoutingNodes.assertShardStats(allocation.routingNodes());
String startedShardsAsString
= firstListElementsToCommaDelimitedString(startedShards, s -> s.shardId().toString(), logger.isDebugEnabled());
return buildResultAndLogHealthChange(clusterState, allocation, "shards started [" + startedShardsAsString + "]");

View File

@ -32,6 +32,7 @@ import org.elasticsearch.cluster.ClusterName;
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 final class ClusterSettings extends AbstractScopedSettings {
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,

View File

@ -25,6 +25,7 @@ import org.elasticsearch.action.admin.indices.create.CreateIndexClusterStateUpda
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.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;
@ -116,8 +116,7 @@ public class TransportResizeActionTests extends ESTestCase {
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 class TransportResizeActionTests extends ESTestCase {
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 class TransportResizeActionTests extends ESTestCase {
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 class TransportResizeActionTests extends ESTestCase {
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));

View File

@ -466,7 +466,7 @@ public class TransportReplicationActionTests extends ESTestCase {
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());

View File

@ -32,7 +32,6 @@ import org.elasticsearch.cluster.metadata.MetaData;
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.AllocationService;
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.Index;
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 class ShardFailedClusterStateTaskExecutorTests extends ESAllocationTestCa
.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 class ShardFailedClusterStateTaskExecutorTests extends ESAllocationTestCa
List<FailedShardEntry> failingTasks = createExistingShards(currentState, reason);
List<FailedShardEntry> 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<FailedShard> failedShards, List<StaleShard> staleShards) {
throw new RuntimeException("simulated applyFailedShards failure");
@ -165,14 +163,14 @@ public class ShardFailedClusterStateTaskExecutorTests extends ESAllocationTestCa
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<String> 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<String> newInSync = appliedState.metaData().index(INDEX).inSyncAllocationIds(0);
assertThat(newInSync, equalTo(oldInSync));
}
@ -180,7 +178,7 @@ public class ShardFailedClusterStateTaskExecutorTests extends ESAllocationTestCa
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<String> newInSync = appliedState.metaData().index(INDEX).inSyncAllocationIds(0);
assertThat(Sets.difference(oldInSync, newInSync), contains(failedAllocationId));
}
@ -192,11 +190,9 @@ public class ShardFailedClusterStateTaskExecutorTests extends ESAllocationTestCa
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<ShardStateAction.FailedShardEntry> createExistingShards(ClusterState currentState, String reason) {

View File

@ -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.IndexShardRoutingTable;
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<Void> 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 {

View File

@ -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<Class<? extends Plugin>> 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<ClusterUpdateSettingsResponse> 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)
}
}

View File

@ -24,13 +24,13 @@ import org.elasticsearch.Version;
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 class MetaDataCreateIndexServiceTests extends ESTestCase {
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 class MetaDataCreateIndexServiceTests extends ESTestCase {
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 @@ public class MetaDataCreateIndexServiceTests extends ESTestCase {
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);

View File

@ -45,7 +45,6 @@ import java.util.concurrent.atomic.AtomicReference;
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 class DelayedAllocationServiceTests extends ESAllocationTestCase {
.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 class DelayedAllocationServiceTests extends ESAllocationTestCase {
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<ShardRouting> allShards = clusterState.getRoutingTable().allShards("test");
@ -228,9 +227,9 @@ public class DelayedAllocationServiceTests extends ESAllocationTestCase {
// 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 class DelayedAllocationServiceTests extends ESAllocationTestCase {
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")) {

View File

@ -110,10 +110,10 @@ public class PrimaryTermsTests extends ESAllocationTestCase {
}
private boolean startInitializingShards(String index) {
final List<ShardRouting> startedShards = this.clusterState.getRoutingNodes().shardsWithState(index, INITIALIZING);
final List<ShardRouting> 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;
}

View File

@ -38,7 +38,6 @@ import java.util.Arrays;
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 @@ public class RoutingTableTests extends ESAllocationTestCase {
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) {

View File

@ -46,7 +46,6 @@ import java.io.IOException;
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 class UnassignedInfoTests extends ESAllocationTestCase {
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 class UnassignedInfoTests extends ESAllocationTestCase {
.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 class UnassignedInfoTests extends ESAllocationTestCase {
.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 class UnassignedInfoTests extends ESAllocationTestCase {
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 class UnassignedInfoTests extends ESAllocationTestCase {
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();

View File

@ -123,7 +123,7 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
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 class AddIncrementallyTests extends ESAllocationTestCase {
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 class AddIncrementallyTests extends ESAllocationTestCase {
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 class AddIncrementallyTests extends ESAllocationTestCase {
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 class AddIncrementallyTests extends ESAllocationTestCase {
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 class AddIncrementallyTests extends ESAllocationTestCase {
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 class AddIncrementallyTests extends ESAllocationTestCase {
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 class AddIncrementallyTests extends ESAllocationTestCase {
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 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
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 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
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 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
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");

View File

@ -101,7 +101,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
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 class AllocationCommandsTests extends ESAllocationTestCase {
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 class AllocationCommandsTests extends ESAllocationTestCase {
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 class AllocationCommandsTests extends ESAllocationTestCase {
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 class AllocationCommandsTests extends ESAllocationTestCase {
Set<String> 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 class AllocationCommandsTests extends ESAllocationTestCase {
}
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 class AllocationCommandsTests extends ESAllocationTestCase {
}
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 class AllocationCommandsTests extends ESAllocationTestCase {
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,7 +491,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
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));
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));
@ -630,7 +630,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
.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 class AllocationCommandsTests extends ESAllocationTestCase {
.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 class AllocationCommandsTests extends ESAllocationTestCase {
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));

View File

@ -80,18 +80,18 @@ public class AllocationPriorityTests extends ESAllocationTestCase {
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());

View File

@ -80,10 +80,10 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 = startInitializingShardsAndReroute(strategy, clusterState);
}
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(10));
@ -341,7 +341,7 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
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));
@ -382,10 +382,10 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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 class AwarenessAllocationTests extends ESAllocationTestCase {
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));

View File

@ -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 org.hamcrest.Matchers;
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 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
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 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
}
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 class BalanceConfigurationTests extends ESAllocationTestCase {
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 class BalanceConfigurationTests extends ESAllocationTestCase {
}
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();

View File

@ -21,6 +21,7 @@ package org.elasticsearch.cluster.routing.allocation;
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 @@ public class BalanceUnbalancedClusterTests extends CatAllocationTestCase {
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<String, Integer> counts = new HashMap<>();
for (IndexShardRoutingTable table : clusterState.routingTable().index(index)) {

View File

@ -160,7 +160,7 @@ public abstract class CatAllocationTestCase extends ESAllocationTestCase {
}
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;

View File

@ -84,8 +84,7 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
.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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
.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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
.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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
.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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
.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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
.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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
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 class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
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));

View File

@ -27,7 +27,6 @@ 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.common.settings.Settings;
@ -81,8 +80,7 @@ public class ConcurrentRebalanceRoutingTests extends ESAllocationTestCase {
}
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 class ConcurrentRebalanceRoutingTests extends ESAllocationTestCase {
}
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));

View File

@ -67,9 +67,9 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
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 class DeadNodesAllocationTests extends ESAllocationTestCase {
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 class DeadNodesAllocationTests extends ESAllocationTestCase {
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));

View File

@ -44,7 +44,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
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 class DiskThresholdMonitorTests extends ESAllocationTestCase {
.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<Set<String>> indices = new AtomicReference<>();

View File

@ -32,7 +32,6 @@ import org.elasticsearch.cluster.routing.RoutingNodes;
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 class ElectReplicaAsPrimaryDuringRelocationTests extends ESAllocationTest
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));

View File

@ -28,7 +28,6 @@ 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.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
@ -36,7 +35,6 @@ import org.elasticsearch.cluster.routing.allocation.command.AllocationCommands;
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 class ExpectedShardSizeAllocationTests extends ESAllocationTestCase {
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 class ExpectedShardSizeAllocationTests extends ESAllocationTestCase {
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 class ExpectedShardSizeAllocationTests extends ESAllocationTestCase {
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));

View File

@ -88,14 +88,11 @@ public class FailedNodeRoutingTests extends ESAllocationTestCase {
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));

View File

@ -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.index.shard.ShardId;
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 class FailedShardsRoutingTests extends ESAllocationTestCase {
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 class FailedShardsRoutingTests extends ESAllocationTestCase {
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 class FailedShardsRoutingTests extends ESAllocationTestCase {
}
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 class FailedShardsRoutingTests extends ESAllocationTestCase {
}
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 class FailedShardsRoutingTests extends ESAllocationTestCase {
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 class FailedShardsRoutingTests extends ESAllocationTestCase {
}
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 class FailedShardsRoutingTests extends ESAllocationTestCase {
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 class FailedShardsRoutingTests extends ESAllocationTestCase {
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 class FailedShardsRoutingTests extends ESAllocationTestCase {
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 class FailedShardsRoutingTests extends ESAllocationTestCase {
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 class FailedShardsRoutingTests extends ESAllocationTestCase {
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 class FailedShardsRoutingTests extends ESAllocationTestCase {
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));

View File

@ -194,10 +194,10 @@ public class FilterRoutingTests extends ESAllocationTestCase {
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<ShardRouting> startedShards = clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED);
@ -297,10 +297,10 @@ public class FilterRoutingTests extends ESAllocationTestCase {
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<ShardRouting> startedShards = clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED);
@ -323,7 +323,7 @@ public class FilterRoutingTests extends ESAllocationTestCase {
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 class FilterRoutingTests extends ESAllocationTestCase {
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 class FilterRoutingTests extends ESAllocationTestCase {
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 class FilterRoutingTests extends ESAllocationTestCase {
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));

View File

@ -32,6 +32,7 @@ import org.elasticsearch.cluster.routing.RoutingTable;
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.HashSet;
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 class InSyncAllocationIdTests extends ESAllocationTestCase {
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 class InSyncAllocationIdTests extends ESAllocationTestCase {
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 class InSyncAllocationIdTests extends ESAllocationTestCase {
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 class InSyncAllocationIdTests extends ESAllocationTestCase {
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 class InSyncAllocationIdTests extends ESAllocationTestCase {
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 class InSyncAllocationIdTests extends ESAllocationTestCase {
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 @@ public class InSyncAllocationIdTests extends ESAllocationTestCase {
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 @@ public class InSyncAllocationIdTests extends ESAllocationTestCase {
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;
}

View File

@ -105,8 +105,7 @@ public class IndexBalanceTests extends ESAllocationTestCase {
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 class IndexBalanceTests extends ESAllocationTestCase {
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 class IndexBalanceTests extends ESAllocationTestCase {
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,11 +244,9 @@ public class IndexBalanceTests extends ESAllocationTestCase {
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));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
@ -282,18 +277,16 @@ public class IndexBalanceTests extends ESAllocationTestCase {
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));
@ -360,8 +353,7 @@ public class IndexBalanceTests extends ESAllocationTestCase {
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 class IndexBalanceTests extends ESAllocationTestCase {
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 class IndexBalanceTests extends ESAllocationTestCase {
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 class IndexBalanceTests extends ESAllocationTestCase {
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();

View File

@ -219,8 +219,7 @@ public class MaxRetryAllocationDeciderTests extends ESAllocationTestCase {
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

View File

@ -122,8 +122,7 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
}
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 class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
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 class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
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 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
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();

View File

@ -74,8 +74,8 @@ public class PreferLocalPrimariesToRelocatingPrimariesTests extends ESAllocation
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");

View File

@ -65,8 +65,8 @@ public class PreferPrimaryAllocationTests extends ESAllocationTestCase {
.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)");

View File

@ -67,11 +67,11 @@ public class PrimaryElectionRoutingTests extends ESAllocationTestCase {
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 class PrimaryElectionRoutingTests extends ESAllocationTestCase {
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));

View File

@ -64,7 +64,7 @@ public class PrimaryNotRelocatedWhileBeingRecoveredTests extends ESAllocationTes
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));

View File

@ -135,7 +135,7 @@ public class RandomAllocationDeciderTests extends ESAllocationTestCase {
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 class RandomAllocationDeciderTests extends ESAllocationTestCase {
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 ||

View File

@ -99,9 +99,7 @@ public class RebalanceAfterActiveTests extends ESAllocationTestCase {
}
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 class RebalanceAfterActiveTests extends ESAllocationTestCase {
.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 class RebalanceAfterActiveTests extends ESAllocationTestCase {
}
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 class RebalanceAfterActiveTests extends ESAllocationTestCase {
}
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 class RebalanceAfterActiveTests extends ESAllocationTestCase {
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

View File

@ -89,9 +89,7 @@ public class ReplicaAllocatedAfterPrimaryTests extends ESAllocationTestCase {
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));

View File

@ -40,7 +40,6 @@ import org.elasticsearch.index.Index;
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 class ResizeAllocationDeciderTests extends ESAllocationTestCase {
}
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 @@ public class ResizeAllocationDeciderTests extends ESAllocationTestCase {
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 @@ public class ResizeAllocationDeciderTests extends ESAllocationTestCase {
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);

View File

@ -69,7 +69,7 @@ public class RetryFailedAllocationTests extends ESAllocationTestCase {
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 class RetryFailedAllocationTests extends ESAllocationTestCase {
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());
}

View File

@ -84,16 +84,14 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
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 class RoutingNodesIntegrityTests extends ESAllocationTestCase {
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 class RoutingNodesIntegrityTests extends ESAllocationTestCase {
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 class RoutingNodesIntegrityTests extends ESAllocationTestCase {
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 class RoutingNodesIntegrityTests extends ESAllocationTestCase {
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 class RoutingNodesIntegrityTests extends ESAllocationTestCase {
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 class RoutingNodesIntegrityTests extends ESAllocationTestCase {
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 class RoutingNodesIntegrityTests extends ESAllocationTestCase {
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 class RoutingNodesIntegrityTests extends ESAllocationTestCase {
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 class RoutingNodesIntegrityTests extends ESAllocationTestCase {
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));

View File

@ -80,7 +80,7 @@ public class SameShardRoutingTests extends ESAllocationTestCase {
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));

View File

@ -23,14 +23,13 @@ import org.apache.logging.log4j.LogManager;
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 class ShardVersioningTests extends ESAllocationTestCase {
}
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));

View File

@ -27,14 +27,12 @@ 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.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 class ShardsLimitAllocationTests extends ESAllocationTestCase {
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 class ShardsLimitAllocationTests extends ESAllocationTestCase {
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 class ShardsLimitAllocationTests extends ESAllocationTestCase {
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 class ShardsLimitAllocationTests extends ESAllocationTestCase {
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 class ShardsLimitAllocationTests extends ESAllocationTestCase {
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 class ShardsLimitAllocationTests extends ESAllocationTestCase {
.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 class ShardsLimitAllocationTests extends ESAllocationTestCase {
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));

View File

@ -91,7 +91,7 @@ public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
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 class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
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 class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
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 class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
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 class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
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;

View File

@ -90,7 +90,7 @@ public class SingleShardOneReplicaRoutingTests extends ESAllocationTestCase {
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 class SingleShardOneReplicaRoutingTests extends ESAllocationTestCase {
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;

View File

@ -38,7 +38,6 @@ import org.elasticsearch.index.Index;
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 class StartedShardsRoutingTests extends ESAllocationTestCase {
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 class StartedShardsRoutingTests extends ESAllocationTestCase {
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 class StartedShardsRoutingTests extends ESAllocationTestCase {
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 class StartedShardsRoutingTests extends ESAllocationTestCase {
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());
}

View File

@ -104,7 +104,7 @@ public class TenShardsOneReplicaRoutingTests extends ESAllocationTestCase {
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 class TenShardsOneReplicaRoutingTests extends ESAllocationTestCase {
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 class TenShardsOneReplicaRoutingTests extends ESAllocationTestCase {
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();

View File

@ -91,28 +91,28 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase {
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 class ThrottlingAllocationTests extends ESAllocationTestCase {
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 class ThrottlingAllocationTests extends ESAllocationTestCase {
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 class ThrottlingAllocationTests extends ESAllocationTestCase {
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 class ThrottlingAllocationTests extends ESAllocationTestCase {
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 class ThrottlingAllocationTests extends ESAllocationTestCase {
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 class ThrottlingAllocationTests extends ESAllocationTestCase {
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));

View File

@ -27,7 +27,6 @@ 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.common.settings.Settings;
@ -75,12 +74,10 @@ public class UpdateNumberOfReplicasTests extends ESAllocationTestCase {
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 class UpdateNumberOfReplicasTests extends ESAllocationTestCase {
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 class UpdateNumberOfReplicasTests extends ESAllocationTestCase {
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 class UpdateNumberOfReplicasTests extends ESAllocationTestCase {
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();

View File

@ -130,7 +130,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
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 class DiskThresholdDeciderTests extends ESAllocationTestCase {
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 class DiskThresholdDeciderTests extends ESAllocationTestCase {
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 class DiskThresholdDeciderTests extends ESAllocationTestCase {
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 class DiskThresholdDeciderTests extends ESAllocationTestCase {
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 class DiskThresholdDeciderTests extends ESAllocationTestCase {
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 class DiskThresholdDeciderTests extends ESAllocationTestCase {
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 class DiskThresholdDeciderTests extends ESAllocationTestCase {
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 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
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 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
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 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
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

View File

@ -381,7 +381,7 @@ public class DiskThresholdDeciderUnitTests extends ESAllocationTestCase {
.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);

View File

@ -75,9 +75,7 @@ public class EnableAllocationShortCircuitTests extends ESAllocationTestCase {
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;

View File

@ -109,8 +109,7 @@ public class EnableAllocationTests extends ESAllocationTestCase {
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 class EnableAllocationTests extends ESAllocationTestCase {
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 class EnableAllocationTests extends ESAllocationTestCase {
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 class EnableAllocationTests extends ESAllocationTestCase {
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 class EnableAllocationTests extends ESAllocationTestCase {
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));

View File

@ -97,14 +97,14 @@ public class FilterAllocationDeciderTests extends ESAllocationTestCase {
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");

View File

@ -49,7 +49,6 @@ import java.util.Map;
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 class RoutingIteratorTests extends ESAllocationTestCase {
).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 class RoutingIteratorTests extends ESAllocationTestCase {
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 class RoutingIteratorTests extends ESAllocationTestCase {
).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));

View File

@ -26,6 +26,7 @@ import org.elasticsearch.cluster.ClusterState;
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 class IndexLifecycleActionIT extends ESIntegTestCase {
// 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 class IndexLifecycleActionIT extends ESIntegTestCase {
// 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 class IndexLifecycleActionIT extends ESIntegTestCase {
// 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 class IndexLifecycleActionIT extends ESIntegTestCase {
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));

View File

@ -74,6 +74,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
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 class ClusterStateChanges {
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);

View File

@ -86,7 +86,7 @@ public class TransportSearchFailuresIT extends ESIntegTestCase {
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));

View File

@ -52,7 +52,6 @@ import java.util.Set;
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 @@ public abstract class ESAllocationTestCase extends ESTestCase {
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 @@ public abstract class ESAllocationTestCase extends ESTestCase {
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<ShardRouting> initializingShards) {
return allocationService.reroute(allocationService.applyStartedShards(clusterState, initializingShards), "reroute after starting");
}
public static class TestAllocateDecision extends AllocationDecider {
private final Decision decision;