Skip to content

Commit

Permalink
Defer reroute when starting shards (elastic#44539)
Browse files Browse the repository at this point in the history
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 elastic#44433 and elastic#44543.
  • Loading branch information
DaveCTurner authored Jul 18, 2019
1 parent 4c95cc3 commit 452f7f6
Show file tree
Hide file tree
Showing 63 changed files with 630 additions and 517 deletions.
25 changes: 24 additions & 1 deletion docs/reference/migration/migrate_7_4.asciidoc
Original file line number Diff line number Diff line change
Expand Up @@ -67,4 +67,27 @@ unsupported on buckets created after September 30th 2020.
Starting in version 7.4, a `+` in a URL will be encoded as `%2B` by all REST API functionality. Prior versions handled a `+` as a single space.
If your application requires handling `+` as a single space you can return to the old behaviour by setting the system property
`es.rest.url_plus_as_space` to `true`. Note that this behaviour is deprecated and setting this system property to `true` will cease
to be supported in version 8.
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.
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.node.NodeClosedException;
Expand All @@ -72,6 +73,7 @@
import java.util.Objects;
import java.util.Set;
import java.util.function.Predicate;
import java.util.function.Supplier;

import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_PRIMARY_TERM;

Expand All @@ -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<>();
Expand All @@ -97,11 +123,18 @@ public ShardStateAction(ClusterService clusterService, TransportService transpor
this.clusterService = clusterService;
this.threadPool = threadPool;

followUpRerouteTaskPriority = FOLLOW_UP_REROUTE_PRIORITY_SETTING.get(clusterService.getSettings());
clusterService.getClusterSettings().addSettingsUpdateConsumer(FOLLOW_UP_REROUTE_PRIORITY_SETTING,
this::setFollowUpRerouteTaskPriority);

transportService.registerRequestHandler(SHARD_STARTED_ACTION_NAME, ThreadPool.Names.SAME, StartedShardEntry::new,
new ShardStartedTransportHandler(clusterService, new ShardStartedClusterStateTaskExecutor(allocationService, logger), logger));
new ShardStartedTransportHandler(clusterService,
new ShardStartedClusterStateTaskExecutor(allocationService, rerouteService, () -> followUpRerouteTaskPriority, logger),
logger));
transportService.registerRequestHandler(SHARD_FAILED_ACTION_NAME, ThreadPool.Names.SAME, FailedShardEntry::new,
new ShardFailedTransportHandler(clusterService,
new ShardFailedClusterStateTaskExecutor(allocationService, rerouteService, logger), logger));
new ShardFailedClusterStateTaskExecutor(allocationService, rerouteService, () -> followUpRerouteTaskPriority, logger),
logger));
}

private void sendShardAction(final String actionName, final ClusterState currentState,
Expand Down Expand Up @@ -218,6 +251,10 @@ public void onTimeout(TimeValue timeout) {
}, changePredicate);
}

private void setFollowUpRerouteTaskPriority(Priority followUpRerouteTaskPriority) {
this.followUpRerouteTaskPriority = followUpRerouteTaskPriority;
}

private static class ShardFailedTransportHandler implements TransportRequestHandler<FailedShardEntry> {
private final ClusterService clusterService;
private final ShardFailedClusterStateTaskExecutor shardFailedClusterStateTaskExecutor;
Expand Down Expand Up @@ -285,11 +322,14 @@ public static class ShardFailedClusterStateTaskExecutor implements ClusterStateT
private final AllocationService allocationService;
private final RerouteService rerouteService;
private final Logger logger;
private final Supplier<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
Expand Down Expand Up @@ -383,7 +423,7 @@ public void clusterStatePublished(ClusterChangedEvent clusterChangedEvent) {
// assign it again, even if that means putting it back on the node on which it previously failed:
final String reason = String.format(Locale.ROOT, "[%d] unassigned shards after failing shards", numberOfUnassignedShards);
logger.trace("{}, scheduling a reroute", reason);
rerouteService.reroute(reason, Priority.HIGH, ActionListener.wrap(
rerouteService.reroute(reason, prioritySupplier.get(), ActionListener.wrap(
r -> logger.trace("{}, reroute completed", reason),
e -> logger.debug(new ParameterizedMessage("{}, reroute failed", reason), e)));
}
Expand Down Expand Up @@ -520,10 +560,15 @@ public static class ShardStartedClusterStateTaskExecutor
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
Expand Down Expand Up @@ -598,6 +643,13 @@ public void onFailure(String source, Exception e) {
logger.error(() -> new ParameterizedMessage("unexpected failure during [{}]", source), e);
}
}

@Override
public void clusterStatePublished(ClusterChangedEvent clusterChangedEvent) {
rerouteService.reroute("reroute after starting shards", prioritySupplier.get(), ActionListener.wrap(
r -> logger.trace("reroute after starting shards succeeded"),
e -> logger.debug("reroute after starting shards failed", e)));
}
}

public static class StartedShardEntry extends TransportRequest {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ public ClusterState applyStartedShards(ClusterState clusterState, List<ShardRout
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 + "]");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.elasticsearch.cluster.InternalClusterInfoService;
import org.elasticsearch.cluster.NodeConnectionsService;
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.coordination.ClusterBootstrapService;
import org.elasticsearch.cluster.coordination.ClusterFormationFailureHelper;
import org.elasticsearch.cluster.coordination.Coordinator;
Expand Down Expand Up @@ -226,6 +227,7 @@ public void apply(Settings value, Settings current, Settings previous) {
DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING,
DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING,
SameShardAllocationDecider.CLUSTER_ROUTING_ALLOCATION_SAME_HOST_SETTING,
ShardStateAction.FOLLOW_UP_REROUTE_PRIORITY_SETTING,
InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING,
InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING,
DestructiveOperations.REQUIRES_NAME_SETTING,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ESAllocationTestCase;
import org.elasticsearch.cluster.EmptyClusterInfoService;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.metadata.IndexMetaData;
Expand All @@ -33,7 +34,6 @@
import org.elasticsearch.cluster.node.DiscoveryNodeRole;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.RoutingTable;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator;
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
Expand Down Expand Up @@ -116,8 +116,7 @@ public void testErrorCondition() {
RoutingTable routingTable = service.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
// now we start the shard
routingTable = service.applyStartedShards(clusterState,
routingTable.index("source").shardsWithState(ShardRoutingState.INITIALIZING)).routingTable();
routingTable = ESAllocationTestCase.startInitializingShardsAndReroute(service, clusterState, "source").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();

TransportResizeAction.prepareCreateIndexRequest(new ResizeRequest("target", "source"), clusterState,
Expand All @@ -135,8 +134,7 @@ public void testPassNumRoutingShards() {
RoutingTable routingTable = service.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
// now we start the shard
routingTable = service.applyStartedShards(clusterState,
routingTable.index("source").shardsWithState(ShardRoutingState.INITIALIZING)).routingTable();
routingTable = ESAllocationTestCase.startInitializingShardsAndReroute(service, clusterState, "source").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();

ResizeRequest resizeRequest = new ResizeRequest("target", "source");
Expand Down Expand Up @@ -165,8 +163,7 @@ public void testPassNumRoutingShardsAndFail() {
RoutingTable routingTable = service.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
// now we start the shard
routingTable = service.applyStartedShards(clusterState,
routingTable.index("source").shardsWithState(ShardRoutingState.INITIALIZING)).routingTable();
routingTable = ESAllocationTestCase.startInitializingShardsAndReroute(service, clusterState, "source").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();

ResizeRequest resizeRequest = new ResizeRequest("target", "source");
Expand Down Expand Up @@ -200,8 +197,7 @@ public void testShrinkIndexSettings() {
RoutingTable routingTable = service.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
// now we start the shard
routingTable = service.applyStartedShards(clusterState,
routingTable.index(indexName).shardsWithState(ShardRoutingState.INITIALIZING)).routingTable();
routingTable = ESAllocationTestCase.startInitializingShardsAndReroute(service, clusterState, indexName).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
int numSourceShards = clusterState.metaData().index(indexName).getNumberOfShards();
DocsStats stats = new DocsStats(between(0, (IndexWriter.MAX_DOCS) / numSourceShards), between(1, 1000), between(1, 10000));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -466,7 +466,7 @@ public void testNoRerouteOnStaleClusterState() {
ShardRouting relocationTarget = clusterService.state().getRoutingTable().shardRoutingTable(shardId)
.shardsWithState(ShardRoutingState.INITIALIZING).get(0);
AllocationService allocationService = ESAllocationTestCase.createAllocationService();
ClusterState updatedState = allocationService.applyStartedShards(state, Collections.singletonList(relocationTarget));
ClusterState updatedState = ESAllocationTestCase.startShardsAndReroute(allocationService, state, relocationTarget);

setState(clusterService, updatedState);
logger.debug("--> relocation complete state:\n{}", clusterService.state());
Expand Down
Loading

0 comments on commit 452f7f6

Please sign in to comment.