|
@@ -83,11 +83,9 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
|
|
|
import org.elasticsearch.common.xcontent.XContentFactory;
|
|
|
import org.elasticsearch.common.xcontent.XContentParser;
|
|
|
import org.elasticsearch.common.xcontent.XContentType;
|
|
|
-import org.elasticsearch.index.Index;
|
|
|
import org.elasticsearch.index.mapper.MapperService;
|
|
|
import org.elasticsearch.index.shard.ShardId;
|
|
|
import org.elasticsearch.index.snapshots.IndexShardRestoreFailedException;
|
|
|
-import org.elasticsearch.index.snapshots.IndexShardSnapshotException;
|
|
|
import org.elasticsearch.index.snapshots.IndexShardSnapshotFailedException;
|
|
|
import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus;
|
|
|
import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot;
|
|
@@ -107,8 +105,6 @@ import org.elasticsearch.repositories.RepositoryOperation;
|
|
|
import org.elasticsearch.repositories.RepositoryStats;
|
|
|
import org.elasticsearch.repositories.RepositoryVerificationException;
|
|
|
import org.elasticsearch.repositories.ShardGenerations;
|
|
|
-import org.elasticsearch.snapshots.ConcurrentSnapshotExecutionException;
|
|
|
-import org.elasticsearch.snapshots.Snapshot;
|
|
|
import org.elasticsearch.snapshots.SnapshotException;
|
|
|
import org.elasticsearch.snapshots.SnapshotId;
|
|
|
import org.elasticsearch.snapshots.SnapshotInfo;
|
|
@@ -494,28 +490,21 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void deleteSnapshot(SnapshotId snapshotId, long repositoryStateId, Version repositoryMetaVersion,
|
|
|
- ActionListener<Void> listener) {
|
|
|
+ public void deleteSnapshots(Collection<SnapshotId> snapshotIds, long repositoryStateId, Version repositoryMetaVersion,
|
|
|
+ ActionListener<Void> listener) {
|
|
|
if (isReadOnly()) {
|
|
|
listener.onFailure(new RepositoryException(metadata.name(), "cannot delete snapshot from a readonly repository"));
|
|
|
} else {
|
|
|
- final long latestKnownGen = latestKnownRepoGen.get();
|
|
|
- if (latestKnownGen > repositoryStateId) {
|
|
|
- listener.onFailure(new ConcurrentSnapshotExecutionException(
|
|
|
- new Snapshot(metadata.name(), snapshotId), "Another concurrent operation moved repo generation to [ " + latestKnownGen
|
|
|
- + "] but this delete assumed generation [" + repositoryStateId + "]"));
|
|
|
- return;
|
|
|
- }
|
|
|
try {
|
|
|
final Map<String, BlobMetadata> rootBlobs = blobContainer().listBlobs();
|
|
|
final RepositoryData repositoryData = safeRepositoryData(repositoryStateId, 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();
|
|
|
- doDeleteShardSnapshots(snapshotId, repositoryStateId, foundIndices, rootBlobs, repositoryData,
|
|
|
+ doDeleteShardSnapshots(snapshotIds, repositoryStateId, foundIndices, rootBlobs, repositoryData,
|
|
|
SnapshotsService.useShardGenerations(repositoryMetaVersion), listener);
|
|
|
} catch (Exception ex) {
|
|
|
- listener.onFailure(new RepositoryException(metadata.name(), "failed to delete snapshot [" + snapshotId + "]", ex));
|
|
|
+ listener.onFailure(new RepositoryException(metadata.name(), "failed to delete snapshots " + snapshotIds, ex));
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -559,7 +548,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
|
|
* After updating the {@link RepositoryData} each of the shards directories is individually first moved to the next shard generation
|
|
|
* and then has all now unreferenced blobs in it deleted.
|
|
|
*
|
|
|
- * @param snapshotId SnapshotId to delete
|
|
|
+ * @param snapshotIds SnapshotIds to delete
|
|
|
* @param repositoryStateId Expected repository state id
|
|
|
* @param foundIndices All indices folders found in the repository before executing any writes to the repository during this
|
|
|
* delete operation
|
|
@@ -568,28 +557,28 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
|
|
* @param repositoryData RepositoryData found the in the repository before executing this delete
|
|
|
* @param listener Listener to invoke once finished
|
|
|
*/
|
|
|
- private void doDeleteShardSnapshots(SnapshotId snapshotId, long repositoryStateId, Map<String, BlobContainer> foundIndices,
|
|
|
+ private void doDeleteShardSnapshots(Collection<SnapshotId> snapshotIds, long repositoryStateId, Map<String, BlobContainer> foundIndices,
|
|
|
Map<String, BlobMetadata> rootBlobs, RepositoryData repositoryData, boolean writeShardGens,
|
|
|
ActionListener<Void> listener) {
|
|
|
|
|
|
if (writeShardGens) {
|
|
|
// First write the new shard state metadata (with the removed snapshot) and compute deletion targets
|
|
|
- final StepListener<Collection<ShardSnapshotMetaDeleteResult>> writeShardMetadataAndComputeDeletesStep = new StepListener<>();
|
|
|
- writeUpdatedShardMetadataAndComputeDeletes(snapshotId, repositoryData, true, writeShardMetadataAndComputeDeletesStep);
|
|
|
+ final StepListener<Collection<ShardSnapshotMetaDeleteResult>> writeShardMetaDataAndComputeDeletesStep = new StepListener<>();
|
|
|
+ writeUpdatedShardMetaDataAndComputeDeletes(snapshotIds, repositoryData, true, writeShardMetaDataAndComputeDeletesStep);
|
|
|
// Once we have put the new shard-level metadata into place, we can update the repository metadata as follows:
|
|
|
- // 1. Remove the snapshot 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
|
|
|
//
|
|
|
// Note: If we fail updating any of the individual shard paths, none of them are changed since the newly created
|
|
|
// index-${gen_uuid} will not be referenced by the existing RepositoryData and new RepositoryData is only
|
|
|
// written if all shard paths have been successfully updated.
|
|
|
final StepListener<RepositoryData> writeUpdatedRepoDataStep = new StepListener<>();
|
|
|
- writeShardMetadataAndComputeDeletesStep.whenComplete(deleteResults -> {
|
|
|
+ writeShardMetaDataAndComputeDeletesStep.whenComplete(deleteResults -> {
|
|
|
final ShardGenerations.Builder builder = ShardGenerations.builder();
|
|
|
for (ShardSnapshotMetaDeleteResult newGen : deleteResults) {
|
|
|
builder.put(newGen.indexId, newGen.shardId, newGen.newGeneration);
|
|
|
}
|
|
|
- final RepositoryData updatedRepoData = repositoryData.removeSnapshot(snapshotId, builder.build());
|
|
|
+ final RepositoryData updatedRepoData = repositoryData.removeSnapshots(snapshotIds, builder.build());
|
|
|
writeIndexGen(updatedRepoData, repositoryStateId, true, Function.identity(),
|
|
|
ActionListener.wrap(v -> writeUpdatedRepoDataStep.onResponse(updatedRepoData), listener::onFailure));
|
|
|
}, listener::onFailure);
|
|
@@ -599,20 +588,20 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
|
|
final ActionListener<Void> afterCleanupsListener =
|
|
|
new GroupedActionListener<>(ActionListener.wrap(() -> listener.onResponse(null)), 2);
|
|
|
asyncCleanupUnlinkedRootAndIndicesBlobs(foundIndices, rootBlobs, updatedRepoData, afterCleanupsListener);
|
|
|
- asyncCleanupUnlinkedShardLevelBlobs(snapshotId, writeShardMetadataAndComputeDeletesStep.result(), afterCleanupsListener);
|
|
|
+ asyncCleanupUnlinkedShardLevelBlobs(snapshotIds, writeShardMetaDataAndComputeDeletesStep.result(), afterCleanupsListener);
|
|
|
}, listener::onFailure);
|
|
|
} else {
|
|
|
// Write the new repository data first (with the removed snapshot), using no shard generations
|
|
|
- final RepositoryData updatedRepoData = repositoryData.removeSnapshot(snapshotId, ShardGenerations.EMPTY);
|
|
|
+ final RepositoryData updatedRepoData = repositoryData.removeSnapshots(snapshotIds, ShardGenerations.EMPTY);
|
|
|
writeIndexGen(updatedRepoData, repositoryStateId, false, Function.identity(), ActionListener.wrap(v -> {
|
|
|
// Run unreferenced blobs cleanup in parallel to shard-level snapshot deletion
|
|
|
final ActionListener<Void> afterCleanupsListener =
|
|
|
new GroupedActionListener<>(ActionListener.wrap(() -> listener.onResponse(null)), 2);
|
|
|
asyncCleanupUnlinkedRootAndIndicesBlobs(foundIndices, rootBlobs, updatedRepoData, afterCleanupsListener);
|
|
|
final StepListener<Collection<ShardSnapshotMetaDeleteResult>> writeMetaAndComputeDeletesStep = new StepListener<>();
|
|
|
- writeUpdatedShardMetadataAndComputeDeletes(snapshotId, repositoryData, false, writeMetaAndComputeDeletesStep);
|
|
|
+ writeUpdatedShardMetaDataAndComputeDeletes(snapshotIds, repositoryData, false, writeMetaAndComputeDeletesStep);
|
|
|
writeMetaAndComputeDeletesStep.whenComplete(deleteResults ->
|
|
|
- asyncCleanupUnlinkedShardLevelBlobs(snapshotId, deleteResults, afterCleanupsListener),
|
|
|
+ asyncCleanupUnlinkedShardLevelBlobs(snapshotIds, deleteResults, afterCleanupsListener),
|
|
|
afterCleanupsListener::onFailure);
|
|
|
}, listener::onFailure));
|
|
|
}
|
|
@@ -625,17 +614,18 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
|
|
l -> cleanupStaleBlobs(foundIndices, rootBlobs, updatedRepoData, ActionListener.map(l, ignored -> null))));
|
|
|
}
|
|
|
|
|
|
- private void asyncCleanupUnlinkedShardLevelBlobs(SnapshotId snapshotId, Collection<ShardSnapshotMetaDeleteResult> deleteResults,
|
|
|
+ private void asyncCleanupUnlinkedShardLevelBlobs(Collection<SnapshotId> snapshotIds,
|
|
|
+ Collection<ShardSnapshotMetaDeleteResult> deleteResults,
|
|
|
ActionListener<Void> listener) {
|
|
|
threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(ActionRunnable.wrap(
|
|
|
listener,
|
|
|
l -> {
|
|
|
try {
|
|
|
- blobContainer().deleteBlobsIgnoringIfNotExists(resolveFilesToDelete(snapshotId, deleteResults));
|
|
|
+ blobContainer().deleteBlobsIgnoringIfNotExists(resolveFilesToDelete(snapshotIds, deleteResults));
|
|
|
l.onResponse(null);
|
|
|
} catch (Exception e) {
|
|
|
logger.warn(
|
|
|
- () -> new ParameterizedMessage("[{}] Failed to delete some blobs during snapshot delete", snapshotId),
|
|
|
+ () -> new ParameterizedMessage("{} Failed to delete some blobs during snapshot delete", snapshotIds),
|
|
|
e);
|
|
|
throw e;
|
|
|
}
|
|
@@ -643,11 +633,11 @@ 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.
|
|
|
- private void writeUpdatedShardMetadataAndComputeDeletes(SnapshotId snapshotId, RepositoryData oldRepositoryData,
|
|
|
+ private void writeUpdatedShardMetaDataAndComputeDeletes(Collection<SnapshotId> snapshotIds, RepositoryData oldRepositoryData,
|
|
|
boolean useUUIDs, ActionListener<Collection<ShardSnapshotMetaDeleteResult>> onAllShardsCompleted) {
|
|
|
|
|
|
final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT);
|
|
|
- final List<IndexId> indices = oldRepositoryData.indicesToUpdateAfterRemovingSnapshot(snapshotId);
|
|
|
+ final List<IndexId> indices = oldRepositoryData.indicesToUpdateAfterRemovingSnapshot(snapshotIds);
|
|
|
|
|
|
if (indices.isEmpty()) {
|
|
|
onAllShardsCompleted.onResponse(Collections.emptyList());
|
|
@@ -661,66 +651,72 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
|
|
|
|
|
for (IndexId indexId : indices) {
|
|
|
final Set<SnapshotId> survivingSnapshots = oldRepositoryData.getSnapshots(indexId).stream()
|
|
|
- .filter(id -> id.equals(snapshotId) == false).collect(Collectors.toSet());
|
|
|
- executor.execute(ActionRunnable.wrap(deleteIndexMetadataListener, deleteIdxMetaListener -> {
|
|
|
- final IndexMetadata indexMetadata;
|
|
|
- try {
|
|
|
- indexMetadata = getSnapshotIndexMetadata(snapshotId, indexId);
|
|
|
- } catch (Exception ex) {
|
|
|
- logger.warn(() ->
|
|
|
- new ParameterizedMessage("[{}] [{}] failed to read metadata for index", snapshotId, indexId.getName()), ex);
|
|
|
- // Just invoke the listener without any shard generations to count it down, this index will be cleaned up
|
|
|
- // by the stale data cleanup in the end.
|
|
|
- // TODO: Getting here means repository corruption. We should find a way of dealing with this instead of just ignoring
|
|
|
- // it and letting the cleanup deal with it.
|
|
|
- deleteIdxMetaListener.onResponse(null);
|
|
|
+ .filter(id -> snapshotIds.contains(id) == false).collect(Collectors.toSet());
|
|
|
+ final StepListener<Collection<Integer>> shardCountListener = new StepListener<>();
|
|
|
+ final ActionListener<Integer> allShardCountsListener = new GroupedActionListener<>(shardCountListener, snapshotIds.size());
|
|
|
+ for (SnapshotId snapshotId : snapshotIds) {
|
|
|
+ executor.execute(ActionRunnable.supply(allShardCountsListener, () -> {
|
|
|
+ try {
|
|
|
+ return getSnapshotIndexMetadata(snapshotId, indexId).getNumberOfShards();
|
|
|
+ } catch (Exception ex) {
|
|
|
+ logger.warn(() -> new ParameterizedMessage(
|
|
|
+ "[{}] [{}] failed to read metadata for index", snapshotId, indexId.getName()), ex);
|
|
|
+ // Just invoke the listener without any shard generations to count it down, this index will be cleaned up
|
|
|
+ // by the stale data cleanup in the end.
|
|
|
+ // TODO: Getting here means repository corruption. We should find a way of dealing with this instead of just
|
|
|
+ // ignoring it and letting the cleanup deal with it.
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ }));
|
|
|
+ }
|
|
|
+ shardCountListener.whenComplete(counts -> {
|
|
|
+ final int shardCount = counts.stream().mapToInt(i -> i).max().orElse(0);
|
|
|
+ if (shardCount == 0) {
|
|
|
+ deleteIndexMetadataListener.onResponse(null);
|
|
|
return;
|
|
|
}
|
|
|
- final int shardCount = indexMetadata.getNumberOfShards();
|
|
|
- assert shardCount > 0 : "index did not have positive shard count, get [" + shardCount + "]";
|
|
|
// Listener for collecting the results of removing the snapshot from each shard's metadata in the current index
|
|
|
final ActionListener<ShardSnapshotMetaDeleteResult> allShardsListener =
|
|
|
- new GroupedActionListener<>(deleteIdxMetaListener, shardCount);
|
|
|
- final Index index = indexMetadata.getIndex();
|
|
|
- for (int shardId = 0; shardId < indexMetadata.getNumberOfShards(); shardId++) {
|
|
|
- final ShardId shard = new ShardId(index, shardId);
|
|
|
+ new GroupedActionListener<>(deleteIndexMetadataListener, shardCount);
|
|
|
+ for (int shardId = 0; shardId < shardCount; shardId++) {
|
|
|
+ final int finalShardId = shardId;
|
|
|
executor.execute(new AbstractRunnable() {
|
|
|
@Override
|
|
|
protected void doRun() throws Exception {
|
|
|
- final BlobContainer shardContainer = shardContainer(indexId, shard);
|
|
|
- final Set<String> blobs = getShardBlobs(shard, shardContainer);
|
|
|
+ final BlobContainer shardContainer = shardContainer(indexId, finalShardId);
|
|
|
+ final Set<String> blobs = shardContainer.listBlobs().keySet();
|
|
|
final BlobStoreIndexShardSnapshots blobStoreIndexShardSnapshots;
|
|
|
final String newGen;
|
|
|
if (useUUIDs) {
|
|
|
newGen = UUIDs.randomBase64UUID();
|
|
|
blobStoreIndexShardSnapshots = buildBlobStoreIndexShardSnapshots(blobs, shardContainer,
|
|
|
- oldRepositoryData.shardGenerations().getShardGen(indexId, shard.getId())).v1();
|
|
|
+ oldRepositoryData.shardGenerations().getShardGen(indexId, finalShardId)).v1();
|
|
|
} else {
|
|
|
- Tuple<BlobStoreIndexShardSnapshots, Long> tuple =
|
|
|
- buildBlobStoreIndexShardSnapshots(blobs, shardContainer);
|
|
|
+ Tuple<BlobStoreIndexShardSnapshots, Long> tuple = buildBlobStoreIndexShardSnapshots(blobs, shardContainer);
|
|
|
newGen = Long.toString(tuple.v2() + 1);
|
|
|
blobStoreIndexShardSnapshots = tuple.v1();
|
|
|
}
|
|
|
- allShardsListener.onResponse(deleteFromShardSnapshotMeta(survivingSnapshots, indexId, shard, snapshotId,
|
|
|
- shardContainer, blobs, blobStoreIndexShardSnapshots, newGen));
|
|
|
+ allShardsListener.onResponse(deleteFromShardSnapshotMeta(survivingSnapshots, indexId, finalShardId,
|
|
|
+ snapshotIds, shardContainer, blobs, blobStoreIndexShardSnapshots, newGen));
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public void onFailure(Exception ex) {
|
|
|
logger.warn(
|
|
|
- () -> new ParameterizedMessage("[{}] failed to delete shard data for shard [{}][{}]",
|
|
|
- snapshotId, indexId.getName(), shard.id()), ex);
|
|
|
+ () -> new ParameterizedMessage("{} failed to delete shard data for shard [{}][{}]",
|
|
|
+ snapshotIds, indexId.getName(), finalShardId), ex);
|
|
|
// Just passing null here to count down the listener instead of failing it, the stale data left behind
|
|
|
// here will be retried in the next delete or repository cleanup
|
|
|
allShardsListener.onResponse(null);
|
|
|
}
|
|
|
});
|
|
|
}
|
|
|
- }));
|
|
|
+ }, deleteIndexMetadataListener::onFailure);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private List<String> resolveFilesToDelete(SnapshotId snapshotId, Collection<ShardSnapshotMetaDeleteResult> deleteResults) {
|
|
|
+ private List<String> resolveFilesToDelete(Collection<SnapshotId> snapshotIds,
|
|
|
+ Collection<ShardSnapshotMetaDeleteResult> deleteResults) {
|
|
|
final String basePath = basePath().buildAsString();
|
|
|
final int basePathLen = basePath.length();
|
|
|
return Stream.concat(
|
|
@@ -729,8 +725,10 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
|
|
shardContainer(shardResult.indexId, shardResult.shardId).path().buildAsString();
|
|
|
return shardResult.blobsToDelete.stream().map(blob -> shardPath + blob);
|
|
|
}),
|
|
|
- deleteResults.stream().map(shardResult -> shardResult.indexId).distinct().map(indexId ->
|
|
|
- indexContainer(indexId).path().buildAsString() + globalMetadataFormat.blobName(snapshotId.getUUID()))
|
|
|
+ deleteResults.stream().map(shardResult -> shardResult.indexId).distinct().flatMap(indexId -> {
|
|
|
+ final String indexContainerPath = indexContainer(indexId).path().buildAsString();
|
|
|
+ return snapshotIds.stream().map(snapshotId -> indexContainerPath + globalMetadataFormat.blobName(snapshotId.getUUID()));
|
|
|
+ })
|
|
|
).map(absolutePath -> {
|
|
|
assert absolutePath.startsWith(basePath);
|
|
|
return absolutePath.substring(basePathLen);
|
|
@@ -1922,9 +1920,10 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
|
|
* Delete snapshot from shard level metadata.
|
|
|
*/
|
|
|
private ShardSnapshotMetaDeleteResult deleteFromShardSnapshotMeta(Set<SnapshotId> survivingSnapshots, IndexId indexId,
|
|
|
- ShardId snapshotShardId, SnapshotId snapshotId,
|
|
|
+ int snapshotShardId, Collection<SnapshotId> snapshotIds,
|
|
|
BlobContainer shardContainer, Set<String> blobs,
|
|
|
- BlobStoreIndexShardSnapshots snapshots, String indexGeneration) {
|
|
|
+ BlobStoreIndexShardSnapshots snapshots,
|
|
|
+ String indexGeneration) {
|
|
|
// Build a list of snapshots that should be preserved
|
|
|
List<SnapshotFiles> newSnapshotsList = new ArrayList<>();
|
|
|
final Set<String> survivingSnapshotNames = survivingSnapshots.stream().map(SnapshotId::getName).collect(Collectors.toSet());
|
|
@@ -1935,18 +1934,17 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
|
|
}
|
|
|
try {
|
|
|
if (newSnapshotsList.isEmpty()) {
|
|
|
- return new ShardSnapshotMetaDeleteResult(indexId, snapshotShardId.id(), ShardGenerations.DELETED_SHARD_GEN, blobs);
|
|
|
+ return new ShardSnapshotMetaDeleteResult(indexId, snapshotShardId, ShardGenerations.DELETED_SHARD_GEN, blobs);
|
|
|
} else {
|
|
|
final BlobStoreIndexShardSnapshots updatedSnapshots = new BlobStoreIndexShardSnapshots(newSnapshotsList);
|
|
|
writeShardIndexBlob(shardContainer, indexGeneration, updatedSnapshots);
|
|
|
final Set<String> survivingSnapshotUUIDs = survivingSnapshots.stream().map(SnapshotId::getUUID).collect(Collectors.toSet());
|
|
|
- return new ShardSnapshotMetaDeleteResult(indexId, snapshotShardId.id(), indexGeneration,
|
|
|
+ return new ShardSnapshotMetaDeleteResult(indexId, snapshotShardId, indexGeneration,
|
|
|
unusedBlobs(blobs, survivingSnapshotUUIDs, updatedSnapshots));
|
|
|
}
|
|
|
} catch (IOException e) {
|
|
|
- throw new IndexShardSnapshotFailedException(snapshotShardId,
|
|
|
- "Failed to finalize snapshot deletion [" + snapshotId + "] with shard index ["
|
|
|
- + indexShardSnapshotsFormat.blobName(indexGeneration) + "]", e);
|
|
|
+ throw new RepositoryException(metadata.name(), "Failed to finalize snapshot deletion " + snapshotIds +
|
|
|
+ " with shard index [" + indexShardSnapshotsFormat.blobName(indexGeneration) + "]", e);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -1957,16 +1955,6 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
|
|
indexShardSnapshotsFormat.writeAtomic(updatedSnapshots, shardContainer, indexGeneration);
|
|
|
}
|
|
|
|
|
|
- private static Set<String> getShardBlobs(final ShardId snapshotShardId, final BlobContainer shardContainer) {
|
|
|
- final Set<String> blobs;
|
|
|
- try {
|
|
|
- blobs = shardContainer.listBlobs().keySet();
|
|
|
- } catch (IOException e) {
|
|
|
- throw new IndexShardSnapshotException(snapshotShardId, "Failed to list content of shard directory", e);
|
|
|
- }
|
|
|
- return blobs;
|
|
|
- }
|
|
|
-
|
|
|
// Unused blobs are all previous index-, data- and meta-blobs and that are not referenced by the new index- as well as all
|
|
|
// temporary blobs
|
|
|
private static List<String> unusedBlobs(Set<String> blobs, Set<String> survivingSnapshotUUIDs,
|