Skip to content

Commit 7bb81be

Browse files
Improve Snapshot Abort Efficiency (#62173)
There is no need to let snapshots that haven't yet written anything to the repo finalize with `FAILED`. When we still had the `INIT` state we would also just remove these snapshots from the state without any further action. This is not just a theoretical optimization. Currently, the situation of having a lot of queued up snapshots is fairly complicated to resolve when all the queued shards move to aborted since it is now necessary to execute tasks on the `SNAPSHOT` pool (that might be very busy) to remove the snapshot from the CS (including a number of redundant CS updates and repo writes for finalizing these snapshots before deleting them right away after).
1 parent a25f06d commit 7bb81be

File tree

3 files changed

+79
-14
lines changed

3 files changed

+79
-14
lines changed

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

Lines changed: 28 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -548,7 +548,8 @@ public void testQueuedDeletesWithFailures() throws Exception {
548548
// Second delete works out cleanly since the repo is unblocked now
549549
assertThat(secondDeleteFuture.get().isAcknowledged(), is(true));
550550
// Snapshot should have been aborted
551-
assertThat(snapshotFuture.get().getSnapshotInfo().state(), is(SnapshotState.FAILED));
551+
final SnapshotException snapshotException = expectThrows(SnapshotException.class, snapshotFuture::actionGet);
552+
assertThat(snapshotException.getMessage(), containsString(SnapshotsInProgress.ABORTED_FAILURE_TEXT));
552553

553554
assertThat(client().admin().cluster().prepareGetSnapshots(repoName).get().getSnapshots(repoName), empty());
554555
}
@@ -574,7 +575,8 @@ public void testQueuedDeletesWithOverlap() throws Exception {
574575
// Second delete works out cleanly since the repo is unblocked now
575576
assertThat(secondDeleteFuture.get().isAcknowledged(), is(true));
576577
// Snapshot should have been aborted
577-
assertThat(snapshotFuture.get().getSnapshotInfo().state(), is(SnapshotState.FAILED));
578+
final SnapshotException snapshotException = expectThrows(SnapshotException.class, snapshotFuture::actionGet);
579+
assertThat(snapshotException.getMessage(), containsString(SnapshotsInProgress.ABORTED_FAILURE_TEXT));
578580

579581
assertThat(client().admin().cluster().prepareGetSnapshots(repoName).get().getSnapshots(repoName), empty());
580582
}
@@ -1215,6 +1217,30 @@ public void testQueuedDeleteAfterFinalizationFailure() throws Exception {
12151217
assertThat(sne.getCause().getMessage(), containsString("exception after block"));
12161218
}
12171219

1220+
public void testAbortNotStartedSnapshotWithoutIO() throws Exception {
1221+
internalCluster().startMasterOnlyNode();
1222+
final String dataNode = internalCluster().startDataOnlyNode();
1223+
final String repoName = "test-repo";
1224+
createRepository(repoName, "mock");
1225+
createIndexWithContent("test-index");
1226+
1227+
final ActionFuture<CreateSnapshotResponse> createSnapshot1Future =
1228+
startFullSnapshotBlockedOnDataNode("first-snapshot", repoName, dataNode);
1229+
1230+
final String snapshotTwo = "second-snapshot";
1231+
final ActionFuture<CreateSnapshotResponse> createSnapshot2Future = startFullSnapshot(repoName, snapshotTwo);
1232+
1233+
awaitNSnapshotsInProgress(2);
1234+
1235+
assertAcked(startDelete(repoName, snapshotTwo).get());
1236+
final SnapshotException sne = expectThrows(SnapshotException.class, createSnapshot2Future::actionGet);
1237+
1238+
assertFalse(createSnapshot1Future.isDone());
1239+
unblockNode(repoName, dataNode);
1240+
assertSuccessful(createSnapshot1Future);
1241+
assertThat(getRepositoryData(repoName).getGenId(), is(0L));
1242+
}
1243+
12181244
private static String startDataNodeWithLargeSnapshotPool() {
12191245
return internalCluster().startDataOnlyNode(LARGE_SNAPSHOT_POOL_SETTINGS);
12201246
}

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

Lines changed: 12 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -59,6 +59,8 @@ public class SnapshotsInProgress extends AbstractNamedDiffable<Custom> implement
5959

6060
public static final String TYPE = "snapshots";
6161

62+
public static final String ABORTED_FAILURE_TEXT = "Snapshot was aborted by deletion";
63+
6264
@Override
6365
public boolean equals(Object o) {
6466
if (this == o) return true;
@@ -178,14 +180,19 @@ public Entry withRepoGen(long newRepoGen) {
178180
* data node or to {@link ShardState#FAILED} if not assigned to any data node.
179181
* If the instance had no in-progress shard snapshots assigned to data nodes it's moved to state {@link State#SUCCESS}, otherwise
180182
* it's moved to state {@link State#ABORTED}.
183+
* In the special case where this instance has not yet made any progress on any shard this method just returns
184+
* {@code null} since no abort is needed and the snapshot can simply be removed from the cluster state outright.
181185
*
182-
* @return aborted snapshot entry
186+
* @return aborted snapshot entry or {@code null} if entry can be removed from the cluster state directly
183187
*/
188+
@Nullable
184189
public Entry abort() {
185190
final ImmutableOpenMap.Builder<ShardId, ShardSnapshotStatus> shardsBuilder = ImmutableOpenMap.builder();
186191
boolean completed = true;
192+
boolean allQueued = true;
187193
for (ObjectObjectCursor<ShardId, ShardSnapshotStatus> shardEntry : shards) {
188194
ShardSnapshotStatus status = shardEntry.value;
195+
allQueued &= status.state() == ShardState.QUEUED;
189196
if (status.state().completed() == false) {
190197
final String nodeId = status.nodeId();
191198
status = new ShardSnapshotStatus(nodeId, nodeId == null ? ShardState.FAILED : ShardState.ABORTED,
@@ -194,7 +201,10 @@ public Entry abort() {
194201
completed &= status.state().completed();
195202
shardsBuilder.put(shardEntry.key, status);
196203
}
197-
return fail(shardsBuilder.build(), completed ? State.SUCCESS : State.ABORTED, "Snapshot was aborted by deletion");
204+
if (allQueued) {
205+
return null;
206+
}
207+
return fail(shardsBuilder.build(), completed ? State.SUCCESS : State.ABORTED, ABORTED_FAILURE_TEXT);
198208
}
199209

200210
public Entry fail(ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards, State state, String failure) {

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

Lines changed: 39 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -99,6 +99,7 @@
9999
import java.util.List;
100100
import java.util.Locale;
101101
import java.util.Map;
102+
import java.util.Objects;
102103
import java.util.Optional;
103104
import java.util.Set;
104105
import java.util.concurrent.ConcurrentHashMap;
@@ -1141,7 +1142,13 @@ public void deleteSnapshots(final DeleteSnapshotRequest request, final ActionLis
11411142

11421143
private boolean reusedExistingDelete = false;
11431144

1144-
private final Collection<SnapshotsInProgress.Entry> completedSnapshots = new ArrayList<>();
1145+
// Snapshots that had all of their shard snapshots in queued state and thus were removed from the
1146+
// cluster state right away
1147+
private final Collection<Snapshot> completedNoCleanup = new ArrayList<>();
1148+
1149+
// Snapshots that were aborted and that already wrote data to the repository and now have to be deleted
1150+
// from the repository after the cluster state update
1151+
private final Collection<SnapshotsInProgress.Entry> completedWithCleanup = new ArrayList<>();
11451152

11461153
@Override
11471154
public ClusterState execute(ClusterState currentState) {
@@ -1172,18 +1179,34 @@ public ClusterState execute(ClusterState currentState) {
11721179
"cannot delete snapshot during a restore in progress in [" + restoreInProgress + "]");
11731180
}
11741181
}
1182+
// Snapshot ids that will have to be physically deleted from the repository
1183+
final Set<SnapshotId> snapshotIdsRequiringCleanup = new HashSet<>(snapshotIds);
11751184
final SnapshotsInProgress updatedSnapshots = SnapshotsInProgress.of(snapshots.entries().stream()
11761185
.map(existing -> {
1177-
// snapshot is started - mark every non completed shard as aborted
1178-
if (existing.state() == State.STARTED && snapshotIds.contains(existing.snapshot().getSnapshotId())) {
1186+
if (existing.state() == State.STARTED &&
1187+
snapshotIdsRequiringCleanup.contains(existing.snapshot().getSnapshotId())) {
1188+
// snapshot is started - mark every non completed shard as aborted
11791189
final SnapshotsInProgress.Entry abortedEntry = existing.abort();
1180-
if (abortedEntry.state().completed()) {
1181-
completedSnapshots.add(abortedEntry);
1190+
if (abortedEntry == null) {
1191+
// No work has been done for this snapshot yet so we remove it from the cluster state directly
1192+
final Snapshot existingNotYetStartedSnapshot = existing.snapshot();
1193+
// Adding the snapshot to #endingSnapshots since we still have to resolve its listeners to not trip
1194+
// any leaked listener assertions
1195+
if (endingSnapshots.add(existingNotYetStartedSnapshot)) {
1196+
completedNoCleanup.add(existingNotYetStartedSnapshot);
1197+
}
1198+
snapshotIdsRequiringCleanup.remove(existingNotYetStartedSnapshot.getSnapshotId());
1199+
} else if (abortedEntry.state().completed()) {
1200+
completedWithCleanup.add(abortedEntry);
11821201
}
11831202
return abortedEntry;
11841203
}
11851204
return existing;
1186-
}).collect(Collectors.toUnmodifiableList()));
1205+
}).filter(Objects::nonNull).collect(Collectors.toUnmodifiableList()));
1206+
if (snapshotIdsRequiringCleanup.isEmpty()) {
1207+
// We only saw snapshots that could be removed from the cluster state right away, no need to update the deletions
1208+
return updateWithSnapshots(currentState, updatedSnapshots, null);
1209+
}
11871210
// add the snapshot deletion to the cluster state
11881211
final SnapshotDeletionsInProgress.Entry replacedEntry = deletionsInProgress.getEntries().stream().filter(entry ->
11891212
entry.repository().equals(repoName) && entry.state() == SnapshotDeletionsInProgress.State.WAITING)
@@ -1198,9 +1221,10 @@ public ClusterState execute(ClusterState currentState) {
11981221
reusedExistingDelete = true;
11991222
return currentState;
12001223
}
1201-
ensureBelowConcurrencyLimit(repoName, snapshotIds.get(0).getName(), snapshots, deletionsInProgress);
1224+
final List<SnapshotId> toDelete = List.copyOf(snapshotIdsRequiringCleanup);
1225+
ensureBelowConcurrencyLimit(repoName, toDelete.get(0).getName(), snapshots, deletionsInProgress);
12021226
newDelete = new SnapshotDeletionsInProgress.Entry(
1203-
snapshotIds,
1227+
toDelete,
12041228
repoName,
12051229
threadPool.absoluteTimeInMillis(),
12061230
repositoryData.getGenId(),
@@ -1210,7 +1234,7 @@ public ClusterState execute(ClusterState currentState) {
12101234
repoName.equals(entry.repository()) && entry.state() == SnapshotDeletionsInProgress.State.STARTED)
12111235
? SnapshotDeletionsInProgress.State.STARTED : SnapshotDeletionsInProgress.State.WAITING);
12121236
} else {
1213-
newDelete = replacedEntry.withAddedSnapshots(snapshotIds);
1237+
newDelete = replacedEntry.withAddedSnapshots(snapshotIdsRequiringCleanup);
12141238
}
12151239
return updateWithSnapshots(currentState, updatedSnapshots,
12161240
(replacedEntry == null ? deletionsInProgress : deletionsInProgress.withRemovedEntry(replacedEntry.uuid()))
@@ -1219,11 +1243,16 @@ public ClusterState execute(ClusterState currentState) {
12191243

12201244
@Override
12211245
public void onFailure(String source, Exception e) {
1246+
endingSnapshots.removeAll(completedNoCleanup);
12221247
listener.onFailure(e);
12231248
}
12241249

12251250
@Override
12261251
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
1252+
for (Snapshot snapshot : completedNoCleanup) {
1253+
failSnapshotCompletionListeners(snapshot,
1254+
new SnapshotException(snapshot, SnapshotsInProgress.ABORTED_FAILURE_TEXT));
1255+
}
12271256
if (newDelete == null) {
12281257
listener.onResponse(null);
12291258
} else {
@@ -1238,7 +1267,7 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS
12381267
logger.trace("Delete [{}] could not execute directly and was queued", newDelete);
12391268
}
12401269
} else {
1241-
for (SnapshotsInProgress.Entry completedSnapshot : completedSnapshots) {
1270+
for (SnapshotsInProgress.Entry completedSnapshot : completedWithCleanup) {
12421271
endSnapshot(completedSnapshot, newState.metadata(), repositoryData);
12431272
}
12441273
}

0 commit comments

Comments
 (0)