-
Notifications
You must be signed in to change notification settings - Fork 25.2k
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
Changes from all commits
6ccd8ab
7b127bf
492defb
fbbd199
594202b
d278c14
e355340
850e2c1
0d70bd5
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -71,7 +71,9 @@ | |
import org.elasticsearch.common.unit.ByteSizeValue; | ||
import org.elasticsearch.common.util.CollectionUtils; | ||
import org.elasticsearch.common.util.Maps; | ||
import org.elasticsearch.common.util.concurrent.AbstractRunnable; | ||
import org.elasticsearch.common.util.concurrent.EsExecutors; | ||
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; | ||
import org.elasticsearch.common.util.concurrent.ListenableFuture; | ||
import org.elasticsearch.core.FixForMultiProject; | ||
import org.elasticsearch.core.Nullable; | ||
|
@@ -1398,9 +1400,34 @@ private void leaveRepoLoop(String repository) { | |
} | ||
|
||
private void finalizeSnapshotEntry(Snapshot snapshot, Metadata metadata, RepositoryData repositoryData) { | ||
assert currentlyFinalizing.contains(snapshot.getRepository()); | ||
assert repositoryOperations.assertNotQueued(snapshot); | ||
try { | ||
threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(new SnapshotFinalization(snapshot, metadata, repositoryData)); | ||
} | ||
|
||
/** | ||
* 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 (another finalization, a batch of deletes, etc.) | ||
*/ | ||
// 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 | ||
private class SnapshotFinalization extends AbstractRunnable { | ||
|
||
private final Snapshot snapshot; | ||
private final Metadata metadata; | ||
private final RepositoryData repositoryData; | ||
|
||
SnapshotFinalization(Snapshot snapshot, Metadata metadata, RepositoryData repositoryData) { | ||
this.snapshot = snapshot; | ||
this.metadata = metadata; | ||
this.repositoryData = repositoryData; | ||
} | ||
|
||
@Override | ||
protected void doRun() { | ||
assert ThreadPool.assertCurrentThreadPool(ThreadPool.Names.SNAPSHOT); | ||
assert currentlyFinalizing.contains(snapshot.getRepository()); | ||
assert repositoryOperations.assertNotQueued(snapshot); | ||
|
||
SnapshotsInProgress.Entry entry = SnapshotsInProgress.get(clusterService.state()).snapshot(snapshot); | ||
final String failure = entry.failure(); | ||
logger.trace("[{}] finalizing snapshot in repository, state: [{}], failure[{}]", snapshot, entry.state(), failure); | ||
|
@@ -1428,7 +1455,9 @@ private void finalizeSnapshotEntry(Snapshot snapshot, Metadata metadata, Reposit | |
final ListenableFuture<Metadata> metadataListener = new ListenableFuture<>(); | ||
final Repository repo = repositoriesService.repository(snapshot.getRepository()); | ||
if (entry.isClone()) { | ||
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, () -> { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. FWIW this is equivalent to There was a problem hiding this comment. Choose a reason for hiding this commentThe 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. There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 |
||
final Metadata existing = repo.getSnapshotGlobalMetadata(entry.source()); | ||
final Metadata.Builder metaBuilder = Metadata.builder(existing); | ||
final Set<Index> existingIndices = new HashSet<>(); | ||
|
@@ -1450,11 +1479,12 @@ private void finalizeSnapshotEntry(Snapshot snapshot, Metadata metadata, Reposit | |
); | ||
metaBuilder.dataStreams(dataStreamsToCopy, dataStreamAliasesToCopy); | ||
return metaBuilder.build(); | ||
})); | ||
}); | ||
} else { | ||
metadataListener.onResponse(metadata); | ||
} | ||
metadataListener.addListener(ActionListener.wrap(meta -> { | ||
assert ThreadPool.assertCurrentThreadPool(ThreadPool.Names.SNAPSHOT); | ||
final Metadata metaForSnapshot = metadataForSnapshot(entry, meta); | ||
|
||
final Map<String, SnapshotInfo.IndexSnapshotDetails> indexSnapshotDetails = Maps.newMapWithExpectedSize( | ||
|
@@ -1554,7 +1584,20 @@ public void onFailure(Exception e) { | |
shardGenerations | ||
) | ||
)); | ||
} catch (Exception e) { | ||
} | ||
|
||
@Override | ||
public void onRejection(Exception e) { | ||
if (e instanceof EsRejectedExecutionException esre && esre.isExecutorShutdown()) { | ||
logger.debug("failing finalization of {} due to shutdown", snapshot); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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. There was a problem hiding this comment. Choose a reason for hiding this commentThe 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. |
||
handleFinalizationFailure(e, snapshot, repositoryData, ShardGenerations.EMPTY); | ||
} else { | ||
onFailure(e); | ||
} | ||
} | ||
|
||
@Override | ||
public void onFailure(Exception e) { | ||
logger.error(Strings.format("unexpected failure finalizing %s", snapshot), e); | ||
assert false : new AssertionError("unexpected failure finalizing " + snapshot, e); | ||
handleFinalizationFailure(e, snapshot, repositoryData, ShardGenerations.EMPTY); | ||
|
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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.