Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.common.xcontent.LoggingDeprecationHandler;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentFactory;
Expand Down Expand Up @@ -100,6 +101,7 @@
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
Expand Down Expand Up @@ -433,9 +435,10 @@ public void deleteSnapshot(SnapshotId snapshotId, long repositoryStateId, Action
return;
}
final SnapshotInfo finalSnapshotInfo = snapshot;
final List<String> snapMetaFilesToDelete =
Arrays.asList(snapshotFormat.blobName(snapshotId.getUUID()), globalMetaDataFormat.blobName(snapshotId.getUUID()));
try {
blobContainer().deleteBlobsIgnoringIfNotExists(
Arrays.asList(snapshotFormat.blobName(snapshotId.getUUID()), globalMetaDataFormat.blobName(snapshotId.getUUID())));
blobContainer().deleteBlobsIgnoringIfNotExists(snapMetaFilesToDelete);
} catch (IOException e) {
logger.warn(() -> new ParameterizedMessage("[{}] Unable to delete global metadata files", snapshotId), e);
}
Expand All @@ -448,12 +451,56 @@ public void deleteSnapshot(SnapshotId snapshotId, long repositoryStateId, Action
snapshotId,
ActionListener.map(listener, v -> {
cleanupStaleIndices(foundIndices, survivingIndices);
cleanupStaleRootFiles(Sets.difference(rootBlobs, new HashSet<>(snapMetaFilesToDelete)), updatedRepositoryData);
return null;
})
);
}
}

private void cleanupStaleRootFiles(Set<String> rootBlobNames, RepositoryData repositoryData) {
final Set<String> allSnapshotIds =
repositoryData.getAllSnapshotIds().stream().map(SnapshotId::getUUID).collect(Collectors.toSet());
final List<String> blobsToDelete = rootBlobNames.stream().filter(
blob -> {
if (FsBlobContainer.isTempBlobName(blob)) {
return true;
}
if (blob.endsWith(".dat")) {
final String foundUUID;
if (blob.startsWith(SNAPSHOT_PREFIX)) {
foundUUID = blob.substring(SNAPSHOT_PREFIX.length(), blob.length() - ".dat".length());
assert snapshotFormat.blobName(foundUUID).equals(blob);
} else if (blob.startsWith(METADATA_PREFIX)) {
foundUUID = blob.substring(METADATA_PREFIX.length(), blob.length() - ".dat".length());
assert globalMetaDataFormat.blobName(foundUUID).equals(blob);
} else {
return false;
}
return allSnapshotIds.contains(foundUUID) == false;
}
return false;
}
).collect(Collectors.toList());
if (blobsToDelete.isEmpty()) {
return;
}
try {
logger.info("[{}] Found stale root level blobs {}. Cleaning them up", metadata.name(), blobsToDelete);
blobContainer().deleteBlobsIgnoringIfNotExists(blobsToDelete);
} catch (IOException e) {
logger.warn(() -> new ParameterizedMessage(
"[{}] The following blobs are no longer part of any snapshot [{}] but failed to remove them",
metadata.name(), blobsToDelete), e);
} catch (Exception e) {
// TODO: We shouldn't be blanket catching and suppressing all exceptions here and instead handle them safely upstream.
// Currently this catch exists as a stop gap solution to tackle unexpected runtime exceptions from implementations
// bubbling up and breaking the snapshot functionality.
assert false : e;
logger.warn(new ParameterizedMessage("[{}] Exception during cleanup of root level blobs", metadata.name()), e);
}
}

private void cleanupStaleIndices(Map<String, BlobContainer> foundIndices, Map<String, IndexId> survivingIndices) {
try {
final Set<String> survivingIndexIds = survivingIndices.values().stream()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -236,6 +236,10 @@ protected void doRun() throws Exception {
final BlobStore blobStore = repo.blobStore();
blobStore.blobContainer(BlobPath.cleanPath().add("indices").add("foo"))
.writeBlob("bar", new ByteArrayInputStream(new byte[0]), 0, false);
for (String prefix : Arrays.asList("snap-", "meta-")) {
blobStore.blobContainer(BlobPath.cleanPath())
.writeBlob(prefix + "foo.dat", new ByteArrayInputStream(new byte[0]), 0, false);
}
future.onResponse(null);
}
});
Expand All @@ -256,6 +260,8 @@ protected void doRun() throws Exception {
future.onResponse(
blobStore.blobContainer(BlobPath.cleanPath().add("indices")).children().containsKey("foo")
&& blobStore.blobContainer(BlobPath.cleanPath().add("indices").add("foo")).blobExists("bar")
&& blobStore.blobContainer(BlobPath.cleanPath()).blobExists("meta-foo.dat")
&& blobStore.blobContainer(BlobPath.cleanPath()).blobExists("snap-foo.dat")
);
}
});
Expand Down