Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 5 additions & 0 deletions docs/changelog/141408.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,5 @@
area: Snapshot/Restore
issues: []
pr: 141408
summary: Ensure paused shard snapshot can be deleted
type: bug
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.elasticsearch.cluster.SnapshotDeletionsInProgress;
import org.elasticsearch.cluster.SnapshotsInProgress;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.metadata.ProjectId;
import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.service.ClusterService;
Expand All @@ -43,6 +44,7 @@
import org.elasticsearch.test.transport.MockTransportService;

import java.util.Collection;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.CyclicBarrier;
import java.util.concurrent.ExecutionException;
Expand All @@ -51,6 +53,7 @@
import static org.elasticsearch.snapshots.SnapshotShutdownProgressTracker.SNAPSHOT_PROGRESS_DURING_SHUTDOWN_LOG_INTERVAL_SETTING;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasSize;
import static org.hamcrest.Matchers.oneOf;

Expand Down Expand Up @@ -655,6 +658,57 @@ && switch (shardEntry.getValue().state()) {
resetMockLog();
}

public void testDeleteSnapshotWithPausedShardSnapshots() throws Exception {
final var originalNode = internalCluster().startDataOnlyNode();
final var indexName = randomIndexName();
createIndexWithContent(indexName, indexSettings(1, 0).put(REQUIRE_NODE_NAME_SETTING, originalNode).build());

final var repoName = randomRepoName();
createRepository(repoName, "mock");

// Start the snapshot and block it on the data node
final String snapshotName = randomSnapshotName();
final var snapshotFuture = startFullSnapshotBlockedOnDataNode(snapshotName, repoName, originalNode);

// Mark data node for shutdown and ensure shard snapshot is paused
final var clusterService = internalCluster().getCurrentMasterNodeInstance(ClusterService.class);
final var shardSnapshotsPausedListener = ClusterServiceUtils.addTemporaryStateListener(clusterService, state -> {
final var snapshotEntry = SnapshotsInProgress.get(state)
.forRepo(ProjectId.DEFAULT, repoName)
.stream()
.filter(entry -> entry.snapshot().getSnapshotId().getName().equals(snapshotName))
.findFirst()
.orElseThrow(() -> new AssertionError("Snapshot [" + snapshotName + "] not found"));

return snapshotEntry.shards()
.values()
.stream()
.allMatch(shardSnapshotStatus -> shardSnapshotStatus.state() == SnapshotsInProgress.ShardState.PAUSED_FOR_NODE_REMOVAL);
});
putShutdownForRemovalMetadata(originalNode, clusterService);
unblockAllDataNodes(repoName);
safeAwait(shardSnapshotsPausedListener);

// Delete the snapshot and ensure it is successfully and clears all snapshot operations from cluster state
final var snapshotClearedListener = ClusterServiceUtils.addTemporaryStateListener(
clusterService,
state -> SnapshotsInProgress.get(state).isEmpty() && SnapshotDeletionsInProgress.get(state).getEntries().isEmpty()
);
assertTrue(safeGet(startDeleteSnapshot(repoName, snapshotName)).isAcknowledged());
safeAwait(snapshotClearedListener);

// Snapshot creation has failed snapshot response
final var createSnapshotResponse = safeGet(snapshotFuture);
assertThat(createSnapshotResponse.getSnapshotInfo().state(), equalTo(SnapshotState.FAILED));
assertThat(createSnapshotResponse.getSnapshotInfo().reason(), containsString("Snapshot was aborted by deletion"));

// No snapshot is in the repository
final List<SnapshotInfo> snapshotInfos = clusterAdmin().prepareGetSnapshots(TEST_REQUEST_TIMEOUT, repoName).get().getSnapshots();
assertThat(snapshotInfos, empty());

clearShutdownMetadata(clusterService);
}

private static void addUnassignedShardsWatcher(ClusterService clusterService, String indexName) {
ClusterServiceUtils.addTemporaryStateListener(clusterService, state -> {
final var indexRoutingTable = state.routingTable().index(indexName);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1240,12 +1240,10 @@ public Entry abort() {
allQueued &= status.state() == ShardState.QUEUED;
if (status.state().completed() == false) {
final String nodeId = status.nodeId();
status = new ShardSnapshotStatus(
nodeId,
nodeId == null ? ShardState.FAILED : ShardState.ABORTED,
status.generation(),
"aborted by snapshot deletion"
);
final var newState = (nodeId == null || status.state() == ShardState.PAUSED_FOR_NODE_REMOVAL)
? ShardState.FAILED
: ShardState.ABORTED;
status = new ShardSnapshotStatus(nodeId, newState, status.generation(), "aborted by snapshot deletion");
}
completed &= status.state().completed();
shardsBuilder.put(shardEntry.getKey(), status);
Expand Down