From 9142a254d75a0a6441bf29fc9e3284db9fd88eb1 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 6 Aug 2024 10:43:00 +0530 Subject: [PATCH 01/30] 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/30] 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/30] 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/30] 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/30] 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/30] 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/30] 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/30] 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 6504169602c532a931a3c80e31857558c9b2a4fc Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 20 Aug 2024 20:01:58 +0530 Subject: [PATCH 09/30] 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 626c2fad9c645ef517fb8e50084dc98ee629a182 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Wed, 21 Aug 2024 11:13:25 +0530 Subject: [PATCH 10/30] 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 11/30] 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 12/30] 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 13/30] 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 0c636ef5d886febd6e77cf920e682fba10277129 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Thu, 22 Aug 2024 18:37:23 +0530 Subject: [PATCH 14/30] 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 15/30] 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 16/30] 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 17/30] 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 18/30] 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 80bf6cc8241d3c9cbc95769d4e5deffc9306468e Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Mon, 26 Aug 2024 10:14:03 +0530 Subject: [PATCH 19/30] 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 20/30] 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 21/30] 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 22/30] 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 23/30] 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 24/30] 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 983d2b585b896dcd07ffad9440179796a55bd088 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 27 Aug 2024 13:34:49 +0530 Subject: [PATCH 25/30] address review comments Signed-off-by: Anshu Agarwal --- .../opensearch/remotestore/RemoteRestoreSnapshotIT.java | 9 ++++++++- .../java/org/opensearch/snapshots/SnapshotsService.java | 4 ++-- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index ef6d5d6c75810..e65661a0440e4 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -994,7 +994,6 @@ public void testCreateSnapshotV2WithRedIndex() throws Exception { internalCluster().startDataOnlyNode(); String indexName1 = "testindex1"; String indexName2 = "testindex2"; - String indexName3 = "testindex3"; String snapshotRepoName = "test-create-snapshot-repo"; String snapshotName1 = "test-create-snapshot1"; Path absolutePath1 = randomRepoPath().toAbsolutePath(); @@ -1163,6 +1162,14 @@ public void testCreateSnapshotV2WithShallowCopySettingDisabled() throws Exceptio assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards())); assertThat(snapshotInfo.getPinnedTimestamp(), equalTo(0L)); + // Validate that snapshot is present in repository data + Repository repository = internalCluster().getInstance(RepositoriesService.class).repository(snapshotRepoName); + PlainActionFuture repositoryDataPlainActionFuture = new PlainActionFuture<>(); + repository.getRepositoryData(repositoryDataPlainActionFuture); + + RepositoryData repositoryData = repositoryDataPlainActionFuture.get(); + assertTrue(repositoryData.getSnapshotIds().contains(snapshotInfo.snapshotId())); + } } diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index fdc822a7fe41a..3684bf1786a87 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -203,7 +203,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus Setting.Property.Dynamic ); - private static final String SNAPSHOT_PINNED_TIMESTAMP_DELIMITER = "__"; + private static final String SNAPSHOT_PINNED_TIMESTAMP_DELIMITER = ":"; private volatile int maxConcurrentOperations; public SnapshotsService( @@ -460,7 +460,7 @@ public void createSnapshotV2(final CreateSnapshotRequest request, final ActionLi repositoryDataListener.whenComplete(repositoryData -> { createSnapshotPreValidations(currentState, repositoryData, repositoryName, snapshotName); - List indices = Arrays.asList(indexNameExpressionResolver.concreteIndexNames(currentState, request)); + List indices = new ArrayList<>(currentState.metadata().indices().keySet()); final List dataStreams = indexNameExpressionResolver.dataStreamNames( currentState, From a423475e4a647c1f4f28a1e395ee8fc81e15589b Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 27 Aug 2024 15:10:02 +0530 Subject: [PATCH 26/30] add integ test for master failover scenario Signed-off-by: Anshu Agarwal --- .../remotestore/RemoteRestoreSnapshotIT.java | 81 +++++++++++++++++++ 1 file changed, 81 insertions(+) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index e65661a0440e4..ea2daaf2ee0bf 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -1169,7 +1169,88 @@ public void testCreateSnapshotV2WithShallowCopySettingDisabled() throws Exceptio RepositoryData repositoryData = repositoryDataPlainActionFuture.get(); assertTrue(repositoryData.getSnapshotIds().contains(snapshotInfo.snapshotId())); + } + + public void testClusterManagerFailoverDuringSnapshotCreation() throws Exception { + + internalCluster().startClusterManagerOnlyNodes(3, Settings.EMPTY); + 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(), true) + .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); + + ensureStableCluster(4, internalCluster().getClusterManagerName()); + + final SnapshotInfo[] snapshotInfo = new SnapshotInfo[1]; + final Boolean[] snapshotFailed = new Boolean[1]; + snapshotFailed[0] = false; + Thread snapshotThread = new Thread(() -> { + try { + // Start snapshot creation + CreateSnapshotResponse createSnapshotResponse = client().admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, snapshotName1) + .get(); + snapshotInfo[0] = createSnapshotResponse.getSnapshotInfo(); + } catch (Exception e) { + System.out.println("snapshot creation failed"); + snapshotFailed[0] = true; + } + }); + snapshotThread.start(); + Thread.sleep(100); + + internalCluster().stopCurrentClusterManagerNode(); + + // Wait for the cluster to elect a new Cluster Manager and stabilize + ensureStableCluster(3, internalCluster().getClusterManagerName()); + + // Wait for the snapshot thread to complete + snapshotThread.join(); + + // Validate that the snapshot was created or handled gracefully + Repository repository = internalCluster().getInstance(RepositoriesService.class).repository(snapshotRepoName); + PlainActionFuture repositoryDataPlainActionFuture = new PlainActionFuture<>(); + repository.getRepositoryData(repositoryDataPlainActionFuture); + + RepositoryData repositoryData = repositoryDataPlainActionFuture.get(); + if (snapshotFailed[0]) { + assertFalse(repositoryData.getSnapshotIds().contains(snapshotInfo[0].snapshotId())); + } else { + assertTrue(repositoryData.getSnapshotIds().contains(snapshotInfo[0].snapshotId())); + } } } From 207b03d685c814b2be94433f37205f3a20b3a7bf Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 27 Aug 2024 17:58:07 +0530 Subject: [PATCH 27/30] Add more integ tests Signed-off-by: Anshu Agarwal --- .../remotestore/RemoteRestoreSnapshotIT.java | 89 +++++++++++++++++++ 1 file changed, 89 insertions(+) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index ea2daaf2ee0bf..4440bc659fe14 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -1253,4 +1253,93 @@ public void testClusterManagerFailoverDuringSnapshotCreation() throws Exception } } + public void testConcurrentV1SnapshotAndV2RepoSettingUpdate() throws Exception { + internalCluster().startClusterManagerOnlyNode(); + internalCluster().startDataOnlyNode(); + internalCluster().startDataOnlyNode(); + 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(); + + for (int i = 0; i < 10; i++) { + createIndex("index" + i, indexSettings); + } + ensureStableCluster(3); + for (int i = 0; i < 10; i++) { + indexDocuments(client, "index" + i, 15); + } + + ensureStableCluster(3); + for (int i = 0; i < 10; i++) { + ensureGreen("index" + i); + } + final CreateSnapshotResponse[] snapshotV1Response = new CreateSnapshotResponse[1]; + // Create a separate thread to create the first snapshot + Thread createV1SnapshotThread = new Thread(() -> { + try { + snapshotV1Response[0] = client().admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(true) + .get(); + + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + + // Create a separate thread to enable snapshot_v2 + Thread enableV2Thread = new Thread(() -> { + try { + + assertThrows( + IllegalStateException.class, + () -> 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) + ) + .get() + ); + + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + + createV1SnapshotThread.start(); + + Thread.sleep(100); + + enableV2Thread.start(); + + enableV2Thread.join(); + createV1SnapshotThread.join(); + } + } From 872e136e03c2fec9485bc65798e42a571508ccf8 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 27 Aug 2024 19:06:19 +0530 Subject: [PATCH 28/30] refactor code Signed-off-by: Anshu Agarwal --- .../remotestore/RemoteRestoreSnapshotIT.java | 27 +++++++++---------- .../create/TransportCreateSnapshotAction.java | 4 +-- .../blobstore/BlobStoreRepository.java | 2 +- .../snapshots/SnapshotsService.java | 6 ++--- 4 files changed, 19 insertions(+), 20 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index 4440bc659fe14..4db8aee974d0d 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -779,7 +779,7 @@ public void testCreateSnapshotV2() 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) ) ); @@ -848,7 +848,7 @@ public void testMixedSnapshotCreationWithV2RepositorySetting() 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(), false) + .put(BlobStoreRepository.SHALLOW_SNAPSHOT_V2.getKey(), false) ) ); Client client = client(); @@ -870,7 +870,7 @@ public void testMixedSnapshotCreationWithV2RepositorySetting() throws Exception assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards())); assertThat(snapshotInfo.getPinnedTimestamp(), equalTo(0L)); - // enable snapshot_v2 + // enable shallow_snapshot_v2 assertAcked( client().admin() .cluster() @@ -882,7 +882,7 @@ public void testMixedSnapshotCreationWithV2RepositorySetting() 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) ) ); indexDocuments(client, indexName1, 10); @@ -928,7 +928,7 @@ public void testConcurrentSnapshotV2CreateOperation() throws InterruptedExceptio .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) ) ); @@ -1010,7 +1010,7 @@ public void testCreateSnapshotV2WithRedIndex() 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) ) ); @@ -1064,7 +1064,7 @@ public void testCreateSnapshotV2WithIndexingLoad() 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) ) ); @@ -1138,7 +1138,7 @@ public void testCreateSnapshotV2WithShallowCopySettingDisabled() throws Exceptio .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) + .put(BlobStoreRepository.SHALLOW_SNAPSHOT_V2.getKey(), true) ) ); @@ -1155,7 +1155,7 @@ public void testCreateSnapshotV2WithShallowCopySettingDisabled() throws Exceptio 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 + // Will create full copy snapshot if `REMOTE_STORE_INDEX_SHALLOW_COPY` is false but `SHALLOW_SNAPSHOT_V2` is true SnapshotInfo snapshotInfo = createSnapshot(snapshotRepoName, snapshotName1, Collections.emptyList()); assertThat(snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); assertThat(snapshotInfo.successfulShards(), greaterThan(0)); @@ -1193,7 +1193,7 @@ public void testClusterManagerFailoverDuringSnapshotCreation() 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) ) ); @@ -1225,7 +1225,6 @@ public void testClusterManagerFailoverDuringSnapshotCreation() throws Exception snapshotInfo[0] = createSnapshotResponse.getSnapshotInfo(); } catch (Exception e) { - System.out.println("snapshot creation failed"); snapshotFailed[0] = true; } }); @@ -1273,7 +1272,7 @@ public void testConcurrentV1SnapshotAndV2RepoSettingUpdate() 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(), false) + .put(BlobStoreRepository.SHALLOW_SNAPSHOT_V2.getKey(), false) ) ); Client client = client(); @@ -1306,7 +1305,7 @@ public void testConcurrentV1SnapshotAndV2RepoSettingUpdate() throws Exception { } }); - // Create a separate thread to enable snapshot_v2 + // Create a separate thread to enable shallow_snapshot_v2 Thread enableV2Thread = new Thread(() -> { try { @@ -1322,7 +1321,7 @@ public void testConcurrentV1SnapshotAndV2RepoSettingUpdate() 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) ) .get() ); 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 1ce5ffe1002d8..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 @@ -50,7 +50,7 @@ import java.io.IOException; -import static org.opensearch.repositories.blobstore.BlobStoreRepository.SNAPSHOT_V2; +import static org.opensearch.repositories.blobstore.BlobStoreRepository.SHALLOW_SNAPSHOT_V2; /** * Transport action for create snapshot operation @@ -112,7 +112,7 @@ protected void clusterManagerOperation( final ActionListener listener ) { Repository repository = repositoriesService.repository(request.repository()); - boolean isSnapshotV2 = SNAPSHOT_V2.get(repository.getMetadata().settings()); + 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/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java index 2905ff7d90a63..e18706824d39d 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -267,7 +267,7 @@ 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); + public static final Setting SHALLOW_SNAPSHOT_V2 = Setting.boolSetting("shallow_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. diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index 3684bf1786a87..b7fea116a12b7 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -136,7 +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.repositories.blobstore.BlobStoreRepository.SHALLOW_SNAPSHOT_V2; import static org.opensearch.snapshots.SnapshotUtils.validateSnapshotsBackingAnyIndex; /** @@ -255,8 +255,8 @@ public SnapshotsService( public void executeSnapshot(final CreateSnapshotRequest request, final ActionListener listener) { Repository repository = repositoriesService.repository(request.repository()); - boolean isSnapshotV2 = SNAPSHOT_V2.get(repository.getMetadata().settings()); - logger.debug("snapshot_v2 is set as [{}]", isSnapshotV2); + boolean isSnapshotV2 = SHALLOW_SNAPSHOT_V2.get(repository.getMetadata().settings()); + logger.debug("shallow_snapshot_v2 is set as [{}]", isSnapshotV2); boolean remoteStoreIndexShallowCopy = remoteStoreShallowCopyEnabled(repository); if (remoteStoreIndexShallowCopy From 1ef061a4ed26340079d5603b8fcb76b077513843 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 27 Aug 2024 19:10:23 +0530 Subject: [PATCH 29/30] add changelog Signed-off-by: Anshu Agarwal --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index cd02af4f625b9..aae63c7efaf33 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -26,6 +26,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add allowlist setting for ingest-geoip and ingest-useragent ([#15325](https://github.com/opensearch-project/OpenSearch/pull/15325)) - Adding access to noSubMatches and noOverlappingMatches in Hyphenation ([#13895](https://github.com/opensearch-project/OpenSearch/pull/13895)) - Add support for index level max slice count setting for concurrent segment search ([#15336](https://github.com/opensearch-project/OpenSearch/pull/15336)) +- Add support for centralize snapshot creation with pinned timestamp ([#15124](https://github.com/opensearch-project/OpenSearch/pull/15124)) ### Dependencies - Bump `netty` from 4.1.111.Final to 4.1.112.Final ([#15081](https://github.com/opensearch-project/OpenSearch/pull/15081)) From fb48a2dbe978b25ee6f48b9568a23a911ae13015 Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Wed, 28 Aug 2024 12:45:45 +0530 Subject: [PATCH 30/30] Add pinned timestamp setting in integ tests Signed-off-by: Anshu Agarwal --- .../remotestore/RemoteRestoreSnapshotIT.java | 55 +++++++++++-------- 1 file changed, 31 insertions(+), 24 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index 4db8aee974d0d..dc0654c623137 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -33,6 +33,7 @@ import org.opensearch.index.remote.RemoteStoreEnums.PathType; import org.opensearch.index.shard.IndexShard; import org.opensearch.indices.IndicesService; +import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; @@ -756,10 +757,9 @@ public void testInvalidRestoreRequestScenarios() throws Exception { } public void testCreateSnapshotV2() throws Exception { - - internalCluster().startClusterManagerOnlyNode(); - internalCluster().startDataOnlyNode(); - internalCluster().startDataOnlyNode(); + internalCluster().startClusterManagerOnlyNode(pinnedTimestampSettings()); + internalCluster().startDataOnlyNode(pinnedTimestampSettings()); + internalCluster().startDataOnlyNode(pinnedTimestampSettings()); String indexName1 = "testindex1"; String indexName2 = "testindex2"; String indexName3 = "testindex3"; @@ -826,9 +826,9 @@ public void testCreateSnapshotV2() throws Exception { public void testMixedSnapshotCreationWithV2RepositorySetting() throws Exception { - internalCluster().startClusterManagerOnlyNode(); - internalCluster().startDataOnlyNode(); - internalCluster().startDataOnlyNode(); + internalCluster().startClusterManagerOnlyNode(pinnedTimestampSettings()); + internalCluster().startDataOnlyNode(pinnedTimestampSettings()); + internalCluster().startDataOnlyNode(pinnedTimestampSettings()); String indexName1 = "testindex1"; String indexName2 = "testindex2"; String indexName3 = "testindex3"; @@ -908,9 +908,9 @@ public void testMixedSnapshotCreationWithV2RepositorySetting() throws Exception } public void testConcurrentSnapshotV2CreateOperation() throws InterruptedException, ExecutionException { - internalCluster().startClusterManagerOnlyNode(); - internalCluster().startDataOnlyNode(); - internalCluster().startDataOnlyNode(); + internalCluster().startClusterManagerOnlyNode(pinnedTimestampSettings()); + internalCluster().startDataOnlyNode(pinnedTimestampSettings()); + internalCluster().startDataOnlyNode(pinnedTimestampSettings()); String indexName1 = "testindex1"; String indexName2 = "testindex2"; String snapshotRepoName = "test-create-snapshot-repo"; @@ -989,9 +989,9 @@ public void testConcurrentSnapshotV2CreateOperation() throws InterruptedExceptio } public void testCreateSnapshotV2WithRedIndex() throws Exception { - internalCluster().startClusterManagerOnlyNode(); - internalCluster().startDataOnlyNode(); - internalCluster().startDataOnlyNode(); + internalCluster().startClusterManagerOnlyNode(pinnedTimestampSettings()); + internalCluster().startDataOnlyNode(pinnedTimestampSettings()); + internalCluster().startDataOnlyNode(pinnedTimestampSettings()); String indexName1 = "testindex1"; String indexName2 = "testindex2"; String snapshotRepoName = "test-create-snapshot-repo"; @@ -1043,9 +1043,9 @@ public void testCreateSnapshotV2WithRedIndex() throws Exception { } public void testCreateSnapshotV2WithIndexingLoad() throws Exception { - internalCluster().startClusterManagerOnlyNode(); - internalCluster().startDataOnlyNode(); - internalCluster().startDataOnlyNode(); + internalCluster().startClusterManagerOnlyNode(pinnedTimestampSettings()); + internalCluster().startDataOnlyNode(pinnedTimestampSettings()); + internalCluster().startDataOnlyNode(pinnedTimestampSettings()); String indexName1 = "testindex1"; String indexName2 = "testindex2"; String snapshotRepoName = "test-create-snapshot-repo"; @@ -1117,9 +1117,9 @@ public void testCreateSnapshotV2WithIndexingLoad() throws Exception { } public void testCreateSnapshotV2WithShallowCopySettingDisabled() throws Exception { - internalCluster().startClusterManagerOnlyNode(); - internalCluster().startDataOnlyNode(); - internalCluster().startDataOnlyNode(); + internalCluster().startClusterManagerOnlyNode(pinnedTimestampSettings()); + internalCluster().startDataOnlyNode(pinnedTimestampSettings()); + internalCluster().startDataOnlyNode(pinnedTimestampSettings()); String indexName1 = "testindex1"; String indexName2 = "testindex2"; String snapshotRepoName = "test-create-snapshot-repo"; @@ -1173,8 +1173,8 @@ public void testCreateSnapshotV2WithShallowCopySettingDisabled() throws Exceptio public void testClusterManagerFailoverDuringSnapshotCreation() throws Exception { - internalCluster().startClusterManagerOnlyNodes(3, Settings.EMPTY); - internalCluster().startDataOnlyNode(); + internalCluster().startClusterManagerOnlyNodes(3, pinnedTimestampSettings()); + internalCluster().startDataOnlyNode(pinnedTimestampSettings()); String indexName1 = "testindex1"; String indexName2 = "testindex2"; String snapshotRepoName = "test-create-snapshot-repo"; @@ -1253,9 +1253,9 @@ public void testClusterManagerFailoverDuringSnapshotCreation() throws Exception } public void testConcurrentV1SnapshotAndV2RepoSettingUpdate() throws Exception { - internalCluster().startClusterManagerOnlyNode(); - internalCluster().startDataOnlyNode(); - internalCluster().startDataOnlyNode(); + internalCluster().startClusterManagerOnlyNode(pinnedTimestampSettings()); + internalCluster().startDataOnlyNode(pinnedTimestampSettings()); + internalCluster().startDataOnlyNode(pinnedTimestampSettings()); String snapshotRepoName = "test-create-snapshot-repo"; String snapshotName1 = "test-create-snapshot-v1"; Path absolutePath1 = randomRepoPath().toAbsolutePath(); @@ -1341,4 +1341,11 @@ public void testConcurrentV1SnapshotAndV2RepoSettingUpdate() throws Exception { createV1SnapshotThread.join(); } + private Settings pinnedTimestampSettings() { + Settings settings = Settings.builder() + .put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_ENABLED.getKey(), true) + .build(); + return settings; + } + }