From 9142a254d75a0a6441bf29fc9e3284db9fd88eb1 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 6 Aug 2024 10:43:00 +0530 Subject: [PATCH 01/55] Initial Commit to support centralize snapshot creation and implicit locking mechanism Signed-off-by: Anshu Agarwal --- .../remotestore/RemoteRestoreSnapshotIT.java | 54 ++++ .../create/TransportCreateSnapshotAction.java | 2 +- .../common/settings/ClusterSettings.java | 1 + .../opensearch/snapshots/SnapshotInfo.java | 61 +++- .../snapshots/SnapshotsService.java | 263 +++++++++++++++++- .../create/CreateSnapshotResponseTests.java | 3 +- .../get/GetSnapshotsResponseTests.java | 3 +- .../BlobStoreRepositoryRestoreTests.java | 3 +- .../snapshots/SnapshotInfoTests.java | 33 ++- ...ckEventuallyConsistentRepositoryTests.java | 9 +- .../AbstractSnapshotIntegTestCase.java | 3 +- 11 files changed, 395 insertions(+), 40 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index f8e5079b01a36..8b7f3c9687fd3 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -746,4 +746,58 @@ public void testInvalidRestoreRequestScenarios() throws Exception { assertTrue(exception.getMessage().contains("cannot remove setting [index.remote_store.segment.repository]" + " on restore")); } + public void testCentralizedCreateAndRestoreShallowCopy() throws Exception { + + Settings snapshotSettings = Settings.builder().put("snapshot.centralized_create_operation", true).build(); + internalCluster().startClusterManagerOnlyNode(Settings.builder().put(snapshotSettings).build()); + internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); + String indexName1 = "testindex1"; + String indexName2 = "testindex2"; + String snapshotRepoName = "test-create-snapshot-repo"; + String snapshotName1 = "test-create-snapshot1"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + logger.info("Snapshot Path [{}]", absolutePath1); + String restoredIndexName1 = indexName1 + "-restored"; + + createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, true)); + + Client client = client(); + Settings indexSettings = getIndexSettings(20, 0).build(); + createIndex(indexName1, indexSettings); + + Settings indexSettings2 = getIndexSettings(15, 0).build(); + createIndex(indexName2, indexSettings2); + + final int numDocsInIndex1 = 10; + final int numDocsInIndex2 = 20; + indexDocuments(client, indexName1, numDocsInIndex1); + indexDocuments(client, indexName2, numDocsInIndex2); + ensureGreen(indexName1, indexName2); + + internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); + logger.info("--> snapshot"); + + SnapshotInfo snapshotInfo = createSnapshot(snapshotRepoName, snapshotName1, new ArrayList<>(Arrays.asList(indexName1, indexName2))); + assertThat(snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); + assertThat(snapshotInfo.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards())); + + // delete indices + DeleteResponse deleteResponse = client().prepareDelete(indexName1, "0").execute().actionGet(); + assertEquals(deleteResponse.getResult(), DocWriteResponse.Result.DELETED); + RestoreSnapshotResponse restoreSnapshotResponse1 = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(false) + .setIndices(indexName1) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1) + .get(); + + assertEquals(restoreSnapshotResponse1.status(), RestStatus.ACCEPTED); + ensureYellowAndNoInitializingShards(restoredIndexName1); + ensureGreen(restoredIndexName1); + assertDocsPresentInIndex(client(), restoredIndexName1, numDocsInIndex1); + } + } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java index bb3bf014f213b..04d20a9616031 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java @@ -106,7 +106,7 @@ protected void clusterManagerOperation( if (request.waitForCompletion()) { snapshotsService.executeSnapshot(request, ActionListener.map(listener, CreateSnapshotResponse::new)); } else { - snapshotsService.createSnapshot(request, ActionListener.map(listener, snapshot -> new CreateSnapshotResponse())); + snapshotsService.startCreateSnapshot(request, ActionListener.map(listener, snapshot -> new CreateSnapshotResponse())); } } } diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index 233a8d732d178..5bfa2ff768a5c 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -628,6 +628,7 @@ public void apply(Settings value, Settings current, Settings previous) { HandshakingTransportAddressConnector.PROBE_CONNECT_TIMEOUT_SETTING, HandshakingTransportAddressConnector.PROBE_HANDSHAKE_TIMEOUT_SETTING, SnapshotsService.MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING, + SnapshotsService.SHALLOW_SNAPSHOT_V2, FsHealthService.ENABLED_SETTING, FsHealthService.REFRESH_INTERVAL_SETTING, FsHealthService.SLOW_PATH_LOGGING_THRESHOLD_SETTING, diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java index 191b872cdd563..07c0e9133a635 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java @@ -98,10 +98,14 @@ public final class SnapshotInfo implements Comparable, ToXContent, private static final String INCLUDE_GLOBAL_STATE = "include_global_state"; private static final String REMOTE_STORE_INDEX_SHALLOW_COPY = "remote_store_index_shallow_copy"; + + private static final String PINNED_TIMESTAMP = "pinned_timestamp"; private static final String USER_METADATA = "metadata"; private static final Comparator COMPARATOR = Comparator.comparing(SnapshotInfo::startTime) .thenComparing(SnapshotInfo::snapshotId); + private XContentBuilder builder; + private XContentBuilder build; /** * Builds snapshot information @@ -121,6 +125,7 @@ public static final class SnapshotInfoBuilder { private Boolean includeGlobalState = null; private Boolean remoteStoreIndexShallowCopy = null; + private long pinnedTimestamp; private Map userMetadata = null; private int version = -1; private List shardFailures = null; @@ -177,6 +182,10 @@ private void setRemoteStoreIndexShallowCopy(Boolean remoteStoreIndexShallowCopy) this.remoteStoreIndexShallowCopy = remoteStoreIndexShallowCopy; } + private void setPinnedTimestamp(long pinnedTimestamp) { + this.pinnedTimestamp = pinnedTimestamp; + } + private void setShardFailures(List shardFailures) { this.shardFailures = shardFailures; } @@ -216,7 +225,8 @@ public SnapshotInfo build() { shardFailures, includeGlobalState, userMetadata, - remoteStoreIndexShallowCopy + remoteStoreIndexShallowCopy, + pinnedTimestamp ); } } @@ -271,6 +281,7 @@ int getSuccessfulShards() { SnapshotInfoBuilder::setRemoteStoreIndexShallowCopy, new ParseField(REMOTE_STORE_INDEX_SHALLOW_COPY) ); + SNAPSHOT_INFO_PARSER.declareLong(SnapshotInfoBuilder::setPinnedTimestamp, new ParseField(PINNED_TIMESTAMP)); SNAPSHOT_INFO_PARSER.declareObjectArray( SnapshotInfoBuilder::setShardFailures, SnapshotShardFailure.SNAPSHOT_SHARD_FAILURE_PARSER, @@ -307,6 +318,7 @@ int getSuccessfulShards() { @Nullable private Boolean remoteStoreIndexShallowCopy; + private long pinnedTimestamp; @Nullable private final Map userMetadata; @@ -316,11 +328,11 @@ int getSuccessfulShards() { private final List shardFailures; public SnapshotInfo(SnapshotId snapshotId, List indices, List dataStreams, SnapshotState state) { - this(snapshotId, indices, dataStreams, state, null, null, 0L, 0L, 0, 0, Collections.emptyList(), null, null, null); + this(snapshotId, indices, dataStreams, state, null, null, 0L, 0L, 0, 0, Collections.emptyList(), null, null, null, 0); } public SnapshotInfo(SnapshotId snapshotId, List indices, List dataStreams, SnapshotState state, Version version) { - this(snapshotId, indices, dataStreams, state, null, version, 0L, 0L, 0, 0, Collections.emptyList(), null, null, null); + this(snapshotId, indices, dataStreams, state, null, version, 0L, 0L, 0, 0, Collections.emptyList(), null, null, null, 0); } public SnapshotInfo(SnapshotsInProgress.Entry entry) { @@ -338,7 +350,8 @@ public SnapshotInfo(SnapshotsInProgress.Entry entry) { Collections.emptyList(), entry.includeGlobalState(), entry.userMetadata(), - entry.remoteStoreIndexShallowCopy() + entry.remoteStoreIndexShallowCopy(), + 0 ); } @@ -353,7 +366,8 @@ public SnapshotInfo( List shardFailures, Boolean includeGlobalState, Map userMetadata, - Boolean remoteStoreIndexShallowCopy + Boolean remoteStoreIndexShallowCopy, + long pinnedTimestamp ) { this( snapshotId, @@ -369,7 +383,8 @@ public SnapshotInfo( shardFailures, includeGlobalState, userMetadata, - remoteStoreIndexShallowCopy + remoteStoreIndexShallowCopy, + pinnedTimestamp ); } @@ -387,7 +402,8 @@ public SnapshotInfo( List shardFailures, Boolean includeGlobalState, Map userMetadata, - Boolean remoteStoreIndexShallowCopy + Boolean remoteStoreIndexShallowCopy, + long pinnedTimestamp ) { this.snapshotId = Objects.requireNonNull(snapshotId); this.indices = Collections.unmodifiableList(Objects.requireNonNull(indices)); @@ -403,6 +419,7 @@ public SnapshotInfo( this.includeGlobalState = includeGlobalState; this.userMetadata = userMetadata; this.remoteStoreIndexShallowCopy = remoteStoreIndexShallowCopy; + this.pinnedTimestamp = pinnedTimestamp; } /** @@ -425,6 +442,9 @@ public SnapshotInfo(final StreamInput in) throws IOException { if (in.getVersion().onOrAfter(Version.V_2_9_0)) { remoteStoreIndexShallowCopy = in.readOptionalBoolean(); } + if (in.getVersion().onOrAfter(Version.V_2_16_0)) { + pinnedTimestamp = in.readLong(); + } } /** @@ -539,6 +559,10 @@ public Boolean isRemoteStoreIndexShallowCopyEnabled() { return remoteStoreIndexShallowCopy; } + public long getPinnedTimestamp() { + return pinnedTimestamp; + } + /** * Returns shard failures; an empty list will be returned if there were no shard * failures, or if {@link #state()} returns {@code null}. @@ -606,6 +630,8 @@ public String toString() { + shardFailures + ", isRemoteStoreInteropEnabled=" + remoteStoreIndexShallowCopy + + ", pinnedTimestamp=" + + pinnedTimestamp + '}'; } @@ -641,6 +667,9 @@ public XContentBuilder toXContent(final XContentBuilder builder, final Params pa if (remoteStoreIndexShallowCopy != null) { builder.field(REMOTE_STORE_INDEX_SHALLOW_COPY, remoteStoreIndexShallowCopy); } + if (pinnedTimestamp != 0) { + builder.field(PINNED_TIMESTAMP); + } builder.startArray(INDICES); for (String index : indices) { builder.value(index); @@ -699,6 +728,9 @@ private XContentBuilder toXContentInternal(final XContentBuilder builder, final if (remoteStoreIndexShallowCopy != null) { builder.field(REMOTE_STORE_INDEX_SHALLOW_COPY, remoteStoreIndexShallowCopy); } + if (pinnedTimestamp != 0) { + builder.field(PINNED_TIMESTAMP, pinnedTimestamp); + } builder.startArray(INDICES); for (String index : indices) { builder.value(index); @@ -747,6 +779,7 @@ public static SnapshotInfo fromXContentInternal(final XContentParser parser) thr long endTime = 0; int totalShards = 0; int successfulShards = 0; + long pinnedTimestamp = 0; Boolean includeGlobalState = null; Boolean remoteStoreIndexShallowCopy = null; Map userMetadata = null; @@ -788,6 +821,8 @@ public static SnapshotInfo fromXContentInternal(final XContentParser parser) thr includeGlobalState = parser.booleanValue(); } else if (REMOTE_STORE_INDEX_SHALLOW_COPY.equals(currentFieldName)) { remoteStoreIndexShallowCopy = parser.booleanValue(); + } else if (PINNED_TIMESTAMP.equals(currentFieldName)) { + pinnedTimestamp = parser.longValue(); } } else if (token == XContentParser.Token.START_ARRAY) { if (DATA_STREAMS.equals(currentFieldName)) { @@ -840,7 +875,8 @@ public static SnapshotInfo fromXContentInternal(final XContentParser parser) thr shardFailures, includeGlobalState, userMetadata, - remoteStoreIndexShallowCopy + remoteStoreIndexShallowCopy, + pinnedTimestamp ); } @@ -872,6 +908,9 @@ public void writeTo(final StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_2_9_0)) { out.writeOptionalBoolean(remoteStoreIndexShallowCopy); } + if (out.getVersion().onOrAfter(Version.V_2_16_0)) { + out.writeVLong(pinnedTimestamp); + } } private static SnapshotState snapshotState(final String reason, final List shardFailures) { @@ -904,7 +943,8 @@ public boolean equals(Object o) { && Objects.equals(version, that.version) && Objects.equals(shardFailures, that.shardFailures) && Objects.equals(userMetadata, that.userMetadata) - && Objects.equals(remoteStoreIndexShallowCopy, that.remoteStoreIndexShallowCopy); + && Objects.equals(remoteStoreIndexShallowCopy, that.remoteStoreIndexShallowCopy) + && Objects.equals(pinnedTimestamp, that.pinnedTimestamp); } @Override @@ -924,7 +964,8 @@ public int hashCode() { version, shardFailures, userMetadata, - remoteStoreIndexShallowCopy + remoteStoreIndexShallowCopy, + pinnedTimestamp ); } } diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index acc2dc83749cd..ad0b029e13a92 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -200,8 +200,17 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus Setting.Property.Dynamic ); + public static final Setting SHALLOW_SNAPSHOT_V2 = Setting.boolSetting( + "snapshot.shallow_snapshot_v2", + false, + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); + private volatile int maxConcurrentOperations; + private volatile boolean isShallowSnapV2; + public SnapshotsService( Settings settings, ClusterService clusterService, @@ -231,6 +240,8 @@ public SnapshotsService( maxConcurrentOperations = MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING.get(settings); clusterService.getClusterSettings() .addSettingsUpdateConsumer(MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING, i -> maxConcurrentOperations = i); + isShallowSnapV2 = SHALLOW_SNAPSHOT_V2.get(settings); + clusterService.getClusterSettings().addSettingsUpdateConsumer(SHALLOW_SNAPSHOT_V2, this::setShallowSnapshotV2); } // Task is onboarded for throttling, it will get retried from associated TransportClusterManagerNodeAction. @@ -239,6 +250,14 @@ public SnapshotsService( updateSnapshotStateTaskKey = clusterService.registerClusterManagerTask(ClusterManagerTaskKeys.UPDATE_SNAPSHOT_STATE_KEY, true); } + private void setShallowSnapshotV2(boolean isShallowSnapV2) { + this.isShallowSnapV2 = isShallowSnapV2; + } + + public boolean isShallowSnapV2() { + return isShallowSnapV2; + } + /** * Same as {@link #createSnapshot(CreateSnapshotRequest, ActionListener)} but invokes its callback on completion of * the snapshot. @@ -247,12 +266,30 @@ public SnapshotsService( * @param listener snapshot completion listener */ public void executeSnapshot(final CreateSnapshotRequest request, final ActionListener listener) { - createSnapshot( + startCreateSnapshot( request, ActionListener.wrap(snapshot -> addListener(snapshot, ActionListener.map(listener, Tuple::v2)), listener::onFailure) ); } + public void startCreateSnapshot(final CreateSnapshotRequest request, final ActionListener listener) { + Repository repository = repositoriesService.repository(request.repository()); + boolean remoteStoreIndexShallowCopy = REMOTE_STORE_INDEX_SHALLOW_COPY.get(repository.getMetadata().settings()); + logger.debug("remote_store_index_shallow_copy setting is set as [{}]", remoteStoreIndexShallowCopy); + + if (remoteStoreIndexShallowCopy + && clusterService.getClusterSettings().get(REMOTE_STORE_COMPATIBILITY_MODE_SETTING).equals(CompatibilityMode.MIXED)) { + // don't allow shallow snapshots if compatibility mode is not strict + logger.warn("Shallow snapshots are not supported during migration. Falling back to full snapshot."); + remoteStoreIndexShallowCopy = false; + } + if (remoteStoreIndexShallowCopy && isShallowSnapV2 && request.indices().length == 0) { + createShallowSnapshotV2(request, listener); + } else { + createSnapshot(request, listener); + } + } + /** * Initializes the snapshotting process. *

@@ -263,6 +300,7 @@ public void executeSnapshot(final CreateSnapshotRequest request, final ActionLis * @param listener snapshot creation listener */ public void createSnapshot(final CreateSnapshotRequest request, final ActionListener listener) { + final String repositoryName = request.repository(); final String snapshotName = indexNameExpressionResolver.resolveDateMathExpression(request.snapshot()); validate(repositoryName, snapshotName); @@ -403,6 +441,168 @@ public TimeValue timeout() { }, "create_snapshot [" + snapshotName + ']', listener::onFailure); } + public void createShallowSnapshotV2(final CreateSnapshotRequest request, final ActionListener listener) { + long pinnedTimestamp = System.currentTimeMillis(); + final String repositoryName = request.repository(); + final String snapshotName = indexNameExpressionResolver.resolveDateMathExpression(request.snapshot()); + validate(repositoryName, snapshotName); + + final SnapshotId snapshotId = new SnapshotId(snapshotName, UUIDs.randomBase64UUID()); // new UUID for the snapshot + Repository repository = repositoriesService.repository(request.repository()); + + if (repository.isReadOnly()) { + listener.onFailure(new RepositoryException(repository.getMetadata().name(), "cannot create snapshot in a readonly repository")); + return; + } + + final Snapshot snapshot = new Snapshot(repositoryName, snapshotId); + ClusterState currentState = clusterService.state(); + final Map userMeta = repository.adaptUserMetadata(request.userMetadata()); + try { + final StepListener repositoryDataListener = new StepListener<>(); + repositoriesService.getRepositoryData(repositoryName, repositoryDataListener); + + repositoryDataListener.whenComplete(repositoryData -> { + ensureSnapshotNameAvailableInRepo(repositoryData, snapshotName, repository); + final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); + final List runningSnapshots = snapshots.entries(); + ensureSnapshotNameNotRunning(runningSnapshots, repositoryName, snapshotName); + validate(repositoryName, snapshotName, currentState); + final SnapshotDeletionsInProgress deletionsInProgress = currentState.custom( + SnapshotDeletionsInProgress.TYPE, + SnapshotDeletionsInProgress.EMPTY + ); + final RepositoryCleanupInProgress repositoryCleanupInProgress = currentState.custom( + RepositoryCleanupInProgress.TYPE, + RepositoryCleanupInProgress.EMPTY + ); + if (repositoryCleanupInProgress.hasCleanupInProgress()) { + throw new ConcurrentSnapshotExecutionException( + repositoryName, + snapshotName, + "cannot snapshot while a repository cleanup is in-progress in [" + repositoryCleanupInProgress + "]" + ); + } + ensureNoCleanupInProgress(currentState, repositoryName, snapshotName); + + List indices = Arrays.asList(indexNameExpressionResolver.concreteIndexNames(currentState, request)); + + final List dataStreams = indexNameExpressionResolver.dataStreamNames( + currentState, + request.indicesOptions(), + request.indices() + ); + + logger.trace("[{}][{}] creating snapshot for indices [{}]", repositoryName, snapshotName, indices); + + final List indexIds = repositoryData.resolveNewIndices( + indices, + getInFlightIndexIds(runningSnapshots, repositoryName) + ); + final Version version = minCompatibleVersion(currentState.nodes().getMinNodeVersion(), repositoryData, null); + final ShardGenerations shardGenerations = buildShardsGenerationFromRepositoryData( + currentState.metadata(), + currentState.routingTable(), + indexIds, + repositoryData + ); + + boolean remoteStoreIndexShallowCopy = REMOTE_STORE_INDEX_SHALLOW_COPY.get(repository.getMetadata().settings()); + assert remoteStoreIndexShallowCopy : "remote_store_index_shallow_copy setting is set as false"; + if (repositoryData.getGenId() == RepositoryData.UNKNOWN_REPO_GEN) { + logger.debug("[{}] was aborted before starting", snapshot); + throw new SnapshotException(snapshot, "Aborted on initialization"); + } + final SnapshotInfo snapshotInfo = new SnapshotInfo( + snapshot.getSnapshotId(), + shardGenerations.indices().stream().map(IndexId::getName).collect(Collectors.toList()), + dataStreams, + pinnedTimestamp, + "", + System.currentTimeMillis(), + shardGenerations.totalShards(), + Collections.emptyList(), + request.includeGlobalState(), + userMeta, + remoteStoreIndexShallowCopy, + pinnedTimestamp + ); + if (!currentState.nodes().isLocalNodeElectedClusterManager()) { + throw new SnapshotException(repositoryName, snapshotName, "Aborting Snapshot, no longer cluster manager"); + } + final StepListener pinnedTimestampListener = new StepListener<>(); + pinnedTimestampListener.whenComplete( + repoData -> completeListenersIgnoringException( + endAndGetListenersToResolve(snapshot), + Tuple.tuple(repoData, snapshotInfo) + ), + e -> failSnapshotCompletionListeners(snapshot, new SnapshotException(snapshot, e.toString())) + ); + + repository.finalizeSnapshot( + shardGenerations, + repositoryData.getGenId(), + metadataForSnapshot(currentState.metadata(), request.includeGlobalState(), false, dataStreams, indexIds), + snapshotInfo, + version, + (Function) currentState, + new ActionListener() { + @Override + public void onResponse(RepositoryData repositoryData) { + if (!currentState.nodes().isLocalNodeElectedClusterManager()) { + failSnapshotCompletionListeners( + snapshot, + new SnapshotException(snapshot, "Aborting Snapshot, no longer cluster manager") + ); + throw new SnapshotException(repositoryName, snapshotName, "Aborting Snapshot, no longer cluster manager"); + + } + updateSnapshotPinnedTimestamp(repositoryData, snapshot, pinnedTimestamp, pinnedTimestampListener); + } + + @Override + public void onFailure(Exception e) { + logger.error("Failed to upload files to snapshot repo {} for snapshot {} ", repositoryName, snapshotName); + failSnapshotCompletionListeners(snapshot, new SnapshotException(snapshot, e.toString())); + listener.onFailure(e); + } + } + ); + + }, listener::onFailure); + } catch (Exception e) { + assert false : new AssertionError(e); + logger.error("Snapshot {} creation failed with exception {}", snapshot.getSnapshotId().getName(), e); + failSnapshotCompletionListeners(snapshot, new SnapshotException(snapshot, e.toString())); + } + } + + private void updateSnapshotPinnedTimestamp( + RepositoryData repositoryData, + Snapshot snapshot, + long timestampToPin, + ActionListener listener + ) { + // remoteStorePinnedTimestampService.pinTimestamp( + // timestampToPin, + // snapshot.getRepository() + "__" + snapshot.getSnapshotId(), + // new ActionListener() { + // @Override + // public void onResponse(Void unused) { + // logger.debug("Timestamp pinned successfully for snapshot {}", snapshot.getSnapshotId().getName()); + // listener.onResponse(repositoryData); + // } + // + // @Override + // public void onFailure(Exception e) { + // logger.error("Failed to pin timestamp for snapshot {} with exception {}", snapshot.getSnapshotId().getName(), e); + // listener.onFailure(e); + // + // } + // } + // ); + } + private static void ensureSnapshotNameNotRunning( List runningSnapshots, String repositoryName, @@ -899,15 +1099,21 @@ private static ShardGenerations buildGenerations(SnapshotsInProgress.Entry snaps return builder.build(); } - private static Metadata metadataForSnapshot(SnapshotsInProgress.Entry snapshot, Metadata metadata) { + private static Metadata metadataForSnapshot( + Metadata metadata, + boolean includeGlobalState, + boolean isPartial, + List dataStreamsList, + List indices + ) { final Metadata.Builder builder; - if (snapshot.includeGlobalState() == false) { + if (includeGlobalState == false) { // Remove global state from the cluster state builder = Metadata.builder(); - for (IndexId index : snapshot.indices()) { + for (IndexId index : indices) { final IndexMetadata indexMetadata = metadata.index(index.getName()); if (indexMetadata == null) { - assert snapshot.partial() : "Index [" + index + "] was deleted during a snapshot but snapshot was not partial."; + assert isPartial : "Index [" + index + "] was deleted during a snapshot but snapshot was not partial."; } else { builder.put(indexMetadata, false); } @@ -917,12 +1123,10 @@ private static Metadata metadataForSnapshot(SnapshotsInProgress.Entry snapshot, } // Only keep those data streams in the metadata that were actually requested by the initial snapshot create operation Map dataStreams = new HashMap<>(); - for (String dataStreamName : snapshot.dataStreams()) { + for (String dataStreamName : dataStreamsList) { DataStream dataStream = metadata.dataStreams().get(dataStreamName); if (dataStream == null) { - assert snapshot.partial() : "Data stream [" - + dataStreamName - + "] was deleted during a snapshot but snapshot was not partial."; + assert isPartial : "Data stream [" + dataStreamName + "] was deleted during a snapshot but snapshot was not partial."; } else { dataStreams.put(dataStreamName, dataStream); } @@ -1470,7 +1674,8 @@ private void finalizeSnapshotEntry(SnapshotsInProgress.Entry entry, Metadata met shardFailures, entry.includeGlobalState(), entry.userMetadata(), - entry.remoteStoreIndexShallowCopy() + entry.remoteStoreIndexShallowCopy(), + 0 ); final StepListener metadataListener = new StepListener<>(); final Repository repo = repositoriesService.repository(snapshot.getRepository()); @@ -1489,7 +1694,7 @@ private void finalizeSnapshotEntry(SnapshotsInProgress.Entry entry, Metadata met meta -> repo.finalizeSnapshot( shardGenerations, repositoryData.getGenId(), - metadataForSnapshot(entry, meta), + metadataForSnapshot(meta, entry.includeGlobalState(), entry.partial(), entry.dataStreams(), entry.indices()), snapshotInfo, entry.version(), state -> stateWithoutSnapshot(state, snapshot), @@ -2669,6 +2874,42 @@ private static Map shards( return Collections.unmodifiableMap(builder); } + private static ShardGenerations buildShardsGenerationFromRepositoryData( + Metadata metadata, + RoutingTable routingTable, + List indices, + RepositoryData repositoryData + ) { + ShardGenerations.Builder builder = ShardGenerations.builder(); + final ShardGenerations shardGenerations = repositoryData.shardGenerations(); + + for (IndexId index : indices) { + final String indexName = index.getName(); + final boolean isNewIndex = repositoryData.getIndices().containsKey(indexName) == false; + IndexMetadata indexMetadata = metadata.index(indexName); + + final IndexRoutingTable indexRoutingTable = routingTable.index(indexName); + for (int i = 0; i < indexMetadata.getNumberOfShards(); i++) { + final ShardId shardId = indexRoutingTable.shard(i).shardId(); + final String shardRepoGeneration; + + if (isNewIndex) { + assert shardGenerations.getShardGen(index, shardId.getId()) == null : "Found shard generation for new index [" + + index + + "]"; + shardRepoGeneration = ShardGenerations.NEW_SHARD_GEN; + } else { + shardRepoGeneration = shardGenerations.getShardGen(index, shardId.id()); + } + builder.put(index, shardId.id(), shardRepoGeneration); + + } + + } + + return builder.build(); + } + /** * Returns the data streams that are currently being snapshotted (with partial == false) and that are contained in the * indices-to-check set. diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/snapshots/create/CreateSnapshotResponseTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/snapshots/create/CreateSnapshotResponseTests.java index 274a548fd98ab..2feb0d3ba9405 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/snapshots/create/CreateSnapshotResponseTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/snapshots/create/CreateSnapshotResponseTests.java @@ -95,7 +95,8 @@ protected CreateSnapshotResponse createTestInstance() { shardFailures, globalState, SnapshotInfoTests.randomUserMetadata(), - false + false, + 0 ) ); } diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/snapshots/get/GetSnapshotsResponseTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/snapshots/get/GetSnapshotsResponseTests.java index 3ef143e36dab9..58af390d194d3 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/snapshots/get/GetSnapshotsResponseTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/snapshots/get/GetSnapshotsResponseTests.java @@ -77,7 +77,8 @@ protected GetSnapshotsResponse createTestInstance() { shardFailures, randomBoolean(), SnapshotInfoTests.randomUserMetadata(), - false + false, + 0 ) ); } diff --git a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java index e4e83f2453fa2..ef2c64f89d3a0 100644 --- a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java +++ b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java @@ -213,7 +213,8 @@ public void testSnapshotWithConflictingName() throws Exception { Collections.emptyList(), true, Collections.emptyMap(), - false + false, + 0 ), Version.CURRENT, Function.identity(), diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotInfoTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotInfoTests.java index 850a392c9619c..74b53723440e1 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotInfoTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotInfoTests.java @@ -86,7 +86,8 @@ protected SnapshotInfo createTestInstance() { shardFailures, includeGlobalState, userMetadata, - remoteStoreIndexShallowCopy + remoteStoreIndexShallowCopy, + 0 ); } @@ -114,7 +115,8 @@ protected SnapshotInfo mutateInstance(SnapshotInfo instance) { instance.shardFailures(), instance.includeGlobalState(), instance.userMetadata(), - instance.isRemoteStoreIndexShallowCopyEnabled() + instance.isRemoteStoreIndexShallowCopyEnabled(), + 0 ); case 1: int indicesSize = randomValueOtherThan(instance.indices().size(), () -> randomIntBetween(1, 10)); @@ -132,7 +134,8 @@ protected SnapshotInfo mutateInstance(SnapshotInfo instance) { instance.shardFailures(), instance.includeGlobalState(), instance.userMetadata(), - instance.isRemoteStoreIndexShallowCopyEnabled() + instance.isRemoteStoreIndexShallowCopyEnabled(), + 0 ); case 2: return new SnapshotInfo( @@ -146,7 +149,8 @@ protected SnapshotInfo mutateInstance(SnapshotInfo instance) { instance.shardFailures(), instance.includeGlobalState(), instance.userMetadata(), - instance.isRemoteStoreIndexShallowCopyEnabled() + instance.isRemoteStoreIndexShallowCopyEnabled(), + 0 ); case 3: return new SnapshotInfo( @@ -160,7 +164,8 @@ protected SnapshotInfo mutateInstance(SnapshotInfo instance) { instance.shardFailures(), instance.includeGlobalState(), instance.userMetadata(), - instance.isRemoteStoreIndexShallowCopyEnabled() + instance.isRemoteStoreIndexShallowCopyEnabled(), + 0 ); case 4: return new SnapshotInfo( @@ -174,7 +179,8 @@ protected SnapshotInfo mutateInstance(SnapshotInfo instance) { instance.shardFailures(), instance.includeGlobalState(), instance.userMetadata(), - instance.isRemoteStoreIndexShallowCopyEnabled() + instance.isRemoteStoreIndexShallowCopyEnabled(), + 0 ); case 5: int totalShards = randomValueOtherThan(instance.totalShards(), () -> randomIntBetween(0, 100)); @@ -200,7 +206,8 @@ protected SnapshotInfo mutateInstance(SnapshotInfo instance) { shardFailures, instance.includeGlobalState(), instance.userMetadata(), - instance.isRemoteStoreIndexShallowCopyEnabled() + instance.isRemoteStoreIndexShallowCopyEnabled(), + 0 ); case 6: return new SnapshotInfo( @@ -214,7 +221,8 @@ protected SnapshotInfo mutateInstance(SnapshotInfo instance) { instance.shardFailures(), Boolean.FALSE.equals(instance.includeGlobalState()), instance.userMetadata(), - instance.isRemoteStoreIndexShallowCopyEnabled() + instance.isRemoteStoreIndexShallowCopyEnabled(), + 0 ); case 7: return new SnapshotInfo( @@ -228,7 +236,8 @@ protected SnapshotInfo mutateInstance(SnapshotInfo instance) { instance.shardFailures(), instance.includeGlobalState(), randomValueOtherThan(instance.userMetadata(), SnapshotInfoTests::randomUserMetadata), - instance.isRemoteStoreIndexShallowCopyEnabled() + instance.isRemoteStoreIndexShallowCopyEnabled(), + 0 ); case 8: List dataStreams = randomValueOtherThan( @@ -246,7 +255,8 @@ protected SnapshotInfo mutateInstance(SnapshotInfo instance) { instance.shardFailures(), instance.includeGlobalState(), instance.userMetadata(), - instance.isRemoteStoreIndexShallowCopyEnabled() + instance.isRemoteStoreIndexShallowCopyEnabled(), + 0 ); case 9: return new SnapshotInfo( @@ -260,7 +270,8 @@ protected SnapshotInfo mutateInstance(SnapshotInfo instance) { instance.shardFailures(), instance.includeGlobalState(), instance.userMetadata(), - Boolean.FALSE.equals(instance.isRemoteStoreIndexShallowCopyEnabled()) + Boolean.FALSE.equals(instance.isRemoteStoreIndexShallowCopyEnabled()), + 0 ); default: throw new IllegalArgumentException("invalid randomization case"); diff --git a/server/src/test/java/org/opensearch/snapshots/mockstore/MockEventuallyConsistentRepositoryTests.java b/server/src/test/java/org/opensearch/snapshots/mockstore/MockEventuallyConsistentRepositoryTests.java index 43dde7281fb2d..c3577885e9cb1 100644 --- a/server/src/test/java/org/opensearch/snapshots/mockstore/MockEventuallyConsistentRepositoryTests.java +++ b/server/src/test/java/org/opensearch/snapshots/mockstore/MockEventuallyConsistentRepositoryTests.java @@ -233,7 +233,8 @@ public void testOverwriteSnapshotInfoBlob() throws Exception { Collections.emptyList(), true, Collections.emptyMap(), - false + false, + 0 ), Version.CURRENT, Function.identity(), @@ -259,7 +260,8 @@ public void testOverwriteSnapshotInfoBlob() throws Exception { Collections.emptyList(), true, Collections.emptyMap(), - false + false, + 0 ), Version.CURRENT, Function.identity(), @@ -287,7 +289,8 @@ public void testOverwriteSnapshotInfoBlob() throws Exception { Collections.emptyList(), true, Collections.emptyMap(), - false + false, + 0 ), Version.CURRENT, Function.identity(), diff --git a/test/framework/src/main/java/org/opensearch/snapshots/AbstractSnapshotIntegTestCase.java b/test/framework/src/main/java/org/opensearch/snapshots/AbstractSnapshotIntegTestCase.java index ce76914882150..b76de4e0b00cd 100644 --- a/test/framework/src/main/java/org/opensearch/snapshots/AbstractSnapshotIntegTestCase.java +++ b/test/framework/src/main/java/org/opensearch/snapshots/AbstractSnapshotIntegTestCase.java @@ -612,7 +612,8 @@ protected void addBwCFailedSnapshot(String repoName, String snapshotName, Mapget( f -> repo.finalizeSnapshot( From d9bbc6500b87e6fa90a6c8421f8ce88b8b24644a Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 6 Aug 2024 16:03:34 +0530 Subject: [PATCH 02/55] Fix deserilization error Signed-off-by: Anshu Agarwal --- .../org/opensearch/snapshots/SnapshotInfo.java | 11 +++++------ .../org/opensearch/snapshots/SnapshotsService.java | 14 +++++++++----- .../opensearch/snapshots/SnapshotInfoTests.java | 4 ++-- 3 files changed, 16 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java index 07c0e9133a635..7e50ebfa702c1 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java @@ -125,7 +125,7 @@ public static final class SnapshotInfoBuilder { private Boolean includeGlobalState = null; private Boolean remoteStoreIndexShallowCopy = null; - private long pinnedTimestamp; + private long pinnedTimestamp = 0L; private Map userMetadata = null; private int version = -1; private List shardFailures = null; @@ -351,7 +351,7 @@ public SnapshotInfo(SnapshotsInProgress.Entry entry) { entry.includeGlobalState(), entry.userMetadata(), entry.remoteStoreIndexShallowCopy(), - 0 + 0L ); } @@ -443,7 +443,7 @@ public SnapshotInfo(final StreamInput in) throws IOException { remoteStoreIndexShallowCopy = in.readOptionalBoolean(); } if (in.getVersion().onOrAfter(Version.V_2_16_0)) { - pinnedTimestamp = in.readLong(); + pinnedTimestamp = in.readVLong(); } } @@ -667,9 +667,8 @@ public XContentBuilder toXContent(final XContentBuilder builder, final Params pa if (remoteStoreIndexShallowCopy != null) { builder.field(REMOTE_STORE_INDEX_SHALLOW_COPY, remoteStoreIndexShallowCopy); } - if (pinnedTimestamp != 0) { - builder.field(PINNED_TIMESTAMP); - } + builder.field(PINNED_TIMESTAMP); + builder.startArray(INDICES); for (String index : indices) { builder.value(index); diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index ad0b029e13a92..65280d02714a9 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -300,7 +300,6 @@ public void startCreateSnapshot(final CreateSnapshotRequest request, final Actio * @param listener snapshot creation listener */ public void createSnapshot(final CreateSnapshotRequest request, final ActionListener listener) { - final String repositoryName = request.repository(); final String snapshotName = indexNameExpressionResolver.resolveDateMathExpression(request.snapshot()); validate(repositoryName, snapshotName); @@ -518,7 +517,7 @@ public void createShallowSnapshotV2(final CreateSnapshotRequest request, final A shardGenerations.indices().stream().map(IndexId::getName).collect(Collectors.toList()), dataStreams, pinnedTimestamp, - "", + null, System.currentTimeMillis(), shardGenerations.totalShards(), Collections.emptyList(), @@ -532,10 +531,14 @@ public void createShallowSnapshotV2(final CreateSnapshotRequest request, final A } final StepListener pinnedTimestampListener = new StepListener<>(); pinnedTimestampListener.whenComplete( - repoData -> completeListenersIgnoringException( + + repoData -> {completeListenersIgnoringException( endAndGetListenersToResolve(snapshot), Tuple.tuple(repoData, snapshotInfo) - ), + ); + listener.onResponse(snapshot); + }, + e -> failSnapshotCompletionListeners(snapshot, new SnapshotException(snapshot, e.toString())) ); @@ -545,7 +548,7 @@ public void createShallowSnapshotV2(final CreateSnapshotRequest request, final A metadataForSnapshot(currentState.metadata(), request.includeGlobalState(), false, dataStreams, indexIds), snapshotInfo, version, - (Function) currentState, + state -> stateWithoutSnapshot(clusterService.state(), snapshot), new ActionListener() { @Override public void onResponse(RepositoryData repositoryData) { @@ -583,6 +586,7 @@ private void updateSnapshotPinnedTimestamp( long timestampToPin, ActionListener listener ) { + listener.onResponse(repositoryData); // remoteStorePinnedTimestampService.pinTimestamp( // timestampToPin, // snapshot.getRepository() + "__" + snapshot.getSnapshotId(), diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotInfoTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotInfoTests.java index 74b53723440e1..684a8dd36fccc 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotInfoTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotInfoTests.java @@ -256,7 +256,7 @@ protected SnapshotInfo mutateInstance(SnapshotInfo instance) { instance.includeGlobalState(), instance.userMetadata(), instance.isRemoteStoreIndexShallowCopyEnabled(), - 0 + 123456 ); case 9: return new SnapshotInfo( @@ -271,7 +271,7 @@ protected SnapshotInfo mutateInstance(SnapshotInfo instance) { instance.includeGlobalState(), instance.userMetadata(), Boolean.FALSE.equals(instance.isRemoteStoreIndexShallowCopyEnabled()), - 0 + 123456 ); default: throw new IllegalArgumentException("invalid randomization case"); From f72702f8ad2c2e2fea272255ad57235897dbe03b Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 6 Aug 2024 16:53:19 +0530 Subject: [PATCH 03/55] Fix gradle spotless check Signed-off-by: Anshu Agarwal --- .../remotestore/RemoteRestoreSnapshotIT.java | 37 ++++++++++--------- .../snapshots/SnapshotsService.java | 19 +++++----- 2 files changed, 29 insertions(+), 27 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index 8b7f3c9687fd3..dcdcda1d58457 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -46,6 +46,7 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; @@ -748,7 +749,7 @@ public void testInvalidRestoreRequestScenarios() throws Exception { public void testCentralizedCreateAndRestoreShallowCopy() throws Exception { - Settings snapshotSettings = Settings.builder().put("snapshot.centralized_create_operation", true).build(); + Settings snapshotSettings = Settings.builder().put("snapshot.shallow_snapshot_v2", true).build(); internalCluster().startClusterManagerOnlyNode(Settings.builder().put(snapshotSettings).build()); internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); String indexName1 = "testindex1"; @@ -777,27 +778,27 @@ public void testCentralizedCreateAndRestoreShallowCopy() throws Exception { internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); logger.info("--> snapshot"); - SnapshotInfo snapshotInfo = createSnapshot(snapshotRepoName, snapshotName1, new ArrayList<>(Arrays.asList(indexName1, indexName2))); + SnapshotInfo snapshotInfo = createSnapshot(snapshotRepoName, snapshotName1, Collections.emptyList()); assertThat(snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); assertThat(snapshotInfo.successfulShards(), greaterThan(0)); assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards())); - // delete indices - DeleteResponse deleteResponse = client().prepareDelete(indexName1, "0").execute().actionGet(); - assertEquals(deleteResponse.getResult(), DocWriteResponse.Result.DELETED); - RestoreSnapshotResponse restoreSnapshotResponse1 = client.admin() - .cluster() - .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) - .setWaitForCompletion(false) - .setIndices(indexName1) - .setRenamePattern(indexName1) - .setRenameReplacement(restoredIndexName1) - .get(); - - assertEquals(restoreSnapshotResponse1.status(), RestStatus.ACCEPTED); - ensureYellowAndNoInitializingShards(restoredIndexName1); - ensureGreen(restoredIndexName1); - assertDocsPresentInIndex(client(), restoredIndexName1, numDocsInIndex1); + // // delete indices + // DeleteResponse deleteResponse = client().prepareDelete(indexName1, "0").execute().actionGet(); + // assertEquals(deleteResponse.getResult(), DocWriteResponse.Result.DELETED); + // RestoreSnapshotResponse restoreSnapshotResponse1 = client.admin() + // .cluster() + // .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + // .setWaitForCompletion(false) + // .setIndices(indexName1) + // .setRenamePattern(indexName1) + // .setRenameReplacement(restoredIndexName1) + // .get(); + // + // assertEquals(restoreSnapshotResponse1.status(), RestStatus.ACCEPTED); + // ensureYellowAndNoInitializingShards(restoredIndexName1); + // ensureGreen(restoredIndexName1); + // assertDocsPresentInIndex(client(), restoredIndexName1, numDocsInIndex1); } } diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index 65280d02714a9..5502c93919739 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -526,17 +526,15 @@ public void createShallowSnapshotV2(final CreateSnapshotRequest request, final A remoteStoreIndexShallowCopy, pinnedTimestamp ); - if (!currentState.nodes().isLocalNodeElectedClusterManager()) { + if (!clusterService.state().nodes().isLocalNodeElectedClusterManager()) { throw new SnapshotException(repositoryName, snapshotName, "Aborting Snapshot, no longer cluster manager"); } final StepListener pinnedTimestampListener = new StepListener<>(); pinnedTimestampListener.whenComplete( - repoData -> {completeListenersIgnoringException( - endAndGetListenersToResolve(snapshot), - Tuple.tuple(repoData, snapshotInfo) - ); - listener.onResponse(snapshot); + repoData -> { + completeListenersIgnoringException(endAndGetListenersToResolve(snapshot), Tuple.tuple(repoData, snapshotInfo)); + listener.onResponse(snapshot); }, e -> failSnapshotCompletionListeners(snapshot, new SnapshotException(snapshot, e.toString())) @@ -552,13 +550,15 @@ public void createShallowSnapshotV2(final CreateSnapshotRequest request, final A new ActionListener() { @Override public void onResponse(RepositoryData repositoryData) { - if (!currentState.nodes().isLocalNodeElectedClusterManager()) { + if (!clusterService.state().nodes().isLocalNodeElectedClusterManager()) { failSnapshotCompletionListeners( snapshot, new SnapshotException(snapshot, "Aborting Snapshot, no longer cluster manager") ); - throw new SnapshotException(repositoryName, snapshotName, "Aborting Snapshot, no longer cluster manager"); - + listener.onFailure( + new SnapshotException(repositoryName, snapshotName, "Aborting Snapshot, no longer cluster manager") + ); + return; } updateSnapshotPinnedTimestamp(repositoryData, snapshot, pinnedTimestamp, pinnedTimestampListener); } @@ -577,6 +577,7 @@ public void onFailure(Exception e) { assert false : new AssertionError(e); logger.error("Snapshot {} creation failed with exception {}", snapshot.getSnapshotId().getName(), e); failSnapshotCompletionListeners(snapshot, new SnapshotException(snapshot, e.toString())); + listener.onFailure(e); } } From 8e852312348df75b5b0886e6c83fab1cbcaa3006 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 6 Aug 2024 17:30:11 +0530 Subject: [PATCH 04/55] Fix listener Signed-off-by: Anshu Agarwal --- .../remotestore/RemoteRestoreSnapshotIT.java | 2 +- .../snapshots/SnapshotsService.java | 31 ++++++++----------- 2 files changed, 14 insertions(+), 19 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index dcdcda1d58457..dc4cc6339fc57 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -747,7 +747,7 @@ public void testInvalidRestoreRequestScenarios() throws Exception { assertTrue(exception.getMessage().contains("cannot remove setting [index.remote_store.segment.repository]" + " on restore")); } - public void testCentralizedCreateAndRestoreShallowCopy() throws Exception { + public void testCreateShallowCopyV2() throws Exception { Settings snapshotSettings = Settings.builder().put("snapshot.shallow_snapshot_v2", true).build(); internalCluster().startClusterManagerOnlyNode(Settings.builder().put(snapshotSettings).build()); diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index 5502c93919739..bc725c2f851c7 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -266,13 +266,10 @@ public boolean isShallowSnapV2() { * @param listener snapshot completion listener */ public void executeSnapshot(final CreateSnapshotRequest request, final ActionListener listener) { - startCreateSnapshot( - request, - ActionListener.wrap(snapshot -> addListener(snapshot, ActionListener.map(listener, Tuple::v2)), listener::onFailure) - ); + startCreateSnapshot(request, listener); } - public void startCreateSnapshot(final CreateSnapshotRequest request, final ActionListener listener) { + public void startCreateSnapshot(final CreateSnapshotRequest request, final ActionListener listener) { Repository repository = repositoriesService.repository(request.repository()); boolean remoteStoreIndexShallowCopy = REMOTE_STORE_INDEX_SHALLOW_COPY.get(repository.getMetadata().settings()); logger.debug("remote_store_index_shallow_copy setting is set as [{}]", remoteStoreIndexShallowCopy); @@ -286,7 +283,10 @@ public void startCreateSnapshot(final CreateSnapshotRequest request, final Actio if (remoteStoreIndexShallowCopy && isShallowSnapV2 && request.indices().length == 0) { createShallowSnapshotV2(request, listener); } else { - createSnapshot(request, listener); + createSnapshot( + request, + ActionListener.wrap(snapshot -> addListener(snapshot, ActionListener.map(listener, Tuple::v2)), listener::onFailure) + ); } } @@ -440,7 +440,7 @@ public TimeValue timeout() { }, "create_snapshot [" + snapshotName + ']', listener::onFailure); } - public void createShallowSnapshotV2(final CreateSnapshotRequest request, final ActionListener listener) { + public void createShallowSnapshotV2(final CreateSnapshotRequest request, final ActionListener listener) { long pinnedTimestamp = System.currentTimeMillis(); final String repositoryName = request.repository(); final String snapshotName = indexNameExpressionResolver.resolveDateMathExpression(request.snapshot()); @@ -530,15 +530,10 @@ public void createShallowSnapshotV2(final CreateSnapshotRequest request, final A throw new SnapshotException(repositoryName, snapshotName, "Aborting Snapshot, no longer cluster manager"); } final StepListener pinnedTimestampListener = new StepListener<>(); - pinnedTimestampListener.whenComplete( - - repoData -> { - completeListenersIgnoringException(endAndGetListenersToResolve(snapshot), Tuple.tuple(repoData, snapshotInfo)); - listener.onResponse(snapshot); - }, - - e -> failSnapshotCompletionListeners(snapshot, new SnapshotException(snapshot, e.toString())) - ); + pinnedTimestampListener.whenComplete(repoData -> { + // completeListenersIgnoringException(endAndGetListenersToResolve(snapshot), Tuple.tuple(repoData, snapshotInfo)); + listener.onResponse(snapshotInfo); + }, listener::onFailure); repository.finalizeSnapshot( shardGenerations, @@ -566,7 +561,7 @@ public void onResponse(RepositoryData repositoryData) { @Override public void onFailure(Exception e) { logger.error("Failed to upload files to snapshot repo {} for snapshot {} ", repositoryName, snapshotName); - failSnapshotCompletionListeners(snapshot, new SnapshotException(snapshot, e.toString())); + // failSnapshotCompletionListeners(snapshot, new SnapshotException(snapshot, e.toString())); listener.onFailure(e); } } @@ -576,7 +571,7 @@ public void onFailure(Exception e) { } catch (Exception e) { assert false : new AssertionError(e); logger.error("Snapshot {} creation failed with exception {}", snapshot.getSnapshotId().getName(), e); - failSnapshotCompletionListeners(snapshot, new SnapshotException(snapshot, e.toString())); + // failSnapshotCompletionListeners(snapshot, new SnapshotException(snapshot, e.toString())); listener.onFailure(e); } } From 2d404e8b2c8db09c3af22310c56e947a406dba3a Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Wed, 7 Aug 2024 10:35:26 +0530 Subject: [PATCH 05/55] Fix test Signed-off-by: Anshu Agarwal --- server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java index 7e50ebfa702c1..a2a733593f5f1 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java @@ -667,7 +667,7 @@ public XContentBuilder toXContent(final XContentBuilder builder, final Params pa if (remoteStoreIndexShallowCopy != null) { builder.field(REMOTE_STORE_INDEX_SHALLOW_COPY, remoteStoreIndexShallowCopy); } - builder.field(PINNED_TIMESTAMP); + builder.field(PINNED_TIMESTAMP, pinnedTimestamp); builder.startArray(INDICES); for (String index : indices) { From 90c860c7c893c05ac61eec9c49bb1a2193912ec2 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Thu, 8 Aug 2024 13:22:03 +0530 Subject: [PATCH 06/55] Fix snapshot generation Signed-off-by: Anshu Agarwal --- .../src/main/java/org/opensearch/snapshots/SnapshotInfo.java | 4 ++-- .../main/java/org/opensearch/snapshots/SnapshotsService.java | 3 +-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java index a2a733593f5f1..3325396a8b8c3 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java @@ -442,7 +442,7 @@ public SnapshotInfo(final StreamInput in) throws IOException { if (in.getVersion().onOrAfter(Version.V_2_9_0)) { remoteStoreIndexShallowCopy = in.readOptionalBoolean(); } - if (in.getVersion().onOrAfter(Version.V_2_16_0)) { + if (in.getVersion().onOrAfter(Version.V_2_17_0)) { pinnedTimestamp = in.readVLong(); } } @@ -907,7 +907,7 @@ public void writeTo(final StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_2_9_0)) { out.writeOptionalBoolean(remoteStoreIndexShallowCopy); } - if (out.getVersion().onOrAfter(Version.V_2_16_0)) { + if (out.getVersion().onOrAfter(Version.V_2_17_0)) { out.writeVLong(pinnedTimestamp); } } diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index bc725c2f851c7..f995d71e208e1 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -534,14 +534,13 @@ public void createShallowSnapshotV2(final CreateSnapshotRequest request, final A // completeListenersIgnoringException(endAndGetListenersToResolve(snapshot), Tuple.tuple(repoData, snapshotInfo)); listener.onResponse(snapshotInfo); }, listener::onFailure); - repository.finalizeSnapshot( shardGenerations, repositoryData.getGenId(), metadataForSnapshot(currentState.metadata(), request.includeGlobalState(), false, dataStreams, indexIds), snapshotInfo, version, - state -> stateWithoutSnapshot(clusterService.state(), snapshot), + state -> stateWithoutSnapshot(state, snapshot), new ActionListener() { @Override public void onResponse(RepositoryData repositoryData) { From 193da651fabe98276f706306326e60afd25df99f Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Wed, 14 Aug 2024 14:48:17 +0530 Subject: [PATCH 07/55] Modify cluster setting name Signed-off-by: Anshu Agarwal --- .../remotestore/RemoteRestoreSnapshotIT.java | 23 +++--------------- .../common/settings/ClusterSettings.java | 2 +- .../snapshots/SnapshotsService.java | 24 +++++++++---------- 3 files changed, 16 insertions(+), 33 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index dc4cc6339fc57..6b362d2f06afd 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -747,18 +747,18 @@ public void testInvalidRestoreRequestScenarios() throws Exception { assertTrue(exception.getMessage().contains("cannot remove setting [index.remote_store.segment.repository]" + " on restore")); } - public void testCreateShallowCopyV2() throws Exception { + public void testCreateSnapshotV2() throws Exception { - Settings snapshotSettings = Settings.builder().put("snapshot.shallow_snapshot_v2", true).build(); + Settings snapshotSettings = Settings.builder().put("snapshot.snapshot_v2", true).build(); internalCluster().startClusterManagerOnlyNode(Settings.builder().put(snapshotSettings).build()); internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); + internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); String indexName1 = "testindex1"; String indexName2 = "testindex2"; String snapshotRepoName = "test-create-snapshot-repo"; String snapshotName1 = "test-create-snapshot1"; Path absolutePath1 = randomRepoPath().toAbsolutePath(); logger.info("Snapshot Path [{}]", absolutePath1); - String restoredIndexName1 = indexName1 + "-restored"; createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, true)); @@ -775,7 +775,6 @@ public void testCreateShallowCopyV2() throws Exception { indexDocuments(client, indexName2, numDocsInIndex2); ensureGreen(indexName1, indexName2); - internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); logger.info("--> snapshot"); SnapshotInfo snapshotInfo = createSnapshot(snapshotRepoName, snapshotName1, Collections.emptyList()); @@ -783,22 +782,6 @@ public void testCreateShallowCopyV2() throws Exception { assertThat(snapshotInfo.successfulShards(), greaterThan(0)); assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards())); - // // delete indices - // DeleteResponse deleteResponse = client().prepareDelete(indexName1, "0").execute().actionGet(); - // assertEquals(deleteResponse.getResult(), DocWriteResponse.Result.DELETED); - // RestoreSnapshotResponse restoreSnapshotResponse1 = client.admin() - // .cluster() - // .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) - // .setWaitForCompletion(false) - // .setIndices(indexName1) - // .setRenamePattern(indexName1) - // .setRenameReplacement(restoredIndexName1) - // .get(); - // - // assertEquals(restoreSnapshotResponse1.status(), RestStatus.ACCEPTED); - // ensureYellowAndNoInitializingShards(restoredIndexName1); - // ensureGreen(restoredIndexName1); - // assertDocsPresentInIndex(client(), restoredIndexName1, numDocsInIndex1); } } diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index 27a23d32f399e..373c5a1e7810c 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -632,7 +632,7 @@ public void apply(Settings value, Settings current, Settings previous) { HandshakingTransportAddressConnector.PROBE_CONNECT_TIMEOUT_SETTING, HandshakingTransportAddressConnector.PROBE_HANDSHAKE_TIMEOUT_SETTING, SnapshotsService.MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING, - SnapshotsService.SHALLOW_SNAPSHOT_V2, + SnapshotsService.SNAPSHOT_V2, FsHealthService.ENABLED_SETTING, FsHealthService.REFRESH_INTERVAL_SETTING, FsHealthService.SLOW_PATH_LOGGING_THRESHOLD_SETTING, diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index f995d71e208e1..120066e806138 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -200,8 +200,8 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus Setting.Property.Dynamic ); - public static final Setting SHALLOW_SNAPSHOT_V2 = Setting.boolSetting( - "snapshot.shallow_snapshot_v2", + public static final Setting SNAPSHOT_V2 = Setting.boolSetting( + "snapshot.snapshot_v2", false, Setting.Property.Dynamic, Setting.Property.NodeScope @@ -209,7 +209,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus private volatile int maxConcurrentOperations; - private volatile boolean isShallowSnapV2; + private volatile boolean isSnapshotV2; public SnapshotsService( Settings settings, @@ -240,8 +240,8 @@ public SnapshotsService( maxConcurrentOperations = MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING.get(settings); clusterService.getClusterSettings() .addSettingsUpdateConsumer(MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING, i -> maxConcurrentOperations = i); - isShallowSnapV2 = SHALLOW_SNAPSHOT_V2.get(settings); - clusterService.getClusterSettings().addSettingsUpdateConsumer(SHALLOW_SNAPSHOT_V2, this::setShallowSnapshotV2); + isSnapshotV2 = SNAPSHOT_V2.get(settings); + clusterService.getClusterSettings().addSettingsUpdateConsumer(SNAPSHOT_V2, this::setSnapshotV2); } // Task is onboarded for throttling, it will get retried from associated TransportClusterManagerNodeAction. @@ -250,12 +250,12 @@ public SnapshotsService( updateSnapshotStateTaskKey = clusterService.registerClusterManagerTask(ClusterManagerTaskKeys.UPDATE_SNAPSHOT_STATE_KEY, true); } - private void setShallowSnapshotV2(boolean isShallowSnapV2) { - this.isShallowSnapV2 = isShallowSnapV2; + private void setSnapshotV2(boolean isSnapshotV2) { + this.isSnapshotV2 = isSnapshotV2; } - public boolean isShallowSnapV2() { - return isShallowSnapV2; + public boolean isSnapshotV2() { + return isSnapshotV2; } /** @@ -280,8 +280,8 @@ public void startCreateSnapshot(final CreateSnapshotRequest request, final Actio logger.warn("Shallow snapshots are not supported during migration. Falling back to full snapshot."); remoteStoreIndexShallowCopy = false; } - if (remoteStoreIndexShallowCopy && isShallowSnapV2 && request.indices().length == 0) { - createShallowSnapshotV2(request, listener); + if (remoteStoreIndexShallowCopy && isSnapshotV2 && request.indices().length == 0) { + createSnapshotV2(request, listener); } else { createSnapshot( request, @@ -440,7 +440,7 @@ public TimeValue timeout() { }, "create_snapshot [" + snapshotName + ']', listener::onFailure); } - public void createShallowSnapshotV2(final CreateSnapshotRequest request, final ActionListener listener) { + public void createSnapshotV2(final CreateSnapshotRequest request, final ActionListener listener) { long pinnedTimestamp = System.currentTimeMillis(); final String repositoryName = request.repository(); final String snapshotName = indexNameExpressionResolver.resolveDateMathExpression(request.snapshot()); From fe2aaaf903dd2c5b008531c50b8b120a58615e87 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Wed, 14 Aug 2024 16:53:53 +0530 Subject: [PATCH 08/55] Add more tests Signed-off-by: Anshu Agarwal --- .../remotestore/RemoteRestoreSnapshotIT.java | 26 ++++++++++++++++-- .../create/TransportCreateSnapshotAction.java | 3 ++- .../opensearch/snapshots/SnapshotInfo.java | 3 +-- .../snapshots/SnapshotsService.java | 27 ++++++++++++++----- 4 files changed, 48 insertions(+), 11 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index 6b362d2f06afd..e83cc1f972d96 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -10,6 +10,7 @@ import org.opensearch.action.DocWriteResponse; import org.opensearch.action.admin.cluster.remotestore.restore.RestoreRemoteStoreRequest; +import org.opensearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; import org.opensearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; import org.opensearch.action.admin.indices.delete.DeleteIndexRequest; import org.opensearch.action.delete.DeleteResponse; @@ -755,6 +756,7 @@ public void testCreateSnapshotV2() throws Exception { internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); String indexName1 = "testindex1"; String indexName2 = "testindex2"; + String indexName3 = "testindex3"; String snapshotRepoName = "test-create-snapshot-repo"; String snapshotName1 = "test-create-snapshot1"; Path absolutePath1 = randomRepoPath().toAbsolutePath(); @@ -775,13 +777,33 @@ public void testCreateSnapshotV2() throws Exception { indexDocuments(client, indexName2, numDocsInIndex2); ensureGreen(indexName1, indexName2); - logger.info("--> snapshot"); - SnapshotInfo snapshotInfo = createSnapshot(snapshotRepoName, snapshotName1, Collections.emptyList()); assertThat(snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); assertThat(snapshotInfo.successfulShards(), greaterThan(0)); assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards())); + // TODO - verify pinned timestamp + indexDocuments(client, indexName1, 10); + indexDocuments(client, indexName2, 20); + + createIndex(indexName3, indexSettings); + indexDocuments(client, indexName3, 10); + + String snapshotName2 = "test-create-snapshot2"; + + // verify even if waitForCompletion is not true, the request executes in a sync manner + CreateSnapshotResponse createSnapshotResponse2 = client().admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, snapshotName2) + .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)); + + // TODO - verify pinned timestamp + } } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java index 04d20a9616031..d0da00bb10308 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java @@ -103,7 +103,8 @@ protected void clusterManagerOperation( ClusterState state, final ActionListener listener ) { - if (request.waitForCompletion()) { + Boolean isSnapshotV2 = clusterService.getClusterSettings().get(SnapshotsService.SNAPSHOT_V2); + if (request.waitForCompletion() || isSnapshotV2) { snapshotsService.executeSnapshot(request, ActionListener.map(listener, CreateSnapshotResponse::new)); } else { snapshotsService.startCreateSnapshot(request, ActionListener.map(listener, snapshot -> new CreateSnapshotResponse())); diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java index 3325396a8b8c3..3efe50d0a06f9 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java @@ -100,12 +100,11 @@ public final class SnapshotInfo implements Comparable, ToXContent, private static final String REMOTE_STORE_INDEX_SHALLOW_COPY = "remote_store_index_shallow_copy"; private static final String PINNED_TIMESTAMP = "pinned_timestamp"; + private static final String USER_METADATA = "metadata"; private static final Comparator COMPARATOR = Comparator.comparing(SnapshotInfo::startTime) .thenComparing(SnapshotInfo::snapshotId); - private XContentBuilder builder; - private XContentBuilder build; /** * Builds snapshot information diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index 120066e806138..c3f610128a033 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -269,6 +269,14 @@ public void executeSnapshot(final CreateSnapshotRequest request, final ActionLis startCreateSnapshot(request, listener); } + /** + * This method calls {@link #createSnapshot(CreateSnapshotRequest, ActionListener)} to create snapshot if snapshot + * V2 is not enabled. + * For V2 enabled snapshots, {@link #createSnapshotV2(CreateSnapshotRequest, ActionListener)} is called and + * appropriate listeners are mapped + * @param request snapshot request + * @param listener snapshot completion listener + */ public void startCreateSnapshot(final CreateSnapshotRequest request, final ActionListener listener) { Repository repository = repositoriesService.repository(request.repository()); boolean remoteStoreIndexShallowCopy = REMOTE_STORE_INDEX_SHALLOW_COPY.get(repository.getMetadata().settings()); @@ -440,6 +448,18 @@ public TimeValue timeout() { }, "create_snapshot [" + snapshotName + ']', listener::onFailure); } + /** + * Initializes the snapshotting process for clients when Snapshot v2 is enabled. This method is responsible for taking + * a shallow snapshot and pinning the snapshot timestamp.The entire process is executed on the cluster manager node. + * + * Unlike traditional snapshot operations, this method performs a synchronous snapshot execution and doesn't + * upload any shard metadata to the snapshot repository. + * The pinned timestamp is later reconciled with remote store segment and translog metadata files during the restore + * operation. + * + * @param request snapshot request + * @param listener snapshot creation listener + */ public void createSnapshotV2(final CreateSnapshotRequest request, final ActionListener listener) { long pinnedTimestamp = System.currentTimeMillis(); final String repositoryName = request.repository(); @@ -530,10 +550,7 @@ public void createSnapshotV2(final CreateSnapshotRequest request, final ActionLi throw new SnapshotException(repositoryName, snapshotName, "Aborting Snapshot, no longer cluster manager"); } final StepListener pinnedTimestampListener = new StepListener<>(); - pinnedTimestampListener.whenComplete(repoData -> { - // completeListenersIgnoringException(endAndGetListenersToResolve(snapshot), Tuple.tuple(repoData, snapshotInfo)); - listener.onResponse(snapshotInfo); - }, listener::onFailure); + pinnedTimestampListener.whenComplete(repoData -> { listener.onResponse(snapshotInfo); }, listener::onFailure); repository.finalizeSnapshot( shardGenerations, repositoryData.getGenId(), @@ -560,7 +577,6 @@ public void onResponse(RepositoryData repositoryData) { @Override public void onFailure(Exception e) { logger.error("Failed to upload files to snapshot repo {} for snapshot {} ", repositoryName, snapshotName); - // failSnapshotCompletionListeners(snapshot, new SnapshotException(snapshot, e.toString())); listener.onFailure(e); } } @@ -570,7 +586,6 @@ public void onFailure(Exception e) { } catch (Exception e) { assert false : new AssertionError(e); logger.error("Snapshot {} creation failed with exception {}", snapshot.getSnapshotId().getName(), e); - // failSnapshotCompletionListeners(snapshot, new SnapshotException(snapshot, e.toString())); listener.onFailure(e); } } From afcc37aab7737bb91586c273d0de318f9dc3655a Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Wed, 14 Aug 2024 23:33:44 +0530 Subject: [PATCH 09/55] Snapshot V2 delete changes Signed-off-by: Anshu Agarwal --- .../remotestore/RemoteRestoreSnapshotIT.java | 82 ++++++++++++++++++ .../opensearch/repositories/Repository.java | 1 + .../blobstore/BlobStoreRepository.java | 27 ++++-- .../snapshots/SnapshotsService.java | 84 ++++++++++++++++--- .../blobstore/BlobStoreTestUtil.java | 15 ++-- 5 files changed, 182 insertions(+), 27 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index e83cc1f972d96..45d5ed7511872 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -15,6 +15,7 @@ import org.opensearch.action.admin.indices.delete.DeleteIndexRequest; import org.opensearch.action.delete.DeleteResponse; import org.opensearch.action.support.PlainActionFuture; +import org.opensearch.action.support.master.AcknowledgedResponse; import org.opensearch.client.Client; import org.opensearch.client.Requests; import org.opensearch.cluster.ClusterState; @@ -24,6 +25,7 @@ import org.opensearch.common.io.PathUtils; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.common.unit.ByteSizeUnit; import org.opensearch.core.index.Index; import org.opensearch.core.rest.RestStatus; import org.opensearch.index.IndexService; @@ -33,6 +35,8 @@ import org.opensearch.index.shard.IndexShard; import org.opensearch.indices.IndicesService; import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.repositories.fs.FsRepository; import org.opensearch.snapshots.AbstractSnapshotIntegTestCase; import org.opensearch.snapshots.SnapshotInfo; import org.opensearch.snapshots.SnapshotRestoreException; @@ -806,4 +810,82 @@ public void testCreateSnapshotV2() throws Exception { } + public void testDeleteShallowCopyV2() throws Exception { + + Settings snapshotSettings = Settings.builder().put("snapshot.snapshot_v2", true).build(); + // Settings snapshotSettings = Settings.builder().put("snapshot.max_concurrent_operations", 1000).build(); + internalCluster().startClusterManagerOnlyNode(Settings.builder().put(snapshotSettings).build()); + internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); + internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); + + 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) + ) + ); + + Settings indexSettings = getIndexSettings(20, 0).build(); + createIndex(indexName1, indexSettings); + + Settings indexSettings2 = getIndexSettings(15, 0).build(); + createIndex(indexName2, indexSettings2); + + final int numDocsInIndex1 = 10; + final int numDocsInIndex2 = 20; + indexDocuments(client, indexName1, numDocsInIndex1); + indexDocuments(client, indexName2, numDocsInIndex2); + ensureGreen(indexName1, indexName2); + + CreateSnapshotResponse createSnapshotResponse = client().admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, snapshotName1) + .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)); + + indexDocuments(client, indexName2, 10); + indexDocuments(client, indexName1, 10); + createIndex(indexName3); + indexDocuments(client, 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, snapshotName2) + .setSnapshots(snapshotName2) + .get(); + assertTrue(deleteResponse.isAcknowledged()); + } + } diff --git a/server/src/main/java/org/opensearch/repositories/Repository.java b/server/src/main/java/org/opensearch/repositories/Repository.java index ed30aad7b4dd2..d6d8c725f339b 100644 --- a/server/src/main/java/org/opensearch/repositories/Repository.java +++ b/server/src/main/java/org/opensearch/repositories/Repository.java @@ -191,6 +191,7 @@ default void deleteSnapshotsAndReleaseLockFiles( long repositoryStateId, Version repositoryMetaVersion, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, + boolean isSnapshotV2, ActionListener listener ) { throw new UnsupportedOperationException(); 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 c4908f8c5fc4b..e44d4ae07c4a6 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -897,6 +897,7 @@ public void deleteSnapshotsAndReleaseLockFiles( long repositoryStateId, Version repositoryMetaVersion, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, + boolean isSnapshotV2, ActionListener listener ) { if (isReadOnly()) { @@ -918,6 +919,7 @@ protected void doRun() throws Exception { repositoryData, repositoryMetaVersion, remoteStoreLockManagerFactory, + isSnapshotV2, listener ); } @@ -942,6 +944,7 @@ public void deleteSnapshots( repositoryStateId, repositoryMetaVersion, null, // Passing null since no remote store lock files need to be cleaned up. + false, listener ); } @@ -1016,6 +1019,7 @@ private void doDeleteShardSnapshots( RepositoryData repositoryData, Version repoMetaVersion, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, + boolean isSnapshotV2, ActionListener listener ) { // First write the new shard state metadata (with the removed snapshot) and compute deletion targets @@ -1051,10 +1055,14 @@ private void doDeleteShardSnapshots( }, listener::onFailure); // Once we have updated the repository, run the clean-ups writeUpdatedRepoDataStep.whenComplete(updatedRepoData -> { + int groupSize = 2; + if (isSnapshotV2) { + groupSize = 1; + } // Run unreferenced blobs cleanup in parallel to shard-level snapshot deletion final ActionListener afterCleanupsListener = new GroupedActionListener<>( ActionListener.wrap(() -> listener.onResponse(updatedRepoData)), - 2 + groupSize ); cleanupUnlinkedRootAndIndicesBlobs( snapshotIds, @@ -1064,13 +1072,16 @@ private void doDeleteShardSnapshots( remoteStoreLockManagerFactory, afterCleanupsListener ); - asyncCleanupUnlinkedShardLevelBlobs( - repositoryData, - snapshotIds, - writeShardMetaDataAndComputeDeletesStep.result(), - remoteStoreLockManagerFactory, - afterCleanupsListener - ); + if (!isSnapshotV2) { + asyncCleanupUnlinkedShardLevelBlobs( + repositoryData, + snapshotIds, + writeShardMetaDataAndComputeDeletesStep.result(), + remoteStoreLockManagerFactory, + afterCleanupsListener + ); + } + }, listener::onFailure); } diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index c3f610128a033..9fa3660734704 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -122,6 +122,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; @@ -617,6 +618,15 @@ private void updateSnapshotPinnedTimestamp( // ); } + private void removeSnapshotPinnedTimestamp( + RepositoryData repositoryData, + Snapshot snapshot, + long timestampToPin, + ActionListener listener + ) { + + } + private static void ensureSnapshotNameNotRunning( List runningSnapshots, String repositoryName, @@ -2462,18 +2472,68 @@ 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) { + List snapshotsWithPinnedTimestamp = Collections.synchronizedList(new ArrayList<>()); + 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.add(snapshotId); + } else { + snapshotsWithLockFiles.add(snapshotId); + } + } catch (Exception e) { + logger.warn("Failed to get snapshot info for {}", snapshotId, e); + removeSnapshotDeletionFromClusterState(deleteEntry, e, repositoryData); + } + } + } finally { + latch.countDown(); + } + }); + try { + latch.await(); + if (snapshotsWithLockFiles.size() > 0) { + repository.deleteSnapshotsAndReleaseLockFiles( + snapshotIds, + repositoryData.getGenId(), + minCompatibleVersion(minNodeVersion, repositoryData, snapshotIds), + remoteStoreLockManagerFactory, + false, + ActionListener.wrap(updatedRepoData -> { + logger.info("snapshots {} deleted", snapshotsWithLockFiles); + removeSnapshotDeletionFromClusterState(deleteEntry, null, updatedRepoData); + }, ex -> removeSnapshotDeletionFromClusterState(deleteEntry, ex, repositoryData)) + ); + } + if (snapshotsWithPinnedTimestamp.size() > 0) { + repository.deleteSnapshotsAndReleaseLockFiles( + snapshotIds, + repositoryData.getGenId(), + minCompatibleVersion(minNodeVersion, repositoryData, snapshotIds), + null, + true, + ActionListener.wrap(updatedRepoData -> { + logger.info("snapshots {} deleted", snapshotsWithPinnedTimestamp); + // removeSnapshotPinnedTimestamp(); + 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/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java b/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java index 32f445bf24a41..14c5433926c0f 100644 --- a/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java +++ b/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java @@ -298,15 +298,16 @@ private static void assertSnapshotUUIDs(BlobStoreRepository repository, Reposito .stream() .noneMatch(shardFailure -> shardFailure.index().equals(index) && shardFailure.shardId() == shardId)) { final Map shardPathContents = shardContainer.listBlobs(); - - assertTrue( - shardPathContents.containsKey( - String.format(Locale.ROOT, BlobStoreRepository.SHALLOW_SNAPSHOT_NAME_FORMAT, snapshotId.getUUID()) - ) - || shardPathContents.containsKey( + if (snapshotInfo.getPinnedTimestamp() == 0) { + assertTrue( + shardPathContents.containsKey( + String.format(Locale.ROOT, BlobStoreRepository.SHALLOW_SNAPSHOT_NAME_FORMAT, snapshotId.getUUID()) + ) + || shardPathContents.containsKey( String.format(Locale.ROOT, BlobStoreRepository.SNAPSHOT_NAME_FORMAT, snapshotId.getUUID()) ) - ); + ); + } assertThat( shardPathContents.keySet() From 6504169602c532a931a3c80e31857558c9b2a4fc Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 20 Aug 2024 20:01:58 +0530 Subject: [PATCH 10/55] Uncomment pin timestamp code Signed-off-by: Anshu Agarwal --- .../snapshots/SnapshotsService.java | 37 +++++++++---------- 1 file changed, 18 insertions(+), 19 deletions(-) diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index 3946a3764e71d..74591ef8fc9cf 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -561,7 +561,7 @@ public void createSnapshotV2(final CreateSnapshotRequest request, final ActionLi metadataForSnapshot(currentState.metadata(), request.includeGlobalState(), false, dataStreams, indexIds), snapshotInfo, version, - state -> stateWithoutSnapshot(state, snapshot), + state -> state, new ActionListener() { @Override public void onResponse(RepositoryData repositoryData) { @@ -601,24 +601,23 @@ private void updateSnapshotPinnedTimestamp( ActionListener listener ) { listener.onResponse(repositoryData); - // remoteStorePinnedTimestampService.pinTimestamp( - // timestampToPin, - // snapshot.getRepository() + "__" + snapshot.getSnapshotId(), - // new ActionListener() { - // @Override - // public void onResponse(Void unused) { - // logger.debug("Timestamp pinned successfully for snapshot {}", snapshot.getSnapshotId().getName()); - // listener.onResponse(repositoryData); - // } - // - // @Override - // public void onFailure(Exception e) { - // logger.error("Failed to pin timestamp for snapshot {} with exception {}", snapshot.getSnapshotId().getName(), e); - // listener.onFailure(e); - // - // } - // } - // ); + remoteStorePinnedTimestampService.pinTimestamp( + timestampToPin, + snapshot.getRepository() + "__" + snapshot.getSnapshotId(), + new ActionListener() { + @Override + public void onResponse(Void unused) { + logger.debug("Timestamp pinned successfully for snapshot {}", snapshot.getSnapshotId().getName()); + listener.onResponse(repositoryData); + } + @Override + public void onFailure(Exception e) { + logger.error("Failed to pin timestamp for snapshot {} with exception {}", snapshot.getSnapshotId().getName(), e); + listener.onFailure(e); + + } + } + ); } private static void ensureSnapshotNameNotRunning( From c91f1b56a00ece9ae278d359d341166c4dd64b27 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 20 Aug 2024 23:47:59 +0530 Subject: [PATCH 11/55] Add code to unpin timestamp on deletion Signed-off-by: Anshu Agarwal --- .../snapshots/SnapshotsService.java | 80 ++++++++++++++++--- .../blobstore/BlobStoreTestUtil.java | 4 +- 2 files changed, 69 insertions(+), 15 deletions(-) diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index 55b20f8c5352b..978d48d6f0da1 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -601,16 +601,16 @@ private void updateSnapshotPinnedTimestamp( long timestampToPin, ActionListener listener ) { - listener.onResponse(repositoryData); remoteStorePinnedTimestampService.pinTimestamp( timestampToPin, - snapshot.getRepository() + "__" + snapshot.getSnapshotId(), + snapshot.getRepository() + "__" + snapshot.getSnapshotId().getUUID(), new ActionListener() { @Override public void onResponse(Void unused) { logger.debug("Timestamp pinned successfully for snapshot {}", snapshot.getSnapshotId().getName()); listener.onResponse(repositoryData); } + @Override public void onFailure(Exception e) { logger.error("Failed to pin timestamp for snapshot {} with exception {}", snapshot.getSnapshotId().getName(), e); @@ -622,12 +622,34 @@ public void onFailure(Exception e) { } private void removeSnapshotPinnedTimestamp( - RepositoryData repositoryData, - Snapshot snapshot, - long timestampToPin, + SnapshotId snapshotId, + String repository, + long timestampToUnpin, ActionListener listener ) { + remoteStorePinnedTimestampService.unpinTimestamp( + timestampToUnpin, + repository + "__" + 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); + } + } + ); } private static void ensureSnapshotNameNotRunning( @@ -2477,7 +2499,7 @@ private void deleteSnapshotsFromRepository( // SEE https://github.com/opensearch-project/OpenSearch/issues/8610 final boolean remoteStoreShallowCopyEnabled = REMOTE_STORE_INDEX_SHALLOW_COPY.get(repository.getMetadata().settings()); if (remoteStoreShallowCopyEnabled) { - List snapshotsWithPinnedTimestamp = Collections.synchronizedList(new ArrayList<>()); + Map snapshotsWithPinnedTimestamp = new ConcurrentHashMap<>(); List snapshotsWithLockFiles = Collections.synchronizedList(new ArrayList<>()); CountDownLatch latch = new CountDownLatch(1); @@ -2488,7 +2510,7 @@ private void deleteSnapshotsFromRepository( try { SnapshotInfo snapshotInfo = repository.getSnapshotInfo(snapshotId); if (snapshotInfo.getPinnedTimestamp() > 0) { - snapshotsWithPinnedTimestamp.add(snapshotId); + snapshotsWithPinnedTimestamp.put(snapshotId, snapshotInfo.getPinnedTimestamp()); } else { snapshotsWithLockFiles.add(snapshotId); } @@ -2505,9 +2527,9 @@ private void deleteSnapshotsFromRepository( latch.await(); if (snapshotsWithLockFiles.size() > 0) { repository.deleteSnapshotsAndReleaseLockFiles( - snapshotIds, + snapshotsWithLockFiles, repositoryData.getGenId(), - minCompatibleVersion(minNodeVersion, repositoryData, snapshotIds), + minCompatibleVersion(minNodeVersion, repositoryData, snapshotsWithLockFiles), remoteStoreLockManagerFactory, false, ActionListener.wrap(updatedRepoData -> { @@ -2517,16 +2539,26 @@ private void deleteSnapshotsFromRepository( ); } if (snapshotsWithPinnedTimestamp.size() > 0) { + final StepListener pinnedTimestampListener = new StepListener<>(); + pinnedTimestampListener.whenComplete( + updatedRepoData -> { removeSnapshotDeletionFromClusterState(deleteEntry, null, updatedRepoData); }, + ex -> removeSnapshotDeletionFromClusterState(deleteEntry, ex, repositoryData) + ); + repository.deleteSnapshotsAndReleaseLockFiles( - snapshotIds, + snapshotsWithPinnedTimestamp.keySet(), repositoryData.getGenId(), - minCompatibleVersion(minNodeVersion, repositoryData, snapshotIds), + minCompatibleVersion(minNodeVersion, repositoryData, snapshotsWithPinnedTimestamp.keySet()), null, true, ActionListener.wrap(updatedRepoData -> { logger.info("snapshots {} deleted", snapshotsWithPinnedTimestamp); - // removeSnapshotPinnedTimestamp(); - removeSnapshotDeletionFromClusterState(deleteEntry, null, updatedRepoData); + removeSnapshotsPinnedTimestamp( + snapshotsWithPinnedTimestamp, + repository, + updatedRepoData, + pinnedTimestampListener + ); }, ex -> removeSnapshotDeletionFromClusterState(deleteEntry, ex, repositoryData)) ); } @@ -2551,6 +2583,28 @@ private void deleteSnapshotsFromRepository( } } + private void removeSnapshotsPinnedTimestamp( + Map snapshotsWithPinnedTimestamp, + Repository repository, + RepositoryData repositoryData, + 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(snapshotId, repository.getMetadata().name(), pinnedTimestamp, groupedListener); + }); + } + /** * Removes a {@link SnapshotDeletionsInProgress.Entry} from {@link SnapshotDeletionsInProgress} in the cluster state after it executed * on the repository. diff --git a/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java b/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java index 14c5433926c0f..f9b2eb64294a3 100644 --- a/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java +++ b/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java @@ -304,8 +304,8 @@ private static void assertSnapshotUUIDs(BlobStoreRepository repository, Reposito String.format(Locale.ROOT, BlobStoreRepository.SHALLOW_SNAPSHOT_NAME_FORMAT, snapshotId.getUUID()) ) || shardPathContents.containsKey( - String.format(Locale.ROOT, BlobStoreRepository.SNAPSHOT_NAME_FORMAT, snapshotId.getUUID()) - ) + String.format(Locale.ROOT, BlobStoreRepository.SNAPSHOT_NAME_FORMAT, snapshotId.getUUID()) + ) ); } From 626c2fad9c645ef517fb8e50084dc98ee629a182 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Wed, 21 Aug 2024 11:13:25 +0530 Subject: [PATCH 12/55] Modify log messages Signed-off-by: Anshu Agarwal --- .../opensearch/snapshots/SnapshotsService.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index 74591ef8fc9cf..b22b2da9f45fe 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -474,7 +474,7 @@ public void createSnapshotV2(final CreateSnapshotRequest request, final ActionLi Repository repository = repositoriesService.repository(request.repository()); if (repository.isReadOnly()) { - listener.onFailure(new RepositoryException(repository.getMetadata().name(), "cannot create snapshot in a readonly repository")); + listener.onFailure(new RepositoryException(repository.getMetadata().name(), "cannot create snapshot-v2 in a readonly repository")); return; } @@ -503,7 +503,7 @@ public void createSnapshotV2(final CreateSnapshotRequest request, final ActionLi throw new ConcurrentSnapshotExecutionException( repositoryName, snapshotName, - "cannot snapshot while a repository cleanup is in-progress in [" + repositoryCleanupInProgress + "]" + "cannot snapshot-v2 while a repository cleanup is in-progress in [" + repositoryCleanupInProgress + "]" ); } ensureNoCleanupInProgress(currentState, repositoryName, snapshotName); @@ -516,7 +516,7 @@ public void createSnapshotV2(final CreateSnapshotRequest request, final ActionLi request.indices() ); - logger.trace("[{}][{}] creating snapshot for indices [{}]", repositoryName, snapshotName, indices); + logger.trace("[{}][{}] creating snapshot-v2 for indices [{}]", repositoryName, snapshotName, indices); final List indexIds = repositoryData.resolveNewIndices( indices, @@ -551,7 +551,7 @@ public void createSnapshotV2(final CreateSnapshotRequest request, final ActionLi pinnedTimestamp ); if (!clusterService.state().nodes().isLocalNodeElectedClusterManager()) { - throw new SnapshotException(repositoryName, snapshotName, "Aborting Snapshot, no longer cluster manager"); + throw new SnapshotException(repositoryName, snapshotName, "Aborting snapshot-v2, no longer cluster manager"); } final StepListener pinnedTimestampListener = new StepListener<>(); pinnedTimestampListener.whenComplete(repoData -> { listener.onResponse(snapshotInfo); }, listener::onFailure); @@ -568,10 +568,10 @@ public void onResponse(RepositoryData repositoryData) { if (!clusterService.state().nodes().isLocalNodeElectedClusterManager()) { failSnapshotCompletionListeners( snapshot, - new SnapshotException(snapshot, "Aborting Snapshot, no longer cluster manager") + new SnapshotException(snapshot, "Aborting snapshot-v2, no longer cluster manager") ); listener.onFailure( - new SnapshotException(repositoryName, snapshotName, "Aborting Snapshot, no longer cluster manager") + new SnapshotException(repositoryName, snapshotName, "Aborting snapshot-v2, no longer cluster manager") ); return; } @@ -580,7 +580,7 @@ public void onResponse(RepositoryData repositoryData) { @Override public void onFailure(Exception e) { - logger.error("Failed to upload files to snapshot repo {} for snapshot {} ", repositoryName, snapshotName); + logger.error("Failed to upload files to snapshot repo {} for snapshot-v2 {} ", repositoryName, snapshotName); listener.onFailure(e); } } @@ -589,7 +589,7 @@ public void onFailure(Exception e) { }, listener::onFailure); } catch (Exception e) { assert false : new AssertionError(e); - logger.error("Snapshot {} creation failed with exception {}", snapshot.getSnapshotId().getName(), e); + logger.error("Snapshot-v2 {} creation failed with exception {}", snapshot.getSnapshotId().getName(), e); listener.onFailure(e); } } @@ -603,7 +603,7 @@ private void updateSnapshotPinnedTimestamp( listener.onResponse(repositoryData); remoteStorePinnedTimestampService.pinTimestamp( timestampToPin, - snapshot.getRepository() + "__" + snapshot.getSnapshotId(), + snapshot.getRepository() + "__" + snapshot.getSnapshotId().getUUID(), new ActionListener() { @Override public void onResponse(Void unused) { From be65f6d3e1ca699a2a59e8741f2a4111f9636e74 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Wed, 21 Aug 2024 11:35:48 +0530 Subject: [PATCH 13/55] Add spotless check failure fix Signed-off-by: Anshu Agarwal --- .../main/java/org/opensearch/snapshots/SnapshotsService.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index b22b2da9f45fe..45d098dd93175 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -474,7 +474,9 @@ public void createSnapshotV2(final CreateSnapshotRequest request, final ActionLi Repository repository = repositoriesService.repository(request.repository()); if (repository.isReadOnly()) { - listener.onFailure(new RepositoryException(repository.getMetadata().name(), "cannot create snapshot-v2 in a readonly repository")); + listener.onFailure( + new RepositoryException(repository.getMetadata().name(), "cannot create snapshot-v2 in a readonly repository") + ); return; } @@ -610,6 +612,7 @@ public void onResponse(Void unused) { logger.debug("Timestamp pinned successfully for snapshot {}", snapshot.getSnapshotId().getName()); listener.onResponse(repositoryData); } + @Override public void onFailure(Exception e) { logger.error("Failed to pin timestamp for snapshot {} with exception {}", snapshot.getSnapshotId().getName(), e); From 62452ee0577037ee00a85c3a93032e86a9861443 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Wed, 21 Aug 2024 12:05:59 +0530 Subject: [PATCH 14/55] Fix completion listener for snapshot v2 Signed-off-by: Anshu Agarwal --- .../create/TransportCreateSnapshotAction.java | 2 +- .../org/opensearch/snapshots/SnapshotsService.java | 12 ------------ 2 files changed, 1 insertion(+), 13 deletions(-) diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java index d0da00bb10308..b7dcfb64dea2f 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java @@ -107,7 +107,7 @@ protected void clusterManagerOperation( if (request.waitForCompletion() || isSnapshotV2) { snapshotsService.executeSnapshot(request, ActionListener.map(listener, CreateSnapshotResponse::new)); } else { - snapshotsService.startCreateSnapshot(request, ActionListener.map(listener, snapshot -> new CreateSnapshotResponse())); + snapshotsService.createSnapshot(request, ActionListener.map(listener, snapshot -> new CreateSnapshotResponse())); } } } diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index 45d098dd93175..3c0ecbce51521 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -270,18 +270,6 @@ public boolean isSnapshotV2() { * @param listener snapshot completion listener */ public void executeSnapshot(final CreateSnapshotRequest request, final ActionListener listener) { - startCreateSnapshot(request, listener); - } - - /** - * This method calls {@link #createSnapshot(CreateSnapshotRequest, ActionListener)} to create snapshot if snapshot - * V2 is not enabled. - * For V2 enabled snapshots, {@link #createSnapshotV2(CreateSnapshotRequest, ActionListener)} is called and - * appropriate listeners are mapped - * @param request snapshot request - * @param listener snapshot completion listener - */ - public void startCreateSnapshot(final CreateSnapshotRequest request, final ActionListener listener) { Repository repository = repositoriesService.repository(request.repository()); boolean remoteStoreIndexShallowCopy = REMOTE_STORE_INDEX_SHALLOW_COPY.get(repository.getMetadata().settings()); logger.debug("remote_store_index_shallow_copy setting is set as [{}]", remoteStoreIndexShallowCopy); From 00031ec3ee4d516ee5c24afc74ad2cbe5d89cf73 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Wed, 21 Aug 2024 13:43:32 +0530 Subject: [PATCH 15/55] Elevate cluster state update priority for repository metadata update task Signed-off-by: Anshu Agarwal --- .../org/opensearch/remotestore/RemoteRestoreSnapshotIT.java | 5 ++--- .../repositories/blobstore/BlobStoreRepository.java | 5 +++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index e83cc1f972d96..33c29b1c2f442 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -781,8 +781,8 @@ public void testCreateSnapshotV2() throws Exception { assertThat(snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); assertThat(snapshotInfo.successfulShards(), greaterThan(0)); assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards())); + assertThat(snapshotInfo.getPinnedTimestamp(), greaterThan(0L)); - // TODO - verify pinned timestamp indexDocuments(client, indexName1, 10); indexDocuments(client, indexName2, 20); @@ -801,8 +801,7 @@ public void testCreateSnapshotV2() throws Exception { assertThat(snapshotInfo.successfulShards(), greaterThan(0)); assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards())); assertThat(snapshotInfo.snapshotId().getName(), equalTo(snapshotName2)); - - // TODO - verify pinned timestamp + assertThat(snapshotInfo.getPinnedTimestamp(), greaterThan(0L)); } 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 3e6a75565891f..39d87af45942a 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -65,6 +65,7 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Nullable; import org.opensearch.common.Numbers; +import org.opensearch.common.Priority; import org.opensearch.common.SetOnce; import org.opensearch.common.UUIDs; import org.opensearch.common.blobstore.BlobContainer; @@ -2315,7 +2316,7 @@ protected void writeIndexGen( final StepListener setPendingStep = new StepListener<>(); clusterService.submitStateUpdateTask( "set pending repository generation [" + metadata.name() + "][" + expectedGen + "]", - new ClusterStateUpdateTask() { + new ClusterStateUpdateTask(Priority.IMMEDIATE) { private long newGen; @@ -2453,7 +2454,7 @@ public void onFailure(Exception e) { // Step 3: Update CS to reflect new repository generation. clusterService.submitStateUpdateTask( "set safe repository generation [" + metadata.name() + "][" + newGen + "]", - new ClusterStateUpdateTask() { + new ClusterStateUpdateTask(Priority.IMMEDIATE) { @Override public ClusterState execute(ClusterState currentState) { final RepositoryMetadata meta = getRepoMetadata(currentState); From a59e98091e05e09f6d0e59e44b26b86955065ea4 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Thu, 22 Aug 2024 16:35:26 +0530 Subject: [PATCH 16/55] Add test for missing snapshot Signed-off-by: Anshu Agarwal --- .../opensearch/remotestore/RemoteRestoreSnapshotIT.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index 45d5ed7511872..604e8e51e3fcb 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -39,6 +39,7 @@ import org.opensearch.repositories.fs.FsRepository; import org.opensearch.snapshots.AbstractSnapshotIntegTestCase; import org.opensearch.snapshots.SnapshotInfo; +import org.opensearch.snapshots.SnapshotMissingException; import org.opensearch.snapshots.SnapshotRestoreException; import org.opensearch.snapshots.SnapshotState; import org.opensearch.test.InternalTestCluster; @@ -886,6 +887,13 @@ public void testDeleteShallowCopyV2() throws Exception { .setSnapshots(snapshotName2) .get(); assertTrue(deleteResponse.isAcknowledged()); + + // test delete non-existent snapshot + assertThrows( + SnapshotMissingException.class, + () -> client().admin().cluster().prepareDeleteSnapshot(snapshotRepoName, "random-snapshot").setSnapshots(snapshotName2).get() + ); + } } From 0c636ef5d886febd6e77cf920e682fba10277129 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Thu, 22 Aug 2024 18:37:23 +0530 Subject: [PATCH 17/55] Add more integ tests Signed-off-by: Anshu Agarwal --- .../remotestore/RemoteRestoreSnapshotIT.java | 179 ++++++++++++++++++ 1 file changed, 179 insertions(+) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index 33c29b1c2f442..950bb5d37e62a 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -33,6 +33,9 @@ import org.opensearch.index.shard.IndexShard; import org.opensearch.indices.IndicesService; import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.repositories.Repository; +import org.opensearch.repositories.RepositoryData; import org.opensearch.snapshots.AbstractSnapshotIntegTestCase; import org.opensearch.snapshots.SnapshotInfo; import org.opensearch.snapshots.SnapshotRestoreException; @@ -805,4 +808,180 @@ public void testCreateSnapshotV2() throws Exception { } + public void testConcurrentSnapshotV2CreateOperation() throws InterruptedException, ExecutionException { + Settings snapshotSettings = Settings.builder().put("snapshot.snapshot_v2", true).build(); + internalCluster().startClusterManagerOnlyNode(Settings.builder().put(snapshotSettings).build()); + internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); + internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); + String indexName1 = "testindex1"; + String indexName2 = "testindex2"; + String snapshotRepoName = "test-create-snapshot-repo"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + logger.info("Snapshot Path [{}]", absolutePath1); + + createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, true)); + + Client client = client(); + Settings indexSettings = getIndexSettings(20, 0).build(); + createIndex(indexName1, indexSettings); + + Settings indexSettings2 = getIndexSettings(15, 0).build(); + createIndex(indexName2, indexSettings2); + + final int numDocsInIndex1 = 10; + final int numDocsInIndex2 = 20; + indexDocuments(client, indexName1, numDocsInIndex1); + indexDocuments(client, indexName2, numDocsInIndex2); + ensureGreen(indexName1, indexName2); + + int concurrentSnapshots = 5; + + // Prepare threads for concurrent snapshot creation + List threads = new ArrayList<>(); + + for (int i = 0; i < concurrentSnapshots; i++) { + int snapshotIndex = i; + Thread thread = new Thread(() -> { + try { + String snapshotName = "snapshot-concurrent-" + snapshotIndex; + CreateSnapshotResponse createSnapshotResponse2 = client().admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, snapshotName) + .get(); + SnapshotInfo 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(snapshotName)); + assertThat(snapshotInfo.getPinnedTimestamp(), greaterThan(0L)); + } catch (Exception e) {} + }); + threads.add(thread); + } + // start all threads + for (Thread thread : threads) { + thread.start(); + } + + // Wait for all threads to complete + for (Thread thread : threads) { + thread.join(); + } + + // Validate that all snapshots have been created + Repository repository = internalCluster().getInstance(RepositoriesService.class).repository(snapshotRepoName); + PlainActionFuture repositoryDataPlainActionFuture = new PlainActionFuture<>(); + repository.getRepositoryData(repositoryDataPlainActionFuture); + + RepositoryData repositoryData = repositoryDataPlainActionFuture.get(); + assertThat(repositoryData.getSnapshotIds().size(), equalTo(1)); + + } + + public void testCreateSnapshotV2WithRedIndex() throws Exception { + Settings snapshotSettings = Settings.builder().put("snapshot.snapshot_v2", true).build(); + internalCluster().startClusterManagerOnlyNode(Settings.builder().put(snapshotSettings).build()); + internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); + internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); + String indexName1 = "testindex1"; + String indexName2 = "testindex2"; + String indexName3 = "testindex3"; + String snapshotRepoName = "test-create-snapshot-repo"; + String snapshotName1 = "test-create-snapshot1"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + logger.info("Snapshot Path [{}]", absolutePath1); + + createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, true)); + + Client client = client(); + Settings indexSettings = getIndexSettings(20, 0).build(); + createIndex(indexName1, indexSettings); + + Settings indexSettings2 = getIndexSettings(15, 0).build(); + createIndex(indexName2, indexSettings2); + + final int numDocsInIndex1 = 10; + final int numDocsInIndex2 = 20; + indexDocuments(client, indexName1, numDocsInIndex1); + indexDocuments(client, indexName2, numDocsInIndex2); + ensureGreen(indexName1, indexName2); + + internalCluster().ensureAtLeastNumDataNodes(0); + ensureRed(indexName1); + ensureRed(indexName2); + CreateSnapshotResponse createSnapshotResponse2 = client().admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, snapshotName1) + .get(); + SnapshotInfo 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(snapshotName1)); + assertThat(snapshotInfo.getPinnedTimestamp(), greaterThan(0L)); + } + + public void testCreateSnapshotV2WithIndexingLoad() throws Exception { + Settings snapshotSettings = Settings.builder().put("snapshot.snapshot_v2", true).build(); + internalCluster().startClusterManagerOnlyNode(Settings.builder().put(snapshotSettings).build()); + internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); + internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); + String indexName1 = "testindex1"; + String indexName2 = "testindex2"; + String snapshotRepoName = "test-create-snapshot-repo"; + String snapshotName1 = "test-create-snapshot1"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + logger.info("Snapshot Path [{}]", absolutePath1); + + createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, true)); + + Client client = client(); + Settings indexSettings = getIndexSettings(20, 0).build(); + createIndex(indexName1, indexSettings); + + Settings indexSettings2 = getIndexSettings(15, 0).build(); + createIndex(indexName2, indexSettings2); + + final int numDocsInIndex1 = 10; + final int numDocsInIndex2 = 20; + indexDocuments(client, indexName1, numDocsInIndex1); + indexDocuments(client, indexName2, numDocsInIndex2); + ensureGreen(indexName1, indexName2); + + Thread indexingThread = new Thread(() -> { + try { + for (int i = 0; i < 50; i++) { + internalCluster().client().prepareIndex("test-index-load").setSource("field", "value" + i).execute().actionGet(); + } + } catch (Exception e) { + fail("indexing failed due to exception: " + e.getMessage()); + } + }); + + // Start indexing + indexingThread.start(); + + // Wait for a bit to let some documents be indexed + + Thread.sleep(1000); + + // Create a snapshot while indexing is ongoing + CreateSnapshotResponse createSnapshotResponse2 = client().admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, snapshotName1) + .get(); + + SnapshotInfo 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(snapshotName1)); + assertThat(snapshotInfo.getPinnedTimestamp(), greaterThan(0L)); + assertTrue(snapshotInfo.indices().contains("test-index-load")); + assertTrue(snapshotInfo.indices().contains(indexName1)); + assertTrue(snapshotInfo.indices().contains(indexName2)); + indexingThread.join(); + + } + } From 623f994be264606aeab439c8f527d015fa0aca22 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Fri, 23 Aug 2024 12:39:22 +0530 Subject: [PATCH 18/55] Add priority as IMMEDIATE for cluster state repo update task only for v2 snapshots Signed-off-by: Anshu Agarwal --- .../remotestore/RemoteRestoreSnapshotIT.java | 4 +--- .../RepositoryFilterUserMetadataIT.java | 3 +++ .../repositories/FilterRepository.java | 3 +++ .../opensearch/repositories/Repository.java | 2 ++ .../blobstore/BlobStoreRepository.java | 18 ++++++++++++------ .../opensearch/snapshots/SnapshotsService.java | 2 ++ .../repositories/RepositoriesServiceTests.java | 2 ++ .../BlobStoreRepositoryRestoreTests.java | 2 ++ .../blobstore/BlobStoreRepositoryTests.java | 5 +++-- ...ockEventuallyConsistentRepositoryTests.java | 4 ++++ .../index/shard/RestoreOnlyRepository.java | 2 ++ .../AbstractSnapshotIntegTestCase.java | 2 ++ 12 files changed, 38 insertions(+), 11 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index 950bb5d37e62a..89f915b2e9247 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -868,14 +868,13 @@ public void testConcurrentSnapshotV2CreateOperation() throws InterruptedExceptio thread.join(); } - // Validate that all snapshots have been created + // Validate that only one snapshot has been created Repository repository = internalCluster().getInstance(RepositoriesService.class).repository(snapshotRepoName); PlainActionFuture repositoryDataPlainActionFuture = new PlainActionFuture<>(); repository.getRepositoryData(repositoryDataPlainActionFuture); RepositoryData repositoryData = repositoryDataPlainActionFuture.get(); assertThat(repositoryData.getSnapshotIds().size(), equalTo(1)); - } public void testCreateSnapshotV2WithRedIndex() throws Exception { @@ -962,7 +961,6 @@ public void testCreateSnapshotV2WithIndexingLoad() throws Exception { indexingThread.start(); // Wait for a bit to let some documents be indexed - Thread.sleep(1000); // Create a snapshot while indexing is ongoing diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/RepositoryFilterUserMetadataIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/RepositoryFilterUserMetadataIT.java index 0eb37703eb0f1..0bebe969b3f3e 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/RepositoryFilterUserMetadataIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/RepositoryFilterUserMetadataIT.java @@ -36,6 +36,7 @@ import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.Priority; import org.opensearch.common.settings.Settings; import org.opensearch.core.action.ActionListener; import org.opensearch.core.xcontent.NamedXContentRegistry; @@ -127,6 +128,7 @@ public void finalizeSnapshot( SnapshotInfo snapshotInfo, Version repositoryMetaVersion, Function stateTransformer, + Priority repositoryUpdatePriority, ActionListener listener ) { super.finalizeSnapshot( @@ -136,6 +138,7 @@ public void finalizeSnapshot( snapshotInfo, repositoryMetaVersion, stateTransformer, + repositoryUpdatePriority, listener ); } diff --git a/server/src/main/java/org/opensearch/repositories/FilterRepository.java b/server/src/main/java/org/opensearch/repositories/FilterRepository.java index d700a92ed4bad..114cd0260fcca 100644 --- a/server/src/main/java/org/opensearch/repositories/FilterRepository.java +++ b/server/src/main/java/org/opensearch/repositories/FilterRepository.java @@ -39,6 +39,7 @@ import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.Priority; import org.opensearch.common.lifecycle.Lifecycle; import org.opensearch.common.lifecycle.LifecycleListener; import org.opensearch.core.action.ActionListener; @@ -104,6 +105,7 @@ public void finalizeSnapshot( SnapshotInfo snapshotInfo, Version repositoryMetaVersion, Function stateTransformer, + Priority repositoryUpdatePriority, ActionListener listener ) { in.finalizeSnapshot( @@ -113,6 +115,7 @@ public void finalizeSnapshot( snapshotInfo, repositoryMetaVersion, stateTransformer, + repositoryUpdatePriority, listener ); } diff --git a/server/src/main/java/org/opensearch/repositories/Repository.java b/server/src/main/java/org/opensearch/repositories/Repository.java index ed30aad7b4dd2..fce3e3ca9c3eb 100644 --- a/server/src/main/java/org/opensearch/repositories/Repository.java +++ b/server/src/main/java/org/opensearch/repositories/Repository.java @@ -41,6 +41,7 @@ import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.common.Nullable; +import org.opensearch.common.Priority; import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.lifecycle.LifecycleComponent; import org.opensearch.common.settings.Setting; @@ -159,6 +160,7 @@ void finalizeSnapshot( SnapshotInfo snapshotInfo, Version repositoryMetaVersion, Function stateTransformer, + Priority repositoryUpdatePriority, ActionListener listener ); 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 39d87af45942a..4ee4e35a66d0b 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -1047,6 +1047,7 @@ private void doDeleteShardSnapshots( repositoryStateId, repoMetaVersion, Function.identity(), + Priority.NORMAL, ActionListener.wrap(writeUpdatedRepoDataStep::onResponse, listener::onFailure) ); }, listener::onFailure); @@ -1521,6 +1522,7 @@ public void cleanup( repositoryStateId, repositoryMetaVersion, Function.identity(), + Priority.NORMAL, ActionListener.wrap( v -> cleanupStaleBlobs( Collections.emptyList(), @@ -1724,6 +1726,7 @@ public void finalizeSnapshot( SnapshotInfo snapshotInfo, Version repositoryMetaVersion, Function stateTransformer, + Priority repositoryUpdatePriority, final ActionListener listener ) { assert repositoryStateId > RepositoryData.UNKNOWN_REPO_GEN : "Must finalize based on a valid repository generation but received [" @@ -1760,6 +1763,7 @@ public void finalizeSnapshot( repositoryStateId, repositoryMetaVersion, stateTransformer, + repositoryUpdatePriority, ActionListener.wrap(newRepoData -> { cleanupOldShardGens(existingRepositoryData, updatedRepositoryData); listener.onResponse(newRepoData); @@ -2281,10 +2285,11 @@ public boolean isSystemRepository() { * Lastly, the {@link RepositoryMetadata} entry for this repository is updated to the new generation {@code P + 1} and thus * pending and safe generation are set to the same value marking the end of the update of the repository data. * - * @param repositoryData RepositoryData to write - * @param expectedGen expected repository generation at the start of the operation - * @param version version of the repository metadata to write - * @param stateFilter filter for the last cluster state update executed by this method + * @param repositoryData RepositoryData to write + * @param expectedGen expected repository generation at the start of the operation + * @param version version of the repository metadata to write + * @param stateFilter filter for the last cluster state update executed by this method + * @param repositoryUpdatePriority priority for the cluster state update task * @param listener completion listener */ protected void writeIndexGen( @@ -2292,6 +2297,7 @@ protected void writeIndexGen( long expectedGen, Version version, Function stateFilter, + Priority repositoryUpdatePriority, ActionListener listener ) { assert isReadOnly() == false; // can not write to a read only repository @@ -2316,7 +2322,7 @@ protected void writeIndexGen( final StepListener setPendingStep = new StepListener<>(); clusterService.submitStateUpdateTask( "set pending repository generation [" + metadata.name() + "][" + expectedGen + "]", - new ClusterStateUpdateTask(Priority.IMMEDIATE) { + new ClusterStateUpdateTask(repositoryUpdatePriority) { private long newGen; @@ -2454,7 +2460,7 @@ public void onFailure(Exception e) { // Step 3: Update CS to reflect new repository generation. clusterService.submitStateUpdateTask( "set safe repository generation [" + metadata.name() + "][" + newGen + "]", - new ClusterStateUpdateTask(Priority.IMMEDIATE) { + new ClusterStateUpdateTask(repositoryUpdatePriority) { @Override public ClusterState execute(ClusterState currentState) { final RepositoryMetadata meta = getRepoMetadata(currentState); diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index 3c0ecbce51521..c23129b69bc32 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -552,6 +552,7 @@ public void createSnapshotV2(final CreateSnapshotRequest request, final ActionLi snapshotInfo, version, state -> state, + Priority.IMMEDIATE, new ActionListener() { @Override public void onResponse(RepositoryData repositoryData) { @@ -1706,6 +1707,7 @@ private void finalizeSnapshotEntry(SnapshotsInProgress.Entry entry, Metadata met snapshotInfo, entry.version(), state -> stateWithoutSnapshot(state, snapshot), + Priority.NORMAL, ActionListener.wrap(newRepoData -> { completeListenersIgnoringException(endAndGetListenersToResolve(snapshot), Tuple.tuple(newRepoData, snapshotInfo)); logger.info("snapshot [{}] completed with state [{}]", snapshot, snapshotInfo.state()); diff --git a/server/src/test/java/org/opensearch/repositories/RepositoriesServiceTests.java b/server/src/test/java/org/opensearch/repositories/RepositoriesServiceTests.java index 43ebb86fd5342..cb0a36c870d07 100644 --- a/server/src/test/java/org/opensearch/repositories/RepositoriesServiceTests.java +++ b/server/src/test/java/org/opensearch/repositories/RepositoriesServiceTests.java @@ -51,6 +51,7 @@ import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.cluster.service.ClusterApplierService; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.Priority; import org.opensearch.common.UUIDs; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStore; @@ -669,6 +670,7 @@ public void finalizeSnapshot( SnapshotInfo snapshotInfo, Version repositoryMetaVersion, Function stateTransformer, + Priority repositoryUpdatePriority, ActionListener listener ) { listener.onResponse(null); diff --git a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java index ef2c64f89d3a0..7fc987dcfa9bb 100644 --- a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java +++ b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java @@ -42,6 +42,7 @@ import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.ShardRoutingHelper; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.Priority; import org.opensearch.common.UUIDs; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; @@ -218,6 +219,7 @@ public void testSnapshotWithConflictingName() throws Exception { ), Version.CURRENT, Function.identity(), + Priority.NORMAL, f ) ); 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 bd47507da4863..eabac37bf3434 100644 --- a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java +++ b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java @@ -39,6 +39,7 @@ import org.opensearch.client.Client; import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.Priority; import org.opensearch.common.UUIDs; import org.opensearch.common.settings.Settings; import org.opensearch.core.common.unit.ByteSizeUnit; @@ -224,7 +225,7 @@ public void testRepositoryDataConcurrentModificationNotAllowed() { RepositoryData repositoryData = generateRandomRepoData(); final long startingGeneration = repositoryData.getGenId(); final PlainActionFuture future1 = PlainActionFuture.newFuture(); - repository.writeIndexGen(repositoryData, startingGeneration, Version.CURRENT, Function.identity(), future1); + repository.writeIndexGen(repositoryData, startingGeneration, Version.CURRENT, Function.identity(), Priority.NORMAL, future1); // write repo data again to index generational file, errors because we already wrote to the // N+1 generation from which this repository data instance was created @@ -295,7 +296,7 @@ public void testFsRepositoryCompressDeprecatedIgnored() { private static void writeIndexGen(BlobStoreRepository repository, RepositoryData repositoryData, long generation) throws Exception { PlainActionFuture.get( - f -> repository.writeIndexGen(repositoryData, generation, Version.CURRENT, Function.identity(), f) + f -> repository.writeIndexGen(repositoryData, generation, Version.CURRENT, Function.identity(), Priority.NORMAL, f) ); } diff --git a/server/src/test/java/org/opensearch/snapshots/mockstore/MockEventuallyConsistentRepositoryTests.java b/server/src/test/java/org/opensearch/snapshots/mockstore/MockEventuallyConsistentRepositoryTests.java index c3577885e9cb1..06a486b3cb997 100644 --- a/server/src/test/java/org/opensearch/snapshots/mockstore/MockEventuallyConsistentRepositoryTests.java +++ b/server/src/test/java/org/opensearch/snapshots/mockstore/MockEventuallyConsistentRepositoryTests.java @@ -36,6 +36,7 @@ import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.Priority; import org.opensearch.common.UUIDs; import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.settings.ClusterSettings; @@ -238,6 +239,7 @@ public void testOverwriteSnapshotInfoBlob() throws Exception { ), Version.CURRENT, Function.identity(), + Priority.NORMAL, f )); @@ -265,6 +267,7 @@ public void testOverwriteSnapshotInfoBlob() throws Exception { ), Version.CURRENT, Function.identity(), + Priority.NORMAL, f ) ) @@ -294,6 +297,7 @@ public void testOverwriteSnapshotInfoBlob() throws Exception { ), Version.CURRENT, Function.identity(), + Priority.NORMAL, f ) ); diff --git a/test/framework/src/main/java/org/opensearch/index/shard/RestoreOnlyRepository.java b/test/framework/src/main/java/org/opensearch/index/shard/RestoreOnlyRepository.java index be2f895301396..1ca1a6969ab2d 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/RestoreOnlyRepository.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/RestoreOnlyRepository.java @@ -39,6 +39,7 @@ import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.Priority; import org.opensearch.common.lifecycle.AbstractLifecycleComponent; import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.shard.ShardId; @@ -125,6 +126,7 @@ public void finalizeSnapshot( SnapshotInfo snapshotInfo, Version repositoryMetaVersion, Function stateTransformer, + Priority repositoryUpdatePriority, ActionListener listener ) { listener.onResponse(null); diff --git a/test/framework/src/main/java/org/opensearch/snapshots/AbstractSnapshotIntegTestCase.java b/test/framework/src/main/java/org/opensearch/snapshots/AbstractSnapshotIntegTestCase.java index b76de4e0b00cd..ec9cd5b64353e 100644 --- a/test/framework/src/main/java/org/opensearch/snapshots/AbstractSnapshotIntegTestCase.java +++ b/test/framework/src/main/java/org/opensearch/snapshots/AbstractSnapshotIntegTestCase.java @@ -48,6 +48,7 @@ import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.cluster.routing.allocation.decider.EnableAllocationDecider; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.Priority; import org.opensearch.common.UUIDs; import org.opensearch.common.action.ActionFuture; import org.opensearch.common.blobstore.BlobContainer; @@ -623,6 +624,7 @@ protected void addBwCFailedSnapshot(String repoName, String snapshotName, Map Date: Fri, 23 Aug 2024 13:33:28 +0530 Subject: [PATCH 19/55] Fix build error Signed-off-by: Anshu Agarwal --- .../main/java/org/opensearch/repositories/s3/S3Repository.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Repository.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Repository.java index 01b75c0b915f2..3581cae7cf616 100644 --- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Repository.java +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Repository.java @@ -32,6 +32,7 @@ package org.opensearch.repositories.s3; +import org.opensearch.common.Priority; import software.amazon.awssdk.services.s3.model.ObjectCannedACL; import software.amazon.awssdk.services.s3.model.StorageClass; @@ -391,6 +392,7 @@ public void finalizeSnapshot( SnapshotInfo snapshotInfo, Version repositoryMetaVersion, Function stateTransformer, + Priority repositoryUpdatePriority, ActionListener listener ) { super.finalizeSnapshot( @@ -400,6 +402,7 @@ public void finalizeSnapshot( snapshotInfo, repositoryMetaVersion, stateTransformer, + repositoryUpdatePriority, listener ); } From a6090a7b8b482aa853af1d03f28ea0d6b4025e03 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Fri, 23 Aug 2024 13:51:00 +0530 Subject: [PATCH 20/55] Fix spotless error Signed-off-by: Anshu Agarwal --- .../main/java/org/opensearch/repositories/s3/S3Repository.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Repository.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Repository.java index 3581cae7cf616..b5c526451899e 100644 --- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Repository.java +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Repository.java @@ -32,7 +32,6 @@ package org.opensearch.repositories.s3; -import org.opensearch.common.Priority; import software.amazon.awssdk.services.s3.model.ObjectCannedACL; import software.amazon.awssdk.services.s3.model.StorageClass; @@ -43,6 +42,7 @@ import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.Priority; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStore; import org.opensearch.common.blobstore.BlobStoreException; From b5d012f9ed59c1b953aaae193ad007837220af6c Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Fri, 23 Aug 2024 14:50:06 +0530 Subject: [PATCH 21/55] Add repository setting for snapshot v2 Signed-off-by: Anshu Agarwal --- .../remotestore/RemoteRestoreSnapshotIT.java | 91 +++++++++++++++---- .../create/TransportCreateSnapshotAction.java | 11 ++- .../common/settings/ClusterSettings.java | 1 - .../blobstore/BlobStoreRepository.java | 2 + .../opensearch/snapshots/SnapshotInfo.java | 4 +- .../snapshots/SnapshotsService.java | 24 +---- .../snapshots/SnapshotResiliencyTests.java | 1 + 7 files changed, 90 insertions(+), 44 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index 89f915b2e9247..5114fa039b8ff 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -24,6 +24,7 @@ import org.opensearch.common.io.PathUtils; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.common.unit.ByteSizeUnit; import org.opensearch.core.index.Index; import org.opensearch.core.rest.RestStatus; import org.opensearch.index.IndexService; @@ -36,6 +37,8 @@ import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; import org.opensearch.repositories.RepositoryData; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.repositories.fs.FsRepository; import org.opensearch.snapshots.AbstractSnapshotIntegTestCase; import org.opensearch.snapshots.SnapshotInfo; import org.opensearch.snapshots.SnapshotRestoreException; @@ -753,10 +756,9 @@ public void testInvalidRestoreRequestScenarios() throws Exception { public void testCreateSnapshotV2() throws Exception { - Settings snapshotSettings = Settings.builder().put("snapshot.snapshot_v2", true).build(); - internalCluster().startClusterManagerOnlyNode(Settings.builder().put(snapshotSettings).build()); - internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); - internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); + internalCluster().startClusterManagerOnlyNode(); + internalCluster().startDataOnlyNode(); + internalCluster().startDataOnlyNode(); String indexName1 = "testindex1"; String indexName2 = "testindex2"; String indexName3 = "testindex3"; @@ -765,7 +767,20 @@ public void testCreateSnapshotV2() throws Exception { Path absolutePath1 = randomRepoPath().toAbsolutePath(); logger.info("Snapshot Path [{}]", absolutePath1); - createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, true)); + 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.SNAPSHOT_V2.getKey(), true) + ) + ); Client client = client(); Settings indexSettings = getIndexSettings(20, 0).build(); @@ -809,17 +824,29 @@ public void testCreateSnapshotV2() throws Exception { } public void testConcurrentSnapshotV2CreateOperation() throws InterruptedException, ExecutionException { - Settings snapshotSettings = Settings.builder().put("snapshot.snapshot_v2", true).build(); - internalCluster().startClusterManagerOnlyNode(Settings.builder().put(snapshotSettings).build()); - internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); - internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); + internalCluster().startClusterManagerOnlyNode(); + internalCluster().startDataOnlyNode(); + internalCluster().startDataOnlyNode(); String indexName1 = "testindex1"; String indexName2 = "testindex2"; String snapshotRepoName = "test-create-snapshot-repo"; Path absolutePath1 = randomRepoPath().toAbsolutePath(); logger.info("Snapshot Path [{}]", absolutePath1); - createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, true)); + 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.SNAPSHOT_V2.getKey(), true) + ) + ); Client client = client(); Settings indexSettings = getIndexSettings(20, 0).build(); @@ -878,10 +905,9 @@ public void testConcurrentSnapshotV2CreateOperation() throws InterruptedExceptio } public void testCreateSnapshotV2WithRedIndex() throws Exception { - Settings snapshotSettings = Settings.builder().put("snapshot.snapshot_v2", true).build(); - internalCluster().startClusterManagerOnlyNode(Settings.builder().put(snapshotSettings).build()); - internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); - internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); + internalCluster().startClusterManagerOnlyNode(); + internalCluster().startDataOnlyNode(); + internalCluster().startDataOnlyNode(); String indexName1 = "testindex1"; String indexName2 = "testindex2"; String indexName3 = "testindex3"; @@ -890,7 +916,20 @@ public void testCreateSnapshotV2WithRedIndex() throws Exception { Path absolutePath1 = randomRepoPath().toAbsolutePath(); logger.info("Snapshot Path [{}]", absolutePath1); - createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, true)); + 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.SNAPSHOT_V2.getKey(), true) + ) + ); Client client = client(); Settings indexSettings = getIndexSettings(20, 0).build(); @@ -921,10 +960,9 @@ public void testCreateSnapshotV2WithRedIndex() throws Exception { } public void testCreateSnapshotV2WithIndexingLoad() throws Exception { - Settings snapshotSettings = Settings.builder().put("snapshot.snapshot_v2", true).build(); - internalCluster().startClusterManagerOnlyNode(Settings.builder().put(snapshotSettings).build()); - internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); - internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); + internalCluster().startClusterManagerOnlyNode(); + internalCluster().startDataOnlyNode(); + internalCluster().startDataOnlyNode(); String indexName1 = "testindex1"; String indexName2 = "testindex2"; String snapshotRepoName = "test-create-snapshot-repo"; @@ -932,7 +970,20 @@ public void testCreateSnapshotV2WithIndexingLoad() throws Exception { Path absolutePath1 = randomRepoPath().toAbsolutePath(); logger.info("Snapshot Path [{}]", absolutePath1); - createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, true)); + 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.SNAPSHOT_V2.getKey(), true) + ) + ); Client client = client(); Settings indexSettings = getIndexSettings(20, 0).build(); diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java index b7dcfb64dea2f..1ce5ffe1002d8 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java @@ -42,12 +42,16 @@ import org.opensearch.common.inject.Inject; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.repositories.Repository; import org.opensearch.snapshots.SnapshotsService; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; import java.io.IOException; +import static org.opensearch.repositories.blobstore.BlobStoreRepository.SNAPSHOT_V2; + /** * Transport action for create snapshot operation * @@ -56,12 +60,15 @@ public class TransportCreateSnapshotAction extends TransportClusterManagerNodeAction { private final SnapshotsService snapshotsService; + private final RepositoriesService repositoriesService; + @Inject public TransportCreateSnapshotAction( TransportService transportService, ClusterService clusterService, ThreadPool threadPool, SnapshotsService snapshotsService, + RepositoriesService repositoriesService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver ) { @@ -75,6 +82,7 @@ public TransportCreateSnapshotAction( indexNameExpressionResolver ); this.snapshotsService = snapshotsService; + this.repositoriesService = repositoriesService; } @Override @@ -103,7 +111,8 @@ protected void clusterManagerOperation( ClusterState state, final ActionListener listener ) { - Boolean isSnapshotV2 = clusterService.getClusterSettings().get(SnapshotsService.SNAPSHOT_V2); + Repository repository = repositoriesService.repository(request.repository()); + boolean isSnapshotV2 = SNAPSHOT_V2.get(repository.getMetadata().settings()); if (request.waitForCompletion() || isSnapshotV2) { snapshotsService.executeSnapshot(request, ActionListener.map(listener, CreateSnapshotResponse::new)); } else { diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index 2323cac3476b3..d5e8e90458390 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -632,7 +632,6 @@ public void apply(Settings value, Settings current, Settings previous) { HandshakingTransportAddressConnector.PROBE_CONNECT_TIMEOUT_SETTING, HandshakingTransportAddressConnector.PROBE_HANDSHAKE_TIMEOUT_SETTING, SnapshotsService.MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING, - SnapshotsService.SNAPSHOT_V2, FsHealthService.ENABLED_SETTING, FsHealthService.REFRESH_INTERVAL_SETTING, FsHealthService.SLOW_PATH_LOGGING_THRESHOLD_SETTING, 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 4ee4e35a66d0b..2905ff7d90a63 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -267,6 +267,8 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp public static final Setting REMOTE_STORE_INDEX_SHALLOW_COPY = Setting.boolSetting("remote_store_index_shallow_copy", false); + public static final Setting SNAPSHOT_V2 = Setting.boolSetting("snapshot_v2", false); + /** * Setting to set batch size of stale snapshot shard blobs that will be deleted by snapshot workers as part of snapshot deletion. * For optimal performance the value of the setting should be equal to or close to repository's max # of keys that can be deleted in single operation diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java index 3efe50d0a06f9..dec6a7778f448 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java @@ -441,7 +441,7 @@ public SnapshotInfo(final StreamInput in) throws IOException { if (in.getVersion().onOrAfter(Version.V_2_9_0)) { remoteStoreIndexShallowCopy = in.readOptionalBoolean(); } - if (in.getVersion().onOrAfter(Version.V_2_17_0)) { + if (in.getVersion().onOrAfter(Version.V_3_0_0)) { pinnedTimestamp = in.readVLong(); } } @@ -906,7 +906,7 @@ public void writeTo(final StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_2_9_0)) { out.writeOptionalBoolean(remoteStoreIndexShallowCopy); } - if (out.getVersion().onOrAfter(Version.V_2_17_0)) { + if (out.getVersion().onOrAfter(Version.V_3_0_0)) { out.writeVLong(pinnedTimestamp); } } diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index c23129b69bc32..da24d2ad480ca 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -136,6 +136,7 @@ import static org.opensearch.node.remotestore.RemoteStoreNodeService.CompatibilityMode; import static org.opensearch.node.remotestore.RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING; import static org.opensearch.repositories.blobstore.BlobStoreRepository.REMOTE_STORE_INDEX_SHALLOW_COPY; +import static org.opensearch.repositories.blobstore.BlobStoreRepository.SNAPSHOT_V2; import static org.opensearch.snapshots.SnapshotUtils.validateSnapshotsBackingAnyIndex; /** @@ -201,18 +202,8 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus Setting.Property.NodeScope, Setting.Property.Dynamic ); - - public static final Setting SNAPSHOT_V2 = Setting.boolSetting( - "snapshot.snapshot_v2", - false, - Setting.Property.Dynamic, - Setting.Property.NodeScope - ); - private volatile int maxConcurrentOperations; - private volatile boolean isSnapshotV2; - public SnapshotsService( Settings settings, ClusterService clusterService, @@ -244,8 +235,6 @@ public SnapshotsService( maxConcurrentOperations = MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING.get(settings); clusterService.getClusterSettings() .addSettingsUpdateConsumer(MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING, i -> maxConcurrentOperations = i); - isSnapshotV2 = SNAPSHOT_V2.get(settings); - clusterService.getClusterSettings().addSettingsUpdateConsumer(SNAPSHOT_V2, this::setSnapshotV2); } // Task is onboarded for throttling, it will get retried from associated TransportClusterManagerNodeAction. @@ -254,14 +243,6 @@ public SnapshotsService( updateSnapshotStateTaskKey = clusterService.registerClusterManagerTask(ClusterManagerTaskKeys.UPDATE_SNAPSHOT_STATE_KEY, true); } - private void setSnapshotV2(boolean isSnapshotV2) { - this.isSnapshotV2 = isSnapshotV2; - } - - public boolean isSnapshotV2() { - return isSnapshotV2; - } - /** * Same as {@link #createSnapshot(CreateSnapshotRequest, ActionListener)} but invokes its callback on completion of * the snapshot. @@ -274,6 +255,9 @@ public void executeSnapshot(final CreateSnapshotRequest request, final ActionLis boolean remoteStoreIndexShallowCopy = REMOTE_STORE_INDEX_SHALLOW_COPY.get(repository.getMetadata().settings()); logger.debug("remote_store_index_shallow_copy setting is set as [{}]", remoteStoreIndexShallowCopy); + boolean isSnapshotV2 = SNAPSHOT_V2.get(repository.getMetadata().settings()); + logger.debug("snapshot_v2 is set as [{}]", isSnapshotV2); + if (remoteStoreIndexShallowCopy && clusterService.getClusterSettings().get(REMOTE_STORE_COMPATIBILITY_MODE_SETTING).equals(CompatibilityMode.MIXED)) { // don't allow shallow snapshots if compatibility mode is not strict diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java index 769dfeb37ff8d..e27223cea0778 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java @@ -2379,6 +2379,7 @@ public void onFailure(final Exception e) { clusterService, threadPool, snapshotsService, + repositoriesService, actionFilters, indexNameExpressionResolver ) From 51c1cdfb97f029751af5cb9d622f6c6284cfb16b Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Mon, 26 Aug 2024 10:05:06 +0530 Subject: [PATCH 22/55] Add integ test on multiple snapshots deletion in one delete request Signed-off-by: Anshu Agarwal --- .../remotestore/RemoteRestoreSnapshotIT.java | 85 +++++++++++++++++++ 1 file changed, 85 insertions(+) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index 604e8e51e3fcb..c15abd25c121e 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -896,4 +896,89 @@ public void testDeleteShallowCopyV2() throws Exception { } + public void testDeleteShallowCopyV2MultipleSnapshots() throws Exception { + + Settings snapshotSettings = Settings.builder().put("snapshot.snapshot_v2", true).build(); + // Settings snapshotSettings = Settings.builder().put("snapshot.max_concurrent_operations", 1000).build(); + internalCluster().startClusterManagerOnlyNode(Settings.builder().put(snapshotSettings).build()); + internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); + internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); + + 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) + ) + ); + + Settings indexSettings = getIndexSettings(20, 0).build(); + createIndex(indexName1, indexSettings); + + Settings indexSettings2 = getIndexSettings(15, 0).build(); + createIndex(indexName2, indexSettings2); + + final int numDocsInIndex1 = 10; + final int numDocsInIndex2 = 20; + indexDocuments(client, indexName1, numDocsInIndex1); + indexDocuments(client, indexName2, numDocsInIndex2); + ensureGreen(indexName1, indexName2); + + CreateSnapshotResponse createSnapshotResponse = client().admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, snapshotName1) + .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)); + + indexDocuments(client, indexName2, 10); + indexDocuments(client, indexName1, 10); + createIndex(indexName3); + indexDocuments(client, 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() + ); + + } + } From 80bf6cc8241d3c9cbc95769d4e5deffc9306468e Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Mon, 26 Aug 2024 10:14:03 +0530 Subject: [PATCH 23/55] Address review comments Signed-off-by: Anshu Agarwal --- .../remotestore/RemoteRestoreSnapshotIT.java | 3 ++- .../org/opensearch/repositories/Repository.java | 17 +++++++++-------- .../opensearch/snapshots/SnapshotsService.java | 1 - 3 files changed, 11 insertions(+), 10 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index 5114fa039b8ff..a94481c2409d0 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -71,6 +71,7 @@ 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.greaterThanOrEqualTo; @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) public class RemoteRestoreSnapshotIT extends AbstractSnapshotIntegTestCase { @@ -901,7 +902,7 @@ public void testConcurrentSnapshotV2CreateOperation() throws InterruptedExceptio repository.getRepositoryData(repositoryDataPlainActionFuture); RepositoryData repositoryData = repositoryDataPlainActionFuture.get(); - assertThat(repositoryData.getSnapshotIds().size(), equalTo(1)); + assertThat(repositoryData.getSnapshotIds().size(), greaterThanOrEqualTo(1)); } public void testCreateSnapshotV2WithRedIndex() throws Exception { diff --git a/server/src/main/java/org/opensearch/repositories/Repository.java b/server/src/main/java/org/opensearch/repositories/Repository.java index fce3e3ca9c3eb..8d29d2f898cbd 100644 --- a/server/src/main/java/org/opensearch/repositories/Repository.java +++ b/server/src/main/java/org/opensearch/repositories/Repository.java @@ -144,14 +144,15 @@ default Repository create(RepositoryMetadata metadata, Function * This method is called on cluster-manager after all shards are snapshotted. * - * @param shardGenerations updated shard generations - * @param repositoryStateId the unique id identifying the state of the repository when the snapshot began - * @param clusterMetadata cluster metadata - * @param snapshotInfo SnapshotInfo instance to write for this snapshot - * @param repositoryMetaVersion version of the updated repository metadata to write - * @param stateTransformer a function that filters the last cluster state update that the snapshot finalization will execute and - * is used to remove any state tracked for the in-progress snapshot from the cluster state - * @param listener listener to be invoked with the new {@link RepositoryData} after completing the snapshot + * @param shardGenerations updated shard generations + * @param repositoryStateId the unique id identifying the state of the repository when the snapshot began + * @param clusterMetadata cluster metadata + * @param snapshotInfo SnapshotInfo instance to write for this snapshot + * @param repositoryMetaVersion version of the updated repository metadata to write + * @param stateTransformer a function that filters the last cluster state update that the snapshot finalization will execute and + * is used to remove any state tracked for the in-progress snapshot from the cluster state + * @param repositoryUpdatePriority priority for the cluster state update task + * @param listener listener to be invoked with the new {@link RepositoryData} after completing the snapshot */ void finalizeSnapshot( ShardGenerations shardGenerations, diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index da24d2ad480ca..ef4b3ba087e4d 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -575,7 +575,6 @@ private void updateSnapshotPinnedTimestamp( long timestampToPin, ActionListener listener ) { - listener.onResponse(repositoryData); remoteStorePinnedTimestampService.pinTimestamp( timestampToPin, snapshot.getRepository() + "__" + snapshot.getSnapshotId().getUUID(), From b0cbc08ebd58d9c307c7ab17a1eeda5149915333 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Mon, 26 Aug 2024 10:53:30 +0530 Subject: [PATCH 24/55] Add integ test to verify snapshot creation if shallow copy repo setting is disabled Signed-off-by: Anshu Agarwal --- .../remotestore/RemoteRestoreSnapshotIT.java | 131 ++++++++++++++++++ .../blobstore/BlobStoreTestUtil.java | 31 +++-- 2 files changed, 148 insertions(+), 14 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index a94481c2409d0..ef6d5d6c75810 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -824,6 +824,89 @@ public void testCreateSnapshotV2() throws Exception { } + public void testMixedSnapshotCreationWithV2RepositorySetting() throws Exception { + + internalCluster().startClusterManagerOnlyNode(); + internalCluster().startDataOnlyNode(); + internalCluster().startDataOnlyNode(); + String indexName1 = "testindex1"; + String indexName2 = "testindex2"; + String indexName3 = "testindex3"; + String snapshotRepoName = "test-create-snapshot-repo"; + String snapshotName1 = "test-create-snapshot-v1"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + logger.info("Snapshot Path [{}]", absolutePath1); + + 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.SNAPSHOT_V2.getKey(), false) + ) + ); + Client client = client(); + Settings indexSettings = getIndexSettings(20, 0).build(); + createIndex(indexName1, indexSettings); + + Settings indexSettings2 = getIndexSettings(15, 0).build(); + createIndex(indexName2, indexSettings2); + + final int numDocsInIndex1 = 10; + final int numDocsInIndex2 = 20; + indexDocuments(client, indexName1, numDocsInIndex1); + indexDocuments(client, indexName2, numDocsInIndex2); + ensureGreen(indexName1, indexName2); + + SnapshotInfo snapshotInfo = createSnapshot(snapshotRepoName, snapshotName1, Collections.emptyList()); + assertThat(snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); + assertThat(snapshotInfo.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards())); + assertThat(snapshotInfo.getPinnedTimestamp(), equalTo(0L)); + + // enable snapshot_v2 + 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.SNAPSHOT_V2.getKey(), true) + ) + ); + indexDocuments(client, indexName1, 10); + indexDocuments(client, indexName2, 20); + + createIndex(indexName3, indexSettings); + indexDocuments(client, indexName3, 10); + + String snapshotName2 = "test-create-snapshot-v2"; + + // verify even if waitForCompletion is not true, the request executes in a sync manner + CreateSnapshotResponse createSnapshotResponse2 = client().admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, snapshotName2) + .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)); + assertThat(snapshotInfo.getPinnedTimestamp(), greaterThan(0L)); + + } + public void testConcurrentSnapshotV2CreateOperation() throws InterruptedException, ExecutionException { internalCluster().startClusterManagerOnlyNode(); internalCluster().startDataOnlyNode(); @@ -1034,4 +1117,52 @@ public void testCreateSnapshotV2WithIndexingLoad() throws Exception { } + public void testCreateSnapshotV2WithShallowCopySettingDisabled() throws Exception { + internalCluster().startClusterManagerOnlyNode(); + internalCluster().startDataOnlyNode(); + internalCluster().startDataOnlyNode(); + String indexName1 = "testindex1"; + String indexName2 = "testindex2"; + String snapshotRepoName = "test-create-snapshot-repo"; + String snapshotName1 = "test-create-snapshot1"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + logger.info("Snapshot Path [{}]", absolutePath1); + + 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(), false) + .put(BlobStoreRepository.SNAPSHOT_V2.getKey(), true) + ) + ); + + Client client = client(); + Settings indexSettings = getIndexSettings(20, 0).build(); + createIndex(indexName1, indexSettings); + + Settings indexSettings2 = getIndexSettings(15, 0).build(); + createIndex(indexName2, indexSettings2); + + final int numDocsInIndex1 = 10; + final int numDocsInIndex2 = 20; + indexDocuments(client, indexName1, numDocsInIndex1); + indexDocuments(client, indexName2, numDocsInIndex2); + ensureGreen(indexName1, indexName2); + + // Will create full copy snapshot if `REMOTE_STORE_INDEX_SHALLOW_COPY` is false but `SNAPSHOT_V2` is true + SnapshotInfo snapshotInfo = createSnapshot(snapshotRepoName, snapshotName1, Collections.emptyList()); + assertThat(snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); + assertThat(snapshotInfo.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards())); + assertThat(snapshotInfo.getPinnedTimestamp(), equalTo(0L)); + + } + } diff --git a/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java b/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java index 32f445bf24a41..4c1d5e8f2b804 100644 --- a/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java +++ b/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java @@ -298,23 +298,26 @@ private static void assertSnapshotUUIDs(BlobStoreRepository repository, Reposito .stream() .noneMatch(shardFailure -> shardFailure.index().equals(index) && shardFailure.shardId() == shardId)) { final Map shardPathContents = shardContainer.listBlobs(); - - assertTrue( - shardPathContents.containsKey( - String.format(Locale.ROOT, BlobStoreRepository.SHALLOW_SNAPSHOT_NAME_FORMAT, snapshotId.getUUID()) - ) - || shardPathContents.containsKey( + if (snapshotInfo.getPinnedTimestamp() == 0) + { + assertTrue( + shardPathContents.containsKey( + String.format(Locale.ROOT, BlobStoreRepository.SHALLOW_SNAPSHOT_NAME_FORMAT, snapshotId.getUUID()) + ) + || shardPathContents.containsKey( String.format(Locale.ROOT, BlobStoreRepository.SNAPSHOT_NAME_FORMAT, snapshotId.getUUID()) ) - ); + ); + + assertThat( + shardPathContents.keySet() + .stream() + .filter(name -> name.startsWith(BlobStoreRepository.INDEX_FILE_PREFIX)) + .count(), + lessThanOrEqualTo(2L) + ); + } - assertThat( - shardPathContents.keySet() - .stream() - .filter(name -> name.startsWith(BlobStoreRepository.INDEX_FILE_PREFIX)) - .count(), - lessThanOrEqualTo(2L) - ); } } } From 38af0f6051daefe3898498c71152dcda73024aea Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Mon, 26 Aug 2024 11:02:59 +0530 Subject: [PATCH 25/55] Fix spotless vilation error Signed-off-by: Anshu Agarwal --- .../repositories/blobstore/BlobStoreTestUtil.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java b/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java index 4c1d5e8f2b804..187c30be0044e 100644 --- a/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java +++ b/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java @@ -298,15 +298,14 @@ private static void assertSnapshotUUIDs(BlobStoreRepository repository, Reposito .stream() .noneMatch(shardFailure -> shardFailure.index().equals(index) && shardFailure.shardId() == shardId)) { final Map shardPathContents = shardContainer.listBlobs(); - if (snapshotInfo.getPinnedTimestamp() == 0) - { + if (snapshotInfo.getPinnedTimestamp() == 0) { assertTrue( shardPathContents.containsKey( String.format(Locale.ROOT, BlobStoreRepository.SHALLOW_SNAPSHOT_NAME_FORMAT, snapshotId.getUUID()) ) || shardPathContents.containsKey( - String.format(Locale.ROOT, BlobStoreRepository.SNAPSHOT_NAME_FORMAT, snapshotId.getUUID()) - ) + String.format(Locale.ROOT, BlobStoreRepository.SNAPSHOT_NAME_FORMAT, snapshotId.getUUID()) + ) ); assertThat( From 73376a8c90cb320402be06c80449051e4067954a Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Mon, 26 Aug 2024 14:23:42 +0530 Subject: [PATCH 26/55] Address review comment Signed-off-by: Anshu Agarwal --- .../org/opensearch/repositories/Repository.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/org/opensearch/repositories/Repository.java b/server/src/main/java/org/opensearch/repositories/Repository.java index 8d29d2f898cbd..637503d3f54df 100644 --- a/server/src/main/java/org/opensearch/repositories/Repository.java +++ b/server/src/main/java/org/opensearch/repositories/Repository.java @@ -144,15 +144,15 @@ default Repository create(RepositoryMetadata metadata, Function * This method is called on cluster-manager after all shards are snapshotted. * - * @param shardGenerations updated shard generations - * @param repositoryStateId the unique id identifying the state of the repository when the snapshot began - * @param clusterMetadata cluster metadata - * @param snapshotInfo SnapshotInfo instance to write for this snapshot - * @param repositoryMetaVersion version of the updated repository metadata to write - * @param stateTransformer a function that filters the last cluster state update that the snapshot finalization will execute and - * is used to remove any state tracked for the in-progress snapshot from the cluster state + * @param shardGenerations updated shard generations + * @param repositoryStateId the unique id identifying the state of the repository when the snapshot began + * @param clusterMetadata cluster metadata + * @param snapshotInfo SnapshotInfo instance to write for this snapshot + * @param repositoryMetaVersion version of the updated repository metadata to write + * @param stateTransformer a function that filters the last cluster state update that the snapshot finalization will execute and + * is used to remove any state tracked for the in-progress snapshot from the cluster state * @param repositoryUpdatePriority priority for the cluster state update task - * @param listener listener to be invoked with the new {@link RepositoryData} after completing the snapshot + * @param listener listener to be invoked with the new {@link RepositoryData} after completing the snapshot */ void finalizeSnapshot( ShardGenerations shardGenerations, From 39b57e3bad33e7d4f170d1085454dadf9fe58c85 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Mon, 26 Aug 2024 19:01:36 +0530 Subject: [PATCH 27/55] Address review comments Signed-off-by: Anshu Agarwal --- .../opensearch/snapshots/SnapshotInfo.java | 5 +- .../snapshots/SnapshotsService.java | 91 +++++++++---------- 2 files changed, 47 insertions(+), 49 deletions(-) diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java index dec6a7778f448..5c316a8a9b9cf 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java @@ -666,7 +666,10 @@ public XContentBuilder toXContent(final XContentBuilder builder, final Params pa if (remoteStoreIndexShallowCopy != null) { builder.field(REMOTE_STORE_INDEX_SHALLOW_COPY, remoteStoreIndexShallowCopy); } - builder.field(PINNED_TIMESTAMP, pinnedTimestamp); + if (pinnedTimestamp != 0) + { + builder.field(PINNED_TIMESTAMP, pinnedTimestamp); + } builder.startArray(INDICES); for (String index : indices) { diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index ef4b3ba087e4d..09260e3e7cf31 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -252,18 +252,11 @@ public SnapshotsService( */ public void executeSnapshot(final CreateSnapshotRequest request, final ActionListener listener) { Repository repository = repositoriesService.repository(request.repository()); - boolean remoteStoreIndexShallowCopy = REMOTE_STORE_INDEX_SHALLOW_COPY.get(repository.getMetadata().settings()); - logger.debug("remote_store_index_shallow_copy setting is set as [{}]", remoteStoreIndexShallowCopy); boolean isSnapshotV2 = SNAPSHOT_V2.get(repository.getMetadata().settings()); logger.debug("snapshot_v2 is set as [{}]", isSnapshotV2); - if (remoteStoreIndexShallowCopy - && clusterService.getClusterSettings().get(REMOTE_STORE_COMPATIBILITY_MODE_SETTING).equals(CompatibilityMode.MIXED)) { - // don't allow shallow snapshots if compatibility mode is not strict - logger.warn("Shallow snapshots are not supported during migration. Falling back to full snapshot."); - remoteStoreIndexShallowCopy = false; - } + boolean remoteStoreIndexShallowCopy = remoteStoreShallowCopyEnabled(repository); if (remoteStoreIndexShallowCopy && isSnapshotV2 && request.indices().length == 0) { createSnapshotV2(request, listener); } else { @@ -274,11 +267,25 @@ public void executeSnapshot(final CreateSnapshotRequest request, final ActionLis } } + private boolean remoteStoreShallowCopyEnabled(Repository repository) { + boolean remoteStoreIndexShallowCopy = REMOTE_STORE_INDEX_SHALLOW_COPY.get(repository.getMetadata().settings()); + logger.debug("remote_store_index_shallow_copy setting is set as [{}]", remoteStoreIndexShallowCopy); + if (remoteStoreIndexShallowCopy + && clusterService.getClusterSettings().get(REMOTE_STORE_COMPATIBILITY_MODE_SETTING).equals(CompatibilityMode.MIXED)) { + // don't allow shallow snapshots if compatibility mode is not strict + logger.warn("Shallow snapshots are not supported during migration. Falling back to full snapshot."); + remoteStoreIndexShallowCopy = false; + } + return remoteStoreIndexShallowCopy; + + } + /** * Initializes the snapshotting process. *

* This method is used by clients to start snapshot. It makes sure that there is no snapshots are currently running and * creates a snapshot record in cluster state metadata. + *

* * @param request snapshot request * @param listener snapshot creation listener @@ -304,27 +311,13 @@ public void createSnapshot(final CreateSnapshotRequest request, final ActionList @Override public ClusterState execute(ClusterState currentState) { - ensureSnapshotNameAvailableInRepo(repositoryData, snapshotName, repository); + createSnapshotPreValidations(currentState,repositoryData,repositoryName,snapshotName); final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); final List runningSnapshots = snapshots.entries(); - ensureSnapshotNameNotRunning(runningSnapshots, repositoryName, snapshotName); - validate(repositoryName, snapshotName, currentState); final SnapshotDeletionsInProgress deletionsInProgress = currentState.custom( SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY ); - final RepositoryCleanupInProgress repositoryCleanupInProgress = currentState.custom( - RepositoryCleanupInProgress.TYPE, - RepositoryCleanupInProgress.EMPTY - ); - if (repositoryCleanupInProgress.hasCleanupInProgress()) { - throw new ConcurrentSnapshotExecutionException( - repositoryName, - snapshotName, - "cannot snapshot while a repository cleanup is in-progress in [" + repositoryCleanupInProgress + "]" - ); - } - ensureNoCleanupInProgress(currentState, repositoryName, snapshotName); ensureBelowConcurrencyLimit(repositoryName, snapshotName, snapshots, deletionsInProgress); // Store newSnapshot here to be processed in clusterStateProcessed List indices = Arrays.asList(indexNameExpressionResolver.concreteIndexNames(currentState, request)); @@ -443,7 +436,7 @@ public void createSnapshotV2(final CreateSnapshotRequest request, final ActionLi validate(repositoryName, snapshotName); final SnapshotId snapshotId = new SnapshotId(snapshotName, UUIDs.randomBase64UUID()); // new UUID for the snapshot - Repository repository = repositoriesService.repository(request.repository()); + Repository repository = repositoriesService.repository(repositoryName); if (repository.isReadOnly()) { listener.onFailure( @@ -460,27 +453,7 @@ public void createSnapshotV2(final CreateSnapshotRequest request, final ActionLi repositoriesService.getRepositoryData(repositoryName, repositoryDataListener); repositoryDataListener.whenComplete(repositoryData -> { - ensureSnapshotNameAvailableInRepo(repositoryData, snapshotName, repository); - final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); - final List runningSnapshots = snapshots.entries(); - ensureSnapshotNameNotRunning(runningSnapshots, repositoryName, snapshotName); - validate(repositoryName, snapshotName, currentState); - final SnapshotDeletionsInProgress deletionsInProgress = currentState.custom( - SnapshotDeletionsInProgress.TYPE, - SnapshotDeletionsInProgress.EMPTY - ); - final RepositoryCleanupInProgress repositoryCleanupInProgress = currentState.custom( - RepositoryCleanupInProgress.TYPE, - RepositoryCleanupInProgress.EMPTY - ); - if (repositoryCleanupInProgress.hasCleanupInProgress()) { - throw new ConcurrentSnapshotExecutionException( - repositoryName, - snapshotName, - "cannot snapshot-v2 while a repository cleanup is in-progress in [" + repositoryCleanupInProgress + "]" - ); - } - ensureNoCleanupInProgress(currentState, repositoryName, snapshotName); + createSnapshotPreValidations(currentState,repositoryData,repositoryName,snapshotName); List indices = Arrays.asList(indexNameExpressionResolver.concreteIndexNames(currentState, request)); @@ -492,6 +465,9 @@ public void createSnapshotV2(final CreateSnapshotRequest request, final ActionLi logger.trace("[{}][{}] creating snapshot-v2 for indices [{}]", repositoryName, snapshotName, indices); + final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); + final List runningSnapshots = snapshots.entries(); + final List indexIds = repositoryData.resolveNewIndices( indices, getInFlightIndexIds(runningSnapshots, repositoryName) @@ -504,8 +480,6 @@ public void createSnapshotV2(final CreateSnapshotRequest request, final ActionLi repositoryData ); - boolean remoteStoreIndexShallowCopy = REMOTE_STORE_INDEX_SHALLOW_COPY.get(repository.getMetadata().settings()); - assert remoteStoreIndexShallowCopy : "remote_store_index_shallow_copy setting is set as false"; if (repositoryData.getGenId() == RepositoryData.UNKNOWN_REPO_GEN) { logger.debug("[{}] was aborted before starting", snapshot); throw new SnapshotException(snapshot, "Aborted on initialization"); @@ -521,7 +495,7 @@ public void createSnapshotV2(final CreateSnapshotRequest request, final ActionLi Collections.emptyList(), request.includeGlobalState(), userMeta, - remoteStoreIndexShallowCopy, + true, pinnedTimestamp ); if (!clusterService.state().nodes().isLocalNodeElectedClusterManager()) { @@ -569,6 +543,27 @@ public void onFailure(Exception e) { } } + private void createSnapshotPreValidations(ClusterState currentState, RepositoryData repositoryData, String repositoryName, String snapshotName) { + Repository repository = repositoriesService.repository(repositoryName); + ensureSnapshotNameAvailableInRepo(repositoryData, snapshotName, repository); + final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); + final List runningSnapshots = snapshots.entries(); + ensureSnapshotNameNotRunning(runningSnapshots, repositoryName, snapshotName); + validate(repositoryName, snapshotName, currentState); + final RepositoryCleanupInProgress repositoryCleanupInProgress = currentState.custom( + RepositoryCleanupInProgress.TYPE, + RepositoryCleanupInProgress.EMPTY + ); + if (repositoryCleanupInProgress.hasCleanupInProgress()) { + throw new ConcurrentSnapshotExecutionException( + repositoryName, + snapshotName, + "cannot snapshot-v2 while a repository cleanup is in-progress in [" + repositoryCleanupInProgress + "]" + ); + } + ensureNoCleanupInProgress(currentState, repositoryName, snapshotName); + } + private void updateSnapshotPinnedTimestamp( RepositoryData repositoryData, Snapshot snapshot, From e1eecbd102319320e63fcaba085b16db8d8eca06 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Mon, 26 Aug 2024 19:27:27 +0530 Subject: [PATCH 28/55] Add min version check for backward compatibility Signed-off-by: Anshu Agarwal --- .../opensearch/snapshots/SnapshotInfo.java | 3 +-- .../snapshots/SnapshotsService.java | 24 +++++++++++++------ 2 files changed, 18 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java index 5c316a8a9b9cf..7558c4456109e 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java @@ -666,8 +666,7 @@ public XContentBuilder toXContent(final XContentBuilder builder, final Params pa if (remoteStoreIndexShallowCopy != null) { builder.field(REMOTE_STORE_INDEX_SHALLOW_COPY, remoteStoreIndexShallowCopy); } - if (pinnedTimestamp != 0) - { + if (pinnedTimestamp != 0) { builder.field(PINNED_TIMESTAMP, pinnedTimestamp); } diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index 09260e3e7cf31..fdc822a7fe41a 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -202,6 +202,8 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus Setting.Property.NodeScope, Setting.Property.Dynamic ); + + private static final String SNAPSHOT_PINNED_TIMESTAMP_DELIMITER = "__"; private volatile int maxConcurrentOperations; public SnapshotsService( @@ -256,8 +258,11 @@ public void executeSnapshot(final CreateSnapshotRequest request, final ActionLis boolean isSnapshotV2 = SNAPSHOT_V2.get(repository.getMetadata().settings()); logger.debug("snapshot_v2 is set as [{}]", isSnapshotV2); - boolean remoteStoreIndexShallowCopy = remoteStoreShallowCopyEnabled(repository); - if (remoteStoreIndexShallowCopy && isSnapshotV2 && request.indices().length == 0) { + boolean remoteStoreIndexShallowCopy = remoteStoreShallowCopyEnabled(repository); + if (remoteStoreIndexShallowCopy + && isSnapshotV2 + && request.indices().length == 0 + && clusterService.state().nodes().getMinNodeVersion().onOrAfter(Version.CURRENT)) { createSnapshotV2(request, listener); } else { createSnapshot( @@ -267,7 +272,7 @@ public void executeSnapshot(final CreateSnapshotRequest request, final ActionLis } } - private boolean remoteStoreShallowCopyEnabled(Repository repository) { + private boolean remoteStoreShallowCopyEnabled(Repository repository) { boolean remoteStoreIndexShallowCopy = REMOTE_STORE_INDEX_SHALLOW_COPY.get(repository.getMetadata().settings()); logger.debug("remote_store_index_shallow_copy setting is set as [{}]", remoteStoreIndexShallowCopy); if (remoteStoreIndexShallowCopy @@ -311,7 +316,7 @@ public void createSnapshot(final CreateSnapshotRequest request, final ActionList @Override public ClusterState execute(ClusterState currentState) { - createSnapshotPreValidations(currentState,repositoryData,repositoryName,snapshotName); + createSnapshotPreValidations(currentState, repositoryData, repositoryName, snapshotName); final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); final List runningSnapshots = snapshots.entries(); final SnapshotDeletionsInProgress deletionsInProgress = currentState.custom( @@ -453,7 +458,7 @@ public void createSnapshotV2(final CreateSnapshotRequest request, final ActionLi repositoriesService.getRepositoryData(repositoryName, repositoryDataListener); repositoryDataListener.whenComplete(repositoryData -> { - createSnapshotPreValidations(currentState,repositoryData,repositoryName,snapshotName); + createSnapshotPreValidations(currentState, repositoryData, repositoryName, snapshotName); List indices = Arrays.asList(indexNameExpressionResolver.concreteIndexNames(currentState, request)); @@ -543,7 +548,12 @@ public void onFailure(Exception e) { } } - private void createSnapshotPreValidations(ClusterState currentState, RepositoryData repositoryData, String repositoryName, String snapshotName) { + private void createSnapshotPreValidations( + ClusterState currentState, + RepositoryData repositoryData, + String repositoryName, + String snapshotName + ) { Repository repository = repositoriesService.repository(repositoryName); ensureSnapshotNameAvailableInRepo(repositoryData, snapshotName, repository); final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); @@ -572,7 +582,7 @@ private void updateSnapshotPinnedTimestamp( ) { remoteStorePinnedTimestampService.pinTimestamp( timestampToPin, - snapshot.getRepository() + "__" + snapshot.getSnapshotId().getUUID(), + snapshot.getRepository() + SNAPSHOT_PINNED_TIMESTAMP_DELIMITER + snapshot.getSnapshotId().getUUID(), new ActionListener() { @Override public void onResponse(Void unused) { From ea7185525680ec4c4d5a4b64d066ec183e52ce99 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 27 Aug 2024 12:24:33 +0530 Subject: [PATCH 29/55] Move integ tests to DeleteSnapshotIT Signed-off-by: Anshu Agarwal --- .../remotestore/RemoteRestoreSnapshotIT.java | 171 ------------------ .../snapshots/DeleteSnapshotIT.java | 170 +++++++++++++++++ 2 files changed, 170 insertions(+), 171 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index 4efd1e79a9419..7519925c20ff3 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -1166,175 +1166,4 @@ public void testCreateSnapshotV2WithShallowCopySettingDisabled() throws Exceptio assertThat(snapshotInfo.getPinnedTimestamp(), equalTo(0L)); } - - public void testDeleteShallowCopyV2() throws Exception { - - Settings snapshotSettings = Settings.builder().put("snapshot.snapshot_v2", true).build(); - // Settings snapshotSettings = Settings.builder().put("snapshot.max_concurrent_operations", 1000).build(); - internalCluster().startClusterManagerOnlyNode(Settings.builder().put(snapshotSettings).build()); - internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); - internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); - - 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) - ) - ); - - Settings indexSettings = getIndexSettings(20, 0).build(); - createIndex(indexName1, indexSettings); - - Settings indexSettings2 = getIndexSettings(15, 0).build(); - createIndex(indexName2, indexSettings2); - - final int numDocsInIndex1 = 10; - final int numDocsInIndex2 = 20; - indexDocuments(client, indexName1, numDocsInIndex1); - indexDocuments(client, indexName2, numDocsInIndex2); - ensureGreen(indexName1, indexName2); - - CreateSnapshotResponse createSnapshotResponse = client().admin() - .cluster() - .prepareCreateSnapshot(snapshotRepoName, snapshotName1) - .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)); - - indexDocuments(client, indexName2, 10); - indexDocuments(client, indexName1, 10); - createIndex(indexName3); - indexDocuments(client, 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, 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 { - - Settings snapshotSettings = Settings.builder().put("snapshot.snapshot_v2", true).build(); - // Settings snapshotSettings = Settings.builder().put("snapshot.max_concurrent_operations", 1000).build(); - internalCluster().startClusterManagerOnlyNode(Settings.builder().put(snapshotSettings).build()); - internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); - internalCluster().startDataOnlyNode(Settings.builder().put(snapshotSettings).build()); - - 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) - ) - ); - - Settings indexSettings = getIndexSettings(20, 0).build(); - createIndex(indexName1, indexSettings); - - Settings indexSettings2 = getIndexSettings(15, 0).build(); - createIndex(indexName2, indexSettings2); - - final int numDocsInIndex1 = 10; - final int numDocsInIndex2 = 20; - indexDocuments(client, indexName1, numDocsInIndex1); - indexDocuments(client, indexName2, numDocsInIndex2); - ensureGreen(indexName1, indexName2); - - CreateSnapshotResponse createSnapshotResponse = client().admin() - .cluster() - .prepareCreateSnapshot(snapshotRepoName, snapshotName1) - .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)); - - indexDocuments(client, indexName2, 10); - indexDocuments(client, indexName1, 10); - createIndex(indexName3); - indexDocuments(client, 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() - ); - - } - } diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java index e688a4491b1a7..05aa43f02387a 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java @@ -18,10 +18,12 @@ import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.unit.ByteSizeUnit; import org.opensearch.index.store.RemoteBufferedOutputDirectory; import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.repositories.fs.FsRepository; import org.opensearch.test.OpenSearchIntegTestCase; import java.nio.file.Path; @@ -32,6 +34,8 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Stream; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; import static org.opensearch.index.remote.RemoteStoreEnums.DataCategory.SEGMENTS; import static org.opensearch.index.remote.RemoteStoreEnums.DataType.LOCK_FILES; import static org.opensearch.remotestore.RemoteStoreBaseIntegTestCase.remoteStoreClusterSettings; @@ -363,6 +367,172 @@ public void testRemoteStoreCleanupForDeletedIndex() throws Exception { }, 30, TimeUnit.SECONDS); } + public void testDeleteShallowCopyV2() throws Exception { + disableRepoConsistencyCheck("Remote store repository is being used in the test"); + + final Path remoteStoreRepoPath = randomRepoPath(); + internalCluster().startClusterManagerOnlyNode(remoteStoreClusterSettings(REMOTE_REPO_NAME, remoteStoreRepoPath)); + + internalCluster().startDataOnlyNode(remoteStoreClusterSettings(REMOTE_REPO_NAME, remoteStoreRepoPath)); + internalCluster().startDataOnlyNode(remoteStoreClusterSettings(REMOTE_REPO_NAME, 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.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) + .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, 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(remoteStoreClusterSettings(REMOTE_REPO_NAME, remoteStoreRepoPath)); + internalCluster().startDataOnlyNode(remoteStoreClusterSettings(REMOTE_REPO_NAME, remoteStoreRepoPath)); + internalCluster().startDataOnlyNode(remoteStoreClusterSettings(REMOTE_REPO_NAME, 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.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) + .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() + ); + + } + private List createNSnapshots(String repoName, int count) { final List snapshotNames = new ArrayList<>(count); final String prefix = "snap-" + UUIDs.randomBase64UUID(random()).toLowerCase(Locale.ROOT) + "-"; From 14e7539a68326130de613d0a8af72418c802bb59 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Wed, 28 Aug 2024 15:43:01 +0530 Subject: [PATCH 30/55] Refactor code Signed-off-by: Anshu Agarwal --- .../opensearch/remotestore/RemoteRestoreSnapshotIT.java | 7 ------- .../java/org/opensearch/snapshots/DeleteSnapshotIT.java | 9 ++++----- .../repositories/blobstore/BlobStoreTestUtil.java | 1 + 3 files changed, 5 insertions(+), 12 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index d859177109f73..dc0654c623137 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -15,7 +15,6 @@ import org.opensearch.action.admin.indices.delete.DeleteIndexRequest; import org.opensearch.action.delete.DeleteResponse; import org.opensearch.action.support.PlainActionFuture; -import org.opensearch.action.support.master.AcknowledgedResponse; import org.opensearch.client.Client; import org.opensearch.client.Requests; import org.opensearch.cluster.ClusterState; @@ -43,7 +42,6 @@ import org.opensearch.repositories.fs.FsRepository; import org.opensearch.snapshots.AbstractSnapshotIntegTestCase; import org.opensearch.snapshots.SnapshotInfo; -import org.opensearch.snapshots.SnapshotMissingException; import org.opensearch.snapshots.SnapshotRestoreException; import org.opensearch.snapshots.SnapshotState; import org.opensearch.test.InternalTestCluster; @@ -759,11 +757,9 @@ public void testInvalidRestoreRequestScenarios() throws Exception { } public void testCreateSnapshotV2() throws Exception { - internalCluster().startClusterManagerOnlyNode(pinnedTimestampSettings()); internalCluster().startDataOnlyNode(pinnedTimestampSettings()); internalCluster().startDataOnlyNode(pinnedTimestampSettings()); - String indexName1 = "testindex1"; String indexName2 = "testindex2"; String indexName3 = "testindex3"; @@ -912,7 +908,6 @@ public void testMixedSnapshotCreationWithV2RepositorySetting() throws Exception } public void testConcurrentSnapshotV2CreateOperation() throws InterruptedException, ExecutionException { - internalCluster().startClusterManagerOnlyNode(pinnedTimestampSettings()); internalCluster().startDataOnlyNode(pinnedTimestampSettings()); internalCluster().startDataOnlyNode(pinnedTimestampSettings()); @@ -994,7 +989,6 @@ public void testConcurrentSnapshotV2CreateOperation() throws InterruptedExceptio } public void testCreateSnapshotV2WithRedIndex() throws Exception { - internalCluster().startClusterManagerOnlyNode(pinnedTimestampSettings()); internalCluster().startDataOnlyNode(pinnedTimestampSettings()); internalCluster().startDataOnlyNode(pinnedTimestampSettings()); @@ -1123,7 +1117,6 @@ public void testCreateSnapshotV2WithIndexingLoad() throws Exception { } public void testCreateSnapshotV2WithShallowCopySettingDisabled() throws Exception { - internalCluster().startClusterManagerOnlyNode(pinnedTimestampSettings()); internalCluster().startDataOnlyNode(pinnedTimestampSettings()); internalCluster().startDataOnlyNode(pinnedTimestampSettings()); diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java index a8e1e8276ea94..8ef3a0d0f3570 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java @@ -35,13 +35,13 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Stream; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThan; import static org.opensearch.index.remote.RemoteStoreEnums.DataCategory.SEGMENTS; import static org.opensearch.index.remote.RemoteStoreEnums.DataType.LOCK_FILES; import static org.opensearch.remotestore.RemoteStoreBaseIntegTestCase.remoteStoreClusterSettings; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; import static org.hamcrest.Matchers.comparesEqualTo; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.is; @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) @@ -405,7 +405,7 @@ public void testDeleteShallowCopyV2() throws Exception { .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.SNAPSHOT_V2.getKey(), true) + .put(BlobStoreRepository.SHALLOW_SNAPSHOT_V2.getKey(), true) ) ); @@ -486,7 +486,7 @@ public void testDeleteShallowCopyV2MultipleSnapshots() throws Exception { .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.SNAPSHOT_V2.getKey(), true) + .put(BlobStoreRepository.SHALLOW_SNAPSHOT_V2.getKey(), true) ) ); @@ -510,7 +510,6 @@ public void testDeleteShallowCopyV2MultipleSnapshots() throws Exception { assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards())); assertThat(snapshotInfo.snapshotId().getName(), equalTo(snapshotName1)); - createIndex(indexName3, getRemoteStoreBackedIndexSettings()); indexRandomDocs(indexName3, 10); diff --git a/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java b/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java index c35f8f9c35459..8398919c8fb8b 100644 --- a/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java +++ b/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java @@ -317,6 +317,7 @@ private static void assertSnapshotUUIDs(BlobStoreRepository repository, Reposito ); } } + } } } From f4b4122503b3b6c2d4090221da270894dc831c9d Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Wed, 28 Aug 2024 15:45:14 +0530 Subject: [PATCH 31/55] refactor code Signed-off-by: Anshu Agarwal --- .../cluster/snapshots/create/TransportCreateSnapshotAction.java | 1 - .../opensearch/repositories/blobstore/BlobStoreTestUtil.java | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java index 4c2a2526dda45..25e71d5598a98 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java @@ -113,7 +113,6 @@ protected void clusterManagerOperation( ) { Repository repository = repositoriesService.repository(request.repository()); boolean isSnapshotV2 = SHALLOW_SNAPSHOT_V2.get(repository.getMetadata().settings()); - if (request.waitForCompletion() || isSnapshotV2) { snapshotsService.executeSnapshot(request, ActionListener.map(listener, CreateSnapshotResponse::new)); } else { diff --git a/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java b/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java index 8398919c8fb8b..c75aaed2d1e07 100644 --- a/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java +++ b/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java @@ -316,8 +316,8 @@ private static void assertSnapshotUUIDs(BlobStoreRepository repository, Reposito lessThanOrEqualTo(2L) ); } + } - } } } From cb03f59659ffd9f3bf382691065bb527e86d44bc Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Wed, 28 Aug 2024 16:44:17 +0530 Subject: [PATCH 32/55] refactor code Signed-off-by: Anshu Agarwal --- .../repositories/blobstore/BlobStoreRepository.java | 4 ++-- .../opensearch/repositories/blobstore/BlobStoreTestUtil.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) 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 f8ad6b992a04d..52f2c6b418df7 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -947,7 +947,7 @@ public void deleteSnapshots( repositoryStateId, repositoryMetaVersion, null, // Passing null since no remote store lock files need to be cleaned up. - false, + false, // true only for shallow snapshot v2 listener ); } @@ -1076,7 +1076,7 @@ private void doDeleteShardSnapshots( remoteStoreLockManagerFactory, afterCleanupsListener ); - if (!isSnapshotV2) { + if (isSnapshotV2 == false) { asyncCleanupUnlinkedShardLevelBlobs( repositoryData, snapshotIds, diff --git a/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java b/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java index c75aaed2d1e07..187c30be0044e 100644 --- a/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java +++ b/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java @@ -316,7 +316,7 @@ private static void assertSnapshotUUIDs(BlobStoreRepository repository, Reposito lessThanOrEqualTo(2L) ); } - + } } } From 1f966fdccc00d10fb53a74d3ead630437c9a1472 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Wed, 28 Aug 2024 18:27:20 +0530 Subject: [PATCH 33/55] fix tests Signed-off-by: Anshu Agarwal --- .../snapshots/DeleteSnapshotIT.java | 20 +++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java index 8ef3a0d0f3570..92ee4a2432532 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java @@ -378,10 +378,10 @@ public void testDeleteShallowCopyV2() throws Exception { disableRepoConsistencyCheck("Remote store repository is being used in the test"); final Path remoteStoreRepoPath = randomRepoPath(); - internalCluster().startClusterManagerOnlyNode(remoteStoreClusterSettings(REMOTE_REPO_NAME, remoteStoreRepoPath)); + internalCluster().startClusterManagerOnlyNode(snapshotV2Settings(remoteStoreRepoPath)); - internalCluster().startDataOnlyNode(remoteStoreClusterSettings(REMOTE_REPO_NAME, remoteStoreRepoPath)); - internalCluster().startDataOnlyNode(remoteStoreClusterSettings(REMOTE_REPO_NAME, remoteStoreRepoPath)); + internalCluster().startDataOnlyNode(snapshotV2Settings(remoteStoreRepoPath)); + internalCluster().startDataOnlyNode(snapshotV2Settings(remoteStoreRepoPath)); String indexName1 = "testindex1"; String indexName2 = "testindex2"; @@ -460,9 +460,9 @@ public void testDeleteShallowCopyV2MultipleSnapshots() throws Exception { disableRepoConsistencyCheck("Remote store repository is being used in the test"); final Path remoteStoreRepoPath = randomRepoPath(); - internalCluster().startClusterManagerOnlyNode(remoteStoreClusterSettings(REMOTE_REPO_NAME, remoteStoreRepoPath)); - internalCluster().startDataOnlyNode(remoteStoreClusterSettings(REMOTE_REPO_NAME, remoteStoreRepoPath)); - internalCluster().startDataOnlyNode(remoteStoreClusterSettings(REMOTE_REPO_NAME, remoteStoreRepoPath)); + internalCluster().startClusterManagerOnlyNode(snapshotV2Settings(remoteStoreRepoPath)); + internalCluster().startDataOnlyNode(snapshotV2Settings(remoteStoreRepoPath)); + internalCluster().startDataOnlyNode(snapshotV2Settings(remoteStoreRepoPath)); String indexName1 = "testindex1"; String indexName2 = "testindex2"; @@ -550,4 +550,12 @@ private List createNSnapshots(String repoName, int count) { logger.info("--> created {} in [{}]", snapshotNames, repoName); return snapshotNames; } + + 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; + } } From b97707ab7af67f6173e1e239f2592e87f970c299 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Wed, 28 Aug 2024 18:41:20 +0530 Subject: [PATCH 34/55] fix build failure Signed-off-by: Anshu Agarwal --- .../main/java/org/opensearch/snapshots/SnapshotsService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index f485fe8b1da02..35e96b87de8b6 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -2495,7 +2495,7 @@ private void deleteSnapshotsFromRepository( snapshotsWithLockFiles.add(snapshotId); } } catch (Exception e) { - logger.warn("Failed to get snapshot info for {}", snapshotId, e); + logger.warn("Failed to get snapshot info for {} with exception {}", snapshotId, e); removeSnapshotDeletionFromClusterState(deleteEntry, e, repositoryData); } } From a94642ca571b4dc339f69bac51e03d68642c8067 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Wed, 28 Aug 2024 18:50:42 +0530 Subject: [PATCH 35/55] Refactor code Signed-off-by: Anshu Agarwal --- .../java/org/opensearch/repositories/Repository.java | 2 +- .../repositories/blobstore/BlobStoreRepository.java | 10 +++++----- .../org/opensearch/snapshots/SnapshotsService.java | 4 ++-- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/org/opensearch/repositories/Repository.java b/server/src/main/java/org/opensearch/repositories/Repository.java index 1af648f3e9508..6e06e22d58fe1 100644 --- a/server/src/main/java/org/opensearch/repositories/Repository.java +++ b/server/src/main/java/org/opensearch/repositories/Repository.java @@ -194,7 +194,7 @@ default void deleteSnapshotsAndReleaseLockFiles( long repositoryStateId, Version repositoryMetaVersion, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, - boolean isSnapshotV2, + boolean isShallowSnapshotV2, ActionListener listener ) { throw new UnsupportedOperationException(); 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 52f2c6b418df7..7c0c649a6f070 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -900,7 +900,7 @@ public void deleteSnapshotsAndReleaseLockFiles( long repositoryStateId, Version repositoryMetaVersion, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, - boolean isSnapshotV2, + boolean isShallowSnapshotV2, ActionListener listener ) { if (isReadOnly()) { @@ -922,7 +922,7 @@ protected void doRun() throws Exception { repositoryData, repositoryMetaVersion, remoteStoreLockManagerFactory, - isSnapshotV2, + isShallowSnapshotV2, listener ); } @@ -1022,7 +1022,7 @@ private void doDeleteShardSnapshots( RepositoryData repositoryData, Version repoMetaVersion, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, - boolean isSnapshotV2, + boolean isShallowSnapshotV2, ActionListener listener ) { // First write the new shard state metadata (with the removed snapshot) and compute deletion targets @@ -1060,7 +1060,7 @@ private void doDeleteShardSnapshots( // Once we have updated the repository, run the clean-ups writeUpdatedRepoDataStep.whenComplete(updatedRepoData -> { int groupSize = 2; - if (isSnapshotV2) { + if (isShallowSnapshotV2) { groupSize = 1; } // Run unreferenced blobs cleanup in parallel to shard-level snapshot deletion @@ -1076,7 +1076,7 @@ private void doDeleteShardSnapshots( remoteStoreLockManagerFactory, afterCleanupsListener ); - if (isSnapshotV2 == false) { + if (isShallowSnapshotV2 == false) { asyncCleanupUnlinkedShardLevelBlobs( repositoryData, snapshotIds, diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index 35e96b87de8b6..5482fe84ab6e1 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -462,6 +462,7 @@ public void createSnapshotV2(final CreateSnapshotRequest request, final ActionLi createSnapshotPreValidations(currentState, repositoryData, repositoryName, snapshotName); List indices = new ArrayList<>(currentState.metadata().indices().keySet()); + final List dataStreams = indexNameExpressionResolver.dataStreamNames( currentState, request.indicesOptions(), @@ -608,7 +609,7 @@ private void removeSnapshotPinnedTimestamp( ) { remoteStorePinnedTimestampService.unpinTimestamp( timestampToUnpin, - repository + "__" + snapshotId.getUUID(), + repository + SNAPSHOT_PINNED_TIMESTAMP_DELIMITER + snapshotId.getUUID(), new ActionListener() { @Override public void onResponse(Void unused) { @@ -625,7 +626,6 @@ public void onFailure(Exception e) { e ); listener.onFailure(e); - } } ); From 27f7554e6043069abef1a338aefc3be1cebcd116 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Fri, 30 Aug 2024 16:17:25 +0530 Subject: [PATCH 36/55] Add support to cleanup remote store segments for deleted index Signed-off-by: Anshu Agarwal --- .../snapshots/DeleteSnapshotIT.java | 3 + .../TransportCleanupRepositoryAction.java | 8 ++ .../opensearch/repositories/Repository.java | 12 ++ .../repositories/RepositoryData.java | 5 + .../blobstore/BlobStoreRepository.java | 113 +++++++++++++++++- .../snapshots/SnapshotsService.java | 12 +- 6 files changed, 147 insertions(+), 6 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java index 92ee4a2432532..098d610737ed4 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java @@ -441,6 +441,9 @@ public void testDeleteShallowCopyV2() throws Exception { 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) 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 774bffa10da4f..db2c3efc80be2 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 @@ -52,6 +52,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.repositories.RepositoriesService; import org.opensearch.repositories.Repository; @@ -96,6 +97,8 @@ public final class TransportCleanupRepositoryAction extends TransportClusterMana private final RemoteStoreLockManagerFactory remoteStoreLockManagerFactory; + private final RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory; + @Override protected String executor() { return ThreadPool.Names.SAME; @@ -123,6 +126,10 @@ public TransportCleanupRepositoryAction( this.repositoriesService = repositoriesService; this.snapshotsService = snapshotsService; this.remoteStoreLockManagerFactory = new RemoteStoreLockManagerFactory(() -> repositoriesService); + this.remoteSegmentStoreDirectoryFactory = new RemoteSegmentStoreDirectoryFactory( + () -> repositoriesService, + threadPool + ); // We add a state applier that will remove any dangling repository cleanup actions on cluster-manager failover. // This is safe to do since cleanups will increment the repository state id before executing any operations to prevent concurrent // operations from corrupting the repository. This is the same safety mechanism used by snapshot deletes. @@ -272,6 +279,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/repositories/Repository.java b/server/src/main/java/org/opensearch/repositories/Repository.java index 6e06e22d58fe1..f3276e7e6231e 100644 --- a/server/src/main/java/org/opensearch/repositories/Repository.java +++ b/server/src/main/java/org/opensearch/repositories/Repository.java @@ -50,6 +50,7 @@ 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; @@ -194,12 +195,23 @@ default void deleteSnapshotsAndReleaseLockFiles( long repositoryStateId, Version repositoryMetaVersion, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, boolean isShallowSnapshotV2, ActionListener listener ) { throw new UnsupportedOperationException(); } + default void deleteSnapshotsWithPinnedTimestamp( + Collection snapshotIds, + long repositoryStateId, + Version repositoryMetaVersion, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, + ActionListener listener + ) { + throw new UnsupportedOperationException(); + } + /** * Returns snapshot throttle time in nanoseconds */ diff --git a/server/src/main/java/org/opensearch/repositories/RepositoryData.java b/server/src/main/java/org/opensearch/repositories/RepositoryData.java index ea48d9b1a49fe..5a2679f64c70c 100644 --- a/server/src/main/java/org/opensearch/repositories/RepositoryData.java +++ b/server/src/main/java/org/opensearch/repositories/RepositoryData.java @@ -111,6 +111,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 7c0c649a6f070..560a6c7600879 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -111,6 +111,7 @@ import org.opensearch.index.mapper.MapperService; import org.opensearch.index.remote.RemoteStorePathStrategy; import org.opensearch.index.remote.RemoteStorePathStrategy.PathInput; +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; @@ -152,6 +153,7 @@ import java.io.IOException; import java.io.InputStream; import java.nio.file.NoSuchFileException; +import java.rmi.Remote; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -900,6 +902,7 @@ public void deleteSnapshotsAndReleaseLockFiles( long repositoryStateId, Version repositoryMetaVersion, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, boolean isShallowSnapshotV2, ActionListener listener ) { @@ -922,6 +925,7 @@ protected void doRun() throws Exception { repositoryData, repositoryMetaVersion, remoteStoreLockManagerFactory, + remoteSegmentStoreDirectoryFactory, isShallowSnapshotV2, listener ); @@ -935,6 +939,25 @@ public void onFailure(Exception e) { } } + @Override + public void deleteSnapshotsWithPinnedTimestamp( + Collection snapshotIds, + long repositoryStateId, + Version repositoryMetaVersion, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, + ActionListener listener + ) { + deleteSnapshotsAndReleaseLockFiles( + snapshotIds, + repositoryStateId, + repositoryMetaVersion, + null, // Passing null since no remote store lock files need to be cleaned up. + remoteSegmentStoreDirectoryFactory, + true, // true only for shallow snapshot v2 + listener + ); + } + @Override public void deleteSnapshots( Collection snapshotIds, @@ -947,6 +970,7 @@ public void deleteSnapshots( 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 false, // true only for shallow snapshot v2 listener ); @@ -1022,6 +1046,7 @@ private void doDeleteShardSnapshots( RepositoryData repositoryData, Version repoMetaVersion, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, boolean isShallowSnapshotV2, ActionListener listener ) { @@ -1073,7 +1098,9 @@ private void doDeleteShardSnapshots( foundIndices, rootBlobs, updatedRepoData, + repositoryData, remoteStoreLockManagerFactory, + remoteSegmentStoreDirectoryFactory, afterCleanupsListener ); if (isShallowSnapshotV2 == false) { @@ -1094,7 +1121,9 @@ private void cleanupUnlinkedRootAndIndicesBlobs( Map foundIndices, Map rootBlobs, RepositoryData updatedRepoData, + RepositoryData oldRepoData, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, ActionListener listener ) { cleanupStaleBlobs( @@ -1102,7 +1131,9 @@ private void cleanupUnlinkedRootAndIndicesBlobs( foundIndices, rootBlobs, updatedRepoData, + oldRepoData, remoteStoreLockManagerFactory, + remoteSegmentStoreDirectoryFactory, ActionListener.map(listener, ignored -> null) ); } @@ -1461,7 +1492,9 @@ private void cleanupStaleBlobs( Map foundIndices, Map rootBlobs, RepositoryData newRepoData, + RepositoryData oldRepoData, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, ActionListener listener ) { final GroupedActionListener groupedListener = new GroupedActionListener<>(ActionListener.wrap(deleteResults -> { @@ -1487,7 +1520,14 @@ private void cleanupStaleBlobs( if (foundIndices.keySet().equals(survivingIndexIds)) { groupedListener.onResponse(DeleteResult.ZERO); } else { - cleanupStaleIndices(foundIndices, survivingIndexIds, remoteStoreLockManagerFactory, groupedListener); + cleanupStaleIndices( + foundIndices, + survivingIndexIds, + remoteStoreLockManagerFactory, + remoteSegmentStoreDirectoryFactory, + oldRepoData, + groupedListener + ); } } @@ -1510,6 +1550,7 @@ public void cleanup( long repositoryStateId, Version repositoryMetaVersion, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, ActionListener listener ) { try { @@ -1541,8 +1582,10 @@ public void cleanup( Collections.emptyList(), foundIndices, rootBlobs, + v, repositoryData, remoteStoreLockManagerFactory, + null, ActionListener.map(listener, RepositoryCleanupResult::new) ), listener::onFailure @@ -1635,6 +1678,8 @@ private void cleanupStaleIndices( Map foundIndices, Set survivingIndexIds, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, + RepositoryData oldRepoData, GroupedActionListener listener ) { final GroupedActionListener groupedListener = new GroupedActionListener<>(ActionListener.wrap(deleteResults -> { @@ -1659,7 +1704,13 @@ private void cleanupStaleIndices( foundIndices.size() - survivingIndexIds.size() ); for (int i = 0; i < workers; ++i) { - executeOneStaleIndexDelete(staleIndicesToDelete, remoteStoreLockManagerFactory, groupedListener); + executeOneStaleIndexDelete( + staleIndicesToDelete, + remoteStoreLockManagerFactory, + remoteSegmentStoreDirectoryFactory, + oldRepoData, + groupedListener + ); } } catch (Exception e) { // TODO: We shouldn't be blanket catching and suppressing all exceptions here and instead handle them safely upstream. @@ -1682,6 +1733,8 @@ private static boolean isIndexPresent(ClusterService clusterService, String inde private void executeOneStaleIndexDelete( BlockingQueue> staleIndicesToDelete, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, + RepositoryData oldRepoData, GroupedActionListener listener ) throws InterruptedException { Map.Entry indexEntry = staleIndicesToDelete.poll(0L, TimeUnit.MILLISECONDS); @@ -1706,7 +1759,10 @@ private void executeOneStaleIndexDelete( } } } + } else { + cleanRemoteStoreDirectoryIfNeeded(indexSnId, oldRepoData, remoteSegmentStoreDirectoryFactory); } + // Deleting the index folder deleteResult = indexEntry.getValue().delete(); logger.debug("[{}] Cleaned up stale index [{}]", metadata.name(), indexSnId); @@ -1725,7 +1781,13 @@ private void executeOneStaleIndexDelete( logger.warn(new ParameterizedMessage("[{}] Exception during single stale index delete", metadata.name()), e); } - executeOneStaleIndexDelete(staleIndicesToDelete, remoteStoreLockManagerFactory, listener); + executeOneStaleIndexDelete( + staleIndicesToDelete, + remoteStoreLockManagerFactory, + remoteSegmentStoreDirectoryFactory, + oldRepoData, + listener + ); return deleteResult; })); } @@ -1822,6 +1884,51 @@ public void finalizeSnapshot( }, onUpdateFailure); } + private void cleanRemoteStoreDirectoryIfNeeded( + String indexSnId, + RepositoryData oldRepoData, + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory + ) { + IndexId indexId = null; + List snapshotIds = Collections.emptyList(); + for (Map.Entry> entry : oldRepoData.getIndexSnapshots().entrySet()) { + indexId = entry.getKey(); + if (indexId.getId().equals(indexSnId)) { + snapshotIds = entry.getValue(); + break; + } + } + + for (SnapshotId snapshotId : snapshotIds) { + try { + 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() + ); + if (remoteStoreRepository == null) { + break; + } + 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 (IOException e) { + throw new RuntimeException(e); + } + } + + } + // Delete all old shard gen blobs that aren't referenced any longer as a result from moving to updated repository data private void cleanupOldShardGens(RepositoryData existingRepositoryData, RepositoryData updatedRepositoryData) { final List toDelete = new ArrayList<>(); diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index 5482fe84ab6e1..b0c699e444431 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -91,6 +91,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.node.remotestore.RemoteStorePinnedTimestampService; import org.opensearch.repositories.IndexId; @@ -2505,12 +2506,17 @@ private void deleteSnapshotsFromRepository( }); try { latch.await(); + RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory = new RemoteSegmentStoreDirectoryFactory( + () -> repositoriesService, + threadPool + ); if (snapshotsWithLockFiles.size() > 0) { repository.deleteSnapshotsAndReleaseLockFiles( snapshotsWithLockFiles, repositoryData.getGenId(), minCompatibleVersion(minNodeVersion, repositoryData, snapshotsWithLockFiles), remoteStoreLockManagerFactory, + remoteSegmentStoreDirectoryFactory, false, ActionListener.wrap(updatedRepoData -> { logger.info("snapshots {} deleted", snapshotsWithLockFiles); @@ -2519,18 +2525,18 @@ private void deleteSnapshotsFromRepository( ); } if (snapshotsWithPinnedTimestamp.size() > 0) { + final StepListener pinnedTimestampListener = new StepListener<>(); pinnedTimestampListener.whenComplete( updatedRepoData -> { removeSnapshotDeletionFromClusterState(deleteEntry, null, updatedRepoData); }, ex -> removeSnapshotDeletionFromClusterState(deleteEntry, ex, repositoryData) ); - repository.deleteSnapshotsAndReleaseLockFiles( + repository.deleteSnapshotsWithPinnedTimestamp( snapshotsWithPinnedTimestamp.keySet(), repositoryData.getGenId(), minCompatibleVersion(minNodeVersion, repositoryData, snapshotsWithPinnedTimestamp.keySet()), - null, - true, + remoteSegmentStoreDirectoryFactory, ActionListener.wrap(updatedRepoData -> { logger.info("snapshots {} deleted", snapshotsWithPinnedTimestamp); removeSnapshotsPinnedTimestamp( From f1adb8e4391c676cafa81cc16cd8f98ae4a3c9f5 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Fri, 30 Aug 2024 16:24:05 +0530 Subject: [PATCH 37/55] refactor code Signed-off-by: Anshu Agarwal --- .../cleanup/TransportCleanupRepositoryAction.java | 5 +---- .../repositories/blobstore/BlobStoreRepository.java | 5 ++--- 2 files changed, 3 insertions(+), 7 deletions(-) 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 db2c3efc80be2..4085ed70cafc0 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 @@ -126,10 +126,7 @@ public TransportCleanupRepositoryAction( this.repositoriesService = repositoriesService; this.snapshotsService = snapshotsService; this.remoteStoreLockManagerFactory = new RemoteStoreLockManagerFactory(() -> repositoriesService); - this.remoteSegmentStoreDirectoryFactory = new RemoteSegmentStoreDirectoryFactory( - () -> repositoriesService, - threadPool - ); + this.remoteSegmentStoreDirectoryFactory = new RemoteSegmentStoreDirectoryFactory(() -> repositoriesService, threadPool); // We add a state applier that will remove any dangling repository cleanup actions on cluster-manager failover. // This is safe to do since cleanups will increment the repository state id before executing any operations to prevent concurrent // operations from corrupting the repository. This is the same safety mechanism used by snapshot deletes. 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 560a6c7600879..7a0f26ff86ef4 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -153,7 +153,6 @@ import java.io.IOException; import java.io.InputStream; import java.nio.file.NoSuchFileException; -import java.rmi.Remote; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -970,7 +969,7 @@ public void deleteSnapshots( 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, // Passing null since no remote store segment files need to be cleaned up false, // true only for shallow snapshot v2 listener ); @@ -1759,7 +1758,7 @@ private void executeOneStaleIndexDelete( } } } - } else { + } else if (remoteSegmentStoreDirectoryFactory != null) { cleanRemoteStoreDirectoryIfNeeded(indexSnId, oldRepoData, remoteSegmentStoreDirectoryFactory); } From 32ed74fc8486cdb11f0ba5a30f493e652bad6644 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Sat, 31 Aug 2024 22:50:25 +0530 Subject: [PATCH 38/55] refactor code Signed-off-by: Anshu Agarwal --- .../blobstore/BlobStoreRepository.java | 74 ++++++++++++------- 1 file changed, 46 insertions(+), 28 deletions(-) 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 7a0f26ff86ef4..91e019ed85d83 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -1888,42 +1888,60 @@ private void cleanRemoteStoreDirectoryIfNeeded( RepositoryData oldRepoData, RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory ) { + assert (indexSnId != null); + IndexId indexId = null; List snapshotIds = Collections.emptyList(); - for (Map.Entry> entry : oldRepoData.getIndexSnapshots().entrySet()) { - indexId = entry.getKey(); - if (indexId.getId().equals(indexSnId)) { - snapshotIds = entry.getValue(); - break; + 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 { + 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); - for (SnapshotId snapshotId : snapshotIds) { - try { - 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() - ); - if (remoteStoreRepository == null) { - break; - } - 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 + 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 (IOException e) { - throw new RuntimeException(e); } + } catch (Exception e) { + logger.error(new ParameterizedMessage("Exception during the remote directory cleanup for indecSnId [{}]", indexSnId), e); } } From 49981744f1f331781a5e7800c2b410f113446b89 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Mon, 2 Sep 2024 11:21:41 +0530 Subject: [PATCH 39/55] Fix tests Signed-off-by: Anshu Agarwal --- .../blobstore/BlobStoreRepository.java | 38 +++++++++---------- .../blobstore/BlobStoreRepositoryTests.java | 6 ++- 2 files changed, 24 insertions(+), 20 deletions(-) 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 c9247cead7b8e..e0bad61652e78 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -113,8 +113,8 @@ import org.opensearch.index.remote.RemoteStoreEnums.PathType; import org.opensearch.index.remote.RemoteStorePathStrategy; import org.opensearch.index.remote.RemoteStorePathStrategy.PathInput; -import org.opensearch.index.remote.RemoteStoreUtils; 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; @@ -778,10 +778,10 @@ public void updateState(ClusterState state) { assert wasBestEffortConsistency || metadata.generation() == RepositoryData.CORRUPTED_REPO_GEN || previousBest < metadata.generation() : "Illegal move from repository generation [" - + previousBest - + "] to generation [" - + metadata.generation() - + "]"; + + previousBest + + "] to generation [" + + metadata.generation() + + "]"; logger.debug("Updated repository generation from [{}] to [{}]", previousBest, metadata.generation()); } } @@ -1186,7 +1186,7 @@ private void doDeleteShardSnapshots( afterCleanupsListener, idToShardInfoMap ); - if (isShallowSnapshotV2==false) { + if (isShallowSnapshotV2 == false) { asyncCleanupUnlinkedShardLevelBlobs( repositoryData, snapshotIds, @@ -1903,7 +1903,7 @@ 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(remoteStoreLockManagerFactory!=null) { + if (remoteStoreLockManagerFactory != null) { cleanRemoteStoreDirectoryIfNeeded(indexSnId, oldRepoData, remoteSegmentStoreDirectoryFactory); } @@ -2481,8 +2481,8 @@ public long getRemoteDownloadThrottleTimeInNanos() { protected void assertSnapshotOrGenericThread() { assert Thread.currentThread().getName().contains('[' + ThreadPool.Names.SNAPSHOT + ']') || Thread.currentThread().getName().contains('[' + ThreadPool.Names.GENERIC + ']') : "Expected current thread [" - + Thread.currentThread() - + "] to be the snapshot or generic thread."; + + Thread.currentThread() + + "] to be the snapshot or generic thread."; } @Override @@ -4018,13 +4018,13 @@ private static List unusedBlobs( .filter( blob -> blob.startsWith(SNAPSHOT_INDEX_PREFIX) || (blob.startsWith(SNAPSHOT_PREFIX) - && blob.endsWith(".dat") - && survivingSnapshotUUIDs.contains( - blob.substring(SNAPSHOT_PREFIX.length(), blob.length() - ".dat".length()) - ) == false) + && blob.endsWith(".dat") + && survivingSnapshotUUIDs.contains( + blob.substring(SNAPSHOT_PREFIX.length(), blob.length() - ".dat".length()) + ) == false) || (remoteStoreLockManagerFactory != null - && extractShallowSnapshotUUID(blob).map(snapshotUUID -> !survivingSnapshotUUIDs.contains(snapshotUUID)) - .orElse(false)) + && extractShallowSnapshotUUID(blob).map(snapshotUUID -> !survivingSnapshotUUIDs.contains(snapshotUUID)) + .orElse(false)) || (blob.startsWith(UPLOADED_DATA_BLOB_PREFIX) && updatedSnapshots.findNameFile(canonicalName(blob)) == null) || FsBlobContainer.isTempBlobName(blob) ) @@ -4094,10 +4094,10 @@ private Tuple buildBlobStoreIndexShardSnapsh return new Tuple<>(shardSnapshots, latest); } else if (blobs.stream() .anyMatch(b -> b.startsWith(SNAPSHOT_PREFIX) || b.startsWith(INDEX_FILE_PREFIX) || b.startsWith(UPLOADED_DATA_BLOB_PREFIX))) { - logger.warn( - "Could not find a readable index-N file in a non-empty shard snapshot directory [" + shardContainer.path() + "]" - ); - } + logger.warn( + "Could not find a readable index-N file in a non-empty shard snapshot directory [" + shardContainer.path() + "]" + ); + } return new Tuple<>(BlobStoreIndexShardSnapshots.EMPTY, latest); } 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 63257a5575970..019ab6ac6aa45 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; @@ -464,6 +465,7 @@ public void testCleanupStaleIndices() throws Exception { // 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); @@ -494,7 +496,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(), anyMap()); AtomicReference> resultReference = new AtomicReference<>(); CountDownLatch latch = new CountDownLatch(1); @@ -512,6 +514,8 @@ public void testCleanupStaleIndices() throws Exception { foundIndices, survivingIndexIds, mockRemoteStoreLockManagerFactory, + mockRemoteSegmentStoreDirectoryFactory, + mock(RepositoryData.class), listener, mockSnapshotShardPaths, Collections.emptyMap() From 52573e0db04e2168520a54bea95e04a7641b43d2 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Mon, 2 Sep 2024 13:34:08 +0530 Subject: [PATCH 40/55] remove pinned timestamp after updating repository data Signed-off-by: Anshu Agarwal --- .../common/blobstore/BlobMetadata.java | 3 + .../blobstore/stream/write/WritePriority.java | 3 + .../common/remote/BlobPathParameters.java | 3 + .../remote/RemoteWriteableBlobEntity.java | 2 + .../remote/model/RemotePinnedTimestamps.java | 2 + .../RemoteStorePinnedTimestampsBlobStore.java | 2 + .../transfer/BlobStoreTransferService.java | 1 + .../index/translog/transfer/FileSnapshot.java | 2 + .../RemoteStorePinnedTimestampService.java | 2 + .../opensearch/repositories/Repository.java | 6 +- .../blobstore/BlobStoreRepository.java | 97 ++++++++++++++++++- .../snapshots/SnapshotsService.java | 76 +++------------ 12 files changed, 130 insertions(+), 69 deletions(-) diff --git a/server/src/main/java/org/opensearch/common/blobstore/BlobMetadata.java b/server/src/main/java/org/opensearch/common/blobstore/BlobMetadata.java index 37c70365b6a11..7d1c0e6bad5a9 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/BlobMetadata.java +++ b/server/src/main/java/org/opensearch/common/blobstore/BlobMetadata.java @@ -32,11 +32,14 @@ package org.opensearch.common.blobstore; +import org.opensearch.common.annotation.ExperimentalApi; + /** * An interface for providing basic metadata about a blob. * * @opensearch.internal */ +@ExperimentalApi public interface BlobMetadata { /** diff --git a/server/src/main/java/org/opensearch/common/blobstore/stream/write/WritePriority.java b/server/src/main/java/org/opensearch/common/blobstore/stream/write/WritePriority.java index 4e8db0a3a8c69..d88129053fc38 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/stream/write/WritePriority.java +++ b/server/src/main/java/org/opensearch/common/blobstore/stream/write/WritePriority.java @@ -8,11 +8,14 @@ package org.opensearch.common.blobstore.stream.write; +import org.opensearch.common.annotation.ExperimentalApi; + /** * WritePriority for upload * * @opensearch.internal */ +@ExperimentalApi public enum WritePriority { // Used for segment transfers during refresh, flush or merges NORMAL, diff --git a/server/src/main/java/org/opensearch/common/remote/BlobPathParameters.java b/server/src/main/java/org/opensearch/common/remote/BlobPathParameters.java index 58c73a804b66a..7c53af00d1659 100644 --- a/server/src/main/java/org/opensearch/common/remote/BlobPathParameters.java +++ b/server/src/main/java/org/opensearch/common/remote/BlobPathParameters.java @@ -8,12 +8,15 @@ package org.opensearch.common.remote; +import org.opensearch.common.annotation.ExperimentalApi; + import java.util.List; /** * Parameters which can be used to construct a blob path * */ +@ExperimentalApi public class BlobPathParameters { private final List pathTokens; diff --git a/server/src/main/java/org/opensearch/common/remote/RemoteWriteableBlobEntity.java b/server/src/main/java/org/opensearch/common/remote/RemoteWriteableBlobEntity.java index f034ce2d1adf1..d49ef69c65ed3 100644 --- a/server/src/main/java/org/opensearch/common/remote/RemoteWriteableBlobEntity.java +++ b/server/src/main/java/org/opensearch/common/remote/RemoteWriteableBlobEntity.java @@ -8,6 +8,7 @@ package org.opensearch.common.remote; +import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.core.compress.Compressor; @@ -17,6 +18,7 @@ * The abstract class which represents a {@link RemoteWriteableEntity} that can be written to a store * @param the entity to be written */ +@ExperimentalApi public abstract class RemoteWriteableBlobEntity implements RemoteWriteableEntity { protected String blobFileName; diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemotePinnedTimestamps.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemotePinnedTimestamps.java index 030491cf8b7b9..ef8f658f4bb52 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/model/RemotePinnedTimestamps.java +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemotePinnedTimestamps.java @@ -10,6 +10,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.io.Streams; import org.opensearch.common.remote.BlobPathParameters; import org.opensearch.common.remote.RemoteWriteableBlobEntity; @@ -42,6 +43,7 @@ public class RemotePinnedTimestamps extends RemoteWriteableBlobEntity> pinnedTimestampPinningEntityMap; diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteStorePinnedTimestampsBlobStore.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteStorePinnedTimestampsBlobStore.java index 2a65dd993d0af..d628d7f12afd0 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteStorePinnedTimestampsBlobStore.java +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteStorePinnedTimestampsBlobStore.java @@ -8,6 +8,7 @@ package org.opensearch.gateway.remote.model; +import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.remote.RemoteWriteableBlobEntity; import org.opensearch.common.remote.RemoteWriteableEntityBlobStore; @@ -18,6 +19,7 @@ /** * Extends the RemoteClusterStateBlobStore to support {@link RemotePinnedTimestamps} */ +@ExperimentalApi public class RemoteStorePinnedTimestampsBlobStore extends RemoteWriteableEntityBlobStore< RemotePinnedTimestamps.PinnedTimestamps, RemotePinnedTimestamps> { diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java index 22bb4cf0514bf..31b0c9a734503 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java @@ -52,6 +52,7 @@ * * @opensearch.internal */ +@ExperimentalApi public class BlobStoreTransferService implements TransferService { private final BlobStore blobStore; diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/FileSnapshot.java b/server/src/main/java/org/opensearch/index/translog/transfer/FileSnapshot.java index 86f042af0584b..cbdfcf8c9b306 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/FileSnapshot.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/FileSnapshot.java @@ -9,6 +9,7 @@ package org.opensearch.index.translog.transfer; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.lucene.store.ByteArrayIndexInput; import org.opensearch.common.lucene.store.InputStreamIndexInput; import org.opensearch.common.util.io.IOUtils; @@ -104,6 +105,7 @@ public void close() throws IOException { * * @opensearch.internal */ + @ExperimentalApi public static class TransferFileSnapshot extends FileSnapshot { private final long primaryTerm; 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 f7b262664d147..f98f9ef8b6fa1 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.opensearch.cluster.ClusterName; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.collect.Tuple; @@ -48,6 +49,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 f3276e7e6231e..b793f04043e44 100644 --- a/server/src/main/java/org/opensearch/repositories/Repository.java +++ b/server/src/main/java/org/opensearch/repositories/Repository.java @@ -54,6 +54,7 @@ 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; @@ -196,6 +197,8 @@ default void deleteSnapshotsAndReleaseLockFiles( Version repositoryMetaVersion, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, + RemoteStorePinnedTimestampService remoteStorePinnedTimestampService, + Map snapshotIdsPinnedTimestampMap, boolean isShallowSnapshotV2, ActionListener listener ) { @@ -203,10 +206,11 @@ default void deleteSnapshotsAndReleaseLockFiles( } default void deleteSnapshotsWithPinnedTimestamp( - Collection snapshotIds, + Map snapshotsWithPinnedTimestamp, long repositoryStateId, Version repositoryMetaVersion, RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, + RemoteStorePinnedTimestampService remoteStorePinnedTimestampService, ActionListener listener ) { throw new UnsupportedOperationException(); 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 e0bad61652e78..da15589f51d27 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -133,6 +133,7 @@ import org.opensearch.index.store.lockmanager.RemoteStoreLockManagerFactory; 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; @@ -188,6 +189,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 @@ -971,6 +973,8 @@ public void deleteSnapshotsAndReleaseLockFiles( Version repositoryMetaVersion, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, + RemoteStorePinnedTimestampService remoteStorePinnedTimestampService, + Map snapshotIdsPinnedTimestampMap, boolean isShallowSnapshotV2, ActionListener listener ) { @@ -994,6 +998,8 @@ protected void doRun() throws Exception { repositoryMetaVersion, remoteStoreLockManagerFactory, remoteSegmentStoreDirectoryFactory, + remoteStorePinnedTimestampService, + snapshotIdsPinnedTimestampMap, isShallowSnapshotV2, listener ); @@ -1009,18 +1015,21 @@ public void onFailure(Exception e) { @Override public void deleteSnapshotsWithPinnedTimestamp( - Collection snapshotIds, + Map snapshotIdPinnedTimestampMap, long repositoryStateId, Version repositoryMetaVersion, RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory, + RemoteStorePinnedTimestampService remoteStorePinnedTimestampService, ActionListener listener ) { deleteSnapshotsAndReleaseLockFiles( - snapshotIds, + snapshotIdPinnedTimestampMap.keySet(), repositoryStateId, repositoryMetaVersion, null, // Passing null since no remote store lock files need to be cleaned up. remoteSegmentStoreDirectoryFactory, + remoteStorePinnedTimestampService, + Collections.emptyMap(), true, // true only for shallow snapshot v2 listener ); @@ -1039,6 +1048,8 @@ public void deleteSnapshots( 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 ); @@ -1104,6 +1115,10 @@ private RepositoryData safeRepositoryData(long repositoryStateId, Map snapshotIdPinnedTimestampMap, boolean isShallowSnapshotV2, ActionListener listener ) { @@ -1151,12 +1168,26 @@ 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.size() == 0) { + pinnedTimestampListener.onResponse(updatedRepoData); + } else { + removeSnapshotsPinnedTimestamp( + snapshotIdPinnedTimestampMap, + this, + updatedRepoData, + remoteStorePinnedTimestampService, + pinnedTimestampListener + ); + } + }, listener::onFailure); + + pinnedTimestampListener.whenComplete(updatedRepoData -> { int groupSize = 2; if (isShallowSnapshotV2) { groupSize = 1; } - // Run unreferenced blobs cleanup in parallel to shard-level snapshot deletion final ActionListener afterCleanupsListener = new GroupedActionListener<>( ActionListener.wrap(() -> listener.onResponse(updatedRepoData)), @@ -1198,6 +1229,66 @@ private void doDeleteShardSnapshots( }, listener::onFailure); } + 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. * diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index 4e80d8d8f8039..e1bb9e7b72c43 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -206,7 +206,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus Setting.Property.Dynamic ); - private static final String SNAPSHOT_PINNED_TIMESTAMP_DELIMITER = ":"; + public static final String SNAPSHOT_PINNED_TIMESTAMP_DELIMITER = ":"; private volatile int maxConcurrentOperations; public SnapshotsService( @@ -608,36 +608,6 @@ public void onFailure(Exception e) { ); } - private void removeSnapshotPinnedTimestamp( - 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); - } - } - ); - } - private static void ensureSnapshotNameNotRunning( List runningSnapshots, String repositoryName, @@ -2523,6 +2493,8 @@ private void deleteSnapshotsFromRepository( minCompatibleVersion(minNodeVersion, repositoryData, snapshotsWithLockFiles), remoteStoreLockManagerFactory, remoteSegmentStoreDirectoryFactory, + null, + Collections.emptyMap(), false, ActionListener.wrap(updatedRepoData -> { logger.info("snapshots {} deleted", snapshotsWithLockFiles); @@ -2532,25 +2504,21 @@ private void deleteSnapshotsFromRepository( } if (snapshotsWithPinnedTimestamp.size() > 0) { - final StepListener pinnedTimestampListener = new StepListener<>(); - pinnedTimestampListener.whenComplete( - updatedRepoData -> { removeSnapshotDeletionFromClusterState(deleteEntry, null, updatedRepoData); }, - ex -> removeSnapshotDeletionFromClusterState(deleteEntry, ex, repositoryData) - ); + // final StepListener pinnedTimestampListener = new StepListener<>(); + // pinnedTimestampListener.whenComplete( + // updatedRepoData -> { removeSnapshotDeletionFromClusterState(deleteEntry, null, updatedRepoData); }, + // ex -> removeSnapshotDeletionFromClusterState(deleteEntry, ex, repositoryData) + // ); repository.deleteSnapshotsWithPinnedTimestamp( - snapshotsWithPinnedTimestamp.keySet(), + snapshotsWithPinnedTimestamp, repositoryData.getGenId(), minCompatibleVersion(minNodeVersion, repositoryData, snapshotsWithPinnedTimestamp.keySet()), remoteSegmentStoreDirectoryFactory, + remoteStorePinnedTimestampService, ActionListener.wrap(updatedRepoData -> { logger.info("snapshots {} deleted", snapshotsWithPinnedTimestamp); - removeSnapshotsPinnedTimestamp( - snapshotsWithPinnedTimestamp, - repository, - updatedRepoData, - pinnedTimestampListener - ); + removeSnapshotDeletionFromClusterState(deleteEntry, null, updatedRepoData); }, ex -> removeSnapshotDeletionFromClusterState(deleteEntry, ex, repositoryData)) ); } @@ -2575,28 +2543,6 @@ private void deleteSnapshotsFromRepository( } } - private void removeSnapshotsPinnedTimestamp( - Map snapshotsWithPinnedTimestamp, - Repository repository, - RepositoryData repositoryData, - 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(snapshotId, repository.getMetadata().name(), pinnedTimestamp, groupedListener); - }); - } - /** * Removes a {@link SnapshotDeletionsInProgress.Entry} from {@link SnapshotDeletionsInProgress} in the cluster state after it executed * on the repository. From 6d5fd217d14405c67b6f0c260e32150201bcc76b Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Mon, 2 Sep 2024 13:38:47 +0530 Subject: [PATCH 41/55] refactor code Signed-off-by: Anshu Agarwal --- .../java/org/opensearch/snapshots/SnapshotsService.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index e1bb9e7b72c43..e8239ae25d2eb 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -2504,12 +2504,6 @@ private void deleteSnapshotsFromRepository( } if (snapshotsWithPinnedTimestamp.size() > 0) { - // final StepListener pinnedTimestampListener = new StepListener<>(); - // pinnedTimestampListener.whenComplete( - // updatedRepoData -> { removeSnapshotDeletionFromClusterState(deleteEntry, null, updatedRepoData); }, - // ex -> removeSnapshotDeletionFromClusterState(deleteEntry, ex, repositoryData) - // ); - repository.deleteSnapshotsWithPinnedTimestamp( snapshotsWithPinnedTimestamp, repositoryData.getGenId(), From e36786a7aabc8d72a18b0c608cbb7c953f44e8d5 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Mon, 2 Sep 2024 14:19:38 +0530 Subject: [PATCH 42/55] Add tests for deleted index cleanup Signed-off-by: Anshu Agarwal --- .../snapshots/DeleteSnapshotIT.java | 85 +++++++++++++++++++ .../blobstore/BlobStoreRepository.java | 4 +- 2 files changed, 87 insertions(+), 2 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java index 098d610737ed4..d3d0db79a722e 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java @@ -542,6 +542,91 @@ public void testDeleteShallowCopyV2MultipleSnapshots() throws Exception { } + public void testRemoteStoreCleanupForDeletedIndexForSnapshotV2() throws Exception { + disableRepoConsistencyCheck("Remote store repository is being used in the test"); + final Path remoteStoreRepoPath = randomRepoPath(); + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + Settings settings = remoteStoreClusterSettings(REMOTE_REPO_NAME, remoteStoreRepoPath); + + internalCluster().startClusterManagerOnlyNode(snapshotV2Settings(remoteStoreRepoPath)); + internalCluster().startDataOnlyNode(snapshotV2Settings(remoteStoreRepoPath)); + final Client clusterManagerClient = internalCluster().clusterManagerClient(); + ensureStableCluster(2); + + final String snapshotRepoName = "snapshot-repo-name"; + final Path snapshotRepoPath = randomRepoPath(); + 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) + ) + ); + + final String remoteStoreEnabledIndexName = "remote-index-1"; + final Settings remoteStoreEnabledIndexSettings = getRemoteStoreBackedIndexSettings(); + createIndex(remoteStoreEnabledIndexName, remoteStoreEnabledIndexSettings); + indexRandomDocs(remoteStoreEnabledIndexName, randomIntBetween(5, 10)); + + String indexUUID = client().admin() + .indices() + .prepareGetSettings(remoteStoreEnabledIndexName) + .get() + .getSetting(remoteStoreEnabledIndexName, IndexMetadata.SETTING_INDEX_UUID); + + logger.info("--> create two remote index shallow snapshots"); + CreateSnapshotResponse createSnapshotResponse = client().admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, "snap1") + .setWaitForCompletion(true) + .get(); + SnapshotInfo snapshotInfo1 = createSnapshotResponse.getSnapshotInfo(); + + 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")); + final RepositoriesService repositoriesService = internalCluster().getCurrentClusterManagerNodeInstance(RepositoriesService.class); + + // delete remote store index + assertAcked(client().admin().indices().prepareDelete(remoteStoreEnabledIndexName)); + + logger.info("--> delete snapshot 1"); + AcknowledgedResponse deleteSnapshotResponse = clusterManagerClient.admin() + .cluster() + .prepareDeleteSnapshot(snapshotRepoName, snapshotInfo1.snapshotId().getName()) + .get(); + assertAcked(deleteSnapshotResponse); + + logger.info("--> delete snapshot 2"); + deleteSnapshotResponse = clusterManagerClient.admin() + .cluster() + .prepareDeleteSnapshot(snapshotRepoName, snapshotInfo2.snapshotId().getName()) + .get(); + assertAcked(deleteSnapshotResponse); + + Path indexPath = Path.of(String.valueOf(remoteStoreRepoPath), indexUUID); + // Delete is async. Give time for it + assertBusy(() -> { + try { + assertThat(RemoteStoreBaseIntegTestCase.getFileCount(indexPath), comparesEqualTo(0)); + } catch (Exception e) {} + }, 30, TimeUnit.SECONDS); + } + private List createNSnapshots(String repoName, int count) { final List snapshotNames = new ArrayList<>(count); final String prefix = "snap-" + UUIDs.randomBase64UUID(random()).toLowerCase(Locale.ROOT) + "-"; 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 da15589f51d27..92197b88b3337 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -1029,7 +1029,7 @@ public void deleteSnapshotsWithPinnedTimestamp( null, // Passing null since no remote store lock files need to be cleaned up. remoteSegmentStoreDirectoryFactory, remoteStorePinnedTimestampService, - Collections.emptyMap(), + snapshotIdPinnedTimestampMap, true, // true only for shallow snapshot v2 listener ); @@ -1994,7 +1994,7 @@ 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 (remoteStoreLockManagerFactory != null) { + if (remoteSegmentStoreDirectoryFactory != null) { cleanRemoteStoreDirectoryIfNeeded(indexSnId, oldRepoData, remoteSegmentStoreDirectoryFactory); } From 2b987b008c40f1ab2c9a0f426c78ee4439be6531 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Mon, 2 Sep 2024 17:45:02 +0530 Subject: [PATCH 43/55] Address review comments Signed-off-by: Anshu Agarwal --- .../blobstore/BlobStoreRepository.java | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) 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 92197b88b3337..a7f28a93b8a54 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -1170,7 +1170,7 @@ private void doDeleteShardSnapshots( // Once we have updated the repository, run the clean-ups final StepListener pinnedTimestampListener = new StepListener<>(); writeUpdatedRepoDataStep.whenComplete(updatedRepoData -> { - if (snapshotIdPinnedTimestampMap == null || snapshotIdPinnedTimestampMap.size() == 0) { + if (snapshotIdPinnedTimestampMap == null || snapshotIdPinnedTimestampMap.isEmpty()) { pinnedTimestampListener.onResponse(updatedRepoData); } else { removeSnapshotsPinnedTimestamp( @@ -1758,6 +1758,7 @@ 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( @@ -2031,6 +2032,22 @@ 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( String indexSnId, RepositoryData oldRepoData, From 4b7376b7c096e80e36d1789cdc93843eb8b5ddad Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Mon, 2 Sep 2024 19:11:59 +0530 Subject: [PATCH 44/55] Refactor code Signed-off-by: Anshu Agarwal --- .../opensearch/repositories/Repository.java | 14 ++++++- .../blobstore/BlobStoreRepository.java | 39 +++++++++++++++++-- .../snapshots/SnapshotsService.java | 4 -- .../blobstore/BlobStoreRepositoryTests.java | 4 +- 4 files changed, 51 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/org/opensearch/repositories/Repository.java b/server/src/main/java/org/opensearch/repositories/Repository.java index b793f04043e44..17d8a291e35ff 100644 --- a/server/src/main/java/org/opensearch/repositories/Repository.java +++ b/server/src/main/java/org/opensearch/repositories/Repository.java @@ -189,9 +189,11 @@ void deleteSnapshots( * @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 to pin/umpin the timestamp * @param listener completion listener */ - default void deleteSnapshotsAndReleaseLockFiles( + default void deleteSnapshotsInternal( Collection snapshotIds, long repositoryStateId, Version repositoryMetaVersion, @@ -216,6 +218,16 @@ default void deleteSnapshotsWithPinnedTimestamp( throw new UnsupportedOperationException(); } + default void deleteSnapshotsAndReleaseLockFiles( + Collection snapshotIds, + long repositoryStateId, + Version repositoryMetaVersion, + RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, + ActionListener listener + ) { + throw new UnsupportedOperationException(); + } + /** * Returns snapshot throttle time in nanoseconds */ 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 a7f28a93b8a54..dfc88f490f0b0 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -967,7 +967,7 @@ public RepositoryStats stats() { return new RepositoryStats(store.stats()); } - public void deleteSnapshotsAndReleaseLockFiles( + public void deleteSnapshotsInternal( Collection snapshotIds, long repositoryStateId, Version repositoryMetaVersion, @@ -1022,7 +1022,7 @@ public void deleteSnapshotsWithPinnedTimestamp( RemoteStorePinnedTimestampService remoteStorePinnedTimestampService, ActionListener listener ) { - deleteSnapshotsAndReleaseLockFiles( + deleteSnapshotsInternal( snapshotIdPinnedTimestampMap.keySet(), repositoryStateId, repositoryMetaVersion, @@ -1035,6 +1035,27 @@ public void deleteSnapshotsWithPinnedTimestamp( ); } + @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, @@ -1042,7 +1063,7 @@ public void deleteSnapshots( Version repositoryMetaVersion, ActionListener listener ) { - deleteSnapshotsAndReleaseLockFiles( + deleteSnapshotsInternal( snapshotIds, repositoryStateId, repositoryMetaVersion, @@ -1724,6 +1745,7 @@ private void cleanupStaleBlobs( } else { Map snapshotShardPaths = getSnapshotShardPaths(); cleanupStaleIndices( + deletedSnapshots, foundIndices, survivingIndexIds, remoteStoreLockManagerFactory, @@ -1891,6 +1913,7 @@ private List cleanupStaleRootFiles( } void cleanupStaleIndices( + Collection deletedSnapshots, Map foundIndices, Set survivingIndexIds, RemoteStoreLockManagerFactory remoteStoreLockManagerFactory, @@ -1923,6 +1946,7 @@ void cleanupStaleIndices( ); for (int i = 0; i < workers; ++i) { executeOneStaleIndexDelete( + deletedSnapshots, staleIndicesToDelete, remoteStoreLockManagerFactory, remoteSegmentStoreDirectoryFactory, @@ -1961,6 +1985,7 @@ 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, @@ -1996,7 +2021,7 @@ private void executeOneStaleIndexDelete( deleteResult = deleteResult.add(cleanUpStaleSnapshotShardPathsFile(matchingShardPaths, snapshotShardPaths)); if (remoteSegmentStoreDirectoryFactory != null) { - cleanRemoteStoreDirectoryIfNeeded(indexSnId, oldRepoData, remoteSegmentStoreDirectoryFactory); + cleanRemoteStoreDirectoryIfNeeded(deletedSnapshots, indexSnId, oldRepoData, remoteSegmentStoreDirectoryFactory); } // Finally, we delete the [base_path]/indexId folder @@ -2020,6 +2045,7 @@ private void executeOneStaleIndexDelete( return DeleteResult.ZERO; } finally { executeOneStaleIndexDelete( + deletedSnapshots, staleIndicesToDelete, remoteStoreLockManagerFactory, remoteSegmentStoreDirectoryFactory, @@ -2049,6 +2075,7 @@ private void executeOneStaleIndexDelete( * store segments */ private void cleanRemoteStoreDirectoryIfNeeded( + Collection deletedSnapshots, String indexSnId, RepositoryData oldRepoData, RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory @@ -2071,6 +2098,10 @@ private void cleanRemoteStoreDirectoryIfNeeded( } 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( diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index e8239ae25d2eb..548d4933898d0 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -2492,10 +2492,6 @@ private void deleteSnapshotsFromRepository( repositoryData.getGenId(), minCompatibleVersion(minNodeVersion, repositoryData, snapshotsWithLockFiles), remoteStoreLockManagerFactory, - remoteSegmentStoreDirectoryFactory, - null, - Collections.emptyMap(), - false, ActionListener.wrap(updatedRepoData -> { logger.info("snapshots {} deleted", snapshotsWithLockFiles); removeSnapshotDeletionFromClusterState(deleteEntry, null, updatedRepoData); 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 019ab6ac6aa45..d86bc38637f8d 100644 --- a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java +++ b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java @@ -463,6 +463,8 @@ public void testCleanupStaleIndices() throws Exception { 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); @@ -515,7 +517,7 @@ public void testCleanupStaleIndices() throws Exception { survivingIndexIds, mockRemoteStoreLockManagerFactory, mockRemoteSegmentStoreDirectoryFactory, - mock(RepositoryData.class), + repositoryData, listener, mockSnapshotShardPaths, Collections.emptyMap() From 5fa4237f7bcc991b716dfbe61002a74a66c32baa Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Mon, 2 Sep 2024 19:25:26 +0530 Subject: [PATCH 45/55] modify java doc Signed-off-by: Anshu Agarwal --- .../opensearch/repositories/Repository.java | 35 +++++++++++++++---- 1 file changed, 28 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/opensearch/repositories/Repository.java b/server/src/main/java/org/opensearch/repositories/Repository.java index 17d8a291e35ff..35d4507c1d018 100644 --- a/server/src/main/java/org/opensearch/repositories/Repository.java +++ b/server/src/main/java/org/opensearch/repositories/Repository.java @@ -185,13 +185,15 @@ 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 remoteSegmentStoreDirectoryFactory RemoteSegmentStoreDirectoryFactory to be used for cleaning up remote store segment files - * @param remoteStorePinnedTimestampService service to pin/umpin the timestamp - * @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, @@ -207,6 +209,16 @@ default void deleteSnapshotsInternal( 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, @@ -218,6 +230,15 @@ default void deleteSnapshotsWithPinnedTimestamp( 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, long repositoryStateId, From 88c3be8d7ae3184163edeb298cbfe3c894d4cded Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Mon, 2 Sep 2024 20:00:03 +0530 Subject: [PATCH 46/55] fix test Signed-off-by: Anshu Agarwal --- .../blobstore/BlobStoreRepositoryTests.java | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) 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 d86bc38637f8d..d9622aae4c378 100644 --- a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java +++ b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java @@ -75,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; @@ -460,6 +461,10 @@ 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"); @@ -483,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; @@ -498,7 +503,7 @@ public void testCleanupStaleIndices() throws Exception { listener.onResponse(result); return null; - }).when(repository).cleanupStaleIndices(any(), any(), 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); @@ -513,10 +518,11 @@ public void testCleanupStaleIndices() throws Exception { // Call the method we're testing repository.cleanupStaleIndices( + snapshotIds, foundIndices, survivingIndexIds, mockRemoteStoreLockManagerFactory, - mockRemoteSegmentStoreDirectoryFactory, + null, repositoryData, listener, mockSnapshotShardPaths, From 8db579058039d6387c03ba18f642172199c2ddc1 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 3 Sep 2024 12:56:59 +0530 Subject: [PATCH 47/55] modify integ test for deleted index Signed-off-by: Anshu Agarwal --- .../snapshots/DeleteSnapshotIT.java | 72 +++++++++++++------ 1 file changed, 50 insertions(+), 22 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java index d3d0db79a722e..3532c72da6b4e 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java @@ -19,6 +19,7 @@ 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.index.store.RemoteBufferedOutputDirectory; import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase; @@ -321,6 +322,8 @@ public void testRemoteStoreCleanupForDeletedIndex() throws Exception { .get() .getSetting(remoteStoreEnabledIndexName, IndexMetadata.SETTING_INDEX_UUID); + Path indexPath1 = Path.of(String.valueOf(remoteStoreRepoPath), indexUUID); + logger.info("--> create two remote index shallow snapshots"); SnapshotInfo snapshotInfo1 = createFullSnapshot(snapshotRepoName, "snap1"); SnapshotInfo snapshotInfo2 = createFullSnapshot(snapshotRepoName, "snap2"); @@ -545,30 +548,24 @@ public void testDeleteShallowCopyV2MultipleSnapshots() throws Exception { public void testRemoteStoreCleanupForDeletedIndexForSnapshotV2() throws Exception { disableRepoConsistencyCheck("Remote store repository is being used in the test"); final Path remoteStoreRepoPath = randomRepoPath(); - Path absolutePath1 = randomRepoPath().toAbsolutePath(); Settings settings = remoteStoreClusterSettings(REMOTE_REPO_NAME, remoteStoreRepoPath); - - internalCluster().startClusterManagerOnlyNode(snapshotV2Settings(remoteStoreRepoPath)); - internalCluster().startDataOnlyNode(snapshotV2Settings(remoteStoreRepoPath)); + // Disabling pinned timestamp as this test is specifically for shallow snapshot. + 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(); + internalCluster().startClusterManagerOnlyNode(settings); + internalCluster().startDataOnlyNode(settings); final Client clusterManagerClient = internalCluster().clusterManagerClient(); ensureStableCluster(2); final String snapshotRepoName = "snapshot-repo-name"; final Path snapshotRepoPath = randomRepoPath(); - 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) - ) - ); + createRepository(snapshotRepoName, "mock", snapshotRepoSettingsForShallowV2(snapshotRepoPath)); + + final String testIndex = "index-test"; + createIndexWithContent(testIndex); final String remoteStoreEnabledIndexName = "remote-index-1"; final Settings remoteStoreEnabledIndexSettings = getRemoteStoreBackedIndexSettings(); @@ -581,6 +578,12 @@ public void testRemoteStoreCleanupForDeletedIndexForSnapshotV2() throws Exceptio .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() @@ -589,6 +592,8 @@ public void testRemoteStoreCleanupForDeletedIndexForSnapshotV2() throws Exceptio .get(); SnapshotInfo snapshotInfo1 = createSnapshotResponse.getSnapshotInfo(); + indexRandomDocs(remoteStoreEnabledIndexName, 25); + CreateSnapshotResponse createSnapshotResponse2 = client().admin() .cluster() .prepareCreateSnapshot(snapshotRepoName, "snap2") @@ -599,10 +604,10 @@ public void testRemoteStoreCleanupForDeletedIndexForSnapshotV2() throws Exceptio assertThat(snapshotInfo2.successfulShards(), greaterThan(0)); assertThat(snapshotInfo2.successfulShards(), equalTo(snapshotInfo2.totalShards())); assertThat(snapshotInfo2.snapshotId().getName(), equalTo("snap2")); - final RepositoriesService repositoriesService = internalCluster().getCurrentClusterManagerNodeInstance(RepositoriesService.class); // delete remote store index assertAcked(client().admin().indices().prepareDelete(remoteStoreEnabledIndexName)); + Thread.sleep(1000); logger.info("--> delete snapshot 1"); AcknowledgedResponse deleteSnapshotResponse = clusterManagerClient.admin() @@ -611,20 +616,34 @@ public void testRemoteStoreCleanupForDeletedIndexForSnapshotV2() throws Exceptio .get(); assertAcked(deleteSnapshotResponse); + Path indexPath = Path.of(String.valueOf(remoteStoreRepoPath), indexUUID); + Path shardPath = Path.of(String.valueOf(indexPath), "0"); + Path segmentsPath = Path.of(String.valueOf(shardPath), "segments"); + + Thread.sleep(1000); + + assertBusy(() -> { + try { + assertThat(RemoteStoreBaseIntegTestCase.getFileCount(segmentsPath), greaterThan(0)); + } catch (Exception e) {} + }, 30, TimeUnit.SECONDS); + int segmentsCount = RemoteStoreBaseIntegTestCase.getFileCount(segmentsPath); + logger.info("--> delete snapshot 2"); + // on snapshot deletion, remote store segment files should get cleaned up for deleted index - `remote-index-1` deleteSnapshotResponse = clusterManagerClient.admin() .cluster() .prepareDeleteSnapshot(snapshotRepoName, snapshotInfo2.snapshotId().getName()) .get(); assertAcked(deleteSnapshotResponse); - Path indexPath = Path.of(String.valueOf(remoteStoreRepoPath), indexUUID); + Thread.sleep(1000); // Delete is async. Give time for it assertBusy(() -> { try { - assertThat(RemoteStoreBaseIntegTestCase.getFileCount(indexPath), comparesEqualTo(0)); + assertThat(RemoteStoreBaseIntegTestCase.getFileCount(segmentsPath), comparesEqualTo(0)); } catch (Exception e) {} - }, 30, TimeUnit.SECONDS); + }, 60, TimeUnit.SECONDS); } private List createNSnapshots(String repoName, int count) { @@ -643,7 +662,16 @@ 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) + .put(BlobStoreRepository.SHALLOW_SNAPSHOT_V2.getKey(), false) .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; + } } From 3fca63c9502c191f23f83b6ccc50a8f56b21e204 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 3 Sep 2024 14:39:55 +0530 Subject: [PATCH 48/55] remove merge conflict Signed-off-by: Anshu Agarwal --- .../remote/model/RemotePinnedTimestamps.java | 146 ------------------ .../RemoteStorePinnedTimestampsBlobStore.java | 45 ------ 2 files changed, 191 deletions(-) delete mode 100644 server/src/main/java/org/opensearch/gateway/remote/model/RemotePinnedTimestamps.java delete mode 100644 server/src/main/java/org/opensearch/gateway/remote/model/RemoteStorePinnedTimestampsBlobStore.java diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemotePinnedTimestamps.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemotePinnedTimestamps.java deleted file mode 100644 index ef8f658f4bb52..0000000000000 --- a/server/src/main/java/org/opensearch/gateway/remote/model/RemotePinnedTimestamps.java +++ /dev/null @@ -1,146 +0,0 @@ -/* - * 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.gateway.remote.model; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.opensearch.common.annotation.ExperimentalApi; -import org.opensearch.common.io.Streams; -import org.opensearch.common.remote.BlobPathParameters; -import org.opensearch.common.remote.RemoteWriteableBlobEntity; -import org.opensearch.core.common.io.stream.StreamInput; -import org.opensearch.core.common.io.stream.StreamOutput; -import org.opensearch.core.common.io.stream.Writeable; -import org.opensearch.core.compress.Compressor; -import org.opensearch.index.remote.RemoteStoreUtils; -import org.opensearch.repositories.blobstore.ChecksumWritableBlobStoreFormat; - -import java.io.IOException; -import java.io.InputStream; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; - -/** - * Wrapper class for uploading/downloading {@link RemotePinnedTimestamps} to/from remote blob store - * - * @opensearch.internal - */ -public class RemotePinnedTimestamps extends RemoteWriteableBlobEntity { - private static final Logger logger = LogManager.getLogger(RemotePinnedTimestamps.class); - - /** - * Represents a collection of pinned timestamps and their associated pinning entities. - * This class is thread-safe and implements the Writeable interface for serialization. - */ - @ExperimentalApi - public static class PinnedTimestamps implements Writeable { - private final Map> pinnedTimestampPinningEntityMap; - - public PinnedTimestamps(Map> pinnedTimestampPinningEntityMap) { - this.pinnedTimestampPinningEntityMap = new ConcurrentHashMap<>(pinnedTimestampPinningEntityMap); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeMap(pinnedTimestampPinningEntityMap, StreamOutput::writeLong, StreamOutput::writeStringCollection); - } - - public static PinnedTimestamps readFrom(StreamInput in) throws IOException { - return new PinnedTimestamps(in.readMap(StreamInput::readLong, StreamInput::readStringList)); - } - - /** - * Pins a timestamp against a pinning entity. - * - * @param timestamp The timestamp to pin. - * @param pinningEntity The entity pinning the timestamp. - */ - public void pin(Long timestamp, String pinningEntity) { - logger.debug("Pinning timestamp = {} against entity = {}", timestamp, pinningEntity); - pinnedTimestampPinningEntityMap.computeIfAbsent(timestamp, k -> new ArrayList<>()).add(pinningEntity); - } - - /** - * Unpins a timestamp for a specific pinning entity. - * - * @param timestamp The timestamp to unpin. - * @param pinningEntity The entity unpinning the timestamp. - */ - public void unpin(Long timestamp, String pinningEntity) { - logger.debug("Unpinning timestamp = {} against entity = {}", timestamp, pinningEntity); - if (pinnedTimestampPinningEntityMap.containsKey(timestamp) == false - || pinnedTimestampPinningEntityMap.get(timestamp).contains(pinningEntity) == false) { - logger.warn("Timestamp: {} is not pinned by entity: {}", timestamp, pinningEntity); - } - pinnedTimestampPinningEntityMap.compute(timestamp, (k, v) -> { - v.remove(pinningEntity); - return v.isEmpty() ? null : v; - }); - } - - public Map> getPinnedTimestampPinningEntityMap() { - return new HashMap<>(pinnedTimestampPinningEntityMap); - } - } - - public static final String PINNED_TIMESTAMPS = "pinned_timestamps"; - public static final ChecksumWritableBlobStoreFormat PINNED_TIMESTAMPS_FORMAT = new ChecksumWritableBlobStoreFormat<>( - PINNED_TIMESTAMPS, - PinnedTimestamps::readFrom - ); - - private PinnedTimestamps pinnedTimestamps; - - public RemotePinnedTimestamps(String clusterUUID, Compressor compressor) { - super(clusterUUID, compressor); - pinnedTimestamps = new PinnedTimestamps(new HashMap<>()); - } - - @Override - public BlobPathParameters getBlobPathParameters() { - return new BlobPathParameters(List.of(PINNED_TIMESTAMPS), PINNED_TIMESTAMPS); - } - - @Override - public String getType() { - return PINNED_TIMESTAMPS; - } - - @Override - public String generateBlobFileName() { - return this.blobFileName = String.join(DELIMITER, PINNED_TIMESTAMPS, RemoteStoreUtils.invertLong(System.currentTimeMillis())); - } - - @Override - public InputStream serialize() throws IOException { - return PINNED_TIMESTAMPS_FORMAT.serialize(pinnedTimestamps, generateBlobFileName(), getCompressor()).streamInput(); - } - - @Override - public PinnedTimestamps deserialize(InputStream inputStream) throws IOException { - return PINNED_TIMESTAMPS_FORMAT.deserialize(blobName, Streams.readFully(inputStream)); - } - - public void setBlobFileName(String blobFileName) { - this.blobFileName = blobFileName; - } - - public void setPinnedTimestamps(PinnedTimestamps pinnedTimestamps) { - this.pinnedTimestamps = pinnedTimestamps; - } - - public PinnedTimestamps getPinnedTimestamps() { - return pinnedTimestamps; - } -} diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteStorePinnedTimestampsBlobStore.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteStorePinnedTimestampsBlobStore.java deleted file mode 100644 index d628d7f12afd0..0000000000000 --- a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteStorePinnedTimestampsBlobStore.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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.gateway.remote.model; - -import org.opensearch.common.annotation.ExperimentalApi; -import org.opensearch.common.blobstore.BlobPath; -import org.opensearch.common.remote.RemoteWriteableBlobEntity; -import org.opensearch.common.remote.RemoteWriteableEntityBlobStore; -import org.opensearch.index.translog.transfer.BlobStoreTransferService; -import org.opensearch.repositories.blobstore.BlobStoreRepository; -import org.opensearch.threadpool.ThreadPool; - -/** - * Extends the RemoteClusterStateBlobStore to support {@link RemotePinnedTimestamps} - */ -@ExperimentalApi -public class RemoteStorePinnedTimestampsBlobStore extends RemoteWriteableEntityBlobStore< - RemotePinnedTimestamps.PinnedTimestamps, - RemotePinnedTimestamps> { - - public static final String PINNED_TIMESTAMPS_PATH_TOKEN = "pinned_timestamps"; - private final BlobStoreRepository blobStoreRepository; - - public RemoteStorePinnedTimestampsBlobStore( - BlobStoreTransferService blobStoreTransferService, - BlobStoreRepository blobStoreRepository, - String clusterName, - ThreadPool threadPool, - String executor - ) { - super(blobStoreTransferService, blobStoreRepository, clusterName, threadPool, executor, PINNED_TIMESTAMPS_PATH_TOKEN); - this.blobStoreRepository = blobStoreRepository; - } - - @Override - public BlobPath getBlobPathForUpload(final RemoteWriteableBlobEntity obj) { - return blobStoreRepository.basePath().add(PINNED_TIMESTAMPS_PATH_TOKEN); - } -} From f7e260959d5729995c5bd2fc437de2d0a8e6950a Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 3 Sep 2024 17:54:00 +0530 Subject: [PATCH 49/55] Add cleanup for deleted index if there are snapshots referencing it Signed-off-by: Anshu Agarwal --- .../snapshots/DeleteSnapshotIT.java | 42 ++++++++++--------- .../blobstore/BlobStoreRepository.java | 39 ++++++++++++++--- 2 files changed, 57 insertions(+), 24 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java index 3532c72da6b4e..22e7fb01d66e0 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java @@ -44,6 +44,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThan; @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) public class DeleteSnapshotIT extends AbstractSnapshotIntegTestCase { @@ -424,6 +425,7 @@ public void testDeleteShallowCopyV2() throws Exception { CreateSnapshotResponse createSnapshotResponse = client().admin() .cluster() .prepareCreateSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(true) .get(); SnapshotInfo snapshotInfo = createSnapshotResponse.getSnapshotInfo(); assertThat(snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); @@ -509,6 +511,7 @@ public void testDeleteShallowCopyV2MultipleSnapshots() throws Exception { CreateSnapshotResponse createSnapshotResponse = client().admin() .cluster() .prepareCreateSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(true) .get(); SnapshotInfo snapshotInfo = createSnapshotResponse.getSnapshotInfo(); assertThat(snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); @@ -549,7 +552,6 @@ public void testRemoteStoreCleanupForDeletedIndexForSnapshotV2() throws Exceptio disableRepoConsistencyCheck("Remote store repository is being used in the test"); final Path remoteStoreRepoPath = randomRepoPath(); Settings settings = remoteStoreClusterSettings(REMOTE_REPO_NAME, remoteStoreRepoPath); - // Disabling pinned timestamp as this test is specifically for shallow snapshot. settings = Settings.builder() .put(settings) .put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_ENABLED.getKey(), true) @@ -564,13 +566,10 @@ public void testRemoteStoreCleanupForDeletedIndexForSnapshotV2() throws Exceptio final Path snapshotRepoPath = randomRepoPath(); createRepository(snapshotRepoName, "mock", snapshotRepoSettingsForShallowV2(snapshotRepoPath)); - final String testIndex = "index-test"; - createIndexWithContent(testIndex); - final String remoteStoreEnabledIndexName = "remote-index-1"; final Settings remoteStoreEnabledIndexSettings = getRemoteStoreBackedIndexSettings(); createIndex(remoteStoreEnabledIndexName, remoteStoreEnabledIndexSettings); - indexRandomDocs(remoteStoreEnabledIndexName, randomIntBetween(5, 10)); + indexRandomDocs(remoteStoreEnabledIndexName, 5); String indexUUID = client().admin() .indices() @@ -609,39 +608,45 @@ public void testRemoteStoreCleanupForDeletedIndexForSnapshotV2() throws Exceptio assertAcked(client().admin().indices().prepareDelete(remoteStoreEnabledIndexName)); Thread.sleep(1000); - logger.info("--> delete snapshot 1"); - AcknowledgedResponse deleteSnapshotResponse = clusterManagerClient.admin() - .cluster() - .prepareDeleteSnapshot(snapshotRepoName, snapshotInfo1.snapshotId().getName()) - .get(); - assertAcked(deleteSnapshotResponse); + 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"); - Thread.sleep(1000); + // Get total segments remote store directory file count for deleted index and shard 0 + int segmentFilesCountBeforeDeletingSnapshot1 = RemoteStoreBaseIntegTestCase.getFileCount(segmentsPath); + + AcknowledgedResponse deleteSnapshotResponse = clusterManagerClient.admin() + .cluster() + .prepareDeleteSnapshot(snapshotRepoName, snapshotInfo2.snapshotId().getName()) + .get(); + assertAcked(deleteSnapshotResponse); + + Thread.sleep(5000); assertBusy(() -> { try { - assertThat(RemoteStoreBaseIntegTestCase.getFileCount(segmentsPath), greaterThan(0)); + assertThat(RemoteStoreBaseIntegTestCase.getFileCount(segmentsPath), lessThan(segmentFilesCountBeforeDeletingSnapshot1)); } catch (Exception e) {} }, 30, TimeUnit.SECONDS); - int segmentsCount = RemoteStoreBaseIntegTestCase.getFileCount(segmentsPath); + int segmentFilesCountAfterDeletingSnapshot1 = RemoteStoreBaseIntegTestCase.getFileCount(segmentsPath); + + logger.info("--> delete snapshot 1"); - logger.info("--> delete snapshot 2"); // on snapshot deletion, remote store segment files should get cleaned up for deleted index - `remote-index-1` deleteSnapshotResponse = clusterManagerClient.admin() .cluster() - .prepareDeleteSnapshot(snapshotRepoName, snapshotInfo2.snapshotId().getName()) + .prepareDeleteSnapshot(snapshotRepoName, snapshotInfo1.snapshotId().getName()) .get(); assertAcked(deleteSnapshotResponse); - Thread.sleep(1000); + Thread.sleep(5000); + // Delete is async. Give time for it assertBusy(() -> { try { - assertThat(RemoteStoreBaseIntegTestCase.getFileCount(segmentsPath), comparesEqualTo(0)); + assertThat(RemoteStoreBaseIntegTestCase.getFileCount(segmentsPath), lessThan(segmentFilesCountAfterDeletingSnapshot1)); } catch (Exception e) {} }, 60, TimeUnit.SECONDS); } @@ -662,7 +667,6 @@ 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) - .put(BlobStoreRepository.SHALLOW_SNAPSHOT_V2.getKey(), false) .build(); return settings; } 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 dfc88f490f0b0..309cbbae55b24 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -1205,14 +1205,11 @@ private void doDeleteShardSnapshots( }, listener::onFailure); pinnedTimestampListener.whenComplete(updatedRepoData -> { - int groupSize = 2; - if (isShallowSnapshotV2) { - groupSize = 1; - } + // Run unreferenced blobs cleanup in parallel to shard-level snapshot deletion final ActionListener afterCleanupsListener = new GroupedActionListener<>( ActionListener.wrap(() -> listener.onResponse(updatedRepoData)), - groupSize + 2 ); // We can create map of indexId to ShardInfo based on the old repository data. This is later used in cleanup @@ -1246,10 +1243,42 @@ private void doDeleteShardSnapshots( remoteStoreLockManagerFactory, afterCleanupsListener ); + } else { + cleanUpRemoteStoreFilesForDeletedIndices( + repositoryData, + snapshotIds, + writeShardMetaDataAndComputeDeletesStep.result(), + remoteSegmentStoreDirectoryFactory, + afterCleanupsListener + ); } }, listener::onFailure); } + private void cleanUpRemoteStoreFilesForDeletedIndices( + 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, From d075b6bc6a746671ec5add179325620e59bfaea8 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 3 Sep 2024 18:30:13 +0530 Subject: [PATCH 50/55] fix integ test Signed-off-by: Anshu Agarwal --- .../org/opensearch/snapshots/DeleteSnapshotIT.java | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java index 22e7fb01d66e0..34742e411189f 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java @@ -22,6 +22,7 @@ import org.opensearch.index.remote.RemoteStoreEnums; import org.opensearch.index.store.RemoteBufferedOutputDirectory; import org.opensearch.indices.RemoteStoreSettings; +import org.opensearch.node.remotestore.RemoteStorePinnedTimestampService; import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.blobstore.BlobStoreRepository; @@ -557,11 +558,17 @@ public void testRemoteStoreCleanupForDeletedIndexForSnapshotV2() throws Exceptio .put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_ENABLED.getKey(), true) .put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), RemoteStoreEnums.PathType.FIXED.toString()) .build(); - internalCluster().startClusterManagerOnlyNode(settings); + 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)); @@ -617,6 +624,8 @@ public void testRemoteStoreCleanupForDeletedIndexForSnapshotV2() throws Exceptio // 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()) @@ -633,7 +642,7 @@ public void testRemoteStoreCleanupForDeletedIndexForSnapshotV2() throws Exceptio 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() From de457b6a50ca6ae9ce17c4c3d535498b56615d45 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 3 Sep 2024 20:23:54 +0530 Subject: [PATCH 51/55] fix spotless failure Signed-off-by: Anshu Agarwal --- .../cleanup/TransportCleanupRepositoryAction.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) 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 8e8487de537c8..54b6d7a914a9b 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 @@ -127,7 +127,11 @@ public TransportCleanupRepositoryAction( ); this.repositoriesService = repositoriesService; this.snapshotsService = snapshotsService; - this.remoteSegmentStoreDirectoryFactory = new RemoteSegmentStoreDirectoryFactory(() -> repositoriesService, threadPool, remoteStoreSettings.getSegmentsPathFixedPrefix()); + this.remoteSegmentStoreDirectoryFactory = new RemoteSegmentStoreDirectoryFactory( + () -> repositoriesService, + threadPool, + remoteStoreSettings.getSegmentsPathFixedPrefix() + ); this.remoteStoreLockManagerFactory = new RemoteStoreLockManagerFactory( () -> repositoriesService, remoteStoreSettings.getSegmentsPathFixedPrefix() From 1f38f2386e3f43fc0da2b807659dad1850b18e22 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 3 Sep 2024 23:23:27 +0530 Subject: [PATCH 52/55] address review comments Signed-off-by: Anshu Agarwal --- .../snapshots/DeleteSnapshotIT.java | 3 - .../snapshots/DeleteSnapshotITV2.java | 332 ++++++++++++++++++ .../blobstore/BlobStoreRepository.java | 12 +- 3 files changed, 338 insertions(+), 9 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotITV2.java diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java index 0f6f5af625762..36d22e657653a 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java @@ -616,7 +616,6 @@ public void testRemoteStoreCleanupForDeletedIndexForSnapshotV2() throws Exceptio // delete remote store index assertAcked(client().admin().indices().prepareDelete(remoteStoreEnabledIndexName)); - Thread.sleep(1000); logger.info("--> delete snapshot 2"); @@ -653,8 +652,6 @@ public void testRemoteStoreCleanupForDeletedIndexForSnapshotV2() throws Exceptio .get(); assertAcked(deleteSnapshotResponse); - Thread.sleep(5000); - // Delete is async. Give time for it assertBusy(() -> { try { 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/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java index 856ab29ddfaf0..2b7defefa871d 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -1239,27 +1239,27 @@ private void doDeleteShardSnapshots( afterCleanupsListener, idToShardInfoMap ); - if (isShallowSnapshotV2 == false) { - asyncCleanupUnlinkedShardLevelBlobs( + if (isShallowSnapshotV2) { + cleanUpRemoteStoreFilesForDeletedIndicesV2( repositoryData, snapshotIds, writeShardMetaDataAndComputeDeletesStep.result(), - remoteStoreLockManagerFactory, + remoteSegmentStoreDirectoryFactory, afterCleanupsListener ); } else { - cleanUpRemoteStoreFilesForDeletedIndices( + asyncCleanupUnlinkedShardLevelBlobs( repositoryData, snapshotIds, writeShardMetaDataAndComputeDeletesStep.result(), - remoteSegmentStoreDirectoryFactory, + remoteStoreLockManagerFactory, afterCleanupsListener ); } }, listener::onFailure); } - private void cleanUpRemoteStoreFilesForDeletedIndices( + private void cleanUpRemoteStoreFilesForDeletedIndicesV2( RepositoryData repositoryData, Collection snapshotIds, Collection result, From 8fe6a24f3235ba69eacfdaf55efbbe123de513fe Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 3 Sep 2024 23:27:45 +0530 Subject: [PATCH 53/55] address review comments Signed-off-by: Anshu Agarwal --- .../snapshots/DeleteSnapshotIT.java | 303 ------------------ 1 file changed, 303 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java index 36d22e657653a..26b30af4c2c50 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java @@ -18,15 +18,11 @@ import org.opensearch.common.blobstore.BlobPath; 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.index.store.RemoteBufferedOutputDirectory; import org.opensearch.indices.RemoteStoreSettings; -import org.opensearch.node.remotestore.RemoteStorePinnedTimestampService; import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.blobstore.BlobStoreRepository; -import org.opensearch.repositories.fs.FsRepository; import org.opensearch.test.OpenSearchIntegTestCase; import java.nio.file.Path; @@ -42,10 +38,7 @@ import static org.opensearch.remotestore.RemoteStoreBaseIntegTestCase.remoteStoreClusterSettings; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; import static org.hamcrest.Matchers.comparesEqualTo; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.lessThan; @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) public class DeleteSnapshotIT extends AbstractSnapshotIntegTestCase { @@ -325,8 +318,6 @@ public void testRemoteStoreCleanupForDeletedIndex() throws Exception { .get() .getSetting(remoteStoreEnabledIndexName, IndexMetadata.SETTING_INDEX_UUID); - Path indexPath1 = Path.of(String.valueOf(remoteStoreRepoPath), indexUUID); - logger.info("--> create two remote index shallow snapshots"); SnapshotInfo snapshotInfo1 = createFullSnapshot(snapshotRepoName, "snap1"); SnapshotInfo snapshotInfo2 = createFullSnapshot(snapshotRepoName, "snap2"); @@ -382,284 +373,6 @@ public void testRemoteStoreCleanupForDeletedIndex() throws Exception { }, 30, TimeUnit.SECONDS); } - 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 List createNSnapshots(String repoName, int count) { final List snapshotNames = new ArrayList<>(count); final String prefix = "snap-" + UUIDs.randomBase64UUID(random()).toLowerCase(Locale.ROOT) + "-"; @@ -671,20 +384,4 @@ private List createNSnapshots(String repoName, int count) { logger.info("--> created {} in [{}]", snapshotNames, repoName); return snapshotNames; } - - 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; - } } From 5b33e8598eae7792ce1587c9ff9b6862d530ab92 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 3 Sep 2024 23:39:54 +0530 Subject: [PATCH 54/55] refactor code Signed-off-by: Anshu Agarwal --- .../java/org/opensearch/common/blobstore/BlobMetadata.java | 3 --- .../common/blobstore/stream/write/WritePriority.java | 3 --- .../java/org/opensearch/common/remote/BlobPathParameters.java | 3 --- .../opensearch/common/remote/RemoteWriteableBlobEntity.java | 2 -- .../index/translog/transfer/BlobStoreTransferService.java | 1 - .../org/opensearch/index/translog/transfer/FileSnapshot.java | 2 -- .../node/remotestore/RemoteStorePinnedTimestampService.java | 2 -- 7 files changed, 16 deletions(-) diff --git a/server/src/main/java/org/opensearch/common/blobstore/BlobMetadata.java b/server/src/main/java/org/opensearch/common/blobstore/BlobMetadata.java index 7d1c0e6bad5a9..37c70365b6a11 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/BlobMetadata.java +++ b/server/src/main/java/org/opensearch/common/blobstore/BlobMetadata.java @@ -32,14 +32,11 @@ package org.opensearch.common.blobstore; -import org.opensearch.common.annotation.ExperimentalApi; - /** * An interface for providing basic metadata about a blob. * * @opensearch.internal */ -@ExperimentalApi public interface BlobMetadata { /** diff --git a/server/src/main/java/org/opensearch/common/blobstore/stream/write/WritePriority.java b/server/src/main/java/org/opensearch/common/blobstore/stream/write/WritePriority.java index d88129053fc38..4e8db0a3a8c69 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/stream/write/WritePriority.java +++ b/server/src/main/java/org/opensearch/common/blobstore/stream/write/WritePriority.java @@ -8,14 +8,11 @@ package org.opensearch.common.blobstore.stream.write; -import org.opensearch.common.annotation.ExperimentalApi; - /** * WritePriority for upload * * @opensearch.internal */ -@ExperimentalApi public enum WritePriority { // Used for segment transfers during refresh, flush or merges NORMAL, diff --git a/server/src/main/java/org/opensearch/common/remote/BlobPathParameters.java b/server/src/main/java/org/opensearch/common/remote/BlobPathParameters.java index 7c53af00d1659..58c73a804b66a 100644 --- a/server/src/main/java/org/opensearch/common/remote/BlobPathParameters.java +++ b/server/src/main/java/org/opensearch/common/remote/BlobPathParameters.java @@ -8,15 +8,12 @@ package org.opensearch.common.remote; -import org.opensearch.common.annotation.ExperimentalApi; - import java.util.List; /** * Parameters which can be used to construct a blob path * */ -@ExperimentalApi public class BlobPathParameters { private final List pathTokens; diff --git a/server/src/main/java/org/opensearch/common/remote/RemoteWriteableBlobEntity.java b/server/src/main/java/org/opensearch/common/remote/RemoteWriteableBlobEntity.java index d49ef69c65ed3..f034ce2d1adf1 100644 --- a/server/src/main/java/org/opensearch/common/remote/RemoteWriteableBlobEntity.java +++ b/server/src/main/java/org/opensearch/common/remote/RemoteWriteableBlobEntity.java @@ -8,7 +8,6 @@ package org.opensearch.common.remote; -import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.core.compress.Compressor; @@ -18,7 +17,6 @@ * The abstract class which represents a {@link RemoteWriteableEntity} that can be written to a store * @param the entity to be written */ -@ExperimentalApi public abstract class RemoteWriteableBlobEntity implements RemoteWriteableEntity { protected String blobFileName; diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java index 31b0c9a734503..22bb4cf0514bf 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java @@ -52,7 +52,6 @@ * * @opensearch.internal */ -@ExperimentalApi public class BlobStoreTransferService implements TransferService { private final BlobStore blobStore; diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/FileSnapshot.java b/server/src/main/java/org/opensearch/index/translog/transfer/FileSnapshot.java index cbdfcf8c9b306..86f042af0584b 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/FileSnapshot.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/FileSnapshot.java @@ -9,7 +9,6 @@ package org.opensearch.index.translog.transfer; import org.opensearch.common.Nullable; -import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.lucene.store.ByteArrayIndexInput; import org.opensearch.common.lucene.store.InputStreamIndexInput; import org.opensearch.common.util.io.IOUtils; @@ -105,7 +104,6 @@ public void close() throws IOException { * * @opensearch.internal */ - @ExperimentalApi public static class TransferFileSnapshot extends FileSnapshot { private final long primaryTerm; 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 3a7734fc0538f..782ba5e9a6540 100644 --- a/server/src/main/java/org/opensearch/node/remotestore/RemoteStorePinnedTimestampService.java +++ b/server/src/main/java/org/opensearch/node/remotestore/RemoteStorePinnedTimestampService.java @@ -12,7 +12,6 @@ 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; @@ -43,7 +42,6 @@ * * @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()); From 54bee31b7d0d06f9e6e61ad7f7b200847fa850f4 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 3 Sep 2024 23:52:37 +0530 Subject: [PATCH 55/55] fix precommit failure Signed-off-by: Anshu Agarwal --- .../node/remotestore/RemoteStorePinnedTimestampService.java | 2 ++ 1 file changed, 2 insertions(+) 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());