diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotITV2.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotITV2.java new file mode 100644 index 0000000000000..02b6ea47172c7 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotITV2.java @@ -0,0 +1,332 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.snapshots; + +import org.opensearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; +import org.opensearch.action.support.master.AcknowledgedResponse; +import org.opensearch.client.Client; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.unit.ByteSizeUnit; +import org.opensearch.index.remote.RemoteStoreEnums; +import org.opensearch.indices.RemoteStoreSettings; +import org.opensearch.node.remotestore.RemoteStorePinnedTimestampService; +import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.repositories.fs.FsRepository; +import org.opensearch.test.OpenSearchIntegTestCase; + +import java.nio.file.Path; +import java.util.concurrent.TimeUnit; + +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.lessThan; + +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) +public class DeleteSnapshotITV2 extends AbstractSnapshotIntegTestCase { + + private static final String REMOTE_REPO_NAME = "remote-store-repo-name"; + + public void testDeleteShallowCopyV2() throws Exception { + disableRepoConsistencyCheck("Remote store repository is being used in the test"); + + final Path remoteStoreRepoPath = randomRepoPath(); + internalCluster().startClusterManagerOnlyNode(snapshotV2Settings(remoteStoreRepoPath)); + + internalCluster().startDataOnlyNode(snapshotV2Settings(remoteStoreRepoPath)); + internalCluster().startDataOnlyNode(snapshotV2Settings(remoteStoreRepoPath)); + + String indexName1 = "testindex1"; + String indexName2 = "testindex2"; + String indexName3 = "testindex3"; + String snapshotRepoName = "test-create-snapshot-repo"; + String snapshotName1 = "test-create-snapshot1"; + String snapshotName2 = "test-create-snapshot2"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + logger.info("Snapshot Path [{}]", absolutePath1); + + Client client = client(); + + assertAcked( + client.admin() + .cluster() + .preparePutRepository(snapshotRepoName) + .setType(FsRepository.TYPE) + .setSettings( + Settings.builder() + .put(FsRepository.LOCATION_SETTING.getKey(), absolutePath1) + .put(FsRepository.COMPRESS_SETTING.getKey(), randomBoolean()) + .put(FsRepository.CHUNK_SIZE_SETTING.getKey(), randomIntBetween(100, 1000), ByteSizeUnit.BYTES) + .put(BlobStoreRepository.REMOTE_STORE_INDEX_SHALLOW_COPY.getKey(), true) + .put(BlobStoreRepository.SHALLOW_SNAPSHOT_V2.getKey(), true) + ) + ); + + createIndex(indexName1, getRemoteStoreBackedIndexSettings()); + createIndex(indexName2, getRemoteStoreBackedIndexSettings()); + + final int numDocsInIndex1 = 10; + final int numDocsInIndex2 = 20; + indexRandomDocs(indexName1, numDocsInIndex1); + indexRandomDocs(indexName2, numDocsInIndex2); + ensureGreen(indexName1, indexName2); + + CreateSnapshotResponse createSnapshotResponse = client().admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(true) + .get(); + SnapshotInfo snapshotInfo = createSnapshotResponse.getSnapshotInfo(); + assertThat(snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); + assertThat(snapshotInfo.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards())); + assertThat(snapshotInfo.snapshotId().getName(), equalTo(snapshotName1)); + + createIndex(indexName3, getRemoteStoreBackedIndexSettings()); + indexRandomDocs(indexName3, 10); + CreateSnapshotResponse createSnapshotResponse2 = client().admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, snapshotName2) + .setWaitForCompletion(true) + .get(); + snapshotInfo = createSnapshotResponse2.getSnapshotInfo(); + assertThat(snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); + assertThat(snapshotInfo.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards())); + assertThat(snapshotInfo.snapshotId().getName(), equalTo(snapshotName2)); + + assertAcked(client().admin().indices().prepareDelete(indexName1)); + Thread.sleep(100); + + AcknowledgedResponse deleteResponse = client().admin() + .cluster() + .prepareDeleteSnapshot(snapshotRepoName, snapshotName2) + .setSnapshots(snapshotName2) + .get(); + assertTrue(deleteResponse.isAcknowledged()); + + // test delete non-existent snapshot + assertThrows( + SnapshotMissingException.class, + () -> client().admin().cluster().prepareDeleteSnapshot(snapshotRepoName, "random-snapshot").setSnapshots(snapshotName2).get() + ); + + } + + public void testDeleteShallowCopyV2MultipleSnapshots() throws Exception { + disableRepoConsistencyCheck("Remote store repository is being used in the test"); + final Path remoteStoreRepoPath = randomRepoPath(); + + internalCluster().startClusterManagerOnlyNode(snapshotV2Settings(remoteStoreRepoPath)); + internalCluster().startDataOnlyNode(snapshotV2Settings(remoteStoreRepoPath)); + internalCluster().startDataOnlyNode(snapshotV2Settings(remoteStoreRepoPath)); + + String indexName1 = "testindex1"; + String indexName2 = "testindex2"; + String indexName3 = "testindex3"; + String snapshotRepoName = "test-create-snapshot-repo"; + String snapshotName1 = "test-create-snapshot1"; + String snapshotName2 = "test-create-snapshot2"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + logger.info("Snapshot Path [{}]", absolutePath1); + + Client client = client(); + + assertAcked( + client.admin() + .cluster() + .preparePutRepository(snapshotRepoName) + .setType(FsRepository.TYPE) + .setSettings( + Settings.builder() + .put(FsRepository.LOCATION_SETTING.getKey(), absolutePath1) + .put(FsRepository.COMPRESS_SETTING.getKey(), randomBoolean()) + .put(FsRepository.CHUNK_SIZE_SETTING.getKey(), randomIntBetween(100, 1000), ByteSizeUnit.BYTES) + .put(BlobStoreRepository.REMOTE_STORE_INDEX_SHALLOW_COPY.getKey(), true) + .put(BlobStoreRepository.SHALLOW_SNAPSHOT_V2.getKey(), true) + ) + ); + + createIndex(indexName1, getRemoteStoreBackedIndexSettings()); + + createIndex(indexName2, getRemoteStoreBackedIndexSettings()); + + final int numDocsInIndex1 = 10; + final int numDocsInIndex2 = 20; + indexRandomDocs(indexName1, numDocsInIndex1); + indexRandomDocs(indexName2, numDocsInIndex2); + ensureGreen(indexName1, indexName2); + + CreateSnapshotResponse createSnapshotResponse = client().admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(true) + .get(); + SnapshotInfo snapshotInfo = createSnapshotResponse.getSnapshotInfo(); + assertThat(snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); + assertThat(snapshotInfo.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards())); + assertThat(snapshotInfo.snapshotId().getName(), equalTo(snapshotName1)); + + createIndex(indexName3, getRemoteStoreBackedIndexSettings()); + indexRandomDocs(indexName3, 10); + + CreateSnapshotResponse createSnapshotResponse2 = client().admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, snapshotName2) + .setWaitForCompletion(true) + .get(); + snapshotInfo = createSnapshotResponse2.getSnapshotInfo(); + assertThat(snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); + assertThat(snapshotInfo.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards())); + assertThat(snapshotInfo.snapshotId().getName(), equalTo(snapshotName2)); + + AcknowledgedResponse deleteResponse = client().admin() + .cluster() + .prepareDeleteSnapshot(snapshotRepoName, snapshotName1, snapshotName2) + .setSnapshots(snapshotName2) + .get(); + assertTrue(deleteResponse.isAcknowledged()); + + // test delete non-existent snapshot + assertThrows( + SnapshotMissingException.class, + () -> client().admin().cluster().prepareDeleteSnapshot(snapshotRepoName, "random-snapshot").setSnapshots(snapshotName2).get() + ); + + } + + public void testRemoteStoreCleanupForDeletedIndexForSnapshotV2() throws Exception { + disableRepoConsistencyCheck("Remote store repository is being used in the test"); + final Path remoteStoreRepoPath = randomRepoPath(); + Settings settings = remoteStoreClusterSettings(REMOTE_REPO_NAME, remoteStoreRepoPath); + settings = Settings.builder() + .put(settings) + .put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_ENABLED.getKey(), true) + .put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), RemoteStoreEnums.PathType.FIXED.toString()) + .build(); + String clusterManagerName = internalCluster().startClusterManagerOnlyNode(settings); + internalCluster().startDataOnlyNode(settings); + final Client clusterManagerClient = internalCluster().clusterManagerClient(); + ensureStableCluster(2); + + RemoteStorePinnedTimestampService remoteStorePinnedTimestampService = internalCluster().getInstance( + RemoteStorePinnedTimestampService.class, + clusterManagerName + ); + remoteStorePinnedTimestampService.rescheduleAsyncUpdatePinnedTimestampTask(TimeValue.timeValueSeconds(1)); + + final String snapshotRepoName = "snapshot-repo-name"; + final Path snapshotRepoPath = randomRepoPath(); + createRepository(snapshotRepoName, "mock", snapshotRepoSettingsForShallowV2(snapshotRepoPath)); + + final String remoteStoreEnabledIndexName = "remote-index-1"; + final Settings remoteStoreEnabledIndexSettings = getRemoteStoreBackedIndexSettings(); + createIndex(remoteStoreEnabledIndexName, remoteStoreEnabledIndexSettings); + indexRandomDocs(remoteStoreEnabledIndexName, 5); + + String indexUUID = client().admin() + .indices() + .prepareGetSettings(remoteStoreEnabledIndexName) + .get() + .getSetting(remoteStoreEnabledIndexName, IndexMetadata.SETTING_INDEX_UUID); + + String numShards = client().admin() + .indices() + .prepareGetSettings(remoteStoreEnabledIndexName) + .get() + .getSetting(remoteStoreEnabledIndexName, IndexMetadata.SETTING_NUMBER_OF_SHARDS); + + logger.info("--> create two remote index shallow snapshots"); + CreateSnapshotResponse createSnapshotResponse = client().admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, "snap1") + .setWaitForCompletion(true) + .get(); + SnapshotInfo snapshotInfo1 = createSnapshotResponse.getSnapshotInfo(); + + indexRandomDocs(remoteStoreEnabledIndexName, 25); + + CreateSnapshotResponse createSnapshotResponse2 = client().admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, "snap2") + .setWaitForCompletion(true) + .get(); + SnapshotInfo snapshotInfo2 = createSnapshotResponse2.getSnapshotInfo(); + assertThat(snapshotInfo2.state(), equalTo(SnapshotState.SUCCESS)); + assertThat(snapshotInfo2.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo2.successfulShards(), equalTo(snapshotInfo2.totalShards())); + assertThat(snapshotInfo2.snapshotId().getName(), equalTo("snap2")); + + // delete remote store index + assertAcked(client().admin().indices().prepareDelete(remoteStoreEnabledIndexName)); + + logger.info("--> delete snapshot 2"); + + Path indexPath = Path.of(String.valueOf(remoteStoreRepoPath), indexUUID); + Path shardPath = Path.of(String.valueOf(indexPath), "0"); + Path segmentsPath = Path.of(String.valueOf(shardPath), "segments"); + + // Get total segments remote store directory file count for deleted index and shard 0 + int segmentFilesCountBeforeDeletingSnapshot1 = RemoteStoreBaseIntegTestCase.getFileCount(segmentsPath); + + RemoteStoreSettings.setPinnedTimestampsLookbackInterval(TimeValue.ZERO); + + AcknowledgedResponse deleteSnapshotResponse = clusterManagerClient.admin() + .cluster() + .prepareDeleteSnapshot(snapshotRepoName, snapshotInfo2.snapshotId().getName()) + .get(); + assertAcked(deleteSnapshotResponse); + + Thread.sleep(5000); + + assertBusy(() -> { + try { + assertThat(RemoteStoreBaseIntegTestCase.getFileCount(segmentsPath), lessThan(segmentFilesCountBeforeDeletingSnapshot1)); + } catch (Exception e) {} + }, 30, TimeUnit.SECONDS); + int segmentFilesCountAfterDeletingSnapshot1 = RemoteStoreBaseIntegTestCase.getFileCount(segmentsPath); + + logger.info("--> delete snapshot 1"); + RemoteStoreSettings.setPinnedTimestampsLookbackInterval(TimeValue.ZERO); + // on snapshot deletion, remote store segment files should get cleaned up for deleted index - `remote-index-1` + deleteSnapshotResponse = clusterManagerClient.admin() + .cluster() + .prepareDeleteSnapshot(snapshotRepoName, snapshotInfo1.snapshotId().getName()) + .get(); + assertAcked(deleteSnapshotResponse); + + // Delete is async. Give time for it + assertBusy(() -> { + try { + assertThat(RemoteStoreBaseIntegTestCase.getFileCount(segmentsPath), lessThan(segmentFilesCountAfterDeletingSnapshot1)); + } catch (Exception e) {} + }, 60, TimeUnit.SECONDS); + } + + private Settings snapshotV2Settings(Path remoteStoreRepoPath) { + Settings settings = Settings.builder() + .put(remoteStoreClusterSettings(REMOTE_REPO_NAME, remoteStoreRepoPath)) + .put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_ENABLED.getKey(), true) + .build(); + return settings; + } + + protected Settings.Builder snapshotRepoSettingsForShallowV2(Path path) { + final Settings.Builder settings = Settings.builder(); + settings.put("location", path); + settings.put(BlobStoreRepository.REMOTE_STORE_INDEX_SHALLOW_COPY.getKey(), Boolean.TRUE); + settings.put(BlobStoreRepository.SHALLOW_SNAPSHOT_V2.getKey(), true); + return settings; + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/repositories/cleanup/TransportCleanupRepositoryAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/repositories/cleanup/TransportCleanupRepositoryAction.java index 65a26037e1cd5..83a63070161e9 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/repositories/cleanup/TransportCleanupRepositoryAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/repositories/cleanup/TransportCleanupRepositoryAction.java @@ -54,6 +54,7 @@ import org.opensearch.common.inject.Inject; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; import org.opensearch.index.store.lockmanager.RemoteStoreLockManagerFactory; import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.repositories.RepositoriesService; @@ -101,6 +102,8 @@ public final class TransportCleanupRepositoryAction extends TransportClusterMana private final RemoteStoreLockManagerFactory remoteStoreLockManagerFactory; + private final RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory; + @Override protected String executor() { return ThreadPool.Names.SAME; @@ -128,6 +131,11 @@ public TransportCleanupRepositoryAction( ); this.repositoriesService = repositoriesService; this.snapshotsService = snapshotsService; + this.remoteSegmentStoreDirectoryFactory = new RemoteSegmentStoreDirectoryFactory( + () -> repositoriesService, + threadPool, + remoteStoreSettings.getSegmentsPathFixedPrefix() + ); this.remoteStoreLockManagerFactory = new RemoteStoreLockManagerFactory( () -> repositoriesService, remoteStoreSettings.getSegmentsPathFixedPrefix() @@ -291,6 +299,7 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS repositoryStateId, snapshotsService.minCompatibleVersion(newState.nodes().getMinNodeVersion(), repositoryData, null), remoteStoreLockManagerFactory, + remoteSegmentStoreDirectoryFactory, ActionListener.wrap(result -> after(null, result), e -> after(e, null)) ) ) diff --git a/server/src/main/java/org/opensearch/node/remotestore/RemoteStorePinnedTimestampService.java b/server/src/main/java/org/opensearch/node/remotestore/RemoteStorePinnedTimestampService.java index 782ba5e9a6540..3a7734fc0538f 100644 --- a/server/src/main/java/org/opensearch/node/remotestore/RemoteStorePinnedTimestampService.java +++ b/server/src/main/java/org/opensearch/node/remotestore/RemoteStorePinnedTimestampService.java @@ -12,6 +12,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.collect.Tuple; @@ -42,6 +43,7 @@ * * @opensearch.internal */ +@ExperimentalApi public class RemoteStorePinnedTimestampService implements Closeable { private static final Logger logger = LogManager.getLogger(RemoteStorePinnedTimestampService.class); private static Tuple> pinnedTimestampsSet = new Tuple<>(-1L, Set.of()); diff --git a/server/src/main/java/org/opensearch/repositories/Repository.java b/server/src/main/java/org/opensearch/repositories/Repository.java index 29d4638f4bfbc..a7c44575465b0 100644 --- a/server/src/main/java/org/opensearch/repositories/Repository.java +++ b/server/src/main/java/org/opensearch/repositories/Repository.java @@ -50,9 +50,11 @@ import org.opensearch.index.mapper.MapperService; import org.opensearch.index.snapshots.IndexShardSnapshotStatus; import org.opensearch.index.snapshots.blobstore.RemoteStoreShardShallowCopySnapshot; +import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; import org.opensearch.index.store.Store; import org.opensearch.index.store.lockmanager.RemoteStoreLockManagerFactory; import org.opensearch.indices.recovery.RecoveryState; +import org.opensearch.node.remotestore.RemoteStorePinnedTimestampService; import org.opensearch.snapshots.SnapshotId; import org.opensearch.snapshots.SnapshotInfo; @@ -220,11 +222,59 @@ void deleteSnapshots( /** * Deletes snapshots and releases respective lock files from remote store repository. * - * @param snapshotIds snapshot ids - * @param repositoryStateId the unique id identifying the state of the repository when the snapshot deletion began - * @param repositoryMetaVersion version of the updated repository metadata to write - * @param remoteStoreLockManagerFactory RemoteStoreLockManagerFactory to be used for cleaning up remote store lock files - * @param listener completion listener + * @param snapshotIds snapshot ids + * @param repositoryStateId the unique id identifying the state of the repository when the snapshot deletion began + * @param repositoryMetaVersion version of the updated repository metadata to write + * @param remoteStoreLockManagerFactory RemoteStoreLockManagerFactory to be used for cleaning up remote store lock files + * @param remoteSegmentStoreDirectoryFactory RemoteSegmentStoreDirectoryFactory to be used for cleaning up remote store segment files + * @param remoteStorePinnedTimestampService service for pinning and unpinning of the timestamp + * @param snapshotIdsPinnedTimestampMap map of snapshots ids and the pinned timestamp + * @param isShallowSnapshotV2 true for shallow snapshots v2 + * @param listener completion listener + */ + default void deleteSnapshotsInternal( + Collection snapshotIds, + long repositoryStateId, + Version repositoryMetaVersion, + RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, + RemoteStorePinnedTimestampService remoteStorePinnedTimestampService, + Map snapshotIdsPinnedTimestampMap, + boolean isShallowSnapshotV2, + ActionListener listener + ) { + throw new UnsupportedOperationException(); + } + + /** + * Deletes snapshots and unpin the snapshot timestamp using remoteStorePinnedTimestampService + * + * @param snapshotsWithPinnedTimestamp map of snapshot ids and the pinned timestamps + * @param repositoryStateId the unique id identifying the state of the repository when the snapshot deletion began + * @param repositoryMetaVersion version of the updated repository metadata to write + * @param remoteSegmentStoreDirectoryFactory RemoteSegmentStoreDirectoryFactory to be used for cleaning up remote store segment files + * @param remoteStorePinnedTimestampService service for pinning and unpinning of the timestamp + * @param listener completion listener + */ + default void deleteSnapshotsWithPinnedTimestamp( + Map snapshotsWithPinnedTimestamp, + long repositoryStateId, + Version repositoryMetaVersion, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, + RemoteStorePinnedTimestampService remoteStorePinnedTimestampService, + ActionListener listener + ) { + throw new UnsupportedOperationException(); + } + + /** + * Deletes snapshots and releases respective lock files from remote store repository + * + * @param snapshotIds + * @param repositoryStateId + * @param repositoryMetaVersion + * @param remoteStoreLockManagerFactory + * @param listener */ default void deleteSnapshotsAndReleaseLockFiles( Collection snapshotIds, diff --git a/server/src/main/java/org/opensearch/repositories/RepositoryData.java b/server/src/main/java/org/opensearch/repositories/RepositoryData.java index b9fb2ba2498e8..5328618f688f4 100644 --- a/server/src/main/java/org/opensearch/repositories/RepositoryData.java +++ b/server/src/main/java/org/opensearch/repositories/RepositoryData.java @@ -112,6 +112,11 @@ public final class RepositoryData { * The indices found in the repository across all snapshots, as a name to {@link IndexId} mapping */ private final Map indices; + + public Map> getIndexSnapshots() { + return indexSnapshots; + } + /** * The snapshots that each index belongs to. */ diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java index 0907326cd8668..7b7606d3b70c7 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -114,6 +114,7 @@ import org.opensearch.index.remote.RemoteStorePathStrategy; import org.opensearch.index.remote.RemoteStorePathStrategy.BasePathInput; import org.opensearch.index.remote.RemoteStorePathStrategy.SnapshotShardPathInput; +import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.snapshots.IndexShardRestoreFailedException; import org.opensearch.index.snapshots.IndexShardSnapshotStatus; import org.opensearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; @@ -133,6 +134,7 @@ import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RecoveryState; +import org.opensearch.node.remotestore.RemoteStorePinnedTimestampService; import org.opensearch.repositories.IndexId; import org.opensearch.repositories.IndexMetaDataGenerations; import org.opensearch.repositories.Repository; @@ -189,6 +191,7 @@ import static org.opensearch.index.remote.RemoteStoreEnums.PathHashAlgorithm.FNV_1A_COMPOSITE_1; import static org.opensearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo.canonicalName; import static org.opensearch.repositories.blobstore.ChecksumBlobStoreFormat.SNAPSHOT_ONLY_FORMAT_PARAMS; +import static org.opensearch.snapshots.SnapshotsService.SNAPSHOT_PINNED_TIMESTAMP_DELIMITER; /** * BlobStore - based implementation of Snapshot Repository @@ -993,11 +996,15 @@ public void initializeSnapshot(SnapshotId snapshotId, List indices, Met } } - public void deleteSnapshotsAndReleaseLockFiles( + public void deleteSnapshotsInternal( Collection snapshotIds, long repositoryStateId, Version repositoryMetaVersion, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, + RemoteStorePinnedTimestampService remoteStorePinnedTimestampService, + Map snapshotIdsPinnedTimestampMap, + boolean isShallowSnapshotV2, ActionListener listener ) { if (isReadOnly()) { @@ -1019,6 +1026,10 @@ protected void doRun() throws Exception { repositoryData, repositoryMetaVersion, remoteStoreLockManagerFactory, + remoteSegmentStoreDirectoryFactory, + remoteStorePinnedTimestampService, + snapshotIdsPinnedTimestampMap, + isShallowSnapshotV2, listener ); } @@ -1031,6 +1042,49 @@ public void onFailure(Exception e) { } } + @Override + public void deleteSnapshotsWithPinnedTimestamp( + Map snapshotIdPinnedTimestampMap, + long repositoryStateId, + Version repositoryMetaVersion, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, + RemoteStorePinnedTimestampService remoteStorePinnedTimestampService, + ActionListener listener + ) { + deleteSnapshotsInternal( + snapshotIdPinnedTimestampMap.keySet(), + repositoryStateId, + repositoryMetaVersion, + null, // Passing null since no remote store lock files need to be cleaned up. + remoteSegmentStoreDirectoryFactory, + remoteStorePinnedTimestampService, + snapshotIdPinnedTimestampMap, + true, // true only for shallow snapshot v2 + listener + ); + } + + @Override + public void deleteSnapshotsAndReleaseLockFiles( + Collection snapshotIds, + long repositoryStateId, + Version repositoryMetaVersion, + RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, + ActionListener listener + ) { + deleteSnapshotsInternal( + snapshotIds, + repositoryStateId, + repositoryMetaVersion, + remoteStoreLockManagerFactory, + null, + null, + Collections.emptyMap(), + false, + listener + ); + } + @Override public void deleteSnapshots( Collection snapshotIds, @@ -1038,11 +1092,15 @@ public void deleteSnapshots( Version repositoryMetaVersion, ActionListener listener ) { - deleteSnapshotsAndReleaseLockFiles( + deleteSnapshotsInternal( snapshotIds, repositoryStateId, repositoryMetaVersion, null, // Passing null since no remote store lock files need to be cleaned up. + null, // Passing null since no remote store segment files need to be cleaned up + null, + Collections.emptyMap(), + false, listener ); } @@ -1107,6 +1165,10 @@ private RepositoryData safeRepositoryData(long repositoryStateId, Map snapshotIdPinnedTimestampMap, + boolean isShallowSnapshotV2, ActionListener listener ) { // We can create map of indexId to ShardInfo based on the old repository data. This is later used in cleanup @@ -1165,29 +1231,59 @@ private void doDeleteShardSnapshots( ); }, listener::onFailure); // Once we have updated the repository, run the clean-ups + final StepListener pinnedTimestampListener = new StepListener<>(); writeUpdatedRepoDataStep.whenComplete(updatedRepoData -> { + if (snapshotIdPinnedTimestampMap == null || snapshotIdPinnedTimestampMap.isEmpty()) { + pinnedTimestampListener.onResponse(updatedRepoData); + } else { + removeSnapshotsPinnedTimestamp( + snapshotIdPinnedTimestampMap, + this, + updatedRepoData, + remoteStorePinnedTimestampService, + pinnedTimestampListener + ); + } + }, listener::onFailure); + + pinnedTimestampListener.whenComplete(updatedRepoData -> { + // Run unreferenced blobs cleanup in parallel to shard-level snapshot deletion final ActionListener afterCleanupsListener = new GroupedActionListener<>( ActionListener.wrap(() -> listener.onResponse(updatedRepoData)), 2 ); + cleanupUnlinkedRootAndIndicesBlobs( snapshotIds, foundIndices, rootBlobs, updatedRepoData, + repositoryData, remoteStoreLockManagerFactory, + remoteSegmentStoreDirectoryFactory, afterCleanupsListener, idToShardInfoMap ); - asyncCleanupUnlinkedShardLevelBlobs( - repositoryData, - snapshotIds, - writeShardMetaDataAndComputeDeletesStep.result(), - remoteStoreLockManagerFactory, - afterCleanupsListener - ); + if (isShallowSnapshotV2) { + cleanUpRemoteStoreFilesForDeletedIndicesV2( + repositoryData, + snapshotIds, + writeShardMetaDataAndComputeDeletesStep.result(), + remoteSegmentStoreDirectoryFactory, + afterCleanupsListener + ); + } else { + asyncCleanupUnlinkedShardLevelBlobs( + repositoryData, + snapshotIds, + writeShardMetaDataAndComputeDeletesStep.result(), + remoteStoreLockManagerFactory, + afterCleanupsListener + ); + } }, listener::onFailure); + } else { // Write the new repository data first (with the removed snapshot), using no shard generations final RepositoryData updatedRepoData = repositoryData.removeSnapshots(snapshotIds, ShardGenerations.EMPTY); @@ -1208,7 +1304,9 @@ private void doDeleteShardSnapshots( foundIndices, rootBlobs, newRepoData, + repositoryData, remoteStoreLockManagerFactory, + remoteSegmentStoreDirectoryFactory, afterCleanupsListener, idToShardInfoMap ); @@ -1235,6 +1333,90 @@ private void doDeleteShardSnapshots( } } + private void cleanUpRemoteStoreFilesForDeletedIndicesV2( + RepositoryData repositoryData, + Collection snapshotIds, + Collection result, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, + ActionListener afterCleanupsListener + ) { + try { + Set uniqueIndexIds = new HashSet<>(); + for (ShardSnapshotMetaDeleteResult shardSnapshotMetaDeleteResult : result) { + uniqueIndexIds.add(shardSnapshotMetaDeleteResult.indexId.getId()); + } + // iterate through all the indices and trigger remote store directory cleanup for deleted index segments + for (String indexId : uniqueIndexIds) { + cleanRemoteStoreDirectoryIfNeeded(snapshotIds, indexId, repositoryData, remoteSegmentStoreDirectoryFactory); + } + afterCleanupsListener.onResponse(null); + } catch (Exception e) { + logger.warn("Exception during cleanup of remote directory files for snapshot v2", e); + afterCleanupsListener.onFailure(e); + } + + } + + private void removeSnapshotsPinnedTimestamp( + Map snapshotsWithPinnedTimestamp, + Repository repository, + RepositoryData repositoryData, + RemoteStorePinnedTimestampService remoteStorePinnedTimestampService, + ActionListener pinnedTimestampListener + ) { + // Create a GroupedActionListener to aggregate the results of all unpin operations + GroupedActionListener groupedListener = new GroupedActionListener<>( + ActionListener.wrap( + // This is called once all operations have succeeded + ignored -> pinnedTimestampListener.onResponse(repositoryData), + // This is called if any operation fails + pinnedTimestampListener::onFailure + ), + snapshotsWithPinnedTimestamp.size() + ); + + snapshotsWithPinnedTimestamp.forEach((snapshotId, pinnedTimestamp) -> { + removeSnapshotPinnedTimestamp( + remoteStorePinnedTimestampService, + snapshotId, + repository.getMetadata().name(), + pinnedTimestamp, + groupedListener + ); + }); + } + + private void removeSnapshotPinnedTimestamp( + RemoteStorePinnedTimestampService remoteStorePinnedTimestampService, + SnapshotId snapshotId, + String repository, + long timestampToUnpin, + ActionListener listener + ) { + remoteStorePinnedTimestampService.unpinTimestamp( + timestampToUnpin, + repository + SNAPSHOT_PINNED_TIMESTAMP_DELIMITER + snapshotId.getUUID(), + new ActionListener() { + @Override + public void onResponse(Void unused) { + logger.debug("Timestamp {} unpinned successfully for snapshot {}", timestampToUnpin, snapshotId.getName()); + listener.onResponse(null); + } + + @Override + public void onFailure(Exception e) { + logger.error( + "Failed to unpin timestamp {} for snapshot {} with exception {}", + timestampToUnpin, + snapshotId.getName(), + e + ); + listener.onFailure(e); + } + } + ); + } + /** * Cleans up the indices and data corresponding to all it's shards. * @@ -1251,7 +1433,9 @@ private void cleanupUnlinkedRootAndIndicesBlobs( Map foundIndices, Map rootBlobs, RepositoryData updatedRepoData, + RepositoryData oldRepoData, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, ActionListener listener, Map idToShardInfoMap ) { @@ -1260,7 +1444,9 @@ private void cleanupUnlinkedRootAndIndicesBlobs( foundIndices, rootBlobs, updatedRepoData, + oldRepoData, remoteStoreLockManagerFactory, + remoteSegmentStoreDirectoryFactory, ActionListener.map(listener, ignored -> null), idToShardInfoMap ); @@ -1636,7 +1822,9 @@ private void cleanupStaleBlobs( Map foundIndices, Map rootBlobs, RepositoryData newRepoData, + RepositoryData oldRepoData, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, ActionListener listener, Map idToShardInfoMap ) { @@ -1665,9 +1853,12 @@ private void cleanupStaleBlobs( } else { Map snapshotShardPaths = getSnapshotShardPaths(); cleanupStaleIndices( + deletedSnapshots, foundIndices, survivingIndexIds, remoteStoreLockManagerFactory, + remoteSegmentStoreDirectoryFactory, + oldRepoData, groupedListener, snapshotShardPaths, idToShardInfoMap @@ -1697,12 +1888,14 @@ private Map getSnapshotShardPaths() { * @param repositoryStateId Current repository state id * @param repositoryMetaVersion version of the updated repository metadata to write * @param remoteStoreLockManagerFactory RemoteStoreLockManagerFactory to be used for cleaning up remote store lock files. + * @param remoteSegmentStoreDirectoryFactory RemoteSegmentStoreDirectoryFactory to be used for cleaning up remote store segments. * @param listener Listener to complete when done */ public void cleanup( long repositoryStateId, Version repositoryMetaVersion, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, ActionListener listener ) { try { @@ -1735,7 +1928,9 @@ public void cleanup( foundIndices, rootBlobs, repositoryData, + repositoryData, remoteStoreLockManagerFactory, + remoteSegmentStoreDirectoryFactory, ActionListener.map(listener, RepositoryCleanupResult::new), Collections.emptyMap() ), @@ -1826,9 +2021,12 @@ private List cleanupStaleRootFiles( } void cleanupStaleIndices( + Collection deletedSnapshots, Map foundIndices, Set survivingIndexIds, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, + RepositoryData oldRepoData, GroupedActionListener listener, Map snapshotShardPaths, Map idToShardInfoMap @@ -1856,8 +2054,11 @@ void cleanupStaleIndices( ); for (int i = 0; i < workers; ++i) { executeOneStaleIndexDelete( + deletedSnapshots, staleIndicesToDelete, remoteStoreLockManagerFactory, + remoteSegmentStoreDirectoryFactory, + oldRepoData, groupedListener, snapshotShardPaths, idToShardInfoMap @@ -1892,8 +2093,11 @@ private static boolean isIndexPresent(ClusterService clusterService, String inde * @throws InterruptedException if the thread is interrupted while waiting */ private void executeOneStaleIndexDelete( + Collection deletedSnapshots, BlockingQueue> staleIndicesToDelete, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, + RepositoryData oldRepoData, GroupedActionListener listener, Map snapshotShardPaths, Map idToShardInfoMap @@ -1924,6 +2128,10 @@ private void executeOneStaleIndexDelete( // If there are matchingShardPaths, then we delete them after we have deleted the shard data. deleteResult = deleteResult.add(cleanUpStaleSnapshotShardPathsFile(matchingShardPaths, snapshotShardPaths)); + if (remoteSegmentStoreDirectoryFactory != null) { + cleanRemoteStoreDirectoryIfNeeded(deletedSnapshots, indexSnId, oldRepoData, remoteSegmentStoreDirectoryFactory); + } + // Finally, we delete the [base_path]/indexId folder deleteResult = deleteResult.add(indexEntry.getValue().delete()); // Deleting the index folder logger.debug("[{}] Cleaned up stale index [{}]", metadata.name(), indexSnId); @@ -1945,8 +2153,11 @@ private void executeOneStaleIndexDelete( return DeleteResult.ZERO; } finally { executeOneStaleIndexDelete( + deletedSnapshots, staleIndicesToDelete, remoteStoreLockManagerFactory, + remoteSegmentStoreDirectoryFactory, + oldRepoData, listener, snapshotShardPaths, idToShardInfoMap @@ -1955,6 +2166,90 @@ private void executeOneStaleIndexDelete( })); } + /** + * Cleans up the remote store directory if needed. + *

This method cleans up segments in the remote store directory for deleted indices. + * This cleanup flow is executed only for v2 snapshots. For v1 snapshots, + * the cleanup is done per shard after releasing the lock files. + *

+ * + *

Since this method requires old repository data to fetch index metadata of the deleted index, + * the cleanup won't happen on retries in case of failures. This is because subsequent retries may + * not have access to the older repository data.

+ * + * @param indexSnId The snapshot index id of the index to be cleaned up + * @param oldRepoData The old repository metadata used to fetch the index metadata. + * @param remoteSegmentStoreDirectoryFactory RemoteSegmentStoreDirectoryFactory to be used for cleaning up remote + * store segments + */ + private void cleanRemoteStoreDirectoryIfNeeded( + Collection deletedSnapshots, + String indexSnId, + RepositoryData oldRepoData, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory + ) { + assert (indexSnId != null); + + IndexId indexId = null; + List snapshotIds = Collections.emptyList(); + try { + for (Map.Entry> entry : oldRepoData.getIndexSnapshots().entrySet()) { + indexId = entry.getKey(); + if (indexId != null && indexId.getId().equals(indexSnId)) { + snapshotIds = entry.getValue(); + break; + } + } + if (snapshotIds.isEmpty()) { + logger.info("No snapshots found for indexSnId: {}", indexSnId); + return; + } + for (SnapshotId snapshotId : snapshotIds) { + try { + // skip cleanup for snapshot not present in deleted snapshots list + if (!deletedSnapshots.contains(snapshotId)) { + continue; + } + IndexMetadata prevIndexMetadata = this.getSnapshotIndexMetaData(oldRepoData, snapshotId, indexId); + if (prevIndexMetadata != null && !isIndexPresent(clusterService, prevIndexMetadata.getIndexUUID())) { + String remoteStoreRepository = IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.get( + prevIndexMetadata.getSettings() + ); + assert (remoteStoreRepository != null); + + RemoteStorePathStrategy remoteStorePathStrategy = RemoteStoreUtils.determineRemoteStorePathStrategy( + prevIndexMetadata + ); + + for (int shardId = 0; shardId < prevIndexMetadata.getNumberOfShards(); shardId++) { + remoteDirectoryCleanupAsync( + remoteSegmentStoreDirectoryFactory, + threadPool, + remoteStoreRepository, + prevIndexMetadata.getIndexUUID(), + new ShardId(Index.UNKNOWN_INDEX_NAME, prevIndexMetadata.getIndexUUID(), shardId), + ThreadPool.Names.REMOTE_PURGE, + remoteStorePathStrategy + ); + } + } + } catch (Exception e) { + logger.warn( + new ParameterizedMessage( + "Exception during cleanup of remote directory for snapshot [{}] deleted index [{}]", + snapshotId, + indexSnId + ), + e + ); + } + } + } catch (Exception e) { + logger.error(new ParameterizedMessage("Exception during the remote directory cleanup for indecSnId [{}]", indexSnId), e); + } + + } + /** * Finds and returns a list of shard paths that match the given index ID. * diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index 99a09cb863a8e..2265fe519fc78 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -92,6 +92,7 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; import org.opensearch.index.store.lockmanager.RemoteStoreLockManagerFactory; import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.node.remotestore.RemoteStorePinnedTimestampService; @@ -125,6 +126,7 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; import java.util.function.Consumer; import java.util.function.Function; @@ -181,6 +183,8 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus private final RemoteStoreLockManagerFactory remoteStoreLockManagerFactory; + private final RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory; + private final ThreadPool threadPool; private final Map>>> snapshotCompletionListeners = @@ -227,6 +231,8 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus Setting.Property.Dynamic ); + public static final String SNAPSHOT_PINNED_TIMESTAMP_DELIMITER = "__"; + /** * Setting to specify the maximum number of shards that can be included in the result for the snapshot status * API call. Note that it does not apply to V2-shallow snapshots. @@ -239,7 +245,6 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus Setting.Property.Dynamic ); - private static final String SNAPSHOT_PINNED_TIMESTAMP_DELIMITER = "__"; private volatile int maxConcurrentOperations; public SnapshotsService( @@ -260,6 +265,11 @@ public SnapshotsService( remoteStoreSettings.getSegmentsPathFixedPrefix() ); this.threadPool = transportService.getThreadPool(); + this.remoteSegmentStoreDirectoryFactory = new RemoteSegmentStoreDirectoryFactory( + () -> repositoriesService, + threadPool, + remoteStoreSettings.getSegmentsPathFixedPrefix() + ); this.transportService = transportService; this.remoteStorePinnedTimestampService = remoteStorePinnedTimestampService; @@ -3053,18 +3063,67 @@ private void deleteSnapshotsFromRepository( // the flag. This can be improved by having the info whether there ever were any shallow snapshot present in this repository // or not in RepositoryData. // SEE https://github.com/opensearch-project/OpenSearch/issues/8610 - final boolean cleanupRemoteStoreLockFiles = REMOTE_STORE_INDEX_SHALLOW_COPY.get(repository.getMetadata().settings()); - if (cleanupRemoteStoreLockFiles) { - repository.deleteSnapshotsAndReleaseLockFiles( - snapshotIds, - repositoryData.getGenId(), - minCompatibleVersion(minNodeVersion, repositoryData, snapshotIds), - remoteStoreLockManagerFactory, - ActionListener.wrap(updatedRepoData -> { - logger.info("snapshots {} deleted", snapshotIds); - removeSnapshotDeletionFromClusterState(deleteEntry, null, updatedRepoData); - }, ex -> removeSnapshotDeletionFromClusterState(deleteEntry, ex, repositoryData)) - ); + final boolean remoteStoreShallowCopyEnabled = REMOTE_STORE_INDEX_SHALLOW_COPY.get(repository.getMetadata().settings()); + if (remoteStoreShallowCopyEnabled) { + Map snapshotsWithPinnedTimestamp = new ConcurrentHashMap<>(); + List snapshotsWithLockFiles = Collections.synchronizedList(new ArrayList<>()); + + CountDownLatch latch = new CountDownLatch(1); + + threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(() -> { + try { + for (SnapshotId snapshotId : snapshotIds) { + try { + SnapshotInfo snapshotInfo = repository.getSnapshotInfo(snapshotId); + if (snapshotInfo.getPinnedTimestamp() > 0) { + snapshotsWithPinnedTimestamp.put(snapshotId, snapshotInfo.getPinnedTimestamp()); + } else { + snapshotsWithLockFiles.add(snapshotId); + } + } catch (Exception e) { + logger.warn("Failed to get snapshot info for {} with exception {}", snapshotId, e); + removeSnapshotDeletionFromClusterState(deleteEntry, e, repositoryData); + } + } + } finally { + latch.countDown(); + } + }); + try { + latch.await(); + if (snapshotsWithLockFiles.size() > 0) { + repository.deleteSnapshotsAndReleaseLockFiles( + snapshotsWithLockFiles, + repositoryData.getGenId(), + minCompatibleVersion(minNodeVersion, repositoryData, snapshotsWithLockFiles), + remoteStoreLockManagerFactory, + ActionListener.wrap(updatedRepoData -> { + logger.info("snapshots {} deleted", snapshotsWithLockFiles); + removeSnapshotDeletionFromClusterState(deleteEntry, null, updatedRepoData); + }, ex -> removeSnapshotDeletionFromClusterState(deleteEntry, ex, repositoryData)) + ); + } + if (snapshotsWithPinnedTimestamp.size() > 0) { + + repository.deleteSnapshotsWithPinnedTimestamp( + snapshotsWithPinnedTimestamp, + repositoryData.getGenId(), + minCompatibleVersion(minNodeVersion, repositoryData, snapshotsWithPinnedTimestamp.keySet()), + remoteSegmentStoreDirectoryFactory, + remoteStorePinnedTimestampService, + ActionListener.wrap(updatedRepoData -> { + logger.info("snapshots {} deleted", snapshotsWithPinnedTimestamp); + removeSnapshotDeletionFromClusterState(deleteEntry, null, updatedRepoData); + }, ex -> removeSnapshotDeletionFromClusterState(deleteEntry, ex, repositoryData)) + ); + } + + } catch (InterruptedException e) { + logger.error("Interrupted while waiting for snapshot info processing", e); + Thread.currentThread().interrupt(); + removeSnapshotDeletionFromClusterState(deleteEntry, e, repositoryData); + } + } else { repository.deleteSnapshots( snapshotIds, diff --git a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java index fcc0c5198894f..6540d2f55a349 100644 --- a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java +++ b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java @@ -54,6 +54,7 @@ import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.Environment; import org.opensearch.index.remote.RemoteStoreEnums; +import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; import org.opensearch.index.store.lockmanager.RemoteStoreLockManager; import org.opensearch.index.store.lockmanager.RemoteStoreLockManagerFactory; import org.opensearch.indices.recovery.RecoverySettings; @@ -74,6 +75,7 @@ import org.opensearch.test.OpenSearchIntegTestCase; import java.nio.file.Path; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -459,11 +461,18 @@ public void testCleanupStaleIndices() throws Exception { foundIndices.put("stale-index", staleIndexContainer); foundIndices.put("current-index", currentIndexContainer); + List snapshotIds = new ArrayList<>(); + snapshotIds.add(new SnapshotId("snap1", UUIDs.randomBase64UUID())); + snapshotIds.add(new SnapshotId("snap2", UUIDs.randomBase64UUID())); + Set survivingIndexIds = new HashSet<>(); survivingIndexIds.add("current-index"); + RepositoryData repositoryData = generateRandomRepoData(); + // Create a mock RemoteStoreLockManagerFactory RemoteStoreLockManagerFactory mockRemoteStoreLockManagerFactory = mock(RemoteStoreLockManagerFactory.class); + RemoteSegmentStoreDirectoryFactory mockRemoteSegmentStoreDirectoryFactory = mock(RemoteSegmentStoreDirectoryFactory.class); RemoteStoreLockManager mockLockManager = mock(RemoteStoreLockManager.class); when(mockRemoteStoreLockManagerFactory.newLockManager(anyString(), anyString(), anyString(), any())).thenReturn(mockLockManager); @@ -479,9 +488,9 @@ public void testCleanupStaleIndices() throws Exception { // Mock the cleanupStaleIndices method to call our test implementation doAnswer(invocation -> { - Map indices = invocation.getArgument(0); - Set surviving = invocation.getArgument(1); - GroupedActionListener listener = invocation.getArgument(3); + Map indices = invocation.getArgument(1); + Set surviving = invocation.getArgument(2); + GroupedActionListener listener = invocation.getArgument(6); // Simulate the cleanup process DeleteResult result = DeleteResult.ZERO; @@ -494,7 +503,7 @@ public void testCleanupStaleIndices() throws Exception { listener.onResponse(result); return null; - }).when(repository).cleanupStaleIndices(any(), any(), any(), any(), any(), anyMap()); + }).when(repository).cleanupStaleIndices(any(), any(), any(), any(), any(), any(), any(), any(), anyMap()); AtomicReference> resultReference = new AtomicReference<>(); CountDownLatch latch = new CountDownLatch(1); @@ -509,9 +518,12 @@ public void testCleanupStaleIndices() throws Exception { // Call the method we're testing repository.cleanupStaleIndices( + snapshotIds, foundIndices, survivingIndexIds, mockRemoteStoreLockManagerFactory, + null, + repositoryData, listener, mockSnapshotShardPaths, Collections.emptyMap()