Encapsulate snapshots deletion process (#100617)

Introduces the `SnapshotsDeletion` class which encapsulates the process
of deleting some collection of snapshots. In particular this class gives
us somewhere to store various deletion-wide data which significantly
reduces the length of some argument lists.

Relates #100568
This commit is contained in:
David Turner 2023-10-11 07:03:31 +01:00 committed by GitHub
parent f35c3b49b5
commit b5843e4b98
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
3 changed files with 443 additions and 409 deletions

View file

@ -167,7 +167,7 @@ teardown:
- match: {count: 3} - match: {count: 3}
- do: - do:
catch: /cannot delete snapshot from a readonly repository/ catch: /repository is readonly/
snapshot.delete: snapshot.delete:
repository: repository-url repository: repository-url
snapshot: snapshot-two snapshot: snapshot-two
@ -229,7 +229,7 @@ teardown:
- match: {count: 3} - match: {count: 3}
- do: - do:
catch: /cannot delete snapshot from a readonly repository/ catch: /repository is readonly/
snapshot.delete: snapshot.delete:
repository: repository-file repository: repository-file
snapshot: snapshot-one snapshot: snapshot-one

View file

@ -1050,7 +1050,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
assertRequestBuilderThrows( assertRequestBuilderThrows(
client.admin().cluster().prepareDeleteSnapshot("readonly-repo", "test-snap"), client.admin().cluster().prepareDeleteSnapshot("readonly-repo", "test-snap"),
RepositoryException.class, RepositoryException.class,
"cannot delete snapshot from a readonly repository" "repository is readonly"
); );
logger.info("--> try making another snapshot"); logger.info("--> try making another snapshot");

View file

@ -836,35 +836,126 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
IndexVersion repositoryFormatIndexVersion, IndexVersion repositoryFormatIndexVersion,
SnapshotDeleteListener listener SnapshotDeleteListener listener
) { ) {
if (isReadOnly()) { createSnapshotsDeletion(snapshotIds, repositoryDataGeneration, repositoryFormatIndexVersion, new ActionListener<>() {
listener.onFailure(new RepositoryException(metadata.name(), "cannot delete snapshot from a readonly repository"));
} else {
threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(new AbstractRunnable() {
@Override @Override
protected void doRun() throws Exception { public void onResponse(SnapshotsDeletion snapshotsDeletion) {
final Map<String, BlobMetadata> rootBlobs = blobContainer().listBlobs(OperationPurpose.SNAPSHOT); snapshotsDeletion.runDelete(listener);
final RepositoryData repositoryData = safeRepositoryData(repositoryDataGeneration, rootBlobs);
// Cache the indices that were found before writing out the new index-N blob so that a stuck master will never
// delete an index that was created by another master node after writing this index-N blob.
final Map<String, BlobContainer> foundIndices = blobStore().blobContainer(indicesPath())
.children(OperationPurpose.SNAPSHOT);
doDeleteShardSnapshots(
snapshotIds,
repositoryDataGeneration,
foundIndices,
rootBlobs,
repositoryData,
repositoryFormatIndexVersion,
listener
);
} }
@Override @Override
public void onFailure(Exception e) { public void onFailure(Exception e) {
listener.onFailure(new RepositoryException(metadata.name(), "failed to delete snapshots " + snapshotIds, e)); listener.onFailure(e);
} }
}); });
} }
private void createSnapshotsDeletion(
Collection<SnapshotId> snapshotIds,
long repositoryDataGeneration,
IndexVersion repositoryFormatIndexVersion,
ActionListener<SnapshotsDeletion> listener
) {
if (isReadOnly()) {
listener.onFailure(new RepositoryException(metadata.name(), "repository is readonly"));
} else {
threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(ActionRunnable.supply(listener, () -> {
final var originalRootBlobs = blobContainer().listBlobs(OperationPurpose.SNAPSHOT);
return new SnapshotsDeletion(
snapshotIds,
repositoryDataGeneration,
repositoryFormatIndexVersion,
originalRootBlobs,
blobStore().blobContainer(indicesPath()).children(OperationPurpose.SNAPSHOT),
safeRepositoryData(repositoryDataGeneration, originalRootBlobs)
);
}));
}
}
/**
* <p>
* Represents the process of deleting some collection of snapshots within this repository which since 7.6.0 looks like this:
* </p>
* <ul>
* <li>Write a new {@link BlobStoreIndexShardSnapshots} for each affected shard, and compute the blobs to delete.</li>
* <li>Update the {@link RepositoryData} to remove references to deleted snapshots/indices and point to the new
* {@link BlobStoreIndexShardSnapshots} files.</li>
* <li>Remove up any now-unreferenced blobs.</li>
* </ul>
* <p>
* Until the {@link RepositoryData} is updated there should be no other activities in the repository, and in particular the root
* blob must not change until it is updated by this deletion and {@link SnapshotDeleteListener#onRepositoryDataWritten} is called.
* </p>
*/
class SnapshotsDeletion {
/**
* The IDs of the snapshots to delete.
*/
private final Collection<SnapshotId> snapshotIds;
/**
* The {@link RepositoryData} generation at the start of the process, to ensure that the {@link RepositoryData} does not change
* while the new {@link BlobStoreIndexShardSnapshots} are being written.
*/
private final long originalRepositoryDataGeneration;
/**
* The minimum {@link IndexVersion} of the nodes in the cluster and the snapshots remaining in the repository. The repository must
* remain readable by all node versions which support this {@link IndexVersion}.
*/
private final IndexVersion repositoryFormatIndexVersion;
/**
* Whether the {@link #repositoryFormatIndexVersion} is new enough to support naming {@link BlobStoreIndexShardSnapshots} blobs with
* UUIDs (i.e. does not need to remain compatible with versions before v7.6.0). Older repositories use (unsafe) numeric indices for
* these blobs instead.
*/
private final boolean useShardGenerations;
/**
* All blobs in the repository root at the start of the operation, obtained by listing the repository contents. Note that this may
* include some blobs which are no longer referenced by the current {@link RepositoryData}, but which have not yet been removed by
* the cleanup that follows an earlier deletion. This cleanup may still be ongoing (we do not wait for it to complete before
* starting the next repository operation) or it may have failed before completion (it could have been running on a different node,
* which crashed for unrelated reasons) so we track all the blobs here and clean them up again at the end.
*/
private final Map<String, BlobMetadata> originalRootBlobs;
/**
* All index containers at the start of the operation, obtained by listing the repository contents. Note that this may include some
* containers which are no longer referenced by the current {@link RepositoryData}, but which have not yet been removed by
* the cleanup that follows an earlier deletion. This cleanup may or may not still be ongoing (it could have been running on a
* different node, which died before completing it) so we track all the blobs here and clean them up again at the end.
*/
private final Map<String, BlobContainer> originalIndexContainers;
/**
* The {@link RepositoryData} at the start of the operation, obtained after verifying that {@link #originalRootBlobs} contains no
* {@link RepositoryData} blob newer than the one identified by {@link #originalRepositoryDataGeneration}.
*/
private final RepositoryData originalRepositoryData;
/**
* Executor to use for all repository interactions.
*/
private final Executor snapshotExecutor = threadPool.executor(ThreadPool.Names.SNAPSHOT);
SnapshotsDeletion(
Collection<SnapshotId> snapshotIds,
long originalRepositoryDataGeneration,
IndexVersion repositoryFormatIndexVersion,
Map<String, BlobMetadata> originalRootBlobs,
Map<String, BlobContainer> originalIndexContainers,
RepositoryData originalRepositoryData
) {
this.snapshotIds = snapshotIds;
this.originalRepositoryDataGeneration = originalRepositoryDataGeneration;
this.repositoryFormatIndexVersion = repositoryFormatIndexVersion;
this.useShardGenerations = SnapshotsService.useShardGenerations(repositoryFormatIndexVersion);
this.originalRootBlobs = originalRootBlobs;
this.originalIndexContainers = originalIndexContainers;
this.originalRepositoryData = originalRepositoryData;
} }
/** /**
@ -885,35 +976,12 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
// --------------------------------------------------------------------------------------------------------------------------------- // ---------------------------------------------------------------------------------------------------------------------------------
// The overall flow of execution // The overall flow of execution
/** private void runDelete(SnapshotDeleteListener listener) {
* After updating the {@link RepositoryData} each of the shards directories is individually first moved to the next shard generation if (useShardGenerations) {
* and then has all now unreferenced blobs in it deleted.
*
* @param snapshotIds SnapshotIds to delete
* @param originalRepositoryDataGeneration {@link RepositoryData} generation at the start of the process.
* @param originalIndexContainers All index containers at the start of the operation, obtained by listing the repository
* contents.
* @param originalRootBlobs All blobs found at the root of the repository at the start of the operation, obtained by
* listing the repository contents.
* @param originalRepositoryData {@link RepositoryData} at the start of the operation.
* @param repositoryFormatIndexVersion The minimum {@link IndexVersion} of the nodes in the cluster and the snapshots remaining in
* the repository.
* @param listener Listener to invoke once finished
*/
private void doDeleteShardSnapshots(
Collection<SnapshotId> snapshotIds,
long originalRepositoryDataGeneration,
Map<String, BlobContainer> originalIndexContainers,
Map<String, BlobMetadata> originalRootBlobs,
RepositoryData originalRepositoryData,
IndexVersion repositoryFormatIndexVersion,
SnapshotDeleteListener listener
) {
if (SnapshotsService.useShardGenerations(repositoryFormatIndexVersion)) {
// First write the new shard state metadata (with the removed snapshot) and compute deletion targets // First write the new shard state metadata (with the removed snapshot) and compute deletion targets
final ListenableFuture<Collection<ShardSnapshotMetaDeleteResult>> writeShardMetaDataAndComputeDeletesStep = final ListenableFuture<Collection<ShardSnapshotMetaDeleteResult>> writeShardMetaDataAndComputeDeletesStep =
new ListenableFuture<>(); new ListenableFuture<>();
writeUpdatedShardMetaDataAndComputeDeletes(snapshotIds, originalRepositoryData, true, writeShardMetaDataAndComputeDeletesStep); writeUpdatedShardMetaDataAndComputeDeletes(writeShardMetaDataAndComputeDeletesStep);
// Once we have put the new shard-level metadata into place, we can update the repository metadata as follows: // Once we have put the new shard-level metadata into place, we can update the repository metadata as follows:
// 1. Remove the snapshots from the list of existing snapshots // 1. Remove the snapshots from the list of existing snapshots
// 2. Update the index shard generations of all updated shard folders // 2. Update the index shard generations of all updated shard folders
@ -941,19 +1009,8 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
listener.onRepositoryDataWritten(newRepositoryData); listener.onRepositoryDataWritten(newRepositoryData);
// Run unreferenced blobs cleanup in parallel to shard-level snapshot deletion // Run unreferenced blobs cleanup in parallel to shard-level snapshot deletion
try (var refs = new RefCountingRunnable(listener::onDone)) { try (var refs = new RefCountingRunnable(listener::onDone)) {
cleanupUnlinkedRootAndIndicesBlobs( cleanupUnlinkedRootAndIndicesBlobs(newRepositoryData, refs.acquireListener());
snapshotIds, cleanupUnlinkedShardLevelBlobs(writeShardMetaDataAndComputeDeletesStep.result(), refs.acquireListener());
originalIndexContainers,
originalRootBlobs,
newRepositoryData,
refs.acquireListener()
);
cleanupUnlinkedShardLevelBlobs(
originalRepositoryData,
snapshotIds,
writeShardMetaDataAndComputeDeletesStep.result(),
refs.acquireListener()
);
} }
}, listener::onFailure)); }, listener::onFailure));
} else { } else {
@ -969,31 +1026,14 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
listener.onDone(); listener.onDone();
})) { })) {
// Run unreferenced blobs cleanup in parallel to shard-level snapshot deletion // Run unreferenced blobs cleanup in parallel to shard-level snapshot deletion
cleanupUnlinkedRootAndIndicesBlobs( cleanupUnlinkedRootAndIndicesBlobs(newRepositoryData, refs.acquireListener());
snapshotIds,
originalIndexContainers,
originalRootBlobs,
newRepositoryData,
refs.acquireListener()
);
// writeIndexGen finishes on master-service thread so must fork here. // writeIndexGen finishes on master-service thread so must fork here.
threadPool.executor(ThreadPool.Names.SNAPSHOT) snapshotExecutor.execute(
.execute(
ActionRunnable.wrap( ActionRunnable.wrap(
refs.acquireListener(), refs.acquireListener(),
l0 -> writeUpdatedShardMetaDataAndComputeDeletes( l0 -> writeUpdatedShardMetaDataAndComputeDeletes(
snapshotIds, l0.delegateFailure((l, shardDeleteResults) -> cleanupUnlinkedShardLevelBlobs(shardDeleteResults, l))
originalRepositoryData,
false,
l0.delegateFailure(
(l, deleteResults) -> cleanupUnlinkedShardLevelBlobs(
originalRepositoryData,
snapshotIds,
deleteResults,
l
)
)
) )
) )
); );
@ -1008,13 +1048,9 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
// updates the shard state metadata for shards of a snapshot that is to be deleted. Also computes the files to be cleaned up. // updates the shard state metadata for shards of a snapshot that is to be deleted. Also computes the files to be cleaned up.
private void writeUpdatedShardMetaDataAndComputeDeletes( private void writeUpdatedShardMetaDataAndComputeDeletes(
Collection<SnapshotId> snapshotIds,
RepositoryData originalRepositoryData,
boolean useShardGenerations,
ActionListener<Collection<ShardSnapshotMetaDeleteResult>> onAllShardsCompleted ActionListener<Collection<ShardSnapshotMetaDeleteResult>> onAllShardsCompleted
) { ) {
final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT);
final List<IndexId> indices = originalRepositoryData.indicesToUpdateAfterRemovingSnapshot(snapshotIds); final List<IndexId> indices = originalRepositoryData.indicesToUpdateAfterRemovingSnapshot(snapshotIds);
if (indices.isEmpty()) { if (indices.isEmpty()) {
@ -1044,7 +1080,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
); );
final BlobContainer indexContainer = indexContainer(indexId); final BlobContainer indexContainer = indexContainer(indexId);
for (String indexMetaGeneration : indexMetaGenerations) { for (String indexMetaGeneration : indexMetaGenerations) {
executor.execute(ActionRunnable.supply(allShardCountsListener, () -> { snapshotExecutor.execute(ActionRunnable.supply(allShardCountsListener, () -> {
try { try {
return INDEX_METADATA_FORMAT.read(metadata.name(), indexContainer, indexMetaGeneration, namedXContentRegistry) return INDEX_METADATA_FORMAT.read(metadata.name(), indexContainer, indexMetaGeneration, namedXContentRegistry)
.getNumberOfShards(); .getNumberOfShards();
@ -1062,7 +1098,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
})); }));
} }
// ----------------------------------------------------------------------------------------------------------------------------- // -------------------------------------------------------------------------------------------------------------------------
// Determining the shard count // Determining the shard count
shardCountListener.addListener(deleteIndexMetadataListener.delegateFailureAndWrap((delegate, counts) -> { shardCountListener.addListener(deleteIndexMetadataListener.delegateFailureAndWrap((delegate, counts) -> {
@ -1072,10 +1108,13 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
return; return;
} }
// Listener for collecting the results of removing the snapshot from each shard's metadata in the current index // Listener for collecting the results of removing the snapshot from each shard's metadata in the current index
final ActionListener<ShardSnapshotMetaDeleteResult> allShardsListener = new GroupedActionListener<>(shardCount, delegate); final ActionListener<ShardSnapshotMetaDeleteResult> allShardsListener = new GroupedActionListener<>(
shardCount,
delegate
);
for (int i = 0; i < shardCount; i++) { for (int i = 0; i < shardCount; i++) {
final int shardId = i; final int shardId = i;
executor.execute(new AbstractRunnable() { snapshotExecutor.execute(new AbstractRunnable() {
@Override @Override
protected void doRun() throws Exception { protected void doRun() throws Exception {
final BlobContainer shardContainer = shardContainer(indexId, shardId); final BlobContainer shardContainer = shardContainer(indexId, shardId);
@ -1114,7 +1153,12 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
@Override @Override
public void onFailure(Exception ex) { public void onFailure(Exception ex) {
logger.warn( logger.warn(
() -> format("%s failed to delete shard data for shard [%s][%s]", snapshotIds, indexId.getName(), shardId), () -> format(
"%s failed to delete shard data for shard [%s][%s]",
snapshotIds,
indexId.getName(),
shardId
),
ex ex
); );
// Just passing null here to count down the listener instead of failing it, the stale data left behind // Just passing null here to count down the listener instead of failing it, the stale data left behind
@ -1133,8 +1177,8 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
/** /**
* Delete snapshot from shard level metadata. * Delete snapshot from shard level metadata.
* *
* @param indexGeneration generation to write the new shard level level metadata to. If negative a uuid id shard generation should be * @param indexGeneration generation to write the new shard level level metadata to. If negative a uuid id shard generation should
* used * be used
*/ */
private ShardSnapshotMetaDeleteResult deleteFromShardSnapshotMeta( private ShardSnapshotMetaDeleteResult deleteFromShardSnapshotMeta(
Set<SnapshotId> survivingSnapshots, Set<SnapshotId> survivingSnapshots,
@ -1160,7 +1204,9 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
writtenGeneration = new ShardGeneration(indexGeneration); writtenGeneration = new ShardGeneration(indexGeneration);
writeShardIndexBlobAtomic(shardContainer, indexGeneration, updatedSnapshots, Collections.emptyMap()); writeShardIndexBlobAtomic(shardContainer, indexGeneration, updatedSnapshots, Collections.emptyMap());
} }
final Set<String> survivingSnapshotUUIDs = survivingSnapshots.stream().map(SnapshotId::getUUID).collect(Collectors.toSet()); final Set<String> survivingSnapshotUUIDs = survivingSnapshots.stream()
.map(SnapshotId::getUUID)
.collect(Collectors.toSet());
return new ShardSnapshotMetaDeleteResult( return new ShardSnapshotMetaDeleteResult(
indexId, indexId,
shardId, shardId,
@ -1209,28 +1255,20 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
* Delete any dangling blobs in the repository root (i.e. {@link RepositoryData}, {@link SnapshotInfo} and {@link Metadata} blobs) * Delete any dangling blobs in the repository root (i.e. {@link RepositoryData}, {@link SnapshotInfo} and {@link Metadata} blobs)
* as well as any containers for indices that are now completely unreferenced. * as well as any containers for indices that are now completely unreferenced.
*/ */
private void cleanupUnlinkedRootAndIndicesBlobs( private void cleanupUnlinkedRootAndIndicesBlobs(RepositoryData newRepositoryData, ActionListener<Void> listener) {
Collection<SnapshotId> snapshotIds,
Map<String, BlobContainer> originalIndexContainers,
Map<String, BlobMetadata> originalRootBlobs,
RepositoryData newRepositoryData,
ActionListener<Void> listener
) {
cleanupStaleBlobs(snapshotIds, originalIndexContainers, originalRootBlobs, newRepositoryData, listener.map(ignored -> null)); cleanupStaleBlobs(snapshotIds, originalIndexContainers, originalRootBlobs, newRepositoryData, listener.map(ignored -> null));
} }
private void cleanupUnlinkedShardLevelBlobs( private void cleanupUnlinkedShardLevelBlobs(
RepositoryData originalRepositoryData,
Collection<SnapshotId> snapshotIds,
Collection<ShardSnapshotMetaDeleteResult> shardDeleteResults, Collection<ShardSnapshotMetaDeleteResult> shardDeleteResults,
ActionListener<Void> listener ActionListener<Void> listener
) { ) {
final Iterator<String> filesToDelete = resolveFilesToDelete(originalRepositoryData, snapshotIds, shardDeleteResults); final Iterator<String> filesToDelete = resolveFilesToDelete(shardDeleteResults);
if (filesToDelete.hasNext() == false) { if (filesToDelete.hasNext() == false) {
listener.onResponse(null); listener.onResponse(null);
return; return;
} }
threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(ActionRunnable.wrap(listener, l -> { snapshotExecutor.execute(ActionRunnable.wrap(listener, l -> {
try { try {
deleteFromContainer(blobContainer(), filesToDelete); deleteFromContainer(blobContainer(), filesToDelete);
l.onResponse(null); l.onResponse(null);
@ -1241,16 +1279,11 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
})); }));
} }
private Iterator<String> resolveFilesToDelete( private Iterator<String> resolveFilesToDelete(Collection<ShardSnapshotMetaDeleteResult> deleteResults) {
RepositoryData oldRepositoryData,
Collection<SnapshotId> snapshotIds,
Collection<ShardSnapshotMetaDeleteResult> deleteResults
) {
final String basePath = basePath().buildAsString(); final String basePath = basePath().buildAsString();
final int basePathLen = basePath.length(); final int basePathLen = basePath.length();
final Map<IndexId, Collection<String>> indexMetaGenerations = oldRepositoryData.indexMetaDataToRemoveAfterRemovingSnapshots( final Map<IndexId, Collection<String>> indexMetaGenerations = originalRepositoryData
snapshotIds .indexMetaDataToRemoveAfterRemovingSnapshots(snapshotIds);
);
return Stream.concat(deleteResults.stream().flatMap(shardResult -> { return Stream.concat(deleteResults.stream().flatMap(shardResult -> {
final String shardPath = shardPath(shardResult.indexId, shardResult.shardId).buildAsString(); final String shardPath = shardPath(shardResult.indexId, shardResult.shardId).buildAsString();
return shardResult.blobsToDelete.stream().map(blob -> shardPath + blob); return shardResult.blobsToDelete.stream().map(blob -> shardPath + blob);
@ -1262,6 +1295,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
return absolutePath.substring(basePathLen); return absolutePath.substring(basePathLen);
}).iterator(); }).iterator();
} }
}
/** /**
* Cleans up stale blobs directly under the repository root as well as all indices paths that aren't referenced by any existing * Cleans up stale blobs directly under the repository root as well as all indices paths that aren't referenced by any existing