Skip to content

Commit 35e7058

Browse files
committed
Limit retries of failed allocations per index (#18467)
Today if a shard fails during initialization phase due to misconfiguration, broken disks, missing analyzers, not installed plugins etc. elasticsaerch keeps on trying to initialize or rather allocate that shard. Yet, in the worst case scenario this ends in an endless allocation loop. To prevent this loop and all it's sideeffects like spamming log files over and over again this commit adds an allocation decider that stops allocating a shard that failed more than N times in a row to allocate. The number or retries can be configured via `index.allocation.max_retry` and it's default is set to `5`. Once the setting is updated shards with less failures than the number set per index will be allowed to allocate again. Internally we maintain a counter on the UnassignedInfo that is reset to `0` once the shards has been started. Relates to #18417
1 parent a2ff002 commit 35e7058

33 files changed

+705
-117
lines changed

core/src/main/java/org/elasticsearch/action/admin/cluster/allocation/TransportClusterAllocationExplainAction.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -250,7 +250,7 @@ protected void masterOperation(final ClusterAllocationExplainRequest request, fi
250250
final ActionListener<ClusterAllocationExplainResponse> listener) {
251251
final RoutingNodes routingNodes = state.getRoutingNodes();
252252
final RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, routingNodes, state,
253-
clusterInfoService.getClusterInfo(), System.nanoTime());
253+
clusterInfoService.getClusterInfo(), System.nanoTime(), false);
254254

255255
ShardRouting foundShard = null;
256256
if (request.useAnyUnassignedShard()) {

core/src/main/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteRequest.java

+30-3
Original file line numberDiff line numberDiff line change
@@ -38,9 +38,10 @@
3838
* Request to submit cluster reroute allocation commands
3939
*/
4040
public class ClusterRerouteRequest extends AcknowledgedRequest<ClusterRerouteRequest> {
41-
AllocationCommands commands = new AllocationCommands();
42-
boolean dryRun;
43-
boolean explain;
41+
private AllocationCommands commands = new AllocationCommands();
42+
private boolean dryRun;
43+
private boolean explain;
44+
private boolean retryFailed;
4445

4546
public ClusterRerouteRequest() {
4647
}
@@ -81,13 +82,30 @@ public ClusterRerouteRequest explain(boolean explain) {
8182
return this;
8283
}
8384

85+
/**
86+
* Sets the retry failed flag (defaults to <tt>false</tt>). If true, the
87+
* request will retry allocating shards that can't currently be allocated due to too many allocation failures.
88+
*/
89+
public ClusterRerouteRequest setRetryFailed(boolean retryFailed) {
90+
this.retryFailed = retryFailed;
91+
return this;
92+
}
93+
8494
/**
8595
* Returns the current explain flag
8696
*/
8797
public boolean explain() {
8898
return this.explain;
8999
}
90100

101+
/**
102+
* Returns the current retry failed flag
103+
*/
104+
public boolean isRetryFailed() {
105+
return this.retryFailed;
106+
}
107+
108+
91109
/**
92110
* Set the allocation commands to execute.
93111
*/
@@ -96,6 +114,13 @@ public ClusterRerouteRequest commands(AllocationCommand... commands) {
96114
return this;
97115
}
98116

117+
/**
118+
* Returns the allocation commands to execute
119+
*/
120+
public AllocationCommands getCommands() {
121+
return commands;
122+
}
123+
99124
/**
100125
* Sets the source for the request.
101126
*/
@@ -136,6 +161,7 @@ public void readFrom(StreamInput in) throws IOException {
136161
commands = AllocationCommands.readFrom(in);
137162
dryRun = in.readBoolean();
138163
explain = in.readBoolean();
164+
retryFailed = in.readBoolean();
139165
readTimeout(in);
140166
}
141167

@@ -145,6 +171,7 @@ public void writeTo(StreamOutput out) throws IOException {
145171
AllocationCommands.writeTo(commands, out);
146172
out.writeBoolean(dryRun);
147173
out.writeBoolean(explain);
174+
out.writeBoolean(retryFailed);
148175
writeTimeout(out);
149176
}
150177
}

core/src/main/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteRequestBuilder.java

+9
Original file line numberDiff line numberDiff line change
@@ -60,6 +60,15 @@ public ClusterRerouteRequestBuilder setExplain(boolean explain) {
6060
return this;
6161
}
6262

63+
/**
64+
* Sets the retry failed flag (defaults to <tt>false</tt>). If true, the
65+
* request will retry allocating shards that can't currently be allocated due to too many allocation failures.
66+
*/
67+
public ClusterRerouteRequestBuilder setRetryFailed(boolean retryFailed) {
68+
request.setRetryFailed(retryFailed);
69+
return this;
70+
}
71+
6372
/**
6473
* Sets the commands for the request to execute.
6574
*/

core/src/main/java/org/elasticsearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java

+46-28
Original file line numberDiff line numberDiff line change
@@ -33,6 +33,7 @@
3333
import org.elasticsearch.cluster.service.ClusterService;
3434
import org.elasticsearch.common.Priority;
3535
import org.elasticsearch.common.inject.Inject;
36+
import org.elasticsearch.common.logging.ESLogger;
3637
import org.elasticsearch.common.settings.Settings;
3738
import org.elasticsearch.threadpool.ThreadPool;
3839
import org.elasticsearch.transport.TransportService;
@@ -68,38 +69,55 @@ protected ClusterRerouteResponse newResponse() {
6869

6970
@Override
7071
protected void masterOperation(final ClusterRerouteRequest request, final ClusterState state, final ActionListener<ClusterRerouteResponse> listener) {
71-
clusterService.submitStateUpdateTask("cluster_reroute (api)", new AckedClusterStateUpdateTask<ClusterRerouteResponse>(Priority.IMMEDIATE, request, listener) {
72+
clusterService.submitStateUpdateTask("cluster_reroute (api)", new ClusterRerouteResponseAckedClusterStateUpdateTask(logger,
73+
allocationService, request, listener));
74+
}
7275

73-
private volatile ClusterState clusterStateToSend;
74-
private volatile RoutingExplanations explanations;
76+
static class ClusterRerouteResponseAckedClusterStateUpdateTask extends AckedClusterStateUpdateTask<ClusterRerouteResponse> {
7577

76-
@Override
77-
protected ClusterRerouteResponse newResponse(boolean acknowledged) {
78-
return new ClusterRerouteResponse(acknowledged, clusterStateToSend, explanations);
79-
}
78+
private final ClusterRerouteRequest request;
79+
private final ActionListener<ClusterRerouteResponse> listener;
80+
private final ESLogger logger;
81+
private final AllocationService allocationService;
82+
private volatile ClusterState clusterStateToSend;
83+
private volatile RoutingExplanations explanations;
8084

81-
@Override
82-
public void onAckTimeout() {
83-
listener.onResponse(new ClusterRerouteResponse(false, clusterStateToSend, new RoutingExplanations()));
84-
}
85+
ClusterRerouteResponseAckedClusterStateUpdateTask(ESLogger logger, AllocationService allocationService, ClusterRerouteRequest request,
86+
ActionListener<ClusterRerouteResponse> listener) {
87+
super(Priority.IMMEDIATE, request, listener);
88+
this.request = request;
89+
this.listener = listener;
90+
this.logger = logger;
91+
this.allocationService = allocationService;
92+
}
8593

86-
@Override
87-
public void onFailure(String source, Throwable t) {
88-
logger.debug("failed to perform [{}]", t, source);
89-
super.onFailure(source, t);
90-
}
94+
@Override
95+
protected ClusterRerouteResponse newResponse(boolean acknowledged) {
96+
return new ClusterRerouteResponse(acknowledged, clusterStateToSend, explanations);
97+
}
98+
99+
@Override
100+
public void onAckTimeout() {
101+
listener.onResponse(new ClusterRerouteResponse(false, clusterStateToSend, new RoutingExplanations()));
102+
}
103+
104+
@Override
105+
public void onFailure(String source, Throwable t) {
106+
logger.debug("failed to perform [{}]", t, source);
107+
super.onFailure(source, t);
108+
}
91109

92-
@Override
93-
public ClusterState execute(ClusterState currentState) {
94-
RoutingAllocation.Result routingResult = allocationService.reroute(currentState, request.commands, request.explain());
95-
ClusterState newState = ClusterState.builder(currentState).routingResult(routingResult).build();
96-
clusterStateToSend = newState;
97-
explanations = routingResult.explanations();
98-
if (request.dryRun) {
99-
return currentState;
100-
}
101-
return newState;
110+
@Override
111+
public ClusterState execute(ClusterState currentState) {
112+
RoutingAllocation.Result routingResult = allocationService.reroute(currentState, request.getCommands(), request.explain(),
113+
request.isRetryFailed());
114+
ClusterState newState = ClusterState.builder(currentState).routingResult(routingResult).build();
115+
clusterStateToSend = newState;
116+
explanations = routingResult.explanations();
117+
if (request.dryRun()) {
118+
return currentState;
102119
}
103-
});
120+
return newState;
121+
}
104122
}
105-
}
123+
}

core/src/main/java/org/elasticsearch/cluster/ClusterModule.java

+2
Original file line numberDiff line numberDiff line change
@@ -49,6 +49,7 @@
4949
import org.elasticsearch.cluster.routing.allocation.decider.NodeVersionAllocationDecider;
5050
import org.elasticsearch.cluster.routing.allocation.decider.RebalanceOnlyWhenActiveAllocationDecider;
5151
import org.elasticsearch.cluster.routing.allocation.decider.ReplicaAfterPrimaryActiveAllocationDecider;
52+
import org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider;
5253
import org.elasticsearch.cluster.routing.allocation.decider.SameShardAllocationDecider;
5354
import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocationDecider;
5455
import org.elasticsearch.cluster.routing.allocation.decider.SnapshotInProgressAllocationDecider;
@@ -79,6 +80,7 @@ public class ClusterModule extends AbstractModule {
7980
new Setting<>("cluster.routing.allocation.type", BALANCED_ALLOCATOR, Function.identity(), Property.NodeScope);
8081
public static final List<Class<? extends AllocationDecider>> DEFAULT_ALLOCATION_DECIDERS =
8182
Collections.unmodifiableList(Arrays.asList(
83+
MaxRetryAllocationDecider.class,
8284
SameShardAllocationDecider.class,
8385
FilterAllocationDecider.class,
8486
ReplicaAfterPrimaryActiveAllocationDecider.class,

core/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java

+27-5
Original file line numberDiff line numberDiff line change
@@ -48,7 +48,6 @@ public final class UnassignedInfo implements ToXContent, Writeable {
4848
public static final Setting<TimeValue> INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING =
4949
Setting.timeSetting("index.unassigned.node_left.delayed_timeout", DEFAULT_DELAYED_NODE_LEFT_TIMEOUT, Property.Dynamic,
5050
Property.IndexScope);
51-
5251
/**
5352
* Reason why the shard is in unassigned state.
5453
* <p>
@@ -103,7 +102,11 @@ public enum Reason {
103102
/**
104103
* A better replica location is identified and causes the existing replica allocation to be cancelled.
105104
*/
106-
REALLOCATED_REPLICA;
105+
REALLOCATED_REPLICA,
106+
/**
107+
* Unassigned as a result of a failed primary while the replica was initializing.
108+
*/
109+
PRIMARY_FAILED;
107110
}
108111

109112
private final Reason reason;
@@ -112,6 +115,7 @@ public enum Reason {
112115
private final long lastComputedLeftDelayNanos; // how long to delay shard allocation, not serialized (always positive, 0 means no delay)
113116
private final String message;
114117
private final Throwable failure;
118+
private final int failedAllocations;
115119

116120
/**
117121
* creates an UnassingedInfo object based **current** time
@@ -120,7 +124,7 @@ public enum Reason {
120124
* @param message more information about cause.
121125
**/
122126
public UnassignedInfo(Reason reason, String message) {
123-
this(reason, message, null, System.nanoTime(), System.currentTimeMillis());
127+
this(reason, message, null, reason == Reason.ALLOCATION_FAILED ? 1 : 0, System.nanoTime(), System.currentTimeMillis());
124128
}
125129

126130
/**
@@ -130,13 +134,16 @@ public UnassignedInfo(Reason reason, String message) {
130134
* @param unassignedTimeNanos the time to use as the base for any delayed re-assignment calculation
131135
* @param unassignedTimeMillis the time of unassignment used to display to in our reporting.
132136
*/
133-
public UnassignedInfo(Reason reason, @Nullable String message, @Nullable Throwable failure, long unassignedTimeNanos, long unassignedTimeMillis) {
137+
public UnassignedInfo(Reason reason, @Nullable String message, @Nullable Throwable failure, int failedAllocations, long unassignedTimeNanos, long unassignedTimeMillis) {
134138
this.reason = reason;
135139
this.unassignedTimeMillis = unassignedTimeMillis;
136140
this.unassignedTimeNanos = unassignedTimeNanos;
137141
this.lastComputedLeftDelayNanos = 0L;
138142
this.message = message;
139143
this.failure = failure;
144+
this.failedAllocations = failedAllocations;
145+
assert (failedAllocations > 0) == (reason == Reason.ALLOCATION_FAILED):
146+
"failedAllocations: " + failedAllocations + " for reason " + reason;
140147
assert !(message == null && failure != null) : "provide a message if a failure exception is provided";
141148
}
142149

@@ -147,17 +154,19 @@ public UnassignedInfo(UnassignedInfo unassignedInfo, long newComputedLeftDelayNa
147154
this.lastComputedLeftDelayNanos = newComputedLeftDelayNanos;
148155
this.message = unassignedInfo.message;
149156
this.failure = unassignedInfo.failure;
157+
this.failedAllocations = unassignedInfo.failedAllocations;
150158
}
151159

152160
public UnassignedInfo(StreamInput in) throws IOException {
153161
this.reason = Reason.values()[(int) in.readByte()];
154162
this.unassignedTimeMillis = in.readLong();
155163
// As System.nanoTime() cannot be compared across different JVMs, reset it to now.
156-
// This means that in master failover situations, elapsed delay time is forgotten.
164+
// This means that in master fail-over situations, elapsed delay time is forgotten.
157165
this.unassignedTimeNanos = System.nanoTime();
158166
this.lastComputedLeftDelayNanos = 0L;
159167
this.message = in.readOptionalString();
160168
this.failure = in.readThrowable();
169+
this.failedAllocations = in.readVInt();
161170
}
162171

163172
public void writeTo(StreamOutput out) throws IOException {
@@ -166,12 +175,18 @@ public void writeTo(StreamOutput out) throws IOException {
166175
// Do not serialize unassignedTimeNanos as System.nanoTime() cannot be compared across different JVMs
167176
out.writeOptionalString(message);
168177
out.writeThrowable(failure);
178+
out.writeVInt(failedAllocations);
169179
}
170180

171181
public UnassignedInfo readFrom(StreamInput in) throws IOException {
172182
return new UnassignedInfo(in);
173183
}
174184

185+
/**
186+
* Returns the number of previously failed allocations of this shard.
187+
*/
188+
public int getNumFailedAllocations() { return failedAllocations; }
189+
175190
/**
176191
* The reason why the shard is unassigned.
177192
*/
@@ -325,7 +340,11 @@ public String shortSummary() {
325340
StringBuilder sb = new StringBuilder();
326341
sb.append("[reason=").append(reason).append("]");
327342
sb.append(", at[").append(DATE_TIME_FORMATTER.printer().print(unassignedTimeMillis)).append("]");
343+
if (failedAllocations > 0) {
344+
sb.append(", failed_attempts[").append(failedAllocations).append("]");
345+
}
328346
String details = getDetails();
347+
329348
if (details != null) {
330349
sb.append(", details[").append(details).append("]");
331350
}
@@ -342,6 +361,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws
342361
builder.startObject("unassigned_info");
343362
builder.field("reason", reason);
344363
builder.field("at", DATE_TIME_FORMATTER.printer().print(unassignedTimeMillis));
364+
if (failedAllocations > 0) {
365+
builder.field("failed_attempts", failedAllocations);
366+
}
345367
String details = getDetails();
346368
if (details != null) {
347369
builder.field("details", details);

core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java

+11-11
Original file line numberDiff line numberDiff line change
@@ -222,8 +222,10 @@ public RoutingAllocation.Result applyFailedShards(ClusterState clusterState, Lis
222222
List<FailedRerouteAllocation.FailedShard> orderedFailedShards = new ArrayList<>(failedShards);
223223
orderedFailedShards.sort(Comparator.comparing(failedShard -> failedShard.shard.primary()));
224224
for (FailedRerouteAllocation.FailedShard failedShard : orderedFailedShards) {
225+
UnassignedInfo unassignedInfo = failedShard.shard.unassignedInfo();
226+
final int failedAllocations = unassignedInfo != null ? unassignedInfo.getNumFailedAllocations() : 0;
225227
changed |= applyFailedShard(allocation, failedShard.shard, true, new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, failedShard.message, failedShard.failure,
226-
System.nanoTime(), System.currentTimeMillis()));
228+
failedAllocations + 1, System.nanoTime(), System.currentTimeMillis()));
227229
}
228230
if (!changed) {
229231
return new RoutingAllocation.Result(false, clusterState.routingTable(), clusterState.metaData());
@@ -257,16 +259,13 @@ private <T> String firstListElementsToCommaDelimitedString(List<T> elements, Fun
257259
.collect(Collectors.joining(", "));
258260
}
259261

260-
public RoutingAllocation.Result reroute(ClusterState clusterState, AllocationCommands commands) {
261-
return reroute(clusterState, commands, false);
262-
}
263-
264-
public RoutingAllocation.Result reroute(ClusterState clusterState, AllocationCommands commands, boolean explain) {
262+
public RoutingAllocation.Result reroute(ClusterState clusterState, AllocationCommands commands, boolean explain, boolean retryFailed) {
265263
RoutingNodes routingNodes = getMutableRoutingNodes(clusterState);
266264
// we don't shuffle the unassigned shards here, to try and get as close as possible to
267265
// a consistent result of the effect the commands have on the routing
268266
// this allows systems to dry run the commands, see the resulting cluster state, and act on it
269-
RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, routingNodes, clusterState, clusterInfoService.getClusterInfo(), currentNanoTime());
267+
RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, routingNodes, clusterState,
268+
clusterInfoService.getClusterInfo(), currentNanoTime(), retryFailed);
270269
// don't short circuit deciders, we want a full explanation
271270
allocation.debugDecision(true);
272271
// we ignore disable allocation, because commands are explicit
@@ -305,7 +304,8 @@ protected RoutingAllocation.Result reroute(ClusterState clusterState, String rea
305304
RoutingNodes routingNodes = getMutableRoutingNodes(clusterState);
306305
// shuffle the unassigned nodes, just so we won't have things like poison failed shards
307306
routingNodes.unassigned().shuffle();
308-
RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, routingNodes, clusterState, clusterInfoService.getClusterInfo(), currentNanoTime());
307+
RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, routingNodes, clusterState,
308+
clusterInfoService.getClusterInfo(), currentNanoTime(), false);
309309
allocation.debugDecision(debug);
310310
if (!reroute(allocation)) {
311311
return new RoutingAllocation.Result(false, clusterState.routingTable(), clusterState.metaData());
@@ -437,7 +437,7 @@ private boolean deassociateDeadNodes(RoutingAllocation allocation) {
437437
// now, go over all the shards routing on the node, and fail them
438438
for (ShardRouting shardRouting : node.copyShards()) {
439439
UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.NODE_LEFT, "node_left[" + node.nodeId() + "]", null,
440-
allocation.getCurrentNanoTime(), System.currentTimeMillis());
440+
0, allocation.getCurrentNanoTime(), System.currentTimeMillis());
441441
applyFailedShard(allocation, shardRouting, false, unassignedInfo);
442442
}
443443
// its a dead node, remove it, note, its important to remove it *after* we apply failed shard
@@ -457,8 +457,8 @@ private boolean failReplicasForUnassignedPrimary(RoutingAllocation allocation, S
457457
boolean changed = false;
458458
for (ShardRouting routing : replicas) {
459459
changed |= applyFailedShard(allocation, routing, false,
460-
new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, "primary failed while replica initializing",
461-
null, allocation.getCurrentNanoTime(), System.currentTimeMillis()));
460+
new UnassignedInfo(UnassignedInfo.Reason.PRIMARY_FAILED, "primary failed while replica initializing",
461+
null, 0, allocation.getCurrentNanoTime(), System.currentTimeMillis()));
462462
}
463463
return changed;
464464
}

core/src/main/java/org/elasticsearch/cluster/routing/allocation/FailedRerouteAllocation.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -58,7 +58,7 @@ public String toString() {
5858
private final List<FailedShard> failedShards;
5959

6060
public FailedRerouteAllocation(AllocationDeciders deciders, RoutingNodes routingNodes, ClusterState clusterState, List<FailedShard> failedShards, ClusterInfo clusterInfo) {
61-
super(deciders, routingNodes, clusterState, clusterInfo, System.nanoTime());
61+
super(deciders, routingNodes, clusterState, clusterInfo, System.nanoTime(), false);
6262
this.failedShards = failedShards;
6363
}
6464

0 commit comments

Comments
 (0)