Skip to content

Commit 8c5c96a

Browse files
authored
[9.2] Fix PAUSED_FOR_NODE_REMOVAL shard blocking QUEUED promotion (elastic#142637) (elastic#142673)
* Fix PAUSED_FOR_NODE_REMOVAL shard blocking QUEUED promotion (elastic#142637) When a snapshot with a PAUSED_FOR_NODE_REMOVAL shard is deleted, the abort previously transitioned it directly to FAILED. This bypassed the normal state propagation that promotes QUEUED shards, allowing a subsequently created snapshot to incorrectly receive INIT instead of QUEUED for the same shard, violating the ordering invariant. Change abort to transition PAUSED_FOR_NODE_REMOVAL to ABORTED (an active state) so that new snapshots correctly get QUEUED. The data node detects the PAUSED local status on an ABORTED entry and reports FAILED to the master, which triggers QUEUED promotion through the existing state propagation. (cherry picked from commit 3393f3a) * fix imports
1 parent 2a35c86 commit 8c5c96a

File tree

4 files changed

+95
-3
lines changed

4 files changed

+95
-3
lines changed

server/src/internalClusterTest/java/org/elasticsearch/snapshots/ConcurrentSnapshotsIT.java

Lines changed: 72 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,8 @@
2626
import org.elasticsearch.cluster.SnapshotDeletionsInProgress;
2727
import org.elasticsearch.cluster.SnapshotsInProgress;
2828
import org.elasticsearch.cluster.metadata.MetadataDeleteIndexService;
29+
import org.elasticsearch.cluster.metadata.ProjectId;
30+
import org.elasticsearch.cluster.service.ClusterService;
2931
import org.elasticsearch.common.Strings;
3032
import org.elasticsearch.common.settings.Settings;
3133
import org.elasticsearch.common.util.concurrent.ListenableFuture;
@@ -2288,6 +2290,76 @@ private ActionFuture<CreateSnapshotResponse> startFullSnapshotFromMasterClient(S
22882290
.execute();
22892291
}
22902292

2293+
/**
2294+
* This test ensures that deleting a snapshot with paused shards works fine when there are shards queued behind it
2295+
* as well as creating new snapshots concurrently. The snapshot state machine should propagate correctly.
2296+
*/
2297+
public void testConcurrentDeletePausedSnapshotAndCreateSnapshot() throws Exception {
2298+
final String masterNode = internalCluster().startMasterOnlyNode();
2299+
final String dataNode = internalCluster().startDataOnlyNode();
2300+
ensureStableCluster(2);
2301+
final String repoName = randomRepoName();
2302+
createRepository(repoName, "mock");
2303+
2304+
final String indexName = randomIndexName();
2305+
createIndexWithContent(indexName, indexSettingsNoReplicas(1).build());
2306+
2307+
final var snap0 = randomSnapshotName();
2308+
final var snap1 = randomSnapshotName();
2309+
final var snap2 = randomSnapshotName();
2310+
final var snap3 = randomSnapshotName();
2311+
2312+
// snap0: block finalization on the master so that later deletion stays WAITING
2313+
blockMasterOnWriteIndexFile(repoName);
2314+
final var snap0Future = startFullSnapshot(repoName, snap0);
2315+
waitForBlock(masterNode, repoName);
2316+
2317+
// Add data so that snap1's shard snapshot has new segment files to write (allowing data node block)
2318+
indexDoc(indexName, "another_id", "foo", "bar");
2319+
2320+
// snap1: block on data node, then pause for node removal
2321+
final var snap1Future = startFullSnapshotBlockedOnDataNode(snap1, repoName, dataNode);
2322+
2323+
// Shutdown and unblock the data node so that the shard snapshot moves to PAUSED_FOR_NODE_REMOVAL
2324+
final var clusterService = internalCluster().getCurrentMasterNodeInstance(ClusterService.class);
2325+
putShutdownForRemovalMetadata(dataNode, clusterService);
2326+
unblockNode(repoName, dataNode);
2327+
awaitClusterState(state -> {
2328+
for (var entry : SnapshotsInProgress.get(state).forRepo(ProjectId.DEFAULT, repoName)) {
2329+
if (entry.snapshot().getSnapshotId().getName().equals(snap1)) {
2330+
for (var shard : entry.shards().values()) {
2331+
if (shard.state() == SnapshotsInProgress.ShardState.PAUSED_FOR_NODE_REMOVAL) {
2332+
return true;
2333+
}
2334+
}
2335+
}
2336+
}
2337+
return false;
2338+
});
2339+
2340+
// snap2: shard is QUEUED behind snap1's active PAUSED_FOR_NODE_REMOVAL shard
2341+
final var snap2Future = startFullSnapshot(repoName, snap2);
2342+
awaitNumberOfSnapshotsInProgress(3);
2343+
2344+
// Delete snap1: PAUSED_FOR_NODE_REMOVAL → ABORTED in abort(), deletion is WAITING because snap0 is still finalizing
2345+
final ActionFuture<AcknowledgedResponse> deleteFuture = startDeleteSnapshot(repoName, snap1);
2346+
awaitNDeletionsInProgress(1);
2347+
2348+
// snap3: start yet another snapshot and its shard correctly gets QUEUED
2349+
final var snap3Future = startFullSnapshot(repoName, snap3);
2350+
2351+
// Clean up: clear shutdown so shard snapshots can run, then unblock finalization
2352+
clearShutdownMetadata(clusterService);
2353+
unblockNode(repoName, masterNode);
2354+
2355+
// All snapshots except the deleted on should complete successfully
2356+
assertSuccessful(snap0Future);
2357+
assertThat(snap1Future.get().getSnapshotInfo().state(), is(SnapshotState.FAILED));
2358+
assertThat(deleteFuture.get().isAcknowledged(), is(true));
2359+
assertSuccessful(snap2Future);
2360+
assertSuccessful(snap3Future);
2361+
}
2362+
22912363
private void createIndexWithContent(String indexName, String nodeInclude, String nodeExclude) {
22922364
createIndexWithContent(
22932365
indexName,

server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1240,9 +1240,7 @@ public Entry abort() {
12401240
allQueued &= status.state() == ShardState.QUEUED;
12411241
if (status.state().completed() == false) {
12421242
final String nodeId = status.nodeId();
1243-
final var newState = (nodeId == null || status.state() == ShardState.PAUSED_FOR_NODE_REMOVAL)
1244-
? ShardState.FAILED
1245-
: ShardState.ABORTED;
1243+
final var newState = nodeId == null ? ShardState.FAILED : ShardState.ABORTED;
12461244
status = new ShardSnapshotStatus(nodeId, newState, status.generation(), "aborted by snapshot deletion");
12471245
}
12481246
completed &= status.state().completed();

server/src/main/java/org/elasticsearch/index/snapshots/IndexShardSnapshotStatus.java

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -242,6 +242,15 @@ public synchronized void moveToFailed(final long endTime, final String failure)
242242
}
243243
}
244244

245+
/**
246+
* Transition from {@link Stage#PAUSED} to {@link Stage#FAILURE}. The abort listeners and timing fields were already
247+
* handled during the earlier PAUSING → PAUSED transition, so only the stage needs updating.
248+
*/
249+
public void moveFromPausedToFailed() {
250+
final boolean moved = stage.compareAndSet(Stage.PAUSED, Stage.FAILURE);
251+
assert moved : "expected stage PAUSED but got " + stage.get();
252+
}
253+
245254
public ShardGeneration generation() {
246255
return generation.get();
247256
}

server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -357,6 +357,19 @@ private void handleUpdatedSnapshotsInProgressEntry(String localNodeId, boolean r
357357
(outcomeInfoString) -> {}
358358
);
359359
}
360+
} else if (snapshotStatus.isPaused()) {
361+
// Shard was paused for node removal then aborted by snapshot deletion.
362+
// abortIfNotCompleted won't transition from PAUSED, so report FAILED directly.
363+
logger.debug("snapshot [{}] is deleted after PAUSED, updating shard snapshot for {} to FAILED", snapshot, sid);
364+
snapshotStatus.moveFromPausedToFailed();
365+
notifyUnsuccessfulSnapshotShard(
366+
snapshot,
367+
sid,
368+
ShardState.FAILED,
369+
shard.getValue().reason(),
370+
shard.getValue().generation(),
371+
(outcomeInfoString) -> {}
372+
);
360373
} else {
361374
snapshotStatus.abortIfNotCompleted("snapshot has been aborted", notifyOnAbortTaskRunner::enqueueTask);
362375
}

0 commit comments

Comments
 (0)