Skip to content

Commit

Permalink
Fail restore when the shard allocations max retries count is reached (#…
Browse files Browse the repository at this point in the history
…27493)

This commit changes the RestoreService so that it now fails the snapshot 
restore if one of the shards to restore has failed to be allocated. It also adds
a new RestoreInProgressAllocationDecider that forbids such shards to be 
allocated again. This way, when a restore is impossible or failed too many 
times, the user is forced to take a manual action (like deleting the index 
which failed shards) in order to try to restore it again.

This behaviour has been implemented because when the allocation of a 
shard has been retried too many times, the MaxRetryDecider is engaged 
to prevent any future allocation of the failed shard. If it happens while 
restoring a snapshot, the restore hanged and was never completed because 
it stayed around waiting for the shards to be assigned (and that won't happen).
It also blocked future attempts to restore the snapshot again. With this commit,
the restore does not hang and is marked as failed, leaving failed shards 
around for investigation.

This is the second part of the #26865 issue.

Closes #26865
  • Loading branch information
tlrx committed Dec 22, 2017
1 parent 0d8732b commit 5752583
Show file tree
Hide file tree
Showing 8 changed files with 541 additions and 28 deletions.
Expand Up @@ -54,6 +54,7 @@
import org.elasticsearch.cluster.routing.allocation.decider.SameShardAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.SnapshotInProgressAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.RestoreInProgressAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.ParseField;
Expand Down Expand Up @@ -191,6 +192,7 @@ public static Collection<AllocationDecider> createAllocationDeciders(Settings se
addAllocationDecider(deciders, new EnableAllocationDecider(settings, clusterSettings));
addAllocationDecider(deciders, new NodeVersionAllocationDecider(settings));
addAllocationDecider(deciders, new SnapshotInProgressAllocationDecider(settings));
addAllocationDecider(deciders, new RestoreInProgressAllocationDecider(settings));
addAllocationDecider(deciders, new FilterAllocationDecider(settings, clusterSettings));
addAllocationDecider(deciders, new SameShardAllocationDecider(settings, clusterSettings));
addAllocationDecider(deciders, new DiskThresholdDecider(settings, clusterSettings));
Expand Down
Expand Up @@ -48,6 +48,8 @@
import java.util.function.Function;
import java.util.stream.Collectors;

import static java.util.Collections.emptyList;
import static java.util.Collections.singletonList;
import static org.elasticsearch.cluster.routing.UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING;


Expand Down Expand Up @@ -135,13 +137,14 @@ protected ClusterState buildResultAndLogHealthChange(ClusterState oldState, Rout
return newState;
}

// Used for testing
public ClusterState applyFailedShard(ClusterState clusterState, ShardRouting failedShard) {
return applyFailedShards(clusterState, Collections.singletonList(new FailedShard(failedShard, null, null)),
Collections.emptyList());
return applyFailedShards(clusterState, singletonList(new FailedShard(failedShard, null, null)), emptyList());
}

// Used for testing
public ClusterState applyFailedShards(ClusterState clusterState, List<FailedShard> failedShards) {
return applyFailedShards(clusterState, failedShards, Collections.emptyList());
return applyFailedShards(clusterState, failedShards, emptyList());
}

/**
Expand Down
@@ -0,0 +1,86 @@
/*
* 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.routing.allocation.decider;

import org.elasticsearch.cluster.RestoreInProgress;
import org.elasticsearch.cluster.routing.RecoverySource;
import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.snapshots.Snapshot;

/**
* This {@link AllocationDecider} prevents shards that have failed to be
* restored from a snapshot to be allocated.
*/
public class RestoreInProgressAllocationDecider extends AllocationDecider {

public static final String NAME = "restore_in_progress";

/**
* Creates a new {@link RestoreInProgressAllocationDecider} instance from
* given settings
*
* @param settings {@link Settings} to use
*/
public RestoreInProgressAllocationDecider(Settings settings) {
super(settings);
}

@Override
public Decision canAllocate(final ShardRouting shardRouting, final RoutingNode node, final RoutingAllocation allocation) {
return canAllocate(shardRouting, allocation);
}

@Override
public Decision canAllocate(final ShardRouting shardRouting, final RoutingAllocation allocation) {
final RecoverySource recoverySource = shardRouting.recoverySource();
if (recoverySource == null || recoverySource.getType() != RecoverySource.Type.SNAPSHOT) {
return allocation.decision(Decision.YES, NAME, "ignored as shard is not being recovered from a snapshot");
}

final Snapshot snapshot = ((RecoverySource.SnapshotRecoverySource) recoverySource).snapshot();
final RestoreInProgress restoresInProgress = allocation.custom(RestoreInProgress.TYPE);

if (restoresInProgress != null) {
for (RestoreInProgress.Entry restoreInProgress : restoresInProgress.entries()) {
if (restoreInProgress.snapshot().equals(snapshot)) {
RestoreInProgress.ShardRestoreStatus shardRestoreStatus = restoreInProgress.shards().get(shardRouting.shardId());
if (shardRestoreStatus != null && shardRestoreStatus.state().completed() == false) {
assert shardRestoreStatus.state() != RestoreInProgress.State.SUCCESS : "expected shard [" + shardRouting
+ "] to be in initializing state but got [" + shardRestoreStatus.state() + "]";
return allocation.decision(Decision.YES, NAME, "shard is currently being restored");
}
break;
}
}
}
return allocation.decision(Decision.NO, NAME, "shard has failed to be restored from the snapshot [%s] because of [%s] - " +
"manually close or delete the index [%s] in order to retry to restore the snapshot again or use the reroute API to force the " +
"allocation of an empty primary shard", snapshot, shardRouting.unassignedInfo().getDetails(), shardRouting.getIndexName());
}

@Override
public Decision canForceAllocatePrimary(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) {
assert shardRouting.primary() : "must not call canForceAllocatePrimary on a non-primary shard " + shardRouting;
return canAllocate(shardRouting, node, allocation);
}
}
37 changes: 22 additions & 15 deletions core/src/main/java/org/elasticsearch/snapshots/RestoreService.java
Expand Up @@ -64,7 +64,6 @@
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
Expand Down Expand Up @@ -534,7 +533,7 @@ public void shardStarted(ShardRouting initializingShard, ShardRouting startedSha
RecoverySource recoverySource = initializingShard.recoverySource();
if (recoverySource.getType() == RecoverySource.Type.SNAPSHOT) {
Snapshot snapshot = ((SnapshotRecoverySource) recoverySource).snapshot();
changes(snapshot).startedShards.put(initializingShard.shardId(),
changes(snapshot).shards.put(initializingShard.shardId(),
new ShardRestoreStatus(initializingShard.currentNodeId(), RestoreInProgress.State.SUCCESS));
}
}
Expand All @@ -550,7 +549,7 @@ public void shardFailed(ShardRouting failedShard, UnassignedInfo unassignedInfo)
// to restore this shard on another node if the snapshot files are corrupt. In case where a node just left or crashed,
// however, we only want to acknowledge the restore operation once it has been successfully restored on another node.
if (unassignedInfo.getFailure() != null && Lucene.isCorruptionException(unassignedInfo.getFailure().getCause())) {
changes(snapshot).failedShards.put(failedShard.shardId(), new ShardRestoreStatus(failedShard.currentNodeId(),
changes(snapshot).shards.put(failedShard.shardId(), new ShardRestoreStatus(failedShard.currentNodeId(),
RestoreInProgress.State.FAILURE, unassignedInfo.getFailure().getCause().getMessage()));
}
}
Expand All @@ -563,11 +562,24 @@ public void shardInitialized(ShardRouting unassignedShard, ShardRouting initiali
if (unassignedShard.recoverySource().getType() == RecoverySource.Type.SNAPSHOT &&
initializedShard.recoverySource().getType() != RecoverySource.Type.SNAPSHOT) {
Snapshot snapshot = ((SnapshotRecoverySource) unassignedShard.recoverySource()).snapshot();
changes(snapshot).failedShards.put(unassignedShard.shardId(), new ShardRestoreStatus(null,
changes(snapshot).shards.put(unassignedShard.shardId(), new ShardRestoreStatus(null,
RestoreInProgress.State.FAILURE, "recovery source type changed from snapshot to " + initializedShard.recoverySource()));
}
}

@Override
public void unassignedInfoUpdated(ShardRouting unassignedShard, UnassignedInfo newUnassignedInfo) {
RecoverySource recoverySource = unassignedShard.recoverySource();
if (recoverySource.getType() == RecoverySource.Type.SNAPSHOT) {
if (newUnassignedInfo.getLastAllocationStatus() == UnassignedInfo.AllocationStatus.DECIDERS_NO) {
Snapshot snapshot = ((SnapshotRecoverySource) recoverySource).snapshot();
String reason = "shard could not be allocated to any of the nodes";
changes(snapshot).shards.put(unassignedShard.shardId(),
new ShardRestoreStatus(unassignedShard.currentNodeId(), RestoreInProgress.State.FAILURE, reason));
}
}
}

/**
* Helper method that creates update entry for the given shard id if such an entry does not exist yet.
*/
Expand All @@ -576,25 +588,21 @@ private Updates changes(Snapshot snapshot) {
}

private static class Updates {
private Map<ShardId, ShardRestoreStatus> failedShards = new HashMap<>();
private Map<ShardId, ShardRestoreStatus> startedShards = new HashMap<>();
private Map<ShardId, ShardRestoreStatus> shards = new HashMap<>();
}

public RestoreInProgress applyChanges(RestoreInProgress oldRestore) {
public RestoreInProgress applyChanges(final RestoreInProgress oldRestore) {
if (shardChanges.isEmpty() == false) {
final List<RestoreInProgress.Entry> entries = new ArrayList<>();
for (RestoreInProgress.Entry entry : oldRestore.entries()) {
Snapshot snapshot = entry.snapshot();
Updates updates = shardChanges.get(snapshot);
assert Sets.haveEmptyIntersection(updates.startedShards.keySet(), updates.failedShards.keySet());
if (updates.startedShards.isEmpty() == false || updates.failedShards.isEmpty() == false) {
if (updates.shards.isEmpty() == false) {
ImmutableOpenMap.Builder<ShardId, ShardRestoreStatus> shardsBuilder = ImmutableOpenMap.builder(entry.shards());
for (Map.Entry<ShardId, ShardRestoreStatus> startedShardEntry : updates.startedShards.entrySet()) {
shardsBuilder.put(startedShardEntry.getKey(), startedShardEntry.getValue());
}
for (Map.Entry<ShardId, ShardRestoreStatus> failedShardEntry : updates.failedShards.entrySet()) {
shardsBuilder.put(failedShardEntry.getKey(), failedShardEntry.getValue());
for (Map.Entry<ShardId, ShardRestoreStatus> shard : updates.shards.entrySet()) {
shardsBuilder.put(shard.getKey(), shard.getValue());
}

ImmutableOpenMap<ShardId, ShardRestoreStatus> shards = shardsBuilder.build();
RestoreInProgress.State newState = overallState(RestoreInProgress.State.STARTED, shards);
entries.add(new RestoreInProgress.Entry(entry.snapshot(), newState, entry.indices(), shards));
Expand All @@ -607,7 +615,6 @@ public RestoreInProgress applyChanges(RestoreInProgress oldRestore) {
return oldRestore;
}
}

}

public static RestoreInProgress.Entry restoreInProgress(ClusterState state, Snapshot snapshot) {
Expand Down
Expand Up @@ -35,6 +35,7 @@
import org.elasticsearch.cluster.routing.allocation.decider.RebalanceOnlyWhenActiveAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.ReplicaAfterPrimaryActiveAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.ResizeAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.RestoreInProgressAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.SameShardAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.SnapshotInProgressAllocationDecider;
Expand Down Expand Up @@ -183,6 +184,7 @@ public void testAllocationDeciderOrder() {
EnableAllocationDecider.class,
NodeVersionAllocationDecider.class,
SnapshotInProgressAllocationDecider.class,
RestoreInProgressAllocationDecider.class,
FilterAllocationDecider.class,
SameShardAllocationDecider.class,
DiskThresholdDecider.class,
Expand Down
Expand Up @@ -25,6 +25,7 @@
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ESAllocationTestCase;
import org.elasticsearch.cluster.RestoreInProgress;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
Expand All @@ -38,6 +39,7 @@
import org.elasticsearch.cluster.routing.allocation.command.AllocationCommands;
import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.Index;
Expand All @@ -46,7 +48,10 @@
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.test.gateway.TestGatewayAllocator;

import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;

import static org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING;
import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING;
Expand Down Expand Up @@ -309,6 +314,8 @@ private ClusterState createRecoveryStateAndInitalizeAllocations(MetaData metaDat
DiscoveryNode node1 = newNode("node1");
MetaData.Builder metaDataBuilder = new MetaData.Builder(metaData);
RoutingTable.Builder routingTableBuilder = RoutingTable.builder();
Snapshot snapshot = new Snapshot("repo", new SnapshotId("snap", "randomId"));
Set<String> snapshotIndices = new HashSet<>();
for (ObjectCursor<IndexMetaData> cursor: metaData.indices().values()) {
Index index = cursor.value.getIndex();
IndexMetaData.Builder indexMetaDataBuilder = IndexMetaData.builder(cursor.value);
Expand All @@ -329,14 +336,14 @@ private ClusterState createRecoveryStateAndInitalizeAllocations(MetaData metaDat
routingTableBuilder.addAsFromDangling(indexMetaData);
break;
case 3:
snapshotIndices.add(index.getName());
routingTableBuilder.addAsNewRestore(indexMetaData,
new SnapshotRecoverySource(new Snapshot("repo", new SnapshotId("snap", "randomId")), Version.CURRENT,
indexMetaData.getIndex().getName()), new IntHashSet());
new SnapshotRecoverySource(snapshot, Version.CURRENT, indexMetaData.getIndex().getName()), new IntHashSet());
break;
case 4:
snapshotIndices.add(index.getName());
routingTableBuilder.addAsRestore(indexMetaData,
new SnapshotRecoverySource(new Snapshot("repo", new SnapshotId("snap", "randomId")), Version.CURRENT,
indexMetaData.getIndex().getName()));
new SnapshotRecoverySource(snapshot, Version.CURRENT, indexMetaData.getIndex().getName()));
break;
case 5:
routingTableBuilder.addAsNew(indexMetaData);
Expand All @@ -345,10 +352,31 @@ private ClusterState createRecoveryStateAndInitalizeAllocations(MetaData metaDat
throw new IndexOutOfBoundsException();
}
}

final RoutingTable routingTable = routingTableBuilder.build();

final ImmutableOpenMap.Builder<String, ClusterState.Custom> restores = ImmutableOpenMap.builder();
if (snapshotIndices.isEmpty() == false) {
// Some indices are restored from snapshot, the RestoreInProgress must be set accordingly
ImmutableOpenMap.Builder<ShardId, RestoreInProgress.ShardRestoreStatus> restoreShards = ImmutableOpenMap.builder();
for (ShardRouting shard : routingTable.allShards()) {
if (shard.primary() && shard.recoverySource().getType() == RecoverySource.Type.SNAPSHOT) {
ShardId shardId = shard.shardId();
restoreShards.put(shardId, new RestoreInProgress.ShardRestoreStatus(node1.getId(), RestoreInProgress.State.INIT));
}
}

RestoreInProgress.Entry restore = new RestoreInProgress.Entry(snapshot, RestoreInProgress.State.INIT,
new ArrayList<>(snapshotIndices), restoreShards.build());
restores.put(RestoreInProgress.TYPE, new RestoreInProgress(restore));
}

return ClusterState.builder(CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY))
.nodes(DiscoveryNodes.builder().add(node1))
.metaData(metaDataBuilder.build())
.routingTable(routingTableBuilder.build()).build();
.routingTable(routingTable)
.customs(restores.build())
.build();
}

private void addInSyncAllocationIds(Index index, IndexMetaData.Builder indexMetaData,
Expand Down

0 comments on commit 5752583

Please sign in to comment.