Skip to content

Commit

Permalink
Finalize all snapshots completed by shard snapshot updates (#105245)
Browse files Browse the repository at this point in the history
Today when processing a batch of `ShardSnapshotUpdate` tasks each
update's listener considers whether the corresponding snapshot has
completed and, if so, it enqueues it for finalization. This is somewhat
inefficient since we may be processing many shard snapshot updates for
the same few snapshots, but there's no need to check each snapshot for
completion more than once. It's also insufficient since the completion
of a shard snapshot may cause the completion of subsequent snapshots too
(e.g. they can go from state `QUEUED` straight to `MISSING`).

This commit detaches the completion handling from the individual shard
snapshot updates and instead makes sure that any snapshot that reaches a
completed state is enqueued for finalization.

Closes #104939
  • Loading branch information
DaveCTurner committed Feb 9, 2024
1 parent c5bbd4c commit a14ad5f
Show file tree
Hide file tree
Showing 4 changed files with 267 additions and 44 deletions.
6 changes: 6 additions & 0 deletions docs/changelog/105245.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,6 @@
pr: 105245
summary: Finalize all snapshots completed by shard snapshot updates
area: Snapshot/Restore
type: bug
issues:
- 104939
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.ContextPreservingActionListener;
import org.elasticsearch.action.support.GroupedActionListener;
import org.elasticsearch.action.support.RefCountingRunnable;
import org.elasticsearch.action.support.master.TransportMasterNodeAction;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterState;
Expand Down Expand Up @@ -70,7 +71,6 @@
import org.elasticsearch.common.util.Maps;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.common.util.concurrent.ListenableFuture;
import org.elasticsearch.common.util.concurrent.RunOnce;
import org.elasticsearch.core.Nullable;
import org.elasticsearch.core.SuppressForbidden;
import org.elasticsearch.core.Tuple;
Expand Down Expand Up @@ -188,6 +188,8 @@ public final class SnapshotsService extends AbstractLifecycleComponent implement

private final MasterServiceTaskQueue<SnapshotTask> masterServiceTaskQueue;

private final ShardSnapshotUpdateCompletionHandler shardSnapshotUpdateCompletionHandler;

/**
* Setting that specifies the maximum number of allowed concurrent snapshot create and delete operations in the
* cluster state. The number of concurrent operations in a cluster state is defined as the sum of
Expand Down Expand Up @@ -238,6 +240,7 @@ public SnapshotsService(
this.systemIndices = systemIndices;

this.masterServiceTaskQueue = clusterService.createTaskQueue("snapshots-service", Priority.NORMAL, new SnapshotTaskExecutor());
this.shardSnapshotUpdateCompletionHandler = this::handleShardSnapshotUpdateCompletion;
}

/**
Expand Down Expand Up @@ -3084,16 +3087,19 @@ static final class SnapshotShardsUpdateContext {
// updates that were used to update an existing in-progress shard snapshot
private final Set<ShardSnapshotUpdate> executedUpdates = new HashSet<>();

// enqueues a reroute because some shard snapshots finished
private final Runnable rerouteRunnable;
// handles the completion of some shard-snapshot updates, performing the next possible actions
private final ShardSnapshotUpdateCompletionHandler completionHandler;

// entries that became complete due to this batch of updates
private final List<SnapshotsInProgress.Entry> newlyCompletedEntries = new ArrayList<>();

SnapshotShardsUpdateContext(
ClusterStateTaskExecutor.BatchExecutionContext<SnapshotTask> batchExecutionContext,
Runnable rerouteRunnable
ShardSnapshotUpdateCompletionHandler completionHandler
) {
this.batchExecutionContext = batchExecutionContext;
this.initialState = batchExecutionContext.initialState();
this.rerouteRunnable = new RunOnce(rerouteRunnable); // RunOnce to avoid enqueueing O(#shards) listeners
this.completionHandler = completionHandler;
this.updatesByRepo = new HashMap<>();
for (final var taskContext : batchExecutionContext.taskContexts()) {
if (taskContext.getTask() instanceof ShardSnapshotUpdate task) {
Expand All @@ -3113,7 +3119,11 @@ SnapshotsInProgress computeUpdatedState() {
}
final List<SnapshotsInProgress.Entry> newEntries = new ArrayList<>(oldEntries.size());
for (SnapshotsInProgress.Entry entry : oldEntries) {
newEntries.add(applyToEntry(entry, updates.getValue()));
final var newEntry = applyToEntry(entry, updates.getValue());
newEntries.add(newEntry);
if (newEntry != entry && newEntry.state().completed()) {
newlyCompletedEntries.add(newEntry);
}
}
updated = updated.withUpdatedEntriesForRepo(repoName, newEntries);
}
Expand All @@ -3132,12 +3142,20 @@ SnapshotsInProgress computeUpdatedState() {
void completeWithUpdatedState(SnapshotsInProgress snapshotsInProgress) {
if (updatesByRepo.isEmpty() == false) {
final var result = new ShardSnapshotUpdateResult(initialState.metadata(), snapshotsInProgress);
for (final var taskContext : batchExecutionContext.taskContexts()) {
if (taskContext.getTask() instanceof ShardSnapshotUpdate task) {
taskContext.success(() -> {
rerouteRunnable.run();
task.listener.onResponse(result);
});
try (
var onCompletionRefs = new RefCountingRunnable(
() -> completionHandler.handleCompletion(result, newlyCompletedEntries, updatesByRepo.keySet())
)
) {
for (final var taskContext : batchExecutionContext.taskContexts()) {
if (taskContext.getTask() instanceof ShardSnapshotUpdate task) {
final var ref = onCompletionRefs.acquire();
taskContext.success(() -> {
try (ref) {
task.listener.onResponse(result);
}
});
}
}
}
}
Expand Down Expand Up @@ -3376,6 +3394,37 @@ private ImmutableOpenMap.Builder<ShardId, ShardSnapshotStatus> shardsBuilder() {
*/
record ShardSnapshotUpdateResult(Metadata metadata, SnapshotsInProgress snapshotsInProgress) {}

interface ShardSnapshotUpdateCompletionHandler {
void handleCompletion(
ShardSnapshotUpdateResult shardSnapshotUpdateResult,
List<SnapshotsInProgress.Entry> newlyCompletedEntries,
Set<String> updatedRepositories
);
}

private void handleShardSnapshotUpdateCompletion(
ShardSnapshotUpdateResult shardSnapshotUpdateResult,
List<SnapshotsInProgress.Entry> newlyCompletedEntries,
Set<String> updatedRepositories
) {
// Maybe this state update completed one or more snapshots. If we are not already ending them because of some earlier update, end
// them now.
final var snapshotsInProgress = shardSnapshotUpdateResult.snapshotsInProgress();
for (final var newlyCompletedEntry : newlyCompletedEntries) {
if (endingSnapshots.contains(newlyCompletedEntry.snapshot()) == false) {
endSnapshot(newlyCompletedEntry, shardSnapshotUpdateResult.metadata, null);
}
}
// Likewise this state update may enable some new shard clones on any affected repository, so check them all.
for (final var updatedRepository : updatedRepositories) {
startExecutableClones(snapshotsInProgress, updatedRepository);
}
// Also shard snapshot completions may free up some shards to move to other nodes, so we must trigger a reroute.
if (updatedRepositories.isEmpty() == false) {
rerouteService.reroute("after shards snapshot update", Priority.NORMAL, ActionListener.noop());
}
}

/**
* An update to the snapshot state of a shard.
*
Expand Down Expand Up @@ -3455,23 +3504,13 @@ private void innerUpdateSnapshotState(
ShardSnapshotStatus updatedState,
ActionListener<Void> listener
) {
var update = new ShardSnapshotUpdate(snapshot, shardId, repoShardId, updatedState, listener.delegateFailure((delegate, result) -> {
try {
delegate.onResponse(null);
} finally {
// Maybe this state update completed the snapshot. If we are not already ending it because of a concurrent
// state update we check if its state is completed and end it if it is.
final SnapshotsInProgress snapshotsInProgress = result.snapshotsInProgress();
if (endingSnapshots.contains(snapshot) == false) {
final SnapshotsInProgress.Entry updatedEntry = snapshotsInProgress.snapshot(snapshot);
// If the entry is still in the cluster state and is completed, try finalizing the snapshot in the repo
if (updatedEntry != null && updatedEntry.state().completed()) {
endSnapshot(updatedEntry, result.metadata(), null);
}
}
startExecutableClones(snapshotsInProgress, snapshot.getRepository());
}
}));
var update = new ShardSnapshotUpdate(
snapshot,
shardId,
repoShardId,
updatedState,
listener.delegateFailure((delegate, result) -> delegate.onResponse(null))
);
logger.trace("received updated snapshot restore state [{}]", update);
masterServiceTaskQueue.submitTask("update snapshot state", update, null);
}
Expand Down Expand Up @@ -3751,7 +3790,7 @@ public ClusterState execute(BatchExecutionContext<SnapshotTask> batchExecutionCo
final ClusterState state = batchExecutionContext.initialState();
final SnapshotShardsUpdateContext shardsUpdateContext = new SnapshotShardsUpdateContext(
batchExecutionContext,
() -> rerouteService.reroute("after shards snapshot update", Priority.NORMAL, ActionListener.noop())
shardSnapshotUpdateCompletionHandler
);
final SnapshotsInProgress initialSnapshots = SnapshotsInProgress.get(state);
SnapshotsInProgress snapshotsInProgress = shardsUpdateContext.computeUpdatedState();
Expand Down
Loading

0 comments on commit a14ad5f

Please sign in to comment.