Skip to content

Tighten up threading in snapshot finalization #124403

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Conversation

DaveCTurner
Copy link
Contributor

Today snapshot finalization does nontrivial work on the calling thread
(often the cluster applier thread) and also in theory may fork back to
the cluster applier thread in getRepositoryData, yet it always forks
at least one task (the SnapshotInfo write) to the SNAPSHOT pool
anyway. With this change we fork to the SNAPSHOT pool straight away
and then make sure to stay on this pool throughout.

Today snapshot finalization does nontrivial work on the calling thread
(often the cluster applier thread) and also in theory may fork back to
the cluster applier thread in `getRepositoryData`, yet it always forks
at least one task (the `SnapshotInfo` write) to the `SNAPSHOT` pool
anyway. With this change we fork to the `SNAPSHOT` pool straight away
and then make sure to stay on this pool throughout.
@DaveCTurner DaveCTurner added >non-issue :Distributed Coordination/Snapshot/Restore Anything directly related to the `_snapshot/*` APIs v8.19.0 v9.1.0 labels Mar 8, 2025
@DaveCTurner DaveCTurner requested review from DiannaHohensee and removed request for DiannaHohensee March 8, 2025 09:22
@DaveCTurner
Copy link
Contributor Author

Somewhat relates #108907: moving this code into a method object will let us split it up into more manageable pieces and eventually move it out of SnapshotsService altogether which will give us more opportunities to simplify finalization. Not doing that yet because it'd be quite noisy, and I've crafted this PR to have a minimal diff for ease of review.

@DaveCTurner DaveCTurner marked this pull request as ready for review March 10, 2025 10:46
@elasticsearchmachine elasticsearchmachine added the Team:Distributed Coordination Meta label for Distributed Coordination team label Mar 10, 2025
@elasticsearchmachine
Copy link
Collaborator

Pinging @elastic/es-distributed-coordination (Team:Distributed Coordination)

threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(ActionRunnable.supply(metadataListener, () -> {
// This listener is kinda unnecessary since we now always complete it synchronously. It's only here to catch exceptions.
// TODO simplify this.
ActionListener.completeWith(metadataListener, () -> {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

FWIW this is equivalent to threadPool.executor(EsExecutors.DIRECT_EXECUTOR_SERVICE).execute(ActionRunnable.supply(metadataListener, () -> {, i.e the only change from before is the move from forking to ThreadPool.Names.SNAPSHOT to running this on the current thread.

Copy link
Contributor

@DiannaHohensee DiannaHohensee left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks good!

I just left some comments picking on the comments :) Whatever improvements you make should be good 👍

* and triggering the next snapshot-related activity.
*/
// This only really makes sense to run against a BlobStoreRepository, and the division of work between this class and
// BlobStoreRepository#finalizeSnapshot is kind of awkward and artificial; TODO consolidate all this stuff into one place and simplify
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe a little more concise, like:

// TODO: This only makes sense to run against a BlobStoreRepository. This logic should be consolidated into the BlobStoreRepository#finalizeSnapshot method, and hopefully simplified thereby.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think it'll work just to move this into BlobStoreRepository#finalizeSnapshot so I didn't want to prescribe that as the solution.

/**
* Implements the finalization process for a snapshot: does some preparatory calculations, builds a {@link SnapshotInfo} and a
* {@link FinalizeSnapshotContext}, calls {@link Repository#finalizeSnapshot} and handles the outcome by notifying waiting listeners
* and triggering the next snapshot-related activity.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This comment is vague. What's the next snapshot-related activity? What's the preparatory calculations (maybe remove this if not important to know)? There's no context about FinalizeSnapshotContext and why it's relevant to surface a mention on the interface here.

How do I use this thing and why?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The comment is more to orient the reader about what the method does because it's a little long right now (will be shorter soon). It's an AbstractRunnable so the only way to use it is to run it, and why is because you are finalizing a snapshot?

The next snapshot-related activity is deliberately vague because it kinda could be anything that was blocked on this finalization. This includes completing another snapshot, starting a batch of deletes, running a cleanup, likely others too.

threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(ActionRunnable.supply(metadataListener, () -> {
// This listener is kinda unnecessary since we now always complete it synchronously. It's only here to catch exceptions.
// TODO simplify this.
ActionListener.completeWith(metadataListener, () -> {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[opt] if you know what this code does, it might be worth documenting what originally caused us to fork this work over the SNAPSHOT pool -- cpu or I/o intensive? But perhaps that's overkill.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This will go away in a (hopefully fairly immediate) follow up. It used to fork because repo.getSnapshotIndexMetaData interacts with the repository.

@Override
public void onRejection(Exception e) {
if (e instanceof EsRejectedExecutionException esre && esre.isExecutorShutdown()) {
logger.debug("failing finalization of {} due to shutdown", snapshot);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice extra handling.

To verify my understanding, and make a note of the behavior change: previously we'd log a warning about failing due to shutdown; now we're making that quiet under debug. That seems reasonable.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Previously we didn't fork here, we just ran all this on the calling thread, so rejection was not a thing that could happen.

@DaveCTurner DaveCTurner added auto-merge-without-approval Automatically merge pull request when CI checks pass (NB doesn't wait for reviews!) auto-backport Automatically create backport pull requests when merged labels Mar 11, 2025
@elasticsearchmachine elasticsearchmachine merged commit f1f2df7 into elastic:main Mar 11, 2025
17 checks passed
@DaveCTurner DaveCTurner deleted the 2025/03/08/snapshot-finalization-threading branch March 11, 2025 07:13
DaveCTurner added a commit to DaveCTurner/elasticsearch that referenced this pull request Mar 11, 2025
Today snapshot finalization does nontrivial work on the calling thread
(often the cluster applier thread) and also in theory may fork back to
the cluster applier thread in `getRepositoryData`, yet it always forks
at least one task (the `SnapshotInfo` write) to the `SNAPSHOT` pool
anyway. With this change we fork to the `SNAPSHOT` pool straight away
and then make sure to stay on this pool throughout.
@elasticsearchmachine
Copy link
Collaborator

💚 Backport successful

Status Branch Result
8.x

elasticsearchmachine pushed a commit that referenced this pull request Mar 11, 2025
Today snapshot finalization does nontrivial work on the calling thread
(often the cluster applier thread) and also in theory may fork back to
the cluster applier thread in `getRepositoryData`, yet it always forks
at least one task (the `SnapshotInfo` write) to the `SNAPSHOT` pool
anyway. With this change we fork to the `SNAPSHOT` pool straight away
and then make sure to stay on this pool throughout.
albertzaharovits pushed a commit to albertzaharovits/elasticsearch that referenced this pull request Mar 13, 2025
Today snapshot finalization does nontrivial work on the calling thread
(often the cluster applier thread) and also in theory may fork back to
the cluster applier thread in `getRepositoryData`, yet it always forks
at least one task (the `SnapshotInfo` write) to the `SNAPSHOT` pool
anyway. With this change we fork to the `SNAPSHOT` pool straight away
and then make sure to stay on this pool throughout.
jfreden pushed a commit to jfreden/elasticsearch that referenced this pull request Mar 13, 2025
Today snapshot finalization does nontrivial work on the calling thread
(often the cluster applier thread) and also in theory may fork back to
the cluster applier thread in `getRepositoryData`, yet it always forks
at least one task (the `SnapshotInfo` write) to the `SNAPSHOT` pool
anyway. With this change we fork to the `SNAPSHOT` pool straight away
and then make sure to stay on this pool throughout.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
auto-backport Automatically create backport pull requests when merged auto-merge-without-approval Automatically merge pull request when CI checks pass (NB doesn't wait for reviews!) :Distributed Coordination/Snapshot/Restore Anything directly related to the `_snapshot/*` APIs >non-issue Team:Distributed Coordination Meta label for Distributed Coordination team v8.19.0 v9.1.0
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants