From 7a9cb35a461384d05b49e14d028feee09593c69a Mon Sep 17 00:00:00 2001 From: Sachin Kale Date: Tue, 3 Sep 2024 18:33:39 +0530 Subject: [PATCH 1/7] Restore snapshot changes for shallow snapshot V2 (#15462) --------- Signed-off-by: Sachin Kale Co-authored-by: Sachin Kale Co-authored-by: Gaurav Bafna --- .../RestoreShallowSnapshotV2IT.java | 805 ++++++++++++++++++ .../restore/RestoreSnapshotRequest.java | 36 +- .../cluster/routing/RecoverySource.java | 65 +- .../opensearch/index/shard/IndexShard.java | 97 ++- .../opensearch/index/shard/StoreRecovery.java | 102 ++- .../index/translog/RemoteFsTranslog.java | 62 +- .../index/translog/TranslogConfig.java | 9 + .../transfer/TranslogTransferManager.java | 20 + .../opensearch/repositories/Repository.java | 12 + .../blobstore/BlobStoreRepository.java | 39 +- .../InternalSnapshotsInfoService.java | 26 +- .../opensearch/snapshots/RestoreService.java | 6 +- .../index/shard/IndexShardTests.java | 2 +- .../TranslogTransferManagerTests.java | 50 ++ 14 files changed, 1237 insertions(+), 94 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/opensearch/remotestore/RestoreShallowSnapshotV2IT.java diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RestoreShallowSnapshotV2IT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RestoreShallowSnapshotV2IT.java new file mode 100644 index 0000000000000..c5a55f16cab2b --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RestoreShallowSnapshotV2IT.java @@ -0,0 +1,805 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.remotestore; + +import org.opensearch.action.DocWriteResponse; +import org.opensearch.action.admin.cluster.remotestore.restore.RestoreRemoteStoreRequest; +import org.opensearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; +import org.opensearch.action.admin.indices.delete.DeleteIndexRequest; +import org.opensearch.action.admin.indices.recovery.RecoveryResponse; +import org.opensearch.action.delete.DeleteResponse; +import org.opensearch.action.support.PlainActionFuture; +import org.opensearch.client.Client; +import org.opensearch.client.Requests; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.routing.RecoverySource; +import org.opensearch.common.Nullable; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.io.PathUtils; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.index.Index; +import org.opensearch.core.rest.RestStatus; +import org.opensearch.index.IndexService; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.remote.RemoteStoreEnums; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.indices.IndicesService; +import org.opensearch.indices.RemoteStoreSettings; +import org.opensearch.indices.recovery.RecoveryState; +import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.snapshots.AbstractSnapshotIntegTestCase; +import org.opensearch.snapshots.SnapshotInfo; +import org.opensearch.snapshots.SnapshotRestoreException; +import org.opensearch.snapshots.SnapshotState; +import org.opensearch.test.InternalTestCluster; +import org.opensearch.test.OpenSearchIntegTestCase; +import org.junit.After; +import org.junit.Before; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; +import static org.opensearch.index.remote.RemoteStoreEnums.DataCategory.SEGMENTS; +import static org.opensearch.index.remote.RemoteStoreEnums.DataCategory.TRANSLOG; +import static org.opensearch.index.remote.RemoteStoreEnums.DataType.DATA; +import static org.opensearch.index.remote.RemoteStoreEnums.DataType.METADATA; +import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING; +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; +import static org.hamcrest.Matchers.lessThanOrEqualTo; + +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) +public class RestoreShallowSnapshotV2IT extends AbstractSnapshotIntegTestCase { + + private static final String BASE_REMOTE_REPO = "test-rs-repo" + TEST_REMOTE_STORE_REPO_SUFFIX; + private Path remoteRepoPath; + + @Before + public void setup() { + remoteRepoPath = randomRepoPath().toAbsolutePath(); + } + + @After + public void teardown() { + clusterAdmin().prepareCleanupRepository(BASE_REMOTE_REPO).get(); + } + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal)) + .put(remoteStoreClusterSettings(BASE_REMOTE_REPO, remoteRepoPath)) + .put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_ENABLED.getKey(), true) + .build(); + } + + @Override + protected Settings.Builder getRepositorySettings(Path location, boolean shallowCopyEnabled) { + Settings.Builder settingsBuilder = randomRepositorySettings(); + settingsBuilder.put("location", location); + if (shallowCopyEnabled) { + settingsBuilder.put(BlobStoreRepository.REMOTE_STORE_INDEX_SHALLOW_COPY.getKey(), true) + .put(BlobStoreRepository.SHALLOW_SNAPSHOT_V2.getKey(), true); + } + return settingsBuilder; + } + + private Settings.Builder getIndexSettings(int numOfShards, int numOfReplicas) { + Settings.Builder settingsBuilder = Settings.builder() + .put(super.indexSettings()) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, numOfShards) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, numOfReplicas) + .put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), "300s"); + return settingsBuilder; + } + + private void indexDocuments(Client client, String indexName, int numOfDocs) { + indexDocuments(client, indexName, 0, numOfDocs); + } + + protected void indexDocuments(Client client, String indexName, int fromId, int toId) { + for (int i = fromId; i < toId; i++) { + String id = Integer.toString(i); + client.prepareIndex(indexName).setId(id).setSource("text", "sometext").get(); + } + } + + private void assertDocsPresentInIndex(Client client, String indexName, int numOfDocs) { + for (int i = 0; i < numOfDocs; i++) { + String id = Integer.toString(i); + logger.info("checking for index " + indexName + " with docId" + id); + assertTrue("doc with id" + id + " is not present for index " + indexName, client.prepareGet(indexName, id).get().isExists()); + } + } + + public void testRestoreOperationsShallowCopyEnabled() throws Exception { + String clusterManagerNode = internalCluster().startClusterManagerOnlyNode(); + String primary = internalCluster().startDataOnlyNode(); + String indexName1 = "testindex1"; + String indexName2 = "testindex2"; + String snapshotRepoName = "test-restore-snapshot-repo"; + String snapshotName1 = "test-restore-snapshot1"; + String snapshotName2 = "test-restore-snapshot2"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + logger.info("Snapshot Path [{}]", absolutePath1); + String restoredIndexName1 = indexName1 + "-restored"; + String restoredIndexName2 = indexName2 + "-restored"; + + createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, true)); + + Client client = client(); + Settings indexSettings = getIndexSettings(1, 0).build(); + createIndex(indexName1, indexSettings); + + Settings indexSettings2 = getIndexSettings(1, 0).build(); + createIndex(indexName2, indexSettings2); + + final int numDocsInIndex1 = 5; + final int numDocsInIndex2 = 6; + indexDocuments(client, indexName1, numDocsInIndex1); + indexDocuments(client, indexName2, numDocsInIndex2); + ensureGreen(indexName1, indexName2); + + internalCluster().startDataOnlyNode(); + logger.info("--> snapshot"); + + SnapshotInfo snapshotInfo = createSnapshot(snapshotRepoName, snapshotName1, new ArrayList<>()); + assertThat(snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); + assertThat(snapshotInfo.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards())); + + updateRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, false)); + SnapshotInfo snapshotInfo2 = createSnapshot( + snapshotRepoName, + snapshotName2, + new ArrayList<>(Arrays.asList(indexName1, indexName2)) + ); + assertThat(snapshotInfo2.state(), equalTo(SnapshotState.SUCCESS)); + assertThat(snapshotInfo2.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo2.successfulShards(), equalTo(snapshotInfo2.totalShards())); + + DeleteResponse deleteResponse = client().prepareDelete(indexName1, "0").execute().actionGet(); + assertEquals(deleteResponse.getResult(), DocWriteResponse.Result.DELETED); + indexDocuments(client, indexName1, numDocsInIndex1, numDocsInIndex1 + randomIntBetween(2, 5)); + ensureGreen(indexName1); + + RestoreSnapshotResponse restoreSnapshotResponse1 = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(false) + .setIndices(indexName1) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1) + .get(); + RestoreSnapshotResponse restoreSnapshotResponse2 = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName2) + .setWaitForCompletion(false) + .setIndices(indexName2) + .setRenamePattern(indexName2) + .setRenameReplacement(restoredIndexName2) + .get(); + assertEquals(restoreSnapshotResponse1.status(), RestStatus.ACCEPTED); + assertEquals(restoreSnapshotResponse2.status(), RestStatus.ACCEPTED); + ensureGreen(restoredIndexName1, restoredIndexName2); + assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1); + assertDocsPresentInIndex(client, restoredIndexName2, numDocsInIndex2); + + // deleting data for restoredIndexName1 and restoring from remote store. + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primary)); + ensureRed(restoredIndexName1); + // Re-initialize client to make sure we are not using client from stopped node. + client = client(clusterManagerNode); + assertAcked(client.admin().indices().prepareClose(restoredIndexName1)); + client.admin() + .cluster() + .restoreRemoteStore( + new RestoreRemoteStoreRequest().indices(restoredIndexName1).restoreAllShards(true), + PlainActionFuture.newFuture() + ); + ensureYellowAndNoInitializingShards(restoredIndexName1); + ensureGreen(restoredIndexName1); + assertDocsPresentInIndex(client(), restoredIndexName1, numDocsInIndex1); + // indexing some new docs and validating + indexDocuments(client, restoredIndexName1, numDocsInIndex1, numDocsInIndex1 + 2); + ensureGreen(restoredIndexName1); + assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1 + 2); + } + + public void testRemoteStoreCustomDataOnIndexCreationAndRestore() { + String clusterManagerNode = internalCluster().startClusterManagerOnlyNode(); + internalCluster().startDataOnlyNode(); + String indexName1 = "testindex1"; + String indexName2 = "testindex2"; + String snapshotRepoName = "test-restore-snapshot-repo"; + String snapshotName1 = "test-restore-snapshot1"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + logger.info("Snapshot Path [{}]", absolutePath1); + String restoredIndexName1version1 = indexName1 + "-restored-1"; + String restoredIndexName1version2 = indexName1 + "-restored-2"; + + client(clusterManagerNode).admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), RemoteStoreEnums.PathType.FIXED)) + .get(); + createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, true)); + Client client = client(); + Settings indexSettings = getIndexSettings(1, 0).build(); + createIndex(indexName1, indexSettings); + + indexDocuments(client, indexName1, randomIntBetween(5, 10)); + ensureGreen(indexName1); + validatePathType(indexName1, RemoteStoreEnums.PathType.FIXED); + + logger.info("--> snapshot"); + SnapshotInfo snapshotInfo = createSnapshot(snapshotRepoName, snapshotName1, new ArrayList<>(Arrays.asList(indexName1))); + assertEquals(SnapshotState.SUCCESS, snapshotInfo.state()); + assertTrue(snapshotInfo.successfulShards() > 0); + assertEquals(snapshotInfo.totalShards(), snapshotInfo.successfulShards()); + + RestoreSnapshotResponse restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(false) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1version1) + .get(); + assertEquals(RestStatus.ACCEPTED, restoreSnapshotResponse.status()); + ensureGreen(restoredIndexName1version1); + validatePathType(restoredIndexName1version1, RemoteStoreEnums.PathType.FIXED); + + client(clusterManagerNode).admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings( + Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), RemoteStoreEnums.PathType.HASHED_PREFIX) + ) + .get(); + + restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(false) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1version2) + .get(); + assertEquals(RestStatus.ACCEPTED, restoreSnapshotResponse.status()); + ensureGreen(restoredIndexName1version2); + validatePathType( + restoredIndexName1version2, + RemoteStoreEnums.PathType.HASHED_PREFIX, + RemoteStoreEnums.PathHashAlgorithm.FNV_1A_COMPOSITE_1 + ); + + // Create index with cluster setting cluster.remote_store.index.path.type as hashed_prefix. + indexSettings = getIndexSettings(1, 0).build(); + createIndex(indexName2, indexSettings); + ensureGreen(indexName2); + validatePathType(indexName2, RemoteStoreEnums.PathType.HASHED_PREFIX, RemoteStoreEnums.PathHashAlgorithm.FNV_1A_COMPOSITE_1); + + // Validating that custom data has not changed for indexes which were created before the cluster setting got updated + validatePathType(indexName1, RemoteStoreEnums.PathType.FIXED); + + // Create Snapshot of index 2 + String snapshotName2 = "test-restore-snapshot2"; + snapshotInfo = createSnapshot(snapshotRepoName, snapshotName2, new ArrayList<>(List.of(indexName2))); + assertEquals(SnapshotState.SUCCESS, snapshotInfo.state()); + assertTrue(snapshotInfo.successfulShards() > 0); + assertEquals(snapshotInfo.totalShards(), snapshotInfo.successfulShards()); + + // Update cluster settings to FIXED + client(clusterManagerNode).admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), RemoteStoreEnums.PathType.FIXED)) + .get(); + + // Close index 2 + assertAcked(client().admin().indices().prepareClose(indexName2)); + restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName2) + .setWaitForCompletion(false) + .setIndices(indexName2) + .get(); + assertEquals(RestStatus.ACCEPTED, restoreSnapshotResponse.status()); + ensureGreen(indexName2); + + // Validating that custom data has not changed for testindex2 which was created before the cluster setting got updated + validatePathType(indexName2, RemoteStoreEnums.PathType.HASHED_PREFIX, RemoteStoreEnums.PathHashAlgorithm.FNV_1A_COMPOSITE_1); + } + + private void validatePathType(String index, RemoteStoreEnums.PathType pathType) { + validatePathType(index, pathType, null); + } + + private void validatePathType( + String index, + RemoteStoreEnums.PathType pathType, + @Nullable RemoteStoreEnums.PathHashAlgorithm pathHashAlgorithm + ) { + ClusterState state = client().admin().cluster().prepareState().execute().actionGet().getState(); + // Validate that the remote_store custom data is present in index metadata for the created index. + Map remoteCustomData = state.metadata().index(index).getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY); + assertNotNull(remoteCustomData); + assertEquals(pathType.name(), remoteCustomData.get(RemoteStoreEnums.PathType.NAME)); + if (Objects.nonNull(pathHashAlgorithm)) { + assertEquals(pathHashAlgorithm.name(), remoteCustomData.get(RemoteStoreEnums.PathHashAlgorithm.NAME)); + } + } + + public void testRestoreInSameRemoteStoreEnabledIndex() throws IOException { + String clusterManagerNode = internalCluster().startClusterManagerOnlyNode(); + String primary = internalCluster().startDataOnlyNode(); + String indexName1 = "testindex1"; + String indexName2 = "testindex2"; + String snapshotRepoName = "test-restore-snapshot-repo"; + String snapshotName1 = "test-restore-snapshot1"; + String snapshotName2 = "test-restore-snapshot2"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + logger.info("Snapshot Path [{}]", absolutePath1); + String restoredIndexName2 = indexName2 + "-restored"; + + boolean enableShallowCopy = randomBoolean(); + createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, enableShallowCopy)); + + Client client = client(); + Settings indexSettings = getIndexSettings(1, 0).build(); + createIndex(indexName1, indexSettings); + + Settings indexSettings2 = getIndexSettings(1, 0).build(); + createIndex(indexName2, indexSettings2); + + final int numDocsInIndex1 = 5; + final int numDocsInIndex2 = 6; + indexDocuments(client, indexName1, numDocsInIndex1); + indexDocuments(client, indexName2, numDocsInIndex2); + ensureGreen(indexName1, indexName2); + + internalCluster().startDataOnlyNode(); + logger.info("--> snapshot"); + SnapshotInfo snapshotInfo1 = createSnapshot( + snapshotRepoName, + snapshotName1, + new ArrayList<>(Arrays.asList(indexName1, indexName2)) + ); + assertThat(snapshotInfo1.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo1.successfulShards(), equalTo(snapshotInfo1.totalShards())); + assertThat(snapshotInfo1.state(), equalTo(SnapshotState.SUCCESS)); + + updateRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, false)); + SnapshotInfo snapshotInfo2 = createSnapshot( + snapshotRepoName, + snapshotName2, + new ArrayList<>(Arrays.asList(indexName1, indexName2)) + ); + assertThat(snapshotInfo2.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo2.successfulShards(), equalTo(snapshotInfo2.totalShards())); + assertThat(snapshotInfo2.state(), equalTo(SnapshotState.SUCCESS)); + + DeleteResponse deleteResponse = client().prepareDelete(indexName1, "0").execute().actionGet(); + assertEquals(deleteResponse.getResult(), DocWriteResponse.Result.DELETED); + indexDocuments(client, indexName1, numDocsInIndex1, numDocsInIndex1 + randomIntBetween(2, 5)); + ensureGreen(indexName1); + + assertAcked(client().admin().indices().prepareClose(indexName1)); + + RestoreSnapshotResponse restoreSnapshotResponse1 = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(false) + .setIndices(indexName1) + .get(); + RestoreSnapshotResponse restoreSnapshotResponse2 = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName2) + .setWaitForCompletion(false) + .setIndices(indexName2) + .setRenamePattern(indexName2) + .setRenameReplacement(restoredIndexName2) + .get(); + assertEquals(restoreSnapshotResponse1.status(), RestStatus.ACCEPTED); + assertEquals(restoreSnapshotResponse2.status(), RestStatus.ACCEPTED); + ensureGreen(indexName1, restoredIndexName2); + + assertRemoteSegmentsAndTranslogUploaded(restoredIndexName2); + assertDocsPresentInIndex(client, indexName1, numDocsInIndex1); + assertDocsPresentInIndex(client, restoredIndexName2, numDocsInIndex2); + // indexing some new docs and validating + indexDocuments(client, indexName1, numDocsInIndex1, numDocsInIndex1 + 2); + ensureGreen(indexName1); + assertDocsPresentInIndex(client, indexName1, numDocsInIndex1 + 2); + + // deleting data for restoredIndexName1 and restoring from remote store. + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primary)); + ensureRed(indexName1); + // Re-initialize client to make sure we are not using client from stopped node. + client = client(clusterManagerNode); + assertAcked(client.admin().indices().prepareClose(indexName1)); + client.admin() + .cluster() + .restoreRemoteStore(new RestoreRemoteStoreRequest().indices(indexName1).restoreAllShards(true), PlainActionFuture.newFuture()); + ensureYellowAndNoInitializingShards(indexName1); + ensureGreen(indexName1); + assertDocsPresentInIndex(client(), indexName1, numDocsInIndex1); + // indexing some new docs and validating + indexDocuments(client, indexName1, numDocsInIndex1 + 2, numDocsInIndex1 + 4); + ensureGreen(indexName1); + assertDocsPresentInIndex(client, indexName1, numDocsInIndex1 + 4); + } + + void assertRemoteSegmentsAndTranslogUploaded(String idx) throws IOException { + Client client = client(); + String translogPathFixedPrefix = RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_PATH_PREFIX.get(getNodeSettings()); + String segmentsPathFixedPrefix = RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX.get(getNodeSettings()); + String path = getShardLevelBlobPath(client, idx, new BlobPath(), "0", TRANSLOG, METADATA, translogPathFixedPrefix).buildAsString(); + Path remoteTranslogMetadataPath = Path.of(remoteRepoPath + "/" + path); + path = getShardLevelBlobPath(client, idx, new BlobPath(), "0", TRANSLOG, DATA, translogPathFixedPrefix).buildAsString(); + Path remoteTranslogDataPath = Path.of(remoteRepoPath + "/" + path); + path = getShardLevelBlobPath(client, idx, new BlobPath(), "0", SEGMENTS, METADATA, segmentsPathFixedPrefix).buildAsString(); + Path segmentMetadataPath = Path.of(remoteRepoPath + "/" + path); + path = getShardLevelBlobPath(client, idx, new BlobPath(), "0", SEGMENTS, DATA, segmentsPathFixedPrefix).buildAsString(); + Path segmentDataPath = Path.of(remoteRepoPath + "/" + path); + + try ( + Stream translogMetadata = Files.list(remoteTranslogMetadataPath); + Stream translogData = Files.list(remoteTranslogDataPath); + Stream segmentMetadata = Files.list(segmentMetadataPath); + Stream segmentData = Files.list(segmentDataPath); + + ) { + assertTrue(translogData.count() > 0); + assertTrue(translogMetadata.count() > 0); + assertTrue(segmentMetadata.count() > 0); + assertTrue(segmentData.count() > 0); + } + + } + + public void testRemoteRestoreIndexRestoredFromSnapshot() throws IOException, ExecutionException, InterruptedException { + internalCluster().startClusterManagerOnlyNode(); + internalCluster().startDataOnlyNodes(2); + + String indexName1 = "testindex1"; + String snapshotRepoName = "test-restore-snapshot-repo"; + String snapshotName1 = "test-restore-snapshot1"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + logger.info("Snapshot Path [{}]", absolutePath1); + + createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, true)); + + Settings indexSettings = getIndexSettings(1, 0).build(); + createIndex(indexName1, indexSettings); + + final int numDocsInIndex1 = randomIntBetween(20, 30); + indexDocuments(client(), indexName1, numDocsInIndex1); + flushAndRefresh(indexName1); + ensureGreen(indexName1); + + logger.info("--> snapshot"); + SnapshotInfo snapshotInfo1 = createSnapshot(snapshotRepoName, snapshotName1, new ArrayList<>(Arrays.asList(indexName1))); + assertThat(snapshotInfo1.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo1.successfulShards(), equalTo(snapshotInfo1.totalShards())); + assertThat(snapshotInfo1.state(), equalTo(SnapshotState.SUCCESS)); + + assertAcked(client().admin().indices().delete(new DeleteIndexRequest(indexName1)).get()); + assertFalse(indexExists(indexName1)); + + RestoreSnapshotResponse restoreSnapshotResponse1 = client().admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(false) + .setIndices(indexName1) + .get(); + + assertEquals(restoreSnapshotResponse1.status(), RestStatus.ACCEPTED); + ensureGreen(indexName1); + assertDocsPresentInIndex(client(), indexName1, numDocsInIndex1); + + assertRemoteSegmentsAndTranslogUploaded(indexName1); + + // Clear the local data before stopping the node. This will make sure that remote translog is empty. + IndexShard indexShard = getIndexShard(primaryNodeName(indexName1), indexName1); + try (Stream files = Files.list(indexShard.shardPath().resolveTranslog())) { + IOUtils.deleteFilesIgnoringExceptions(files.collect(Collectors.toList())); + } + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primaryNodeName(indexName1))); + + ensureRed(indexName1); + + client().admin() + .cluster() + .restoreRemoteStore(new RestoreRemoteStoreRequest().indices(indexName1).restoreAllShards(false), PlainActionFuture.newFuture()); + + ensureGreen(indexName1); + assertDocsPresentInIndex(client(), indexName1, numDocsInIndex1); + } + + private IndexShard getIndexShard(String node, String indexName) { + final Index index = resolveIndex(indexName); + IndicesService indicesService = internalCluster().getInstance(IndicesService.class, node); + IndexService indexService = indicesService.indexService(index); + assertNotNull(indexService); + final Optional shardId = indexService.shardIds().stream().findFirst(); + return shardId.map(indexService::getShard).orElse(null); + } + + public void testRestoreShallowSnapshotRepository() throws ExecutionException, InterruptedException { + String indexName1 = "testindex1"; + String snapshotRepoName = "test-restore-snapshot-repo"; + String remoteStoreRepoNameUpdated = "test-rs-repo-updated" + TEST_REMOTE_STORE_REPO_SUFFIX; + String snapshotName1 = "test-restore-snapshot1"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + Path absolutePath2 = randomRepoPath().toAbsolutePath(); + String[] pathTokens = absolutePath1.toString().split("/"); + String basePath = pathTokens[pathTokens.length - 1]; + Arrays.copyOf(pathTokens, pathTokens.length - 1); + Path location = PathUtils.get(String.join("/", pathTokens)); + pathTokens = absolutePath2.toString().split("/"); + String basePath2 = pathTokens[pathTokens.length - 1]; + Arrays.copyOf(pathTokens, pathTokens.length - 1); + Path location2 = PathUtils.get(String.join("/", pathTokens)); + logger.info("Path 1 [{}]", absolutePath1); + logger.info("Path 2 [{}]", absolutePath2); + String restoredIndexName1 = indexName1 + "-restored"; + + createRepository(snapshotRepoName, "fs", getRepositorySettings(location, basePath, true)); + + Client client = client(); + Settings indexSettings = Settings.builder() + .put(super.indexSettings()) + .put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), "300s") + .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .build(); + createIndex(indexName1, indexSettings); + + int numDocsInIndex1 = randomIntBetween(2, 5); + indexDocuments(client, indexName1, numDocsInIndex1); + + ensureGreen(indexName1); + + logger.info("--> snapshot"); + SnapshotInfo snapshotInfo1 = createSnapshot(snapshotRepoName, snapshotName1, new ArrayList<>(List.of(indexName1))); + assertThat(snapshotInfo1.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo1.successfulShards(), equalTo(snapshotInfo1.totalShards())); + assertThat(snapshotInfo1.state(), equalTo(SnapshotState.SUCCESS)); + + client().admin().indices().close(Requests.closeIndexRequest(indexName1)).get(); + createRepository(remoteStoreRepoNameUpdated, "fs", remoteRepoPath); + RestoreSnapshotResponse restoreSnapshotResponse2 = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(true) + .setIndices(indexName1) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1) + .setSourceRemoteStoreRepository(remoteStoreRepoNameUpdated) + .get(); + + assertTrue(restoreSnapshotResponse2.getRestoreInfo().failedShards() == 0); + ensureGreen(restoredIndexName1); + assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1); + + // ensure recovery details are non-zero + RecoveryResponse recoveryResponse = client().admin().indices().prepareRecoveries(restoredIndexName1).execute().actionGet(); + assertEquals(1, recoveryResponse.getTotalShards()); + assertEquals(1, recoveryResponse.getSuccessfulShards()); + assertEquals(0, recoveryResponse.getFailedShards()); + assertEquals(1, recoveryResponse.shardRecoveryStates().size()); + assertTrue(recoveryResponse.shardRecoveryStates().containsKey(restoredIndexName1)); + assertEquals(1, recoveryResponse.shardRecoveryStates().get(restoredIndexName1).size()); + + RecoveryState recoveryState = recoveryResponse.shardRecoveryStates().get(restoredIndexName1).get(0); + assertEquals(RecoveryState.Stage.DONE, recoveryState.getStage()); + assertEquals(0, recoveryState.getShardId().getId()); + assertTrue(recoveryState.getPrimary()); + assertEquals(RecoverySource.Type.SNAPSHOT, recoveryState.getRecoverySource().getType()); + assertThat(recoveryState.getIndex().time(), greaterThanOrEqualTo(0L)); + + // ensure populated file details + assertTrue(recoveryState.getIndex().totalFileCount() > 0); + assertTrue(recoveryState.getIndex().totalRecoverFiles() > 0); + assertTrue(recoveryState.getIndex().recoveredFileCount() > 0); + assertThat(recoveryState.getIndex().recoveredFilesPercent(), greaterThanOrEqualTo(0.0f)); + assertThat(recoveryState.getIndex().recoveredFilesPercent(), lessThanOrEqualTo(100.0f)); + assertFalse(recoveryState.getIndex().fileDetails().isEmpty()); + + // ensure populated bytes details + assertTrue(recoveryState.getIndex().recoveredBytes() > 0L); + assertTrue(recoveryState.getIndex().totalBytes() > 0L); + assertTrue(recoveryState.getIndex().totalRecoverBytes() > 0L); + assertThat(recoveryState.getIndex().recoveredBytesPercent(), greaterThanOrEqualTo(0.0f)); + assertThat(recoveryState.getIndex().recoveredBytesPercent(), lessThanOrEqualTo(100.0f)); + + // indexing some new docs and validating + indexDocuments(client, restoredIndexName1, numDocsInIndex1, numDocsInIndex1 + 2); + ensureGreen(restoredIndexName1); + assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1 + 2); + } + + public void testRestoreShallowSnapshotIndexAfterSnapshot() throws ExecutionException, InterruptedException { + String indexName1 = "testindex1"; + String snapshotRepoName = "test-restore-snapshot-repo"; + String remoteStoreRepoNameUpdated = "test-rs-repo-updated" + TEST_REMOTE_STORE_REPO_SUFFIX; + String snapshotName1 = "test-restore-snapshot1"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + Path absolutePath2 = randomRepoPath().toAbsolutePath(); + String[] pathTokens = absolutePath1.toString().split("/"); + String basePath = pathTokens[pathTokens.length - 1]; + Arrays.copyOf(pathTokens, pathTokens.length - 1); + Path location = PathUtils.get(String.join("/", pathTokens)); + pathTokens = absolutePath2.toString().split("/"); + String basePath2 = pathTokens[pathTokens.length - 1]; + Arrays.copyOf(pathTokens, pathTokens.length - 1); + Path location2 = PathUtils.get(String.join("/", pathTokens)); + logger.info("Path 1 [{}]", absolutePath1); + logger.info("Path 2 [{}]", absolutePath2); + String restoredIndexName1 = indexName1 + "-restored"; + + createRepository(snapshotRepoName, "fs", getRepositorySettings(location, basePath, true)); + + Client client = client(); + Settings indexSettings = Settings.builder() + .put(super.indexSettings()) + .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .build(); + createIndex(indexName1, indexSettings); + + int numDocsInIndex1 = randomIntBetween(2, 5); + indexDocuments(client, indexName1, numDocsInIndex1); + + ensureGreen(indexName1); + + logger.info("--> snapshot"); + SnapshotInfo snapshotInfo1 = createSnapshot(snapshotRepoName, snapshotName1, new ArrayList<>(List.of(indexName1))); + assertThat(snapshotInfo1.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo1.successfulShards(), equalTo(snapshotInfo1.totalShards())); + assertThat(snapshotInfo1.state(), equalTo(SnapshotState.SUCCESS)); + + int extraNumDocsInIndex1 = randomIntBetween(20, 50); + indexDocuments(client, indexName1, extraNumDocsInIndex1); + refresh(indexName1); + + client().admin().indices().close(Requests.closeIndexRequest(indexName1)).get(); + createRepository(remoteStoreRepoNameUpdated, "fs", remoteRepoPath); + RestoreSnapshotResponse restoreSnapshotResponse2 = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(true) + .setIndices(indexName1) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1) + .setSourceRemoteStoreRepository(remoteStoreRepoNameUpdated) + .get(); + + assertTrue(restoreSnapshotResponse2.getRestoreInfo().failedShards() == 0); + ensureGreen(restoredIndexName1); + assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1); + + // indexing some new docs and validating + indexDocuments(client, restoredIndexName1, numDocsInIndex1, numDocsInIndex1 + 2); + ensureGreen(restoredIndexName1); + assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1 + 2); + } + + public void testInvalidRestoreRequestScenarios() throws Exception { + internalCluster().startClusterManagerOnlyNode(); + internalCluster().startDataOnlyNode(); + String index = "test-index"; + String snapshotRepo = "test-restore-snapshot-repo"; + String newRemoteStoreRepo = "test-new-rs-repo"; + String snapshotName1 = "test-restore-snapshot1"; + String snapshotName2 = "test-restore-snapshot2"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + logger.info("Snapshot Path [{}]", absolutePath1); + String restoredIndex = index + "-restored"; + + createRepository(snapshotRepo, "fs", getRepositorySettings(absolutePath1, true)); + + Client client = client(); + Settings indexSettings = getIndexSettings(1, 0).build(); + createIndex(index, indexSettings); + + final int numDocsInIndex = 5; + indexDocuments(client, index, numDocsInIndex); + ensureGreen(index); + + internalCluster().startDataOnlyNode(); + logger.info("--> snapshot"); + + SnapshotInfo snapshotInfo = createSnapshot(snapshotRepo, snapshotName1, new ArrayList<>(List.of(index))); + assertThat(snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); + assertThat(snapshotInfo.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards())); + + updateRepository(snapshotRepo, "fs", getRepositorySettings(absolutePath1, false)); + SnapshotInfo snapshotInfo2 = createSnapshot(snapshotRepo, snapshotName2, new ArrayList<>(List.of(index))); + assertThat(snapshotInfo2.state(), equalTo(SnapshotState.SUCCESS)); + assertThat(snapshotInfo2.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo2.successfulShards(), equalTo(snapshotInfo2.totalShards())); + + DeleteResponse deleteResponse = client().prepareDelete(index, "0").execute().actionGet(); + assertEquals(deleteResponse.getResult(), DocWriteResponse.Result.DELETED); + indexDocuments(client, index, numDocsInIndex, numDocsInIndex + randomIntBetween(2, 5)); + ensureGreen(index); + + // try index restore with remote store disabled + SnapshotRestoreException exception = expectThrows( + SnapshotRestoreException.class, + () -> client().admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepo, snapshotName1) + .setWaitForCompletion(false) + .setIgnoreIndexSettings(SETTING_REMOTE_STORE_ENABLED) + .setIndices(index) + .setRenamePattern(index) + .setRenameReplacement(restoredIndex) + .get() + ); + assertTrue(exception.getMessage().contains("cannot remove setting [index.remote_store.enabled] on restore")); + + // try index restore with remote store repository modified + Settings remoteStoreIndexSettings = Settings.builder() + .put(IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY, newRemoteStoreRepo) + .build(); + + exception = expectThrows( + SnapshotRestoreException.class, + () -> client().admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepo, snapshotName1) + .setWaitForCompletion(false) + .setIndexSettings(remoteStoreIndexSettings) + .setIndices(index) + .setRenamePattern(index) + .setRenameReplacement(restoredIndex) + .get() + ); + assertTrue(exception.getMessage().contains("cannot modify setting [index.remote_store.segment.repository]" + " on restore")); + + // try index restore with remote store repository and translog store repository disabled + exception = expectThrows( + SnapshotRestoreException.class, + () -> client().admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepo, snapshotName1) + .setWaitForCompletion(false) + .setIgnoreIndexSettings( + IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY, + IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY + ) + .setIndices(index) + .setRenamePattern(index) + .setRenameReplacement(restoredIndex) + .get() + ); + assertTrue(exception.getMessage().contains("cannot remove setting [index.remote_store.segment.repository]" + " on restore")); + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequest.java index 492ef86bb7843..409c48cabad35 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequest.java @@ -121,6 +121,8 @@ private static StorageType fromString(String string) { private StorageType storageType = StorageType.LOCAL; @Nullable private String sourceRemoteStoreRepository = null; + @Nullable + private String sourceRemoteTranslogRepository = null; @Nullable // if any snapshot UUID will do private String snapshotUuid; @@ -159,6 +161,9 @@ public RestoreSnapshotRequest(StreamInput in) throws IOException { if (in.getVersion().onOrAfter(Version.V_2_10_0)) { sourceRemoteStoreRepository = in.readOptionalString(); } + if (in.getVersion().onOrAfter(Version.CURRENT)) { + sourceRemoteTranslogRepository = in.readOptionalString(); + } } @Override @@ -183,6 +188,9 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_2_10_0)) { out.writeOptionalString(sourceRemoteStoreRepository); } + if (out.getVersion().onOrAfter(Version.CURRENT)) { + out.writeOptionalString(sourceRemoteTranslogRepository); + } } @Override @@ -545,6 +553,16 @@ public RestoreSnapshotRequest setSourceRemoteStoreRepository(String sourceRemote return this; } + /** + * Sets Source Remote Translog Repository for all the restored indices + * + * @param sourceRemoteTranslogRepository name of the remote translog repository that should be used for all restored indices. + */ + public RestoreSnapshotRequest setSourceRemoteTranslogRepository(String sourceRemoteTranslogRepository) { + this.sourceRemoteTranslogRepository = sourceRemoteTranslogRepository; + return this; + } + /** * Returns Source Remote Store Repository for all the restored indices * @@ -554,6 +572,15 @@ public String getSourceRemoteStoreRepository() { return sourceRemoteStoreRepository; } + /** + * Returns Source Remote Translog Repository for all the restored indices + * + * @return source Remote Translog Repository + */ + public String getSourceRemoteTranslogRepository() { + return sourceRemoteTranslogRepository; + } + /** * Parses restore definition * @@ -673,6 +700,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (sourceRemoteStoreRepository != null) { builder.field("source_remote_store_repository", sourceRemoteStoreRepository); } + if (sourceRemoteTranslogRepository != null) { + builder.field("source_remote_translog_repository", sourceRemoteTranslogRepository); + } builder.endObject(); return builder; } @@ -701,7 +731,8 @@ public boolean equals(Object o) { && Arrays.equals(ignoreIndexSettings, that.ignoreIndexSettings) && Objects.equals(snapshotUuid, that.snapshotUuid) && Objects.equals(storageType, that.storageType) - && Objects.equals(sourceRemoteStoreRepository, that.sourceRemoteStoreRepository); + && Objects.equals(sourceRemoteStoreRepository, that.sourceRemoteStoreRepository) + && Objects.equals(sourceRemoteTranslogRepository, that.sourceRemoteTranslogRepository); return equals; } @@ -721,7 +752,8 @@ public int hashCode() { indexSettings, snapshotUuid, storageType, - sourceRemoteStoreRepository + sourceRemoteStoreRepository, + sourceRemoteTranslogRepository ); result = 31 * result + Arrays.hashCode(indices); result = 31 * result + Arrays.hashCode(ignoreIndexSettings); diff --git a/server/src/main/java/org/opensearch/cluster/routing/RecoverySource.java b/server/src/main/java/org/opensearch/cluster/routing/RecoverySource.java index 8d407c6aff5b6..43e195ed47553 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RecoverySource.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RecoverySource.java @@ -48,6 +48,8 @@ import java.io.IOException; import java.util.Objects; +import static org.opensearch.Version.CURRENT; + /** * Represents the recovery source of a shard. Available recovery types are: *

@@ -264,6 +266,9 @@ public static class SnapshotRecoverySource extends RecoverySource { private final boolean isSearchableSnapshot; private final boolean remoteStoreIndexShallowCopy; private final String sourceRemoteStoreRepository; + private final String sourceRemoteTranslogRepository; + + private final long pinnedTimestamp; public SnapshotRecoverySource(String restoreUUID, Snapshot snapshot, Version version, IndexId indexId) { this(restoreUUID, snapshot, version, indexId, false, false, null); @@ -277,6 +282,30 @@ public SnapshotRecoverySource( boolean isSearchableSnapshot, boolean remoteStoreIndexShallowCopy, @Nullable String sourceRemoteStoreRepository + ) { + this( + restoreUUID, + snapshot, + version, + indexId, + isSearchableSnapshot, + remoteStoreIndexShallowCopy, + sourceRemoteStoreRepository, + null, + 0L + ); + } + + public SnapshotRecoverySource( + String restoreUUID, + Snapshot snapshot, + Version version, + IndexId indexId, + boolean isSearchableSnapshot, + boolean remoteStoreIndexShallowCopy, + @Nullable String sourceRemoteStoreRepository, + @Nullable String sourceRemoteTranslogRepository, + long pinnedTimestamp ) { this.restoreUUID = restoreUUID; this.snapshot = Objects.requireNonNull(snapshot); @@ -285,6 +314,8 @@ public SnapshotRecoverySource( this.isSearchableSnapshot = isSearchableSnapshot; this.remoteStoreIndexShallowCopy = remoteStoreIndexShallowCopy; this.sourceRemoteStoreRepository = sourceRemoteStoreRepository; + this.sourceRemoteTranslogRepository = sourceRemoteTranslogRepository; + this.pinnedTimestamp = pinnedTimestamp; } SnapshotRecoverySource(StreamInput in) throws IOException { @@ -304,6 +335,13 @@ public SnapshotRecoverySource( remoteStoreIndexShallowCopy = false; sourceRemoteStoreRepository = null; } + if (in.getVersion().onOrAfter(CURRENT)) { + sourceRemoteTranslogRepository = in.readOptionalString(); + pinnedTimestamp = in.readLong(); + } else { + sourceRemoteTranslogRepository = null; + pinnedTimestamp = 0L; + } } public String restoreUUID() { @@ -336,10 +374,18 @@ public String sourceRemoteStoreRepository() { return sourceRemoteStoreRepository; } + public String sourceRemoteTranslogRepository() { + return sourceRemoteTranslogRepository; + } + public boolean remoteStoreIndexShallowCopy() { return remoteStoreIndexShallowCopy; } + public long pinnedTimestamp() { + return pinnedTimestamp; + } + @Override protected void writeAdditionalFields(StreamOutput out) throws IOException { out.writeString(restoreUUID); @@ -353,6 +399,10 @@ protected void writeAdditionalFields(StreamOutput out) throws IOException { out.writeBoolean(remoteStoreIndexShallowCopy); out.writeOptionalString(sourceRemoteStoreRepository); } + if (out.getVersion().onOrAfter(CURRENT)) { + out.writeOptionalString(sourceRemoteTranslogRepository); + out.writeLong(pinnedTimestamp); + } } @Override @@ -369,7 +419,8 @@ public void addAdditionalFields(XContentBuilder builder, ToXContent.Params param .field("restoreUUID", restoreUUID) .field("isSearchableSnapshot", isSearchableSnapshot) .field("remoteStoreIndexShallowCopy", remoteStoreIndexShallowCopy) - .field("sourceRemoteStoreRepository", sourceRemoteStoreRepository); + .field("sourceRemoteStoreRepository", sourceRemoteStoreRepository) + .field("sourceRemoteTranslogRepository", sourceRemoteTranslogRepository); } @Override @@ -394,8 +445,11 @@ public boolean equals(Object o) { && isSearchableSnapshot == that.isSearchableSnapshot && remoteStoreIndexShallowCopy == that.remoteStoreIndexShallowCopy && sourceRemoteStoreRepository != null - ? sourceRemoteStoreRepository.equals(that.sourceRemoteStoreRepository) - : that.sourceRemoteStoreRepository == null; + ? sourceRemoteStoreRepository.equals(that.sourceRemoteStoreRepository) + : that.sourceRemoteStoreRepository == null && sourceRemoteTranslogRepository != null + ? sourceRemoteTranslogRepository.equals(that.sourceRemoteTranslogRepository) + : that.sourceRemoteTranslogRepository == null && pinnedTimestamp == that.pinnedTimestamp; + } @Override @@ -407,10 +461,11 @@ public int hashCode() { version, isSearchableSnapshot, remoteStoreIndexShallowCopy, - sourceRemoteStoreRepository + sourceRemoteStoreRepository, + sourceRemoteTranslogRepository, + pinnedTimestamp ); } - } /** diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 354ecd72b4b08..6e12e4ed3da1a 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -150,6 +150,7 @@ import org.opensearch.index.recovery.RecoveryStats; import org.opensearch.index.refresh.RefreshStats; import org.opensearch.index.remote.RemoteSegmentStats; +import org.opensearch.index.remote.RemoteStorePathStrategy; import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; import org.opensearch.index.search.stats.SearchStats; import org.opensearch.index.search.stats.ShardSearchStats; @@ -2479,6 +2480,10 @@ private void loadGlobalCheckpointToReplicationTracker() throws IOException { * Operations from the translog will be replayed to bring lucene up to date. **/ public void openEngineAndRecoverFromTranslog() throws IOException { + openEngineAndRecoverFromTranslog(true); + } + + public void openEngineAndRecoverFromTranslog(boolean syncFromRemote) throws IOException { recoveryState.validateCurrentStage(RecoveryState.Stage.INDEX); maybeCheckIndex(); recoveryState.setStage(RecoveryState.Stage.TRANSLOG); @@ -2499,7 +2504,16 @@ public void openEngineAndRecoverFromTranslog() throws IOException { loadGlobalCheckpointToReplicationTracker(); } - innerOpenEngineAndTranslog(replicationTracker); + if (isSnapshotV2Restore()) { + translogConfig.setDownloadRemoteTranslogOnInit(false); + } + + innerOpenEngineAndTranslog(replicationTracker, syncFromRemote); + + if (isSnapshotV2Restore()) { + translogConfig.setDownloadRemoteTranslogOnInit(true); + } + getEngine().translogManager() .recoverFromTranslog(translogRecoveryRunner, getEngine().getProcessedLocalCheckpoint(), Long.MAX_VALUE); } @@ -2561,7 +2575,7 @@ private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier, b if (shardRouting.primary()) { if (syncFromRemote) { syncRemoteTranslogAndUpdateGlobalCheckpoint(); - } else { + } else if (isSnapshotV2Restore() == false) { // we will enter this block when we do not want to recover from remote translog. // currently only during snapshot restore, we are coming into this block. // here, as while initiliazing remote translog we cannot skip downloading translog files, @@ -2607,6 +2621,11 @@ private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier, b recoveryState.validateCurrentStage(RecoveryState.Stage.TRANSLOG); } + private boolean isSnapshotV2Restore() { + return routingEntry().recoverySource().getType() == RecoverySource.Type.SNAPSHOT + && ((SnapshotRecoverySource) routingEntry().recoverySource()).pinnedTimestamp() > 0; + } + private boolean assertSequenceNumbersInCommit() throws IOException { final Map userData = fetchUserData(); assert userData.containsKey(SequenceNumbers.LOCAL_CHECKPOINT_KEY) : "commit point doesn't contains a local checkpoint"; @@ -2892,14 +2911,26 @@ public void restoreFromSnapshotAndRemoteStore( assert recoveryState.getRecoverySource().getType() == RecoverySource.Type.SNAPSHOT : "invalid recovery type: " + recoveryState.getRecoverySource(); StoreRecovery storeRecovery = new StoreRecovery(shardId, logger); - storeRecovery.recoverFromSnapshotAndRemoteStore( - this, - repository, - repositoriesService, - listener, - remoteStoreSettings.getSegmentsPathFixedPrefix(), - threadPool - ); + SnapshotRecoverySource recoverySource = (SnapshotRecoverySource) recoveryState().getRecoverySource(); + if (recoverySource.pinnedTimestamp() != 0) { + storeRecovery.recoverShallowSnapshotV2( + this, + repository, + repositoriesService, + listener, + remoteStoreSettings.getSegmentsPathFixedPrefix(), + threadPool + ); + } else { + storeRecovery.recoverFromSnapshotAndRemoteStore( + this, + repository, + repositoriesService, + listener, + remoteStoreSettings.getSegmentsPathFixedPrefix(), + threadPool + ); + } } catch (Exception e) { listener.onFailure(e); } @@ -5007,16 +5038,33 @@ public void syncTranslogFilesFromRemoteTranslog() throws IOException { TranslogFactory translogFactory = translogFactorySupplier.apply(indexSettings, shardRouting); assert translogFactory instanceof RemoteBlobStoreInternalTranslogFactory; Repository repository = ((RemoteBlobStoreInternalTranslogFactory) translogFactory).getRepository(); + syncTranslogFilesFromGivenRemoteTranslog( + repository, + shardId, + indexSettings.getRemoteStorePathStrategy(), + indexSettings().isTranslogMetadataEnabled(), + 0 + ); + } + + public void syncTranslogFilesFromGivenRemoteTranslog( + Repository repository, + ShardId shardId, + RemoteStorePathStrategy remoteStorePathStrategy, + boolean isTranslogMetadataEnabled, + long timestamp + ) throws IOException { RemoteFsTranslog.download( repository, shardId, getThreadPool(), shardPath().resolveTranslog(), - indexSettings.getRemoteStorePathStrategy(), + remoteStorePathStrategy, remoteStoreSettings, logger, shouldSeedRemoteStore(), - indexSettings().isTranslogMetadataEnabled() + isTranslogMetadataEnabled, + timestamp ); } @@ -5105,15 +5153,13 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, final Runn * Downloads segments from given remote segment store for a specific commit. * @param overrideLocal flag to override local segment files with those in remote store * @param sourceRemoteDirectory RemoteSegmentDirectory Instance from which we need to sync segments - * @param primaryTerm Primary Term for shard at the time of commit operation for which we are syncing segments - * @param commitGeneration commit generation at the time of commit operation for which we are syncing segments * @throws IOException if exception occurs while reading segments from remote store */ public void syncSegmentsFromGivenRemoteSegmentStore( boolean overrideLocal, RemoteSegmentStoreDirectory sourceRemoteDirectory, - long primaryTerm, - long commitGeneration + RemoteSegmentMetadata remoteSegmentMetadata, + boolean pinnedTimestamp ) throws IOException { logger.trace("Downloading segments from given remote segment store"); RemoteSegmentStoreDirectory remoteDirectory = null; @@ -5149,12 +5195,29 @@ public void syncSegmentsFromGivenRemoteSegmentStore( overrideLocal, () -> {} ); - if (segmentsNFile != null) { + if (pinnedTimestamp) { + final SegmentInfos infosSnapshot = store.buildSegmentInfos( + remoteSegmentMetadata.getSegmentInfosBytes(), + remoteSegmentMetadata.getGeneration() + ); + long processedLocalCheckpoint = Long.parseLong(infosSnapshot.getUserData().get(LOCAL_CHECKPOINT_KEY)); + // delete any other commits, we want to start the engine only from a new commit made with the downloaded infos bytes. + // Extra segments will be wiped on engine open. + for (String file : List.of(store.directory().listAll())) { + if (file.startsWith(IndexFileNames.SEGMENTS)) { + store.deleteQuiet(file); + } + } + assert Arrays.stream(store.directory().listAll()).filter(f -> f.startsWith(IndexFileNames.SEGMENTS)).findAny().isEmpty() + : "There should not be any segments file in the dir"; + store.commitSegmentInfos(infosSnapshot, processedLocalCheckpoint, processedLocalCheckpoint); + } else if (segmentsNFile != null) { try ( ChecksumIndexInput indexInput = new BufferedChecksumIndexInput( storeDirectory.openInput(segmentsNFile, IOContext.DEFAULT) ) ) { + long commitGeneration = SegmentInfos.generationFromSegmentsFileName(segmentsNFile); SegmentInfos infosSnapshot = SegmentInfos.readCommit(store.directory(), indexInput, commitGeneration); long processedLocalCheckpoint = Long.parseLong(infosSnapshot.getUserData().get(LOCAL_CHECKPOINT_KEY)); if (remoteStore != null) { diff --git a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java index 8649da936d81c..6933e4e161dd1 100644 --- a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java @@ -58,12 +58,15 @@ import org.opensearch.index.engine.Engine; import org.opensearch.index.engine.EngineException; import org.opensearch.index.mapper.MapperService; +import org.opensearch.index.remote.RemoteStorePathStrategy; +import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.snapshots.IndexShardRestoreFailedException; import org.opensearch.index.snapshots.blobstore.RemoteStoreShardShallowCopySnapshot; import org.opensearch.index.store.RemoteSegmentStoreDirectory; import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; import org.opensearch.index.store.Store; +import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; import org.opensearch.index.translog.Checkpoint; import org.opensearch.index.translog.Translog; import org.opensearch.index.translog.TranslogHeader; @@ -72,6 +75,7 @@ import org.opensearch.repositories.IndexId; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; +import org.opensearch.repositories.RepositoryData; import org.opensearch.threadpool.ThreadPool; import java.io.IOException; @@ -407,14 +411,14 @@ void recoverFromSnapshotAndRemoteStore( shardId, shallowCopyShardMetadata.getRemoteStorePathStrategy() ); - sourceRemoteDirectory.initializeToSpecificCommit( + RemoteSegmentMetadata remoteSegmentMetadata = sourceRemoteDirectory.initializeToSpecificCommit( primaryTerm, commitGeneration, recoverySource.snapshot().getSnapshotId().getUUID() ); - indexShard.syncSegmentsFromGivenRemoteSegmentStore(true, sourceRemoteDirectory, primaryTerm, commitGeneration); + indexShard.syncSegmentsFromGivenRemoteSegmentStore(true, sourceRemoteDirectory, remoteSegmentMetadata, false); final Store store = indexShard.store(); - if (indexShard.indexSettings.isRemoteTranslogStoreEnabled() == false) { + if (indexShard.indexSettings.isRemoteStoreEnabled() == false) { bootstrap(indexShard, store); } else { bootstrapForSnapshot(indexShard, store); @@ -443,6 +447,98 @@ void recoverFromSnapshotAndRemoteStore( } } + void recoverShallowSnapshotV2( + final IndexShard indexShard, + Repository repository, + RepositoriesService repositoriesService, + ActionListener listener, + String segmentsPathFixedPrefix, + ThreadPool threadPool + ) { + try { + if (canRecover(indexShard)) { + indexShard.preRecovery(); + RecoverySource.Type recoveryType = indexShard.recoveryState().getRecoverySource().getType(); + assert recoveryType == RecoverySource.Type.SNAPSHOT : "expected snapshot recovery type: " + recoveryType; + SnapshotRecoverySource recoverySource = (SnapshotRecoverySource) indexShard.recoveryState().getRecoverySource(); + indexShard.prepareForIndexRecovery(); + + assert recoverySource.pinnedTimestamp() != 0; + final StepListener repositoryDataListener = new StepListener<>(); + repository.getRepositoryData(repositoryDataListener); + repositoryDataListener.whenComplete(repositoryData -> { + IndexId indexId = repositoryData.resolveIndexId(recoverySource.index().getName()); + IndexMetadata prevIndexMetadata = repository.getSnapshotIndexMetaData( + repositoryData, + recoverySource.snapshot().getSnapshotId(), + indexId + ); + RemoteSegmentStoreDirectoryFactory directoryFactory = new RemoteSegmentStoreDirectoryFactory( + () -> repositoriesService, + threadPool, + segmentsPathFixedPrefix + ); + String remoteSegmentStoreRepository = ((SnapshotRecoverySource) indexShard.recoveryState().getRecoverySource()) + .sourceRemoteStoreRepository(); + if (remoteSegmentStoreRepository == null) { + remoteSegmentStoreRepository = IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING.get( + prevIndexMetadata.getSettings() + ); + } + RemoteStorePathStrategy remoteStorePathStrategy = RemoteStoreUtils.determineRemoteStorePathStrategy(prevIndexMetadata); + RemoteSegmentStoreDirectory sourceRemoteDirectory = (RemoteSegmentStoreDirectory) directoryFactory.newDirectory( + remoteSegmentStoreRepository, + prevIndexMetadata.getIndexUUID(), + shardId, + remoteStorePathStrategy + ); + RemoteSegmentMetadata remoteSegmentMetadata = sourceRemoteDirectory.initializeToSpecificTimestamp( + recoverySource.pinnedTimestamp() + ); + + String remoteTranslogRepository = ((SnapshotRecoverySource) indexShard.recoveryState().getRecoverySource()) + .sourceRemoteTranslogRepository(); + if (remoteTranslogRepository == null) { + remoteTranslogRepository = IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.get( + prevIndexMetadata.getSettings() + ); + } + + indexShard.syncSegmentsFromGivenRemoteSegmentStore(true, sourceRemoteDirectory, remoteSegmentMetadata, true); + indexShard.syncTranslogFilesFromGivenRemoteTranslog( + repositoriesService.repository(remoteTranslogRepository), + new ShardId(prevIndexMetadata.getIndex(), shardId.id()), + remoteStorePathStrategy, + RemoteStoreUtils.determineTranslogMetadataEnabled(prevIndexMetadata), + recoverySource.pinnedTimestamp() + ); + + assert indexShard.shardRouting.primary() : "only primary shards can recover from store"; + writeEmptyRetentionLeasesFile(indexShard); + indexShard.recoveryState().getIndex().setFileDetailsComplete(); + indexShard.openEngineAndRecoverFromTranslog(false); + indexShard.getEngine().fillSeqNoGaps(indexShard.getPendingPrimaryTerm()); + indexShard.finalizeRecovery(); + if (indexShard.isRemoteTranslogEnabled() && indexShard.shardRouting.primary()) { + indexShard.waitForRemoteStoreSync(); + } + indexShard.postRecovery("post recovery from remote_store"); + SegmentInfos committedSegmentInfos = indexShard.store().readLastCommittedSegmentsInfo(); + try { + indexShard.getEngine() + .translogManager() + .setMinSeqNoToKeep(Long.parseLong(committedSegmentInfos.getUserData().get(SequenceNumbers.MAX_SEQ_NO)) + 1); + } catch (IllegalArgumentException e) { + logger.warn("MinSeqNoToKeep is already past the maxSeqNo from commited segment infos"); + } + listener.onResponse(true); + }, listener::onFailure); + } + } catch (Exception e) { + listener.onFailure(e); + } + } + private boolean canRecover(IndexShard indexShard) { if (indexShard.state() == IndexShardState.CLOSED) { // got closed on us, just ignore this recovery diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java index 66eea2a51b36a..16ebac0a44139 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java @@ -123,36 +123,6 @@ public RemoteFsTranslog( BooleanSupplier startedPrimarySupplier, RemoteTranslogTransferTracker remoteTranslogTransferTracker, RemoteStoreSettings remoteStoreSettings - ) throws IOException { - this( - config, - translogUUID, - deletionPolicy, - globalCheckpointSupplier, - primaryTermSupplier, - persistedSequenceNumberConsumer, - blobStoreRepository, - threadPool, - startedPrimarySupplier, - remoteTranslogTransferTracker, - remoteStoreSettings, - 0 - ); - } - - public RemoteFsTranslog( - TranslogConfig config, - String translogUUID, - TranslogDeletionPolicy deletionPolicy, - LongSupplier globalCheckpointSupplier, - LongSupplier primaryTermSupplier, - LongConsumer persistedSequenceNumberConsumer, - BlobStoreRepository blobStoreRepository, - ThreadPool threadPool, - BooleanSupplier startedPrimarySupplier, - RemoteTranslogTransferTracker remoteTranslogTransferTracker, - RemoteStoreSettings remoteStoreSettings, - long timestamp ) throws IOException { super(config, translogUUID, deletionPolicy, globalCheckpointSupplier, primaryTermSupplier, persistedSequenceNumberConsumer); logger = Loggers.getLogger(getClass(), shardId); @@ -173,7 +143,9 @@ public RemoteFsTranslog( isTranslogMetadataEnabled ); try { - download(translogTransferManager, location, logger, config.shouldSeedRemote(), timestamp); + if (config.downloadRemoteTranslogOnInit()) { + download(translogTransferManager, location, logger, config.shouldSeedRemote(), 0); + } Checkpoint checkpoint = readCheckpoint(location); logger.info("Downloaded data from remote translog till maxSeqNo = {}", checkpoint.maxSeqNo); this.readers.addAll(recoverFromFiles(checkpoint)); @@ -182,6 +154,9 @@ public RemoteFsTranslog( logger.error(errorMsg); throw new IllegalStateException(errorMsg); } + if (config.downloadRemoteTranslogOnInit() == false) { + translogTransferManager.populateFileTrackerWithLocalState(this.readers); + } boolean success = false; current = null; try { @@ -214,31 +189,6 @@ RemoteTranslogTransferTracker getRemoteTranslogTracker() { return remoteTranslogTransferTracker; } - public static void download( - Repository repository, - ShardId shardId, - ThreadPool threadPool, - Path location, - RemoteStorePathStrategy pathStrategy, - RemoteStoreSettings remoteStoreSettings, - Logger logger, - boolean seedRemote, - boolean isTranslogMetadataEnabled - ) throws IOException { - download( - repository, - shardId, - threadPool, - location, - pathStrategy, - remoteStoreSettings, - logger, - seedRemote, - isTranslogMetadataEnabled, - 0 - ); - } - public static void download( Repository repository, ShardId shardId, diff --git a/server/src/main/java/org/opensearch/index/translog/TranslogConfig.java b/server/src/main/java/org/opensearch/index/translog/TranslogConfig.java index f720f041b287c..52e20d9838fca 100644 --- a/server/src/main/java/org/opensearch/index/translog/TranslogConfig.java +++ b/server/src/main/java/org/opensearch/index/translog/TranslogConfig.java @@ -60,6 +60,7 @@ public final class TranslogConfig { private final ByteSizeValue bufferSize; private final String nodeId; private final boolean seedRemote; + private boolean downloadRemoteTranslogOnInit = true; /** * Creates a new TranslogConfig instance @@ -140,4 +141,12 @@ public String getNodeId() { public boolean shouldSeedRemote() { return seedRemote; } + + public boolean downloadRemoteTranslogOnInit() { + return downloadRemoteTranslogOnInit; + } + + public void setDownloadRemoteTranslogOnInit(boolean downloadRemoteTranslogOnInit) { + this.downloadRemoteTranslogOnInit = downloadRemoteTranslogOnInit; + } } diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java index 527fb0123d800..56a9aa6447dec 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java @@ -28,6 +28,7 @@ import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.translog.Translog; +import org.opensearch.index.translog.TranslogReader; import org.opensearch.index.translog.transfer.listener.TranslogTransferListener; import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.threadpool.ThreadPool; @@ -712,4 +713,23 @@ public void onFailure(Exception e) { public int getMaxRemoteTranslogReadersSettings() { return this.remoteStoreSettings.getMaxRemoteTranslogReaders(); } + + public void populateFileTrackerWithLocalState(List readers) { + if (readers == null) { + return; + } + for (TranslogReader reader : readers) { + long generation = reader.getGeneration(); + String tlogFilename = Translog.getFilename(generation); + fileTransferTracker.add(tlogFilename, true); + if (isTranslogMetadataEnabled) { + String ckpFilename = Translog.getCommitCheckpointFileName(generation); + fileTransferTracker.add(ckpFilename, true); + } + } + } + + protected FileTransferTracker getFileTransferTracker() { + return fileTransferTracker; + } } diff --git a/server/src/main/java/org/opensearch/repositories/Repository.java b/server/src/main/java/org/opensearch/repositories/Repository.java index 136e9fb26c3dc..4338c42e20b4b 100644 --- a/server/src/main/java/org/opensearch/repositories/Repository.java +++ b/server/src/main/java/org/opensearch/repositories/Repository.java @@ -418,6 +418,18 @@ default RemoteStoreShardShallowCopySnapshot getRemoteStoreShallowCopyShardMetada */ IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, IndexId indexId, ShardId shardId); + /** + * Retrieve shard snapshot status for the stored snapshot + * + * @param snapshotInfo snapshot info + * @param indexId the snapshotted index id for the shard to get status for + * @param shardId shard id + * @return snapshot status + */ + default IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotInfo snapshotInfo, IndexId indexId, ShardId shardId) { + return getShardSnapshotStatus(snapshotInfo.snapshotId(), indexId, shardId); + } + /** * Update the repository with the incoming cluster state. This method is invoked from {@link RepositoriesService#applyClusterState} and * thus the same semantics as with {@link org.opensearch.cluster.ClusterStateApplier#applyClusterState} apply for the 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 0a877a9ccea4b..d193ab3c14154 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -831,7 +831,7 @@ boolean getPrefixModeVerification() { * maintains single lazy instance of {@link BlobContainer} */ protected BlobContainer blobContainer() { - assertSnapshotOrGenericThread(); + // assertSnapshotOrGenericThread(); BlobContainer blobContainer = this.blobContainer.get(); if (blobContainer == null) { @@ -3749,6 +3749,11 @@ public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, In return snapshot.getIndexShardSnapshotStatus(); } + public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotInfo snapshotInfo, IndexId indexId, ShardId shardId) { + IndexShardSnapshot snapshot = loadShardSnapshot(shardContainer(indexId, shardId), snapshotInfo); + return snapshot.getIndexShardSnapshotStatus(); + } + @Override public void verify(String seed, DiscoveryNode localNode) { if (isSystemRepository == false) { @@ -3957,6 +3962,38 @@ public IndexShardSnapshot loadShardSnapshot(BlobContainer shardContainer, Snapsh } } + public IndexShardSnapshot loadShardSnapshot(BlobContainer shardContainer, SnapshotInfo snapshotInfo) { + try { + SnapshotId snapshotId = snapshotInfo.snapshotId(); + if (snapshotInfo.getPinnedTimestamp() != 0) { + return () -> IndexShardSnapshotStatus.newDone(0L, 0L, 0, 0, 0, 0, "1"); + } else if (snapshotInfo.isRemoteStoreIndexShallowCopyEnabled()) { + if (shardContainer.blobExists(REMOTE_STORE_SHARD_SHALLOW_COPY_SNAPSHOT_FORMAT.blobName(snapshotId.getUUID()))) { + return REMOTE_STORE_SHARD_SHALLOW_COPY_SNAPSHOT_FORMAT.read( + shardContainer, + snapshotId.getUUID(), + namedXContentRegistry + ); + } else { + throw new SnapshotMissingException(metadata.name(), snapshotId.getName()); + } + } else { + if (shardContainer.blobExists(INDEX_SHARD_SNAPSHOT_FORMAT.blobName(snapshotId.getUUID()))) { + return INDEX_SHARD_SNAPSHOT_FORMAT.read(shardContainer, snapshotId.getUUID(), namedXContentRegistry); + } else { + throw new SnapshotMissingException(metadata.name(), snapshotId.getName()); + } + } + } catch (IOException ex) { + throw new SnapshotException( + metadata.name(), + snapshotInfo.snapshotId(), + "failed to read shard snapshot file for [" + shardContainer.path() + ']', + ex + ); + } + } + /** * Loads all available snapshots in the repository using the given {@code generation} or falling back to trying to determine it from * the given list of blobs in the shard container. diff --git a/server/src/main/java/org/opensearch/snapshots/InternalSnapshotsInfoService.java b/server/src/main/java/org/opensearch/snapshots/InternalSnapshotsInfoService.java index 797a58f3b0d9b..e300f845e6f58 100644 --- a/server/src/main/java/org/opensearch/snapshots/InternalSnapshotsInfoService.java +++ b/server/src/main/java/org/opensearch/snapshots/InternalSnapshotsInfoService.java @@ -238,14 +238,18 @@ protected void doRun() throws Exception { final Repository repository = repositories.repository(snapshotShard.snapshot.getRepository()); logger.debug("fetching snapshot shard size for {}", snapshotShard); - final long snapshotShardSize = repository.getShardSnapshotStatus( - snapshotShard.snapshot().getSnapshotId(), - snapshotShard.index(), - snapshotShard.shardId() - ).asCopy().getTotalSize(); + long snapshotShardSize; + if (snapshotShard.pinnedTimestamp > 0) { + snapshotShardSize = 0; + } else { + snapshotShardSize = repository.getShardSnapshotStatus( + snapshotShard.snapshot().getSnapshotId(), + snapshotShard.index(), + snapshotShard.shardId() + ).asCopy().getTotalSize(); + } logger.debug("snapshot shard size for {}: {} bytes", snapshotShard, snapshotShardSize); - boolean updated = false; synchronized (mutex) { removed = unknownSnapshotShards.remove(snapshotShard); @@ -354,7 +358,8 @@ private static Set listOfSnapshotShards(final ClusterState state) final SnapshotShard snapshotShard = new SnapshotShard( snapshotRecoverySource.snapshot(), snapshotRecoverySource.index(), - shardRouting.shardId() + shardRouting.shardId(), + snapshotRecoverySource.pinnedTimestamp() ); snapshotShards.add(snapshotShard); } @@ -374,10 +379,17 @@ public static class SnapshotShard { private final IndexId index; private final ShardId shardId; + private long pinnedTimestamp; + public SnapshotShard(Snapshot snapshot, IndexId index, ShardId shardId) { + this(snapshot, index, shardId, 0L); + } + + public SnapshotShard(Snapshot snapshot, IndexId index, ShardId shardId, long pinnedTimestamp) { this.snapshot = snapshot; this.index = index; this.shardId = shardId; + this.pinnedTimestamp = pinnedTimestamp; } public Snapshot snapshot() { diff --git a/server/src/main/java/org/opensearch/snapshots/RestoreService.java b/server/src/main/java/org/opensearch/snapshots/RestoreService.java index 174cad50b00e2..79a70d835f773 100644 --- a/server/src/main/java/org/opensearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/opensearch/snapshots/RestoreService.java @@ -428,7 +428,9 @@ public ClusterState execute(ClusterState currentState) { snapshotIndexId, isSearchableSnapshot, isRemoteStoreShallowCopy, - request.getSourceRemoteStoreRepository() + request.getSourceRemoteStoreRepository(), + request.getSourceRemoteTranslogRepository(), + snapshotInfo.getPinnedTimestamp() ); final Version minIndexCompatibilityVersion; if (isSearchableSnapshot && isSearchableSnapshotsExtendedCompatibilityEnabled()) { @@ -551,7 +553,7 @@ public ClusterState execute(ClusterState currentState) { for (int shard = 0; shard < snapshotIndexMetadata.getNumberOfShards(); shard++) { if (isRemoteSnapshot) { IndexShardSnapshotStatus.Copy shardStatus = repository.getShardSnapshotStatus( - snapshotInfo.snapshotId(), + snapshotInfo, snapshotIndexId, new ShardId(metadata.index(index).getIndex(), shard) ).asCopy(); diff --git a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java index 377e4e99e9964..96794a83ef762 100644 --- a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java @@ -2899,7 +2899,7 @@ public void testSyncSegmentsFromGivenRemoteSegmentStore() throws IOException { target = reinitShard(target, routing); DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); target.markAsRecovering("from snapshot", new RecoveryState(routing, localNode, null)); - target.syncSegmentsFromGivenRemoteSegmentStore(false, tempRemoteSegmentDirectory, primaryTerm, commitGeneration); + target.syncSegmentsFromGivenRemoteSegmentStore(false, tempRemoteSegmentDirectory, null, false); RemoteSegmentStoreDirectory remoteStoreDirectory = ((RemoteSegmentStoreDirectory) ((FilterDirectory) ((FilterDirectory) target .remoteStore() .directory()).getDelegate()).getDelegate()); diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java index 2426a14c0c93b..8605043ddd5b5 100644 --- a/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java +++ b/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java @@ -26,6 +26,7 @@ import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.translog.Translog; +import org.opensearch.index.translog.TranslogReader; import org.opensearch.index.translog.transfer.FileSnapshot.CheckpointFileSnapshot; import org.opensearch.index.translog.transfer.FileSnapshot.TransferFileSnapshot; import org.opensearch.index.translog.transfer.FileSnapshot.TranslogFileSnapshot; @@ -874,4 +875,53 @@ public void testReadMetadataForGivenTimestampException() throws IOException { assertThrows(IOException.class, () -> translogTransferManager.readMetadata(3000L)); assertNoDownloadStats(true); } + + public void testPopulateFileTrackerWithLocalStateNoReaders() { + translogTransferManager.populateFileTrackerWithLocalState(null); + assertTrue(translogTransferManager.getFileTransferTracker().allUploaded().isEmpty()); + + translogTransferManager.populateFileTrackerWithLocalState(List.of()); + assertTrue(translogTransferManager.getFileTransferTracker().allUploaded().isEmpty()); + } + + public void testPopulateFileTrackerWithLocalState() { + TranslogReader reader1 = mock(TranslogReader.class); + when(reader1.getGeneration()).thenReturn(12L); + TranslogReader reader2 = mock(TranslogReader.class); + when(reader2.getGeneration()).thenReturn(23L); + TranslogReader reader3 = mock(TranslogReader.class); + when(reader3.getGeneration()).thenReturn(34L); + TranslogReader reader4 = mock(TranslogReader.class); + when(reader4.getGeneration()).thenReturn(45L); + + translogTransferManager.populateFileTrackerWithLocalState(List.of(reader1, reader2, reader3, reader4)); + assertEquals( + Set.of("translog-12.tlog", "translog-23.tlog", "translog-34.tlog", "translog-45.tlog"), + translogTransferManager.getFileTransferTracker().allUploaded() + ); + } + + public void testPopulateFileTrackerWithLocalStateNoCkpAsMetadata() { + TranslogTransferManager translogTransferManager = new TranslogTransferManager( + shardId, + transferService, + remoteBaseTransferPath.add(TRANSLOG.getName()), + remoteBaseTransferPath.add(METADATA.getName()), + tracker, + remoteTranslogTransferTracker, + DefaultRemoteStoreSettings.INSTANCE, + true + ); + + TranslogReader reader1 = mock(TranslogReader.class); + when(reader1.getGeneration()).thenReturn(12L); + TranslogReader reader2 = mock(TranslogReader.class); + when(reader2.getGeneration()).thenReturn(23L); + + translogTransferManager.populateFileTrackerWithLocalState(List.of(reader1, reader2)); + assertEquals( + Set.of("translog-12.tlog", "translog-12.ckp", "translog-23.tlog", "translog-23.ckp"), + translogTransferManager.getFileTransferTracker().allUploaded() + ); + } } From 451606535752a73be80d5203ae417e7d57fc5cef Mon Sep 17 00:00:00 2001 From: rishavz_sagar Date: Tue, 3 Sep 2024 20:11:02 +0530 Subject: [PATCH 2/7] Limiting node attribute serialisation in DiscoveryNode (#15341) Signed-off-by: RS146BIJAY --- .../core/common/io/stream/StreamOutput.java | 6 ++++- .../ClusterAllocationExplanation.java | 2 +- .../support/nodes/BaseNodeResponse.java | 2 +- .../org/opensearch/cluster/ClusterState.java | 14 ++++++++-- .../opensearch/cluster/coordination/Join.java | 4 +-- .../cluster/coordination/JoinRequest.java | 2 +- .../coordination/StartJoinRequest.java | 2 +- .../cluster/node/DiscoveryNode.java | 27 ++++++++++++++++--- .../cluster/node/DiscoveryNodes.java | 10 ++++++- .../AbstractAllocationDecision.java | 2 +- .../allocation/NodeAllocationResult.java | 2 +- .../remote/model/RemoteDiscoveryNodes.java | 7 ++++- .../recovery/PeerRecoveryTargetService.java | 1 + .../recovery/StartRecoveryRequest.java | 4 +-- .../ChecksumWritableBlobStoreFormat.java | 7 ++++- .../transport/TransportService.java | 2 +- ...oteClusterStateAttributesManagerTests.java | 7 ++++- .../RemoteClusterStateServiceTests.java | 7 ++++- .../model/RemoteDiscoveryNodesTests.java | 2 +- .../ChecksumWritableBlobStoreFormatTests.java | 8 +++++- 20 files changed, 93 insertions(+), 25 deletions(-) diff --git a/libs/core/src/main/java/org/opensearch/core/common/io/stream/StreamOutput.java b/libs/core/src/main/java/org/opensearch/core/common/io/stream/StreamOutput.java index 867fc99cc6a56..cac8ddc8f94e3 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/io/stream/StreamOutput.java +++ b/libs/core/src/main/java/org/opensearch/core/common/io/stream/StreamOutput.java @@ -969,9 +969,13 @@ public void writeOptionalArray(@Nullable T[] array) throws } public void writeOptionalWriteable(@Nullable Writeable writeable) throws IOException { + writeOptionalWriteable((out, writable) -> writable.writeTo(out), writeable); + } + + public void writeOptionalWriteable(final Writer writer, @Nullable T writeable) throws IOException { if (writeable != null) { writeBoolean(true); - writeable.writeTo(this); + writer.write(this, writeable); } else { writeBoolean(false); } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/allocation/ClusterAllocationExplanation.java b/server/src/main/java/org/opensearch/action/admin/cluster/allocation/ClusterAllocationExplanation.java index 3c8f07613561d..70a223d60069a 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/allocation/ClusterAllocationExplanation.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/allocation/ClusterAllocationExplanation.java @@ -95,7 +95,7 @@ public ClusterAllocationExplanation(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { shardRouting.writeTo(out); - out.writeOptionalWriteable(currentNode); + out.writeOptionalWriteable((stream, node) -> node.writeToWithAttribute(stream), currentNode); out.writeOptionalWriteable(relocationTargetNode); out.writeOptionalWriteable(clusterInfo); shardAllocationDecision.writeTo(out); diff --git a/server/src/main/java/org/opensearch/action/support/nodes/BaseNodeResponse.java b/server/src/main/java/org/opensearch/action/support/nodes/BaseNodeResponse.java index 8a4e12567b515..7e10b583ef21a 100644 --- a/server/src/main/java/org/opensearch/action/support/nodes/BaseNodeResponse.java +++ b/server/src/main/java/org/opensearch/action/support/nodes/BaseNodeResponse.java @@ -67,6 +67,6 @@ public DiscoveryNode getNode() { @Override public void writeTo(StreamOutput out) throws IOException { - node.writeTo(out); + node.writeToWithAttribute(out); } } diff --git a/server/src/main/java/org/opensearch/cluster/ClusterState.java b/server/src/main/java/org/opensearch/cluster/ClusterState.java index 054233305726d..1e4fd2dfffe0f 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterState.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterState.java @@ -781,7 +781,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(stateUUID); metadata.writeTo(out); routingTable.writeTo(out); - nodes.writeTo(out); + nodes.writeToWithAttribute(out); blocks.writeTo(out); // filter out custom states not supported by the other node int numberOfCustoms = 0; @@ -887,13 +887,23 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(toUuid); out.writeLong(toVersion); routingTable.writeTo(out); - nodes.writeTo(out); + nodesWriteToWithAttributes(nodes, out); metadata.writeTo(out); blocks.writeTo(out); customs.writeTo(out); out.writeVInt(minimumClusterManagerNodesOnPublishingClusterManager); } + private void nodesWriteToWithAttributes(Diff nodes, StreamOutput out) throws IOException { + DiscoveryNodes part = nodes.apply(null); + if (part != null) { + out.writeBoolean(true); + part.writeToWithAttribute(out); + } else { + out.writeBoolean(false); + } + } + @Override public ClusterState apply(ClusterState state) { Builder builder = new Builder(clusterName); diff --git a/server/src/main/java/org/opensearch/cluster/coordination/Join.java b/server/src/main/java/org/opensearch/cluster/coordination/Join.java index 58fa85992ebc8..ce1a234998690 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/Join.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/Join.java @@ -78,8 +78,8 @@ public Join(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { - sourceNode.writeTo(out); - targetNode.writeTo(out); + sourceNode.writeToWithAttribute(out); + targetNode.writeToWithAttribute(out); out.writeLong(term); out.writeLong(lastAcceptedTerm); out.writeLong(lastAcceptedVersion); diff --git a/server/src/main/java/org/opensearch/cluster/coordination/JoinRequest.java b/server/src/main/java/org/opensearch/cluster/coordination/JoinRequest.java index 04f87d16ee400..1447838a41502 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/JoinRequest.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/JoinRequest.java @@ -84,7 +84,7 @@ public JoinRequest(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - sourceNode.writeTo(out); + sourceNode.writeToWithAttribute(out); out.writeLong(minimumTerm); out.writeOptionalWriteable(optionalJoin.orElse(null)); } diff --git a/server/src/main/java/org/opensearch/cluster/coordination/StartJoinRequest.java b/server/src/main/java/org/opensearch/cluster/coordination/StartJoinRequest.java index de58eb721b28f..287418aaf378e 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/StartJoinRequest.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/StartJoinRequest.java @@ -64,7 +64,7 @@ public StartJoinRequest(StreamInput input) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - sourceNode.writeTo(out); + sourceNode.writeToWithAttribute(out); out.writeLong(term); } diff --git a/server/src/main/java/org/opensearch/cluster/node/DiscoveryNode.java b/server/src/main/java/org/opensearch/cluster/node/DiscoveryNode.java index 50086d458fab2..8ac978a72b21b 100644 --- a/server/src/main/java/org/opensearch/cluster/node/DiscoveryNode.java +++ b/server/src/main/java/org/opensearch/cluster/node/DiscoveryNode.java @@ -330,6 +330,7 @@ public DiscoveryNode(StreamInput in) throws IOException { for (int i = 0; i < size; i++) { this.attributes.put(in.readString(), in.readString()); } + int rolesSize = in.readVInt(); final Set roles = new HashSet<>(rolesSize); for (int i = 0; i < rolesSize; i++) { @@ -359,13 +360,31 @@ public DiscoveryNode(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { + if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + writeToUtil(out, false); + } else { + writeToUtil(out, true); + } + + } + + public void writeToWithAttribute(StreamOutput out) throws IOException { + writeToUtil(out, true); + } + + public void writeToUtil(StreamOutput out, boolean includeAllAttributes) throws IOException { writeNodeDetails(out); - out.writeVInt(attributes.size()); - for (Map.Entry entry : attributes.entrySet()) { - out.writeString(entry.getKey()); - out.writeString(entry.getValue()); + if (includeAllAttributes) { + out.writeVInt(attributes.size()); + for (Map.Entry entry : attributes.entrySet()) { + out.writeString(entry.getKey()); + out.writeString(entry.getValue()); + } + } else { + out.writeVInt(0); } + writeRolesAndVersion(out); } diff --git a/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodes.java b/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodes.java index ba81012e386d7..52d830aafda38 100644 --- a/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodes.java +++ b/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodes.java @@ -690,10 +690,18 @@ public String shortSummary() { @Override public void writeTo(StreamOutput out) throws IOException { + writeToUtil((output, value) -> value.writeTo(output), out); + } + + public void writeToWithAttribute(StreamOutput out) throws IOException { + writeToUtil((output, value) -> value.writeToWithAttribute(output), out); + } + + public void writeToUtil(final Writer writer, StreamOutput out) throws IOException { writeClusterManager(out); out.writeVInt(nodes.size()); for (DiscoveryNode node : this) { - node.writeTo(out); + writer.write(out, node); } } diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/AbstractAllocationDecision.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/AbstractAllocationDecision.java index 59a39b358cb70..614e9f49c8726 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/AbstractAllocationDecision.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/AbstractAllocationDecision.java @@ -107,7 +107,7 @@ public List getNodeDecisions() { @Override public void writeTo(StreamOutput out) throws IOException { - out.writeOptionalWriteable(targetNode); + out.writeOptionalWriteable((stream, node) -> node.writeToWithAttribute(stream), targetNode); if (nodeDecisions != null) { out.writeBoolean(true); out.writeList(nodeDecisions); diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/NodeAllocationResult.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/NodeAllocationResult.java index 4163a5fd4c16f..6b805ca91fa58 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/NodeAllocationResult.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/NodeAllocationResult.java @@ -104,7 +104,7 @@ public NodeAllocationResult(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { - node.writeTo(out); + node.writeToWithAttribute(out); out.writeOptionalWriteable(shardStoreInfo); out.writeOptionalWriteable(canAllocateDecision); nodeDecision.writeTo(out); diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteDiscoveryNodes.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteDiscoveryNodes.java index 446207a767009..829036c6d122b 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteDiscoveryNodes.java +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteDiscoveryNodes.java @@ -88,7 +88,12 @@ public UploadedMetadata getUploadedMetadata() { @Override public InputStream serialize() throws IOException { - return DISCOVERY_NODES_FORMAT.serialize(discoveryNodes, generateBlobFileName(), getCompressor()).streamInput(); + return DISCOVERY_NODES_FORMAT.serialize( + (out, discoveryNode) -> discoveryNode.writeToWithAttribute(out), + discoveryNodes, + generateBlobFileName(), + getCompressor() + ).streamInput(); } @Override diff --git a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java index 6279a8ec3646c..d746aaa2a0783 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java @@ -294,6 +294,7 @@ private void doRecovery(final long recoveryId, final StartRecoveryRequest preExi logger.debug("{} reestablishing recovery from {}", startRequest.shardId(), startRequest.sourceNode()); } } + transportService.sendRequest( startRequest.sourceNode(), actionName, diff --git a/server/src/main/java/org/opensearch/indices/recovery/StartRecoveryRequest.java b/server/src/main/java/org/opensearch/indices/recovery/StartRecoveryRequest.java index 1df0d3861f686..7309712314acd 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/StartRecoveryRequest.java +++ b/server/src/main/java/org/opensearch/indices/recovery/StartRecoveryRequest.java @@ -144,8 +144,8 @@ public void writeTo(StreamOutput out) throws IOException { out.writeLong(recoveryId); shardId.writeTo(out); out.writeString(targetAllocationId); - sourceNode.writeTo(out); - targetNode.writeTo(out); + sourceNode.writeToWithAttribute(out); + targetNode.writeToWithAttribute(out); metadataSnapshot.writeTo(out); out.writeBoolean(primaryRelocation); out.writeLong(startingSeqNo); diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormat.java b/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormat.java index 0add86ab88a16..88672995f4fd6 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormat.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormat.java @@ -28,6 +28,7 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.core.common.io.stream.Writeable.Writer; import org.opensearch.core.compress.Compressor; import org.opensearch.core.compress.CompressorRegistry; import org.opensearch.gateway.CorruptStateException; @@ -56,6 +57,10 @@ public ChecksumWritableBlobStoreFormat(String codec, CheckedFunction unSerializedObj.writeTo(out), obj, blobName, compressor); + } + + public BytesReference serialize(final Writer writer, T obj, final String blobName, final Compressor compressor) throws IOException { try (BytesStreamOutput outputStream = new BytesStreamOutput()) { try ( OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput( @@ -76,7 +81,7 @@ public void close() throws IOException { }; StreamOutput stream = new OutputStreamStreamOutput(compressor.threadLocalOutputStream(indexOutputOutputStream));) { // TODO The stream version should be configurable stream.setVersion(Version.CURRENT); - obj.writeTo(stream); + writer.write(stream, obj); } CodecUtil.writeFooter(indexOutput); } diff --git a/server/src/main/java/org/opensearch/transport/TransportService.java b/server/src/main/java/org/opensearch/transport/TransportService.java index d08b28730d417..fff6d82b23c7e 100644 --- a/server/src/main/java/org/opensearch/transport/TransportService.java +++ b/server/src/main/java/org/opensearch/transport/TransportService.java @@ -752,7 +752,7 @@ public HandshakeResponse(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { - out.writeOptionalWriteable(discoveryNode); + out.writeOptionalWriteable((stream, node) -> node.writeToWithAttribute(stream), discoveryNode); clusterName.writeTo(out); out.writeVersion(version); } diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java index 4ef459e6657a1..67b1528466a9e 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java @@ -140,7 +140,12 @@ public void testGetAsyncReadRunnable_DiscoveryNodes() throws IOException, Interr DiscoveryNodes discoveryNodes = getDiscoveryNodes(); String fileName = randomAlphaOfLength(10); when(blobStoreTransferService.downloadBlob(anyIterable(), anyString())).thenReturn( - DISCOVERY_NODES_FORMAT.serialize(discoveryNodes, fileName, compressor).streamInput() + DISCOVERY_NODES_FORMAT.serialize( + (out, discoveryNode) -> discoveryNode.writeToWithAttribute(out), + discoveryNodes, + fileName, + compressor + ).streamInput() ); RemoteDiscoveryNodes remoteObjForDownload = new RemoteDiscoveryNodes(fileName, "cluster-uuid", compressor); CountDownLatch latch = new CountDownLatch(1); diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java index 8dd35e59c43c9..ccadbfb3fbebc 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java @@ -1220,7 +1220,12 @@ public void testGetClusterStateUsingDiff() throws IOException { diffManifestBuilder.discoveryNodesUpdated(true); manifestBuilder.discoveryNodesMetadata(new UploadedMetadataAttribute(DISCOVERY_NODES, DISCOVERY_NODES_FILENAME)); when(blobContainer.readBlob(DISCOVERY_NODES_FILENAME)).thenAnswer(invocationOnMock -> { - BytesReference bytes = DISCOVERY_NODES_FORMAT.serialize(nodesBuilder.build(), DISCOVERY_NODES_FILENAME, compressor); + BytesReference bytes = DISCOVERY_NODES_FORMAT.serialize( + (out, nodes) -> nodes.writeToWithAttribute(out), + nodesBuilder.build(), + DISCOVERY_NODES_FILENAME, + compressor + ); return new ByteArrayInputStream(bytes.streamInput().readAllBytes()); }); } diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteDiscoveryNodesTests.java b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteDiscoveryNodesTests.java index f1bced2bdf855..1b988ee1f37ec 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteDiscoveryNodesTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteDiscoveryNodesTests.java @@ -143,7 +143,7 @@ public void testSerDe() throws IOException { public void testExceptionDuringSerialization() throws IOException { DiscoveryNodes nodes = mock(DiscoveryNodes.class); RemoteDiscoveryNodes remoteObjectForUpload = new RemoteDiscoveryNodes(nodes, METADATA_VERSION, clusterUUID, compressor); - doThrow(new IOException("mock-exception")).when(nodes).writeTo(any()); + doThrow(new IOException("mock-exception")).when(nodes).writeToWithAttribute(any()); IOException iea = assertThrows(IOException.class, remoteObjectForUpload::serialize); } diff --git a/server/src/test/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormatTests.java b/server/src/test/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormatTests.java index 536df880b2597..c4e53c1eea138 100644 --- a/server/src/test/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormatTests.java +++ b/server/src/test/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormatTests.java @@ -35,7 +35,12 @@ public class ChecksumWritableBlobStoreFormatTests extends OpenSearchTestCase { public void testSerDe() throws IOException { IndexMetadata indexMetadata = getIndexMetadata(); - BytesReference bytesReference = clusterBlocksFormat.serialize(indexMetadata, TEST_BLOB_FILE_NAME, CompressorRegistry.none()); + BytesReference bytesReference = clusterBlocksFormat.serialize( + (out, metadata) -> metadata.writeTo(out), + indexMetadata, + TEST_BLOB_FILE_NAME, + CompressorRegistry.none() + ); IndexMetadata readIndexMetadata = clusterBlocksFormat.deserialize(TEST_BLOB_FILE_NAME, bytesReference); assertThat(readIndexMetadata, is(indexMetadata)); } @@ -43,6 +48,7 @@ public void testSerDe() throws IOException { public void testSerDeForCompressed() throws IOException { IndexMetadata indexMetadata = getIndexMetadata(); BytesReference bytesReference = clusterBlocksFormat.serialize( + (out, metadata) -> metadata.writeTo(out), indexMetadata, TEST_BLOB_FILE_NAME, CompressorRegistry.getCompressor(DeflateCompressor.NAME) From 12f6493ec4f489bf1932b2ce757c28ebafbb0ca8 Mon Sep 17 00:00:00 2001 From: Sarthak Aggarwal Date: Tue, 3 Sep 2024 21:30:39 +0530 Subject: [PATCH 3/7] interfaces fixes for star tree search (#15603) Signed-off-by: Sarthak Aggarwal --- .../fileformats/meta/StarTreeMetadata.java | 2 ++ .../node/FixedLengthStarTreeNode.java | 1 - .../startree/index/StarTreeValues.java | 19 ++++++++--- .../datacube/startree/StarTreeTestUtils.java | 8 ++--- .../builder/AbstractStarTreeBuilderTests.java | 33 +++++++++++++++---- .../node/FixedLengthStarTreeNodeTests.java | 2 +- 6 files changed, 46 insertions(+), 19 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadata.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadata.java index 2e20c9b9e0737..e7bb32282ece7 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadata.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadata.java @@ -11,6 +11,7 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.store.IndexInput; +import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.index.compositeindex.CompositeIndexMetadata; import org.opensearch.index.compositeindex.datacube.Metric; import org.opensearch.index.compositeindex.datacube.MetricStat; @@ -30,6 +31,7 @@ * * @opensearch.experimental */ +@ExperimentalApi public class StarTreeMetadata extends CompositeIndexMetadata { private static final Logger logger = LogManager.getLogger(StarTreeMetadata.class); diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNode.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNode.java index 72c08d96d26ff..0ae704afa53b9 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNode.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNode.java @@ -201,7 +201,6 @@ public StarTreeNode getChildForDimensionValue(Long dimensionValue) throws IOExce StarTreeNode resultStarTreeNode = null; if (null != dimensionValue) { resultStarTreeNode = binarySearchChild(dimensionValue); - assert null != resultStarTreeNode; } return resultStarTreeNode; } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/index/StarTreeValues.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/index/StarTreeValues.java index 8181a69f49bd4..7d5f5ba02b9f8 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/index/StarTreeValues.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/index/StarTreeValues.java @@ -9,7 +9,6 @@ package org.opensearch.index.compositeindex.datacube.startree.index; import org.apache.lucene.codecs.DocValuesProducer; -import org.apache.lucene.index.DocValues; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SortedNumericDocValues; @@ -74,6 +73,11 @@ public class StarTreeValues implements CompositeIndexValues { */ private final Map attributes; + /** + * A metadata for the star-tree + */ + private final StarTreeMetadata starTreeMetadata; + /** * Constructs a new StarTreeValues object with the provided parameters. * Used for testing. @@ -89,13 +93,15 @@ public StarTreeValues( StarTreeNode root, Map> dimensionDocValuesIteratorMap, Map> metricDocValuesIteratorMap, - Map attributes + Map attributes, + StarTreeMetadata compositeIndexMetadata ) { this.starTreeField = starTreeField; this.root = root; this.dimensionDocValuesIteratorMap = dimensionDocValuesIteratorMap; this.metricDocValuesIteratorMap = metricDocValuesIteratorMap; this.attributes = attributes; + this.starTreeMetadata = compositeIndexMetadata; } /** @@ -114,7 +120,7 @@ public StarTreeValues( SegmentReadState readState ) throws IOException { - StarTreeMetadata starTreeMetadata = (StarTreeMetadata) compositeIndexMetadata; + starTreeMetadata = (StarTreeMetadata) compositeIndexMetadata; // build skip star node dimensions Set skipStarNodeCreationInDims = starTreeMetadata.getSkipStarNodeCreationInDims(); @@ -244,7 +250,7 @@ public DocIdSetIterator getDimensionDocIdSetIterator(String dimension) { return dimensionDocValuesIteratorMap.get(dimension).get(); } - return DocValues.emptySortedNumeric(); + throw new IllegalArgumentException("dimension [" + dimension + "] does not exist in the segment."); } /** @@ -259,7 +265,10 @@ public DocIdSetIterator getMetricDocIdSetIterator(String fullyQualifiedMetricNam return metricDocValuesIteratorMap.get(fullyQualifiedMetricName).get(); } - return DocValues.emptySortedNumeric(); + throw new IllegalArgumentException("metric [" + fullyQualifiedMetricName + "] does not exist in the segment."); } + public int getStarTreeDocumentCount() { + return starTreeMetadata.getStarTreeDocCount(); + } } diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/StarTreeTestUtils.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/StarTreeTestUtils.java index 53c9e05ccb490..7b5672529dc8d 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/StarTreeTestUtils.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/StarTreeTestUtils.java @@ -37,7 +37,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; public class StarTreeTestUtils { @@ -208,11 +208,7 @@ public static void validateFileFormats( if (child.getStarTreeNodeType() != StarTreeNodeType.NULL.getValue()) { assertNotNull(starTreeNode.getChildForDimensionValue(child.getDimensionValue())); } else { - StarTreeNode finalStarTreeNode = starTreeNode; - assertThrows( - AssertionError.class, - () -> finalStarTreeNode.getChildForDimensionValue(child.getDimensionValue()) - ); + assertNull(starTreeNode.getChildForDimensionValue(child.getDimensionValue())); } assertStarTreeNode(child, resultChildNode); assertNotEquals(child.getStarTreeNodeType(), StarTreeNodeType.STAR.getValue()); diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java index 79d1b0e4b9785..6e23e5eacaecd 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java @@ -1894,6 +1894,7 @@ private void validateStarTreeFileFormats( IndexInput metaIn = readState.directory.openInput(metaFileName, IOContext.DEFAULT); StarTreeValues starTreeValues = new StarTreeValues(expectedStarTreeMetadata, dataIn, compositeDocValuesProducer, readState); + assertEquals(expectedStarTreeMetadata.getStarTreeDocCount(), starTreeValues.getStarTreeDocumentCount()); List starTreeNumericTypes = new ArrayList<>(); builder.metricAggregatorInfos.forEach( metricAggregatorInfo -> starTreeNumericTypes.add(metricAggregatorInfo.getValueAggregators().getAggregatedValueType()) @@ -2527,7 +2528,8 @@ private StarTreeValues getStarTreeValues( null, dimDocIdSetIterators, metricDocIdSetIterators, - Map.of(CompositeIndexConstants.SEGMENT_DOCS_COUNT, number) + Map.of(CompositeIndexConstants.SEGMENT_DOCS_COUNT, number), + null ); return starTreeValues; } @@ -3678,7 +3680,14 @@ private StarTreeValues getStarTreeValues( ); // metricDocIdSetIterators.put("field2", () -> m1sndv); // metricDocIdSetIterators.put("_doc_count", () -> m2sndv); - StarTreeValues starTreeValues = new StarTreeValues(sf, null, dimDocIdSetIterators, metricDocIdSetIterators, getAttributes(500)); + StarTreeValues starTreeValues = new StarTreeValues( + sf, + null, + dimDocIdSetIterators, + metricDocIdSetIterators, + getAttributes(500), + null + ); return starTreeValues; } @@ -4088,14 +4097,20 @@ public void testMergeFlow() throws IOException { () -> d4sndv ); - Map> metricDocIdSetIterators = Map.of("field2", () -> m1sndv, "_doc_count", () -> m2sndv); + Map> metricDocIdSetIterators = Map.of( + "sf_field2_sum_metric", + () -> m1sndv, + "sf__doc_count_doc_count_metric", + () -> m2sndv + ); StarTreeValues starTreeValues = new StarTreeValues( compositeField, null, dimDocIdSetIterators, metricDocIdSetIterators, - getAttributes(1000) + getAttributes(1000), + null ); SortedNumericDocValues f2d1sndv = getSortedNumericMock(dimList1, docsWithField1); @@ -4115,13 +4130,19 @@ public void testMergeFlow() throws IOException { () -> f2d4sndv ); - Map> f2metricDocIdSetIterators = Map.of("field2", () -> f2m1sndv, "_doc_count", () -> f2m2sndv); + Map> f2metricDocIdSetIterators = Map.of( + "sf_field2_sum_metric", + () -> f2m1sndv, + "sf__doc_count_doc_count_metric", + () -> f2m2sndv + ); StarTreeValues starTreeValues2 = new StarTreeValues( compositeField, null, f2dimDocIdSetIterators, f2metricDocIdSetIterators, - getAttributes(1000) + getAttributes(1000), + null ); this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNodeTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNodeTests.java index 9657f867422a0..08815d5ef55f5 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNodeTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNodeTests.java @@ -191,7 +191,7 @@ public void testGetChildForNullNode() throws IOException { public void testGetChildForInvalidDimensionValue() throws IOException { long invalidDimensionValue = Long.MAX_VALUE; - assertThrows(AssertionError.class, () -> starTreeNode.getChildForDimensionValue(invalidDimensionValue)); + assertNull(starTreeNode.getChildForDimensionValue(invalidDimensionValue)); } public void testOnlyRootNodePresent() throws IOException { From c62626d25430bea94123d6176caef56ae94deb3f Mon Sep 17 00:00:00 2001 From: Anshu Agarwal Date: Tue, 3 Sep 2024 22:09:17 +0530 Subject: [PATCH 4/7] SnapshotInfo pinned timestamp version change to V_2_17_0 (#15616) Signed-off-by: Anshu Agarwal Co-authored-by: Anshu Agarwal --- .../src/main/java/org/opensearch/snapshots/SnapshotInfo.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java index bc33928acf3da..08433dc6f2e0b 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java @@ -473,7 +473,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_3_0_0)) { + if (in.getVersion().onOrAfter(Version.V_2_17_0)) { pinnedTimestamp = in.readVLong(); } } @@ -940,7 +940,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_3_0_0)) { + if (out.getVersion().onOrAfter(Version.V_2_17_0)) { out.writeVLong(pinnedTimestamp); } } From deeb2ded71f05652900ca2915623fd3d0c81c97f Mon Sep 17 00:00:00 2001 From: Sachin Kale Date: Tue, 3 Sep 2024 23:26:35 +0530 Subject: [PATCH 5/7] Abstract out RemoteFsTranslogWithPinnedTimestamps (#15579) * Abstract out RemoteFsTranslog with pinned timestamps Signed-off-by: Sachin Kale --- ...emoteBlobStoreInternalTranslogFactory.java | 42 +- .../RemoteFsTimestampAwareTranslog.java | 377 ++++++++++++++++++ .../index/translog/RemoteFsTranslog.java | 309 ++------------ ... RemoteFsTimestampAwareTranslogTests.java} | 75 +++- .../index/translog/RemoteFsTranslogTests.java | 38 +- 5 files changed, 531 insertions(+), 310 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslog.java rename server/src/test/java/org/opensearch/index/translog/{RemoteFsTranslogWithPinnedTimestampTests.java => RemoteFsTimestampAwareTranslogTests.java} (93%) diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteBlobStoreInternalTranslogFactory.java b/server/src/main/java/org/opensearch/index/translog/RemoteBlobStoreInternalTranslogFactory.java index 4599aa32325c1..5dc2ad076d21c 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteBlobStoreInternalTranslogFactory.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteBlobStoreInternalTranslogFactory.java @@ -69,19 +69,35 @@ public Translog newTranslog( assert repository instanceof BlobStoreRepository : "repository should be instance of BlobStoreRepository"; BlobStoreRepository blobStoreRepository = ((BlobStoreRepository) repository); - return new RemoteFsTranslog( - config, - translogUUID, - deletionPolicy, - globalCheckpointSupplier, - primaryTermSupplier, - persistedSequenceNumberConsumer, - blobStoreRepository, - threadPool, - startedPrimarySupplier, - remoteTranslogTransferTracker, - remoteStoreSettings - ); + if (RemoteStoreSettings.isPinnedTimestampsEnabled()) { + return new RemoteFsTimestampAwareTranslog( + config, + translogUUID, + deletionPolicy, + globalCheckpointSupplier, + primaryTermSupplier, + persistedSequenceNumberConsumer, + blobStoreRepository, + threadPool, + startedPrimarySupplier, + remoteTranslogTransferTracker, + remoteStoreSettings + ); + } else { + return new RemoteFsTranslog( + config, + translogUUID, + deletionPolicy, + globalCheckpointSupplier, + primaryTermSupplier, + persistedSequenceNumberConsumer, + blobStoreRepository, + threadPool, + startedPrimarySupplier, + remoteTranslogTransferTracker, + remoteStoreSettings + ); + } } public Repository getRepository() { diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslog.java b/server/src/main/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslog.java new file mode 100644 index 0000000000000..0b134b3bddbec --- /dev/null +++ b/server/src/main/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslog.java @@ -0,0 +1,377 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.translog; + +import org.apache.logging.log4j.Logger; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.blobstore.BlobMetadata; +import org.opensearch.common.collect.Tuple; +import org.opensearch.common.logging.Loggers; +import org.opensearch.core.action.ActionListener; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.index.remote.RemoteTranslogTransferTracker; +import org.opensearch.index.translog.transfer.TranslogTransferManager; +import org.opensearch.index.translog.transfer.TranslogTransferMetadata; +import org.opensearch.indices.RemoteStoreSettings; +import org.opensearch.node.remotestore.RemoteStorePinnedTimestampService; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.threadpool.ThreadPool; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.TreeSet; +import java.util.function.BooleanSupplier; +import java.util.function.LongConsumer; +import java.util.function.LongSupplier; +import java.util.stream.Collectors; +import java.util.stream.LongStream; + +import static org.opensearch.index.translog.transfer.TranslogTransferMetadata.METADATA_SEPARATOR; + +/** + * A Translog implementation which syncs local FS with a remote store + * The current impl uploads translog , ckp and metadata to remote store + * for every sync, post syncing to disk. Post that, a new generation is + * created. This implementation is also aware of pinned timestamp and makes + * sure data against pinned timestamp is retained. + * + * @opensearch.internal + */ +public class RemoteFsTimestampAwareTranslog extends RemoteFsTranslog { + + private final Logger logger; + private final Map metadataFilePinnedTimestampMap; + // For metadata files, with no min generation in the name, we cache generation data to avoid multiple reads. + private final Map> oldFormatMetadataFileGenerationMap; + + public RemoteFsTimestampAwareTranslog( + TranslogConfig config, + String translogUUID, + TranslogDeletionPolicy deletionPolicy, + LongSupplier globalCheckpointSupplier, + LongSupplier primaryTermSupplier, + LongConsumer persistedSequenceNumberConsumer, + BlobStoreRepository blobStoreRepository, + ThreadPool threadPool, + BooleanSupplier startedPrimarySupplier, + RemoteTranslogTransferTracker remoteTranslogTransferTracker, + RemoteStoreSettings remoteStoreSettings + ) throws IOException { + super( + config, + translogUUID, + deletionPolicy, + globalCheckpointSupplier, + primaryTermSupplier, + persistedSequenceNumberConsumer, + blobStoreRepository, + threadPool, + startedPrimarySupplier, + remoteTranslogTransferTracker, + remoteStoreSettings + ); + logger = Loggers.getLogger(getClass(), shardId); + this.metadataFilePinnedTimestampMap = new HashMap<>(); + this.oldFormatMetadataFileGenerationMap = new HashMap<>(); + } + + @Override + protected void onDelete() { + ClusterService.assertClusterOrClusterManagerStateThread(); + // clean up all remote translog files + try { + trimUnreferencedReaders(true, false); + } catch (IOException e) { + logger.error("Exception while deleting translog files from remote store", e); + } + } + + @Override + public void trimUnreferencedReaders() throws IOException { + trimUnreferencedReaders(false, true); + } + + // Visible for testing + protected void trimUnreferencedReaders(boolean indexDeleted, boolean trimLocal) throws IOException { + if (trimLocal) { + // clean up local translog files and updates readers + super.trimUnreferencedReaders(); + } + + // Update file tracker to reflect local translog state + Optional minLiveGeneration = readers.stream().map(BaseTranslogReader::getGeneration).min(Long::compareTo); + if (minLiveGeneration.isPresent()) { + List staleFilesInTracker = new ArrayList<>(); + for (String file : fileTransferTracker.allUploaded()) { + if (file.endsWith(TRANSLOG_FILE_SUFFIX)) { + long generation = Translog.parseIdFromFileName(file); + if (generation < minLiveGeneration.get()) { + staleFilesInTracker.add(file); + staleFilesInTracker.add(Translog.getCommitCheckpointFileName(generation)); + } + } + fileTransferTracker.delete(staleFilesInTracker); + } + } + + // This is to ensure that after the permits are acquired during primary relocation, there are no further modification on remote + // store. + if (startedPrimarySupplier.getAsBoolean() == false || pauseSync.get()) { + return; + } + + // This is to fail fast and avoid listing md files un-necessarily. + if (indexDeleted == false && RemoteStoreUtils.isPinnedTimestampStateStale()) { + logger.warn("Skipping remote segment store garbage collection as last fetch of pinned timestamp is stale"); + return; + } + + // Since remote generation deletion is async, this ensures that only one generation deletion happens at a time. + // Remote generations involves 2 async operations - 1) Delete translog generation files 2) Delete metadata files + // We try to acquire 2 permits and if we can not, we return from here itself. + if (remoteGenerationDeletionPermits.tryAcquire(REMOTE_DELETION_PERMITS) == false) { + return; + } + + ActionListener> listMetadataFilesListener = new ActionListener<>() { + @Override + public void onResponse(List blobMetadata) { + List metadataFiles = blobMetadata.stream().map(BlobMetadata::name).collect(Collectors.toList()); + + try { + if (metadataFiles.size() <= 1) { + logger.debug("No stale translog metadata files found"); + remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); + return; + } + + // Check last fetch status of pinned timestamps. If stale, return. + if (indexDeleted == false && RemoteStoreUtils.isPinnedTimestampStateStale()) { + logger.warn("Skipping remote segment store garbage collection as last fetch of pinned timestamp is stale"); + remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); + return; + } + + List metadataFilesToBeDeleted = getMetadataFilesToBeDeleted(metadataFiles); + + // If index is not deleted, make sure to keep latest metadata file + if (indexDeleted == false) { + metadataFilesToBeDeleted.remove(metadataFiles.get(0)); + } + + if (metadataFilesToBeDeleted.isEmpty()) { + logger.debug("No metadata files to delete"); + remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); + return; + } + + logger.debug(() -> "metadataFilesToBeDeleted = " + metadataFilesToBeDeleted); + // For all the files that we are keeping, fetch min and max generations + List metadataFilesNotToBeDeleted = new ArrayList<>(metadataFiles); + metadataFilesNotToBeDeleted.removeAll(metadataFilesToBeDeleted); + + logger.debug(() -> "metadataFilesNotToBeDeleted = " + metadataFilesNotToBeDeleted); + Set generationsToBeDeleted = getGenerationsToBeDeleted( + metadataFilesNotToBeDeleted, + metadataFilesToBeDeleted, + indexDeleted + ); + + logger.debug(() -> "generationsToBeDeleted = " + generationsToBeDeleted); + if (generationsToBeDeleted.isEmpty() == false) { + // Delete stale generations + translogTransferManager.deleteGenerationAsync( + primaryTermSupplier.getAsLong(), + generationsToBeDeleted, + remoteGenerationDeletionPermits::release + ); + + // Delete stale metadata files + translogTransferManager.deleteMetadataFilesAsync( + metadataFilesToBeDeleted, + remoteGenerationDeletionPermits::release + ); + + // Update cache to keep only those metadata files that are not getting deleted + oldFormatMetadataFileGenerationMap.keySet().retainAll(metadataFilesNotToBeDeleted); + + // Delete stale primary terms + deleteStaleRemotePrimaryTerms(metadataFiles); + } else { + remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); + } + } catch (Exception e) { + remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); + } + } + + @Override + public void onFailure(Exception e) { + remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); + logger.error("Exception while listing translog metadata files", e); + } + }; + translogTransferManager.listTranslogMetadataFilesAsync(listMetadataFilesListener); + } + + // Visible for testing + protected Set getGenerationsToBeDeleted( + List metadataFilesNotToBeDeleted, + List metadataFilesToBeDeleted, + boolean indexDeleted + ) throws IOException { + long maxGenerationToBeDeleted = Long.MAX_VALUE; + + if (indexDeleted == false) { + maxGenerationToBeDeleted = minRemoteGenReferenced - 1 - indexSettings().getRemoteTranslogExtraKeep(); + } + + Set generationsFromMetadataFilesToBeDeleted = new HashSet<>(); + for (String mdFile : metadataFilesToBeDeleted) { + Tuple minMaxGen = getMinMaxTranslogGenerationFromMetadataFile(mdFile, translogTransferManager); + generationsFromMetadataFilesToBeDeleted.addAll( + LongStream.rangeClosed(minMaxGen.v1(), minMaxGen.v2()).boxed().collect(Collectors.toList()) + ); + } + + Map> metadataFileNotToBeDeletedGenerationMap = getGenerationForMetadataFiles(metadataFilesNotToBeDeleted); + TreeSet> pinnedGenerations = getOrderedPinnedMetadataGenerations(metadataFileNotToBeDeletedGenerationMap); + Set generationsToBeDeleted = new HashSet<>(); + for (long generation : generationsFromMetadataFilesToBeDeleted) { + // Check if the generation is not referred by metadata file matching pinned timestamps + if (generation <= maxGenerationToBeDeleted && isGenerationPinned(generation, pinnedGenerations) == false) { + generationsToBeDeleted.add(generation); + } + } + return generationsToBeDeleted; + } + + // Visible for testing + protected List getMetadataFilesToBeDeleted(List metadataFiles) { + Tuple> pinnedTimestampsState = RemoteStorePinnedTimestampService.getPinnedTimestamps(); + + // Keep files since last successful run of scheduler + List metadataFilesToBeDeleted = RemoteStoreUtils.filterOutMetadataFilesBasedOnAge( + metadataFiles, + file -> RemoteStoreUtils.invertLong(file.split(METADATA_SEPARATOR)[3]), + pinnedTimestampsState.v1() + ); + + logger.trace( + "metadataFiles.size = {}, metadataFilesToBeDeleted based on age based filtering = {}", + metadataFiles.size(), + metadataFilesToBeDeleted.size() + ); + + // Get md files matching pinned timestamps + Set implicitLockedFiles = RemoteStoreUtils.getPinnedTimestampLockedFiles( + metadataFilesToBeDeleted, + pinnedTimestampsState.v2(), + metadataFilePinnedTimestampMap, + file -> RemoteStoreUtils.invertLong(file.split(METADATA_SEPARATOR)[3]), + TranslogTransferMetadata::getNodeIdByPrimaryTermAndGen + ); + + // Filter out metadata files matching pinned timestamps + metadataFilesToBeDeleted.removeAll(implicitLockedFiles); + + logger.trace( + "implicitLockedFiles.size = {}, metadataFilesToBeDeleted based on pinned timestamp filtering = {}", + implicitLockedFiles.size(), + metadataFilesToBeDeleted.size() + ); + + return metadataFilesToBeDeleted; + } + + // Visible for testing + protected boolean isGenerationPinned(long generation, TreeSet> pinnedGenerations) { + Tuple ceilingGenerationRange = pinnedGenerations.ceiling(new Tuple<>(generation, generation)); + if (ceilingGenerationRange != null && generation >= ceilingGenerationRange.v1() && generation <= ceilingGenerationRange.v2()) { + return true; + } + Tuple floorGenerationRange = pinnedGenerations.floor(new Tuple<>(generation, generation)); + if (floorGenerationRange != null && generation >= floorGenerationRange.v1() && generation <= floorGenerationRange.v2()) { + return true; + } + return false; + } + + private TreeSet> getOrderedPinnedMetadataGenerations(Map> metadataFileGenerationMap) { + TreeSet> pinnedGenerations = new TreeSet<>((o1, o2) -> { + if (Objects.equals(o1.v1(), o2.v1()) == false) { + return o1.v1().compareTo(o2.v1()); + } else { + return o1.v2().compareTo(o2.v2()); + } + }); + pinnedGenerations.addAll(metadataFileGenerationMap.values()); + return pinnedGenerations; + } + + // Visible for testing + protected Map> getGenerationForMetadataFiles(List metadataFiles) throws IOException { + Map> metadataFileGenerationMap = new HashMap<>(); + for (String metadataFile : metadataFiles) { + metadataFileGenerationMap.put(metadataFile, getMinMaxTranslogGenerationFromMetadataFile(metadataFile, translogTransferManager)); + } + return metadataFileGenerationMap; + } + + // Visible for testing + protected Tuple getMinMaxTranslogGenerationFromMetadataFile( + String metadataFile, + TranslogTransferManager translogTransferManager + ) throws IOException { + Tuple minMaxGenerationFromFileName = TranslogTransferMetadata.getMinMaxTranslogGenerationFromFilename(metadataFile); + if (minMaxGenerationFromFileName != null) { + return minMaxGenerationFromFileName; + } else { + if (oldFormatMetadataFileGenerationMap.containsKey(metadataFile)) { + return oldFormatMetadataFileGenerationMap.get(metadataFile); + } else { + TranslogTransferMetadata metadata = translogTransferManager.readMetadata(metadataFile); + Tuple minMaxGenTuple = new Tuple<>(metadata.getMinTranslogGeneration(), metadata.getGeneration()); + oldFormatMetadataFileGenerationMap.put(metadataFile, minMaxGenTuple); + return minMaxGenTuple; + } + } + } + + /** + * This method must be called only after there are valid generations to delete in trimUnreferencedReaders as it ensures + * implicitly that minimum primary term in latest translog metadata in remote store is the current primary term. + *
+ * This will also delete all stale translog metadata files from remote except the latest basis the metadata file comparator. + */ + private void deleteStaleRemotePrimaryTerms(List metadataFiles) { + // The deletion of older translog files in remote store is on best-effort basis, there is a possibility that there + // are older files that are no longer needed and should be cleaned up. In here, we delete all files that are part + // of older primary term. + if (olderPrimaryCleaned.trySet(Boolean.TRUE)) { + if (metadataFiles.isEmpty()) { + logger.trace("No metadata is uploaded yet, returning from deleteStaleRemotePrimaryTerms"); + return; + } + Optional minPrimaryTerm = metadataFiles.stream() + .map(file -> RemoteStoreUtils.invertLong(file.split(METADATA_SEPARATOR)[1])) + .min(Long::compareTo); + // First we delete all stale primary terms folders from remote store + long minimumReferencedPrimaryTerm = minPrimaryTerm.get() - 1; + translogTransferManager.deletePrimaryTermsAsync(minimumReferencedPrimaryTerm); + } + } +} diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java index 16ebac0a44139..812852d107682 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java @@ -11,19 +11,15 @@ import org.apache.logging.log4j.Logger; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.SetOnce; -import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; -import org.opensearch.common.collect.Tuple; import org.opensearch.common.lease.Releasable; import org.opensearch.common.lease.Releasables; import org.opensearch.common.logging.Loggers; import org.opensearch.common.util.concurrent.ReleasableLock; import org.opensearch.common.util.io.IOUtils; -import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.util.FileSystemUtils; import org.opensearch.index.remote.RemoteStorePathStrategy; -import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.translog.transfer.BlobStoreTransferService; @@ -34,7 +30,6 @@ import org.opensearch.index.translog.transfer.TranslogTransferMetadata; import org.opensearch.index.translog.transfer.listener.TranslogTransferListener; import org.opensearch.indices.RemoteStoreSettings; -import org.opensearch.node.remotestore.RemoteStorePinnedTimestampService; import org.opensearch.repositories.Repository; import org.opensearch.repositories.blobstore.BlobStoreRepository; import org.opensearch.threadpool.ThreadPool; @@ -44,16 +39,11 @@ import java.nio.file.Files; import java.nio.file.NoSuchFileException; import java.nio.file.Path; -import java.util.ArrayList; -import java.util.HashMap; import java.util.HashSet; -import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.Set; -import java.util.TreeSet; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -61,13 +51,10 @@ import java.util.function.BooleanSupplier; import java.util.function.LongConsumer; import java.util.function.LongSupplier; -import java.util.stream.Collectors; -import java.util.stream.LongStream; import static org.opensearch.index.remote.RemoteStoreEnums.DataCategory.TRANSLOG; import static org.opensearch.index.remote.RemoteStoreEnums.DataType.DATA; import static org.opensearch.index.remote.RemoteStoreEnums.DataType.METADATA; -import static org.opensearch.index.translog.transfer.TranslogTransferMetadata.METADATA_SEPARATOR; /** * A Translog implementation which syncs local FS with a remote store @@ -80,36 +67,31 @@ public class RemoteFsTranslog extends Translog { private final Logger logger; - private final TranslogTransferManager translogTransferManager; - // This tracker keeps track of local tranlog files that are uploaded to remote store. - // Once tlog files are deleted from local, we remove them from tracker even if the files still exist in remote translog. - private final FileTransferTracker fileTransferTracker; - private final BooleanSupplier startedPrimarySupplier; + protected final TranslogTransferManager translogTransferManager; + protected final FileTransferTracker fileTransferTracker; + protected final BooleanSupplier startedPrimarySupplier; private final RemoteTranslogTransferTracker remoteTranslogTransferTracker; private volatile long maxRemoteTranslogGenerationUploaded; private volatile long minSeqNoToKeep; // min generation referred by last uploaded translog - private volatile long minRemoteGenReferenced; + protected volatile long minRemoteGenReferenced; // clean up translog folder uploaded by previous primaries once - private final SetOnce olderPrimaryCleaned = new SetOnce<>(); + protected final SetOnce olderPrimaryCleaned = new SetOnce<>(); - private static final int REMOTE_DELETION_PERMITS = 2; + protected static final int REMOTE_DELETION_PERMITS = 2; private static final int DOWNLOAD_RETRIES = 2; // Semaphore used to allow only single remote generation to happen at a time - private final Semaphore remoteGenerationDeletionPermits = new Semaphore(REMOTE_DELETION_PERMITS); + protected final Semaphore remoteGenerationDeletionPermits = new Semaphore(REMOTE_DELETION_PERMITS); // These permits exist to allow any inflight background triggered upload. private static final int SYNC_PERMIT = 1; private final Semaphore syncPermit = new Semaphore(SYNC_PERMIT); - private final AtomicBoolean pauseSync = new AtomicBoolean(false); + protected final AtomicBoolean pauseSync = new AtomicBoolean(false); private final boolean isTranslogMetadataEnabled; - private final Map metadataFilePinnedTimestampMap; - // For metadata files, with no min generation in the name, we cache generation data to avoid multiple reads. - private final Map> oldFormatMetadataFileGenerationMap; public RemoteFsTranslog( TranslogConfig config, @@ -130,8 +112,6 @@ public RemoteFsTranslog( this.remoteTranslogTransferTracker = remoteTranslogTransferTracker; fileTransferTracker = new FileTransferTracker(shardId, remoteTranslogTransferTracker); isTranslogMetadataEnabled = indexSettings().isTranslogMetadataEnabled(); - this.metadataFilePinnedTimestampMap = new HashMap<>(); - this.oldFormatMetadataFileGenerationMap = new HashMap<>(); this.translogTransferManager = buildTranslogTransferManager( blobStoreRepository, threadPool, @@ -569,31 +549,8 @@ protected Releasable drainSync() { @Override public void trimUnreferencedReaders() throws IOException { - trimUnreferencedReaders(false, true); - } - - // Visible for testing - protected void trimUnreferencedReaders(boolean indexDeleted, boolean trimLocal) throws IOException { - if (trimLocal) { - // clean up local translog files and updates readers - super.trimUnreferencedReaders(); - } - - // Update file tracker to reflect local translog state - Optional minLiveGeneration = readers.stream().map(BaseTranslogReader::getGeneration).min(Long::compareTo); - if (minLiveGeneration.isPresent()) { - List staleFilesInTracker = new ArrayList<>(); - for (String file : fileTransferTracker.allUploaded()) { - if (file.endsWith(TRANSLOG_FILE_SUFFIX)) { - long generation = Translog.parseIdFromFileName(file); - if (generation < minLiveGeneration.get()) { - staleFilesInTracker.add(file); - staleFilesInTracker.add(Translog.getCommitCheckpointFileName(generation)); - } - } - fileTransferTracker.delete(staleFilesInTracker); - } - } + // clean up local translog files and updates readers + super.trimUnreferencedReaders(); // This is to ensure that after the permits are acquired during primary relocation, there are no further modification on remote // store. @@ -601,12 +558,6 @@ protected void trimUnreferencedReaders(boolean indexDeleted, boolean trimLocal) return; } - // This is to fail fast and avoid listing md files un-necessarily. - if (indexDeleted == false && RemoteStoreUtils.isPinnedTimestampStateStale()) { - logger.warn("Skipping remote segment store garbage collection as last fetch of pinned timestamp is stale"); - return; - } - // Since remote generation deletion is async, this ensures that only one generation deletion happens at a time. // Remote generations involves 2 async operations - 1) Delete translog generation files 2) Delete metadata files // We try to acquire 2 permits and if we can not, we return from here itself. @@ -614,209 +565,34 @@ protected void trimUnreferencedReaders(boolean indexDeleted, boolean trimLocal) return; } - ActionListener> listMetadataFilesListener = new ActionListener<>() { - @Override - public void onResponse(List blobMetadata) { - List metadataFiles = blobMetadata.stream().map(BlobMetadata::name).collect(Collectors.toList()); - - try { - if (metadataFiles.size() <= 1) { - logger.debug("No stale translog metadata files found"); - remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); - return; - } - - // Check last fetch status of pinned timestamps. If stale, return. - if (indexDeleted == false && RemoteStoreUtils.isPinnedTimestampStateStale()) { - logger.warn("Skipping remote segment store garbage collection as last fetch of pinned timestamp is stale"); - remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); - return; - } - - List metadataFilesToBeDeleted = getMetadataFilesToBeDeleted(metadataFiles); - - // If index is not deleted, make sure to keep latest metadata file - if (indexDeleted == false) { - metadataFilesToBeDeleted.remove(metadataFiles.get(0)); - } - - if (metadataFilesToBeDeleted.isEmpty()) { - logger.debug("No metadata files to delete"); - remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); - return; - } - - logger.debug("metadataFilesToBeDeleted = {}", metadataFilesToBeDeleted); - // For all the files that we are keeping, fetch min and max generations - List metadataFilesNotToBeDeleted = new ArrayList<>(metadataFiles); - metadataFilesNotToBeDeleted.removeAll(metadataFilesToBeDeleted); - - logger.debug("metadataFilesNotToBeDeleted = {}", metadataFilesNotToBeDeleted); - Set generationsToBeDeleted = getGenerationsToBeDeleted( - metadataFilesNotToBeDeleted, - metadataFilesToBeDeleted, - indexDeleted - ); - - logger.debug("generationsToBeDeleted = {}", generationsToBeDeleted); - if (generationsToBeDeleted.isEmpty() == false) { - // Delete stale generations - translogTransferManager.deleteGenerationAsync( - primaryTermSupplier.getAsLong(), - generationsToBeDeleted, - remoteGenerationDeletionPermits::release - ); - - // Delete stale metadata files - translogTransferManager.deleteMetadataFilesAsync( - metadataFilesToBeDeleted, - remoteGenerationDeletionPermits::release - ); - - // Update cache to keep only those metadata files that are not getting deleted - oldFormatMetadataFileGenerationMap.keySet().retainAll(metadataFilesNotToBeDeleted); - - // Delete stale primary terms - deleteStaleRemotePrimaryTerms(metadataFiles); - } else { - remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); - } - } catch (Exception e) { - remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); - } - } - - @Override - public void onFailure(Exception e) { - remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); - logger.error("Exception while listing translog metadata files", e); + // cleans up remote translog files not referenced in latest uploaded metadata. + // This enables us to restore translog from the metadata in case of failover or relocation. + Set generationsToDelete = new HashSet<>(); + for (long generation = minRemoteGenReferenced - 1 - indexSettings().getRemoteTranslogExtraKeep(); generation >= 0; generation--) { + if (fileTransferTracker.uploaded(Translog.getFilename(generation)) == false) { + break; } - }; - translogTransferManager.listTranslogMetadataFilesAsync(listMetadataFilesListener); - } - - // Visible for testing - protected Set getGenerationsToBeDeleted( - List metadataFilesNotToBeDeleted, - List metadataFilesToBeDeleted, - boolean indexDeleted - ) throws IOException { - long maxGenerationToBeDeleted = Long.MAX_VALUE; - - if (indexDeleted == false) { - maxGenerationToBeDeleted = minRemoteGenReferenced - 1 - indexSettings().getRemoteTranslogExtraKeep(); + generationsToDelete.add(generation); } - - Set generationsFromMetadataFilesToBeDeleted = new HashSet<>(); - for (String mdFile : metadataFilesToBeDeleted) { - Tuple minMaxGen = getMinMaxTranslogGenerationFromMetadataFile(mdFile, translogTransferManager); - generationsFromMetadataFilesToBeDeleted.addAll( - LongStream.rangeClosed(minMaxGen.v1(), minMaxGen.v2()).boxed().collect(Collectors.toList()) - ); - } - - Map> metadataFileNotToBeDeletedGenerationMap = getGenerationForMetadataFiles(metadataFilesNotToBeDeleted); - TreeSet> pinnedGenerations = getOrderedPinnedMetadataGenerations(metadataFileNotToBeDeletedGenerationMap); - Set generationsToBeDeleted = new HashSet<>(); - for (long generation : generationsFromMetadataFilesToBeDeleted) { - // Check if the generation is not referred by metadata file matching pinned timestamps - if (generation <= maxGenerationToBeDeleted && isGenerationPinned(generation, pinnedGenerations) == false) { - generationsToBeDeleted.add(generation); - } + if (generationsToDelete.isEmpty() == false) { + deleteRemoteGeneration(generationsToDelete); + translogTransferManager.deleteStaleTranslogMetadataFilesAsync(remoteGenerationDeletionPermits::release); + deleteStaleRemotePrimaryTerms(); + } else { + remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); } - return generationsToBeDeleted; } - // Visible for testing - protected List getMetadataFilesToBeDeleted(List metadataFiles) { - Tuple> pinnedTimestampsState = RemoteStorePinnedTimestampService.getPinnedTimestamps(); - - // Keep files since last successful run of scheduler - List metadataFilesToBeDeleted = RemoteStoreUtils.filterOutMetadataFilesBasedOnAge( - metadataFiles, - file -> RemoteStoreUtils.invertLong(file.split(METADATA_SEPARATOR)[3]), - pinnedTimestampsState.v1() - ); - - logger.trace( - "metadataFiles.size = {}, metadataFilesToBeDeleted based on age based filtering = {}", - metadataFiles.size(), - metadataFilesToBeDeleted.size() - ); - - // Get md files matching pinned timestamps - Set implicitLockedFiles = RemoteStoreUtils.getPinnedTimestampLockedFiles( - metadataFilesToBeDeleted, - pinnedTimestampsState.v2(), - metadataFilePinnedTimestampMap, - file -> RemoteStoreUtils.invertLong(file.split(METADATA_SEPARATOR)[3]), - TranslogTransferMetadata::getNodeIdByPrimaryTermAndGen - ); - - // Filter out metadata files matching pinned timestamps - metadataFilesToBeDeleted.removeAll(implicitLockedFiles); - - logger.trace( - "implicitLockedFiles.size = {}, metadataFilesToBeDeleted based on pinned timestamp filtering = {}", - implicitLockedFiles.size(), - metadataFilesToBeDeleted.size() + /** + * Deletes remote translog and metadata files asynchronously corresponding to the generations. + * @param generations generations to be deleted. + */ + private void deleteRemoteGeneration(Set generations) { + translogTransferManager.deleteGenerationAsync( + primaryTermSupplier.getAsLong(), + generations, + remoteGenerationDeletionPermits::release ); - - return metadataFilesToBeDeleted; - } - - // Visible for testing - protected boolean isGenerationPinned(long generation, TreeSet> pinnedGenerations) { - Tuple ceilingGenerationRange = pinnedGenerations.ceiling(new Tuple<>(generation, generation)); - if (ceilingGenerationRange != null && generation >= ceilingGenerationRange.v1() && generation <= ceilingGenerationRange.v2()) { - return true; - } - Tuple floorGenerationRange = pinnedGenerations.floor(new Tuple<>(generation, generation)); - if (floorGenerationRange != null && generation >= floorGenerationRange.v1() && generation <= floorGenerationRange.v2()) { - return true; - } - return false; - } - - private TreeSet> getOrderedPinnedMetadataGenerations(Map> metadataFileGenerationMap) { - TreeSet> pinnedGenerations = new TreeSet<>((o1, o2) -> { - if (Objects.equals(o1.v1(), o2.v1()) == false) { - return o1.v1().compareTo(o2.v1()); - } else { - return o1.v2().compareTo(o2.v2()); - } - }); - pinnedGenerations.addAll(metadataFileGenerationMap.values()); - return pinnedGenerations; - } - - // Visible for testing - protected Map> getGenerationForMetadataFiles(List metadataFiles) throws IOException { - Map> metadataFileGenerationMap = new HashMap<>(); - for (String metadataFile : metadataFiles) { - metadataFileGenerationMap.put(metadataFile, getMinMaxTranslogGenerationFromMetadataFile(metadataFile, translogTransferManager)); - } - return metadataFileGenerationMap; - } - - // Visible for testing - protected Tuple getMinMaxTranslogGenerationFromMetadataFile( - String metadataFile, - TranslogTransferManager translogTransferManager - ) throws IOException { - Tuple minMaxGenerationFromFileName = TranslogTransferMetadata.getMinMaxTranslogGenerationFromFilename(metadataFile); - if (minMaxGenerationFromFileName != null) { - return minMaxGenerationFromFileName; - } else { - if (oldFormatMetadataFileGenerationMap.containsKey(metadataFile)) { - return oldFormatMetadataFileGenerationMap.get(metadataFile); - } else { - TranslogTransferMetadata metadata = translogTransferManager.readMetadata(metadataFile); - Tuple minMaxGenTuple = new Tuple<>(metadata.getMinTranslogGeneration(), metadata.getGeneration()); - oldFormatMetadataFileGenerationMap.put(metadataFile, minMaxGenTuple); - return minMaxGenTuple; - } - } } /** @@ -825,20 +601,17 @@ protected Tuple getMinMaxTranslogGenerationFromMetadataFile( *
* This will also delete all stale translog metadata files from remote except the latest basis the metadata file comparator. */ - private void deleteStaleRemotePrimaryTerms(List metadataFiles) { + private void deleteStaleRemotePrimaryTerms() { // The deletion of older translog files in remote store is on best-effort basis, there is a possibility that there // are older files that are no longer needed and should be cleaned up. In here, we delete all files that are part // of older primary term. if (olderPrimaryCleaned.trySet(Boolean.TRUE)) { - if (metadataFiles.isEmpty()) { - logger.trace("No metadata is uploaded yet, returning from deleteStaleRemotePrimaryTerms"); + if (readers.isEmpty()) { + logger.trace("Translog reader list is empty, returning from deleteStaleRemotePrimaryTerms"); return; } - Optional minPrimaryTerm = metadataFiles.stream() - .map(file -> RemoteStoreUtils.invertLong(file.split(METADATA_SEPARATOR)[1])) - .min(Long::compareTo); // First we delete all stale primary terms folders from remote store - long minimumReferencedPrimaryTerm = minPrimaryTerm.get() - 1; + long minimumReferencedPrimaryTerm = readers.stream().map(BaseTranslogReader::getPrimaryTerm).min(Long::compare).get(); translogTransferManager.deletePrimaryTermsAsync(minimumReferencedPrimaryTerm); } } @@ -874,15 +647,7 @@ public static void cleanup( protected void onDelete() { ClusterService.assertClusterOrClusterManagerStateThread(); // clean up all remote translog files - if (RemoteStoreSettings.isPinnedTimestampsEnabled()) { - try { - trimUnreferencedReaders(true, false); - } catch (IOException e) { - logger.error("Exception while deleting translog files from remote store", e); - } - } else { - translogTransferManager.delete(); - } + translogTransferManager.delete(); } // Visible for testing diff --git a/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogWithPinnedTimestampTests.java b/server/src/test/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslogTests.java similarity index 93% rename from server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogWithPinnedTimestampTests.java rename to server/src/test/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslogTests.java index 386dde4dffc48..1f82dd9d7e641 100644 --- a/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogWithPinnedTimestampTests.java +++ b/server/src/test/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslogTests.java @@ -29,6 +29,7 @@ import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.translog.transfer.TranslogTransferManager; import org.opensearch.index.translog.transfer.TranslogTransferMetadata; +import org.opensearch.index.translog.transfer.TranslogUploadFailedException; import org.opensearch.indices.DefaultRemoteStoreSettings; import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.node.Node; @@ -42,6 +43,7 @@ import org.junit.Before; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -60,14 +62,13 @@ import static org.opensearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy; import static org.opensearch.index.translog.transfer.TranslogTransferMetadata.METADATA_SEPARATOR; import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_ENABLED; -import static org.junit.Assert.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @LuceneTestCase.SuppressFileSystems("ExtrasFS") -public class RemoteFsTranslogWithPinnedTimestampTests extends RemoteFsTranslogTests { +public class RemoteFsTimestampAwareTranslogTests extends RemoteFsTranslogTests { Runnable updatePinnedTimstampTask; BlobContainer blobContainer; @@ -125,6 +126,48 @@ public void setUp() throws Exception { remoteStorePinnedTimestampServiceSpy.start(); } + @Override + protected RemoteFsTranslog createTranslogInstance( + TranslogConfig translogConfig, + String translogUUID, + TranslogDeletionPolicy deletionPolicy + ) throws IOException { + return new RemoteFsTimestampAwareTranslog( + translogConfig, + translogUUID, + deletionPolicy, + () -> globalCheckpoint.get(), + primaryTerm::get, + getPersistedSeqNoConsumer(), + repository, + threadPool, + primaryMode::get, + new RemoteTranslogTransferTracker(shardId, 10), + DefaultRemoteStoreSettings.INSTANCE + ); + } + + @Override + public void testSyncUpAlwaysFailure() throws IOException { + int translogOperations = randomIntBetween(1, 20); + int count = 0; + fail.failAlways(); + for (int op = 0; op < translogOperations; op++) { + translog.add( + new Translog.Index(String.valueOf(op), count, primaryTerm.get(), Integer.toString(count).getBytes(StandardCharsets.UTF_8)) + ); + try { + translog.sync(); + fail("io exception expected"); + } catch (TranslogUploadFailedException e) { + assertTrue("at least one operation pending", translog.syncNeeded()); + } + } + assertTrue(translog.isOpen()); + fail.failNever(); + translog.sync(); + } + public void testGetMinMaxTranslogGenerationFromFilename() throws Exception { RemoteStoreSettings.setPinnedTimestampsLookbackInterval(TimeValue.ZERO); ArrayList ops = new ArrayList<>(); @@ -198,7 +241,7 @@ public void testIndexDeletionWithNoPinnedTimestampNoRecentMdFiles() throws Excep assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); updatePinnedTimstampTask.run(); - translog.trimUnreferencedReaders(true, false); + ((RemoteFsTimestampAwareTranslog) translog).trimUnreferencedReaders(true, false); assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); @@ -221,7 +264,7 @@ public void testIndexDeletionWithNoPinnedTimestampButRecentFiles() throws Except addToTranslogAndListAndUpload(translog, ops, new Translog.Index("4", 4, primaryTerm.get(), new byte[] { 1 })); updatePinnedTimstampTask.run(); - translog.trimUnreferencedReaders(true, false); + ((RemoteFsTimestampAwareTranslog) translog).trimUnreferencedReaders(true, false); assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); assertBusy(() -> { @@ -567,7 +610,11 @@ public void testGetGenerationsToBeDeletedEmptyMetadataFilesNotToBeDeleted() thro // 27 to 42 "metadata__9223372036438563903__9223372036854775765__9223370311919910403__31__9223372036854775780__1" ); - Set generations = translog.getGenerationsToBeDeleted(metadataFilesNotToBeDeleted, metadataFilesToBeDeleted, true); + Set generations = ((RemoteFsTimestampAwareTranslog) translog).getGenerationsToBeDeleted( + metadataFilesNotToBeDeleted, + metadataFilesToBeDeleted, + true + ); Set md1Generations = LongStream.rangeClosed(4, 7).boxed().collect(Collectors.toSet()); Set md2Generations = LongStream.rangeClosed(17, 37).boxed().collect(Collectors.toSet()); Set md3Generations = LongStream.rangeClosed(27, 42).boxed().collect(Collectors.toSet()); @@ -598,7 +645,11 @@ public void testGetGenerationsToBeDeleted() throws IOException { // 27 to 42 "metadata__9223372036438563903__9223372036854775765__9223370311919910403__31__9223372036854775780__1" ); - Set generations = translog.getGenerationsToBeDeleted(metadataFilesNotToBeDeleted, metadataFilesToBeDeleted, true); + Set generations = ((RemoteFsTimestampAwareTranslog) translog).getGenerationsToBeDeleted( + metadataFilesNotToBeDeleted, + metadataFilesToBeDeleted, + true + ); Set md1Generations = LongStream.rangeClosed(5, 7).boxed().collect(Collectors.toSet()); Set md2Generations = LongStream.rangeClosed(17, 25).boxed().collect(Collectors.toSet()); Set md3Generations = LongStream.rangeClosed(31, 41).boxed().collect(Collectors.toSet()); @@ -621,7 +672,7 @@ public void testGetMetadataFilesToBeDeletedNoExclusion() { "metadata__9223372036438563903__9223372036854775701__9223370311919910403__31__9223372036854775701__1" ); - assertEquals(metadataFiles, translog.getMetadataFilesToBeDeleted(metadataFiles)); + assertEquals(metadataFiles, ((RemoteFsTimestampAwareTranslog) translog).getMetadataFilesToBeDeleted(metadataFiles)); } public void testGetMetadataFilesToBeDeletedExclusionBasedOnAgeOnly() { @@ -637,7 +688,7 @@ public void testGetMetadataFilesToBeDeletedExclusionBasedOnAgeOnly() { "metadata__9223372036438563903__9223372036854775701__" + md3Timestamp + "__31__9223372036854775701__1" ); - List metadataFilesToBeDeleted = translog.getMetadataFilesToBeDeleted(metadataFiles); + List metadataFilesToBeDeleted = ((RemoteFsTimestampAwareTranslog) translog).getMetadataFilesToBeDeleted(metadataFiles); assertEquals(1, metadataFilesToBeDeleted.size()); assertEquals(metadataFiles.get(0), metadataFilesToBeDeleted.get(0)); } @@ -659,7 +710,7 @@ public void testGetMetadataFilesToBeDeletedExclusionBasedOnPinningOnly() throws "metadata__9223372036438563903__9223372036854775701__" + md3Timestamp + "__31__9223372036854775701__1" ); - List metadataFilesToBeDeleted = translog.getMetadataFilesToBeDeleted(metadataFiles); + List metadataFilesToBeDeleted = ((RemoteFsTimestampAwareTranslog) translog).getMetadataFilesToBeDeleted(metadataFiles); assertEquals(2, metadataFilesToBeDeleted.size()); assertEquals(metadataFiles.get(0), metadataFilesToBeDeleted.get(0)); assertEquals(metadataFiles.get(2), metadataFilesToBeDeleted.get(1)); @@ -682,7 +733,7 @@ public void testGetMetadataFilesToBeDeletedExclusionBasedOnAgeAndPinning() throw "metadata__9223372036438563903__9223372036854775701__" + md3Timestamp + "__31__9223372036854775701__1" ); - List metadataFilesToBeDeleted = translog.getMetadataFilesToBeDeleted(metadataFiles); + List metadataFilesToBeDeleted = ((RemoteFsTimestampAwareTranslog) translog).getMetadataFilesToBeDeleted(metadataFiles); assertEquals(1, metadataFilesToBeDeleted.size()); assertEquals(metadataFiles.get(2), metadataFilesToBeDeleted.get(0)); } @@ -707,6 +758,8 @@ public void testIsGenerationPinned() { pinnedGenerations.add(new Tuple<>(142L, 180L)); pinnedGenerations.add(new Tuple<>(4L, 9L)); + RemoteFsTimestampAwareTranslog translog = (RemoteFsTimestampAwareTranslog) this.translog; + assertFalse(translog.isGenerationPinned(3, pinnedGenerations)); assertFalse(translog.isGenerationPinned(10, pinnedGenerations)); assertFalse(translog.isGenerationPinned(141, pinnedGenerations)); @@ -724,6 +777,8 @@ public void testIsGenerationPinned() { public void testGetMinMaxTranslogGenerationFromMetadataFile() throws IOException { TranslogTransferManager translogTransferManager = mock(TranslogTransferManager.class); + RemoteFsTimestampAwareTranslog translog = (RemoteFsTimestampAwareTranslog) this.translog; + // Fetch generations directly from the filename assertEquals( new Tuple<>(701L, 1008L), diff --git a/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java index 163340e8ec7d5..339d876274557 100644 --- a/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java +++ b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java @@ -118,11 +118,11 @@ public class RemoteFsTranslogTests extends OpenSearchTestCase { protected final ShardId shardId = new ShardId("index", "_na_", 1); protected RemoteFsTranslog translog; - private AtomicLong globalCheckpoint; + protected AtomicLong globalCheckpoint; protected Path translogDir; // A default primary term is used by translog instances created in this test. protected final AtomicLong primaryTerm = new AtomicLong(); - private final AtomicBoolean primaryMode = new AtomicBoolean(true); + protected final AtomicBoolean primaryMode = new AtomicBoolean(true); private final AtomicReference persistedSeqNoConsumer = new AtomicReference<>(); protected ThreadPool threadPool; protected final static String METADATA_DIR = "metadata"; @@ -136,7 +136,7 @@ public class RemoteFsTranslogTests extends OpenSearchTestCase { TestTranslog.SlowDownWriteSwitch slowDown; - private LongConsumer getPersistedSeqNoConsumer() { + protected LongConsumer getPersistedSeqNoConsumer() { return seqNo -> { final LongConsumer consumer = persistedSeqNoConsumer.get(); if (consumer != null) { @@ -167,7 +167,7 @@ public void tearDown() throws Exception { } } - private RemoteFsTranslog create(Path path) throws IOException { + protected RemoteFsTranslog create(Path path) throws IOException { final String translogUUID = Translog.createEmptyTranslog(path, SequenceNumbers.NO_OPS_PERFORMED, shardId, primaryTerm.get()); return create(path, createRepository(), translogUUID, 0); } @@ -179,6 +179,14 @@ private RemoteFsTranslog create(Path path, BlobStoreRepository repository, Strin final TranslogDeletionPolicy deletionPolicy = createTranslogDeletionPolicy(translogConfig.getIndexSettings()); threadPool = new TestThreadPool(getClass().getName()); blobStoreTransferService = new BlobStoreTransferService(repository.blobStore(), threadPool); + return createTranslogInstance(translogConfig, translogUUID, deletionPolicy); + } + + protected RemoteFsTranslog createTranslogInstance( + TranslogConfig translogConfig, + String translogUUID, + TranslogDeletionPolicy deletionPolicy + ) throws IOException { return new RemoteFsTranslog( translogConfig, translogUUID, @@ -671,13 +679,13 @@ public void testSimpleOperationsUpload() throws Exception { assertThat(snapshot.totalOperations(), equalTo(ops.size())); } - assertEquals(2, translog.allUploaded().size()); + assertEquals(4, translog.allUploaded().size()); addToTranslogAndListAndUpload(translog, ops, new Translog.Index("1", 1, primaryTerm.get(), new byte[] { 1 })); - assertEquals(4, translog.allUploaded().size()); + assertEquals(6, translog.allUploaded().size()); translog.rollGeneration(); - assertEquals(4, translog.allUploaded().size()); + assertEquals(6, translog.allUploaded().size()); Set mdFiles = blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR)); assertEquals(2, mdFiles.size()); @@ -736,7 +744,7 @@ public void testSimpleOperationsUpload() throws Exception { assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); assertEquals(1, translog.readers.size()); assertBusy(() -> { - assertEquals(2, translog.allUploaded().size()); + assertEquals(4, translog.allUploaded().size()); assertEquals( 4, blobStoreTransferService.listAll(getTranslogDirectory().add(DATA_DIR).add(String.valueOf(primaryTerm.get()))).size() @@ -755,7 +763,7 @@ public void testSimpleOperationsUpload() throws Exception { assertEquals(1, translog.readers.size()); assertEquals(1, translog.stats().estimatedNumberOfOperations()); assertBusy(() -> { - assertEquals(2, translog.allUploaded().size()); + assertEquals(4, translog.allUploaded().size()); assertEquals( 4, blobStoreTransferService.listAll(getTranslogDirectory().add(DATA_DIR).add(String.valueOf(primaryTerm.get()))).size() @@ -774,7 +782,7 @@ public void testMetadataFileDeletion() throws Exception { assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); assertEquals(1, translog.readers.size()); } - assertBusy(() -> assertEquals(2, translog.allUploaded().size())); + assertBusy(() -> assertEquals(4, translog.allUploaded().size())); assertBusy(() -> assertEquals(1, blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR)).size())); int moreDocs = randomIntBetween(3, 10); logger.info("numDocs={} moreDocs={}", numDocs, moreDocs); @@ -872,7 +880,7 @@ public void testDrainSync() throws Exception { assertBusy(() -> assertEquals(0, latch.getCount())); assertEquals(0, translog.availablePermits()); slowDown.setSleepSeconds(0); - assertEquals(4, translog.allUploaded().size()); + assertEquals(6, translog.allUploaded().size()); assertEquals(2, translog.readers.size()); Set mdFiles = blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR)); @@ -881,7 +889,7 @@ public void testDrainSync() throws Exception { translog.trimUnreferencedReaders(); assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); assertEquals(1, translog.readers.size()); - assertEquals(2, translog.allUploaded().size()); + assertEquals(6, translog.allUploaded().size()); assertEquals(mdFiles, blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR))); // Case 4 - After drainSync, if an upload is an attempted, we do not upload to remote store. @@ -891,21 +899,21 @@ public void testDrainSync() throws Exception { new Translog.Index(String.valueOf(2), 2, primaryTerm.get(), new byte[] { 1 }) ); assertEquals(1, translog.readers.size()); - assertEquals(2, translog.allUploaded().size()); + assertEquals(6, translog.allUploaded().size()); assertEquals(mdFiles, blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR))); // Refill the permits back Releasables.close(releasable); addToTranslogAndListAndUpload(translog, ops, new Translog.Index(String.valueOf(3), 3, primaryTerm.get(), new byte[] { 1 })); assertEquals(2, translog.readers.size()); - assertEquals(4, translog.allUploaded().size()); + assertEquals(8, translog.allUploaded().size()); assertEquals(3, blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR)).size()); translog.setMinSeqNoToKeep(3); translog.trimUnreferencedReaders(); assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); assertEquals(1, translog.readers.size()); - assertBusy(() -> assertEquals(2, translog.allUploaded().size())); + assertBusy(() -> assertEquals(4, translog.allUploaded().size())); assertBusy(() -> assertEquals(1, blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR)).size())); } From 3fc0139ca68a1ff843ec1492c3cd52c2c4c67f02 Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Tue, 3 Sep 2024 23:42:26 +0530 Subject: [PATCH 6/7] Optimise snapshot deletion to speed up snapshot deletion and creation (#15568) --------- Signed-off-by: Ashish Singh --- CHANGELOG.md | 1 + .../blobstore/BlobStoreRepository.java | 92 ++++++++++++++++--- .../org/opensearch/threadpool/ThreadPool.java | 7 ++ .../threadpool/ScalingThreadPoolTests.java | 1 + 4 files changed, 86 insertions(+), 15 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 93a98de2c5af8..4df028f12d2fe 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -46,6 +46,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Remote Publication] Add remote download stats ([#15291](https://github.com/opensearch-project/OpenSearch/pull/15291))) - Add support for comma-separated list of index names to be used with Snapshot Status API ([#15409](https://github.com/opensearch-project/OpenSearch/pull/15409)) - Add prefix support to hashed prefix & infix path types on remote store ([#15557](https://github.com/opensearch-project/OpenSearch/pull/15557)) +- Optimise snapshot deletion to speed up snapshot deletion and creation ([#15568](https://github.com/opensearch-project/OpenSearch/pull/15568)) - [Remote Publication] Added checksum validation for cluster state behind a cluster setting ([#15218](https://github.com/opensearch-project/OpenSearch/pull/15218)) ### Dependencies 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 d193ab3c14154..aff48f717eb44 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -66,6 +66,7 @@ import org.opensearch.common.Nullable; import org.opensearch.common.Numbers; import org.opensearch.common.Priority; +import org.opensearch.common.Randomness; import org.opensearch.common.SetOnce; import org.opensearch.common.UUIDs; import org.opensearch.common.blobstore.BlobContainer; @@ -831,7 +832,7 @@ boolean getPrefixModeVerification() { * maintains single lazy instance of {@link BlobContainer} */ protected BlobContainer blobContainer() { - // assertSnapshotOrGenericThread(); + assertSnapshotOrGenericThread(); BlobContainer blobContainer = this.blobContainer.get(); if (blobContainer == null) { @@ -1204,6 +1205,10 @@ private void asyncCleanupUnlinkedShardLevelBlobs( ActionListener listener ) { final List> filesToDelete = resolveFilesToDelete(oldRepositoryData, snapshotIds, deleteResults); + long startTimeNs = System.nanoTime(); + Randomness.shuffle(filesToDelete); + logger.debug("[{}] shuffled the filesToDelete with timeElapsedNs={}", metadata.name(), (System.nanoTime() - startTimeNs)); + if (filesToDelete.isEmpty()) { listener.onResponse(null); return; @@ -1221,8 +1226,8 @@ private void asyncCleanupUnlinkedShardLevelBlobs( staleFilesToDeleteInBatch.size() ); - // Start as many workers as fit into the snapshot pool at once at the most - final int workers = Math.min(threadPool.info(ThreadPool.Names.SNAPSHOT).getMax(), staleFilesToDeleteInBatch.size()); + // Start as many workers as fit into the snapshot_deletion pool at once at the most + final int workers = Math.min(threadPool.info(ThreadPool.Names.SNAPSHOT_DELETION).getMax(), staleFilesToDeleteInBatch.size()); for (int i = 0; i < workers; ++i) { executeStaleShardDelete(staleFilesToDeleteInBatch, remoteStoreLockManagerFactory, groupedListener); } @@ -1326,7 +1331,7 @@ private void executeStaleShardDelete( if (filesToDelete == null) { return; } - threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(ActionRunnable.wrap(listener, l -> { + threadPool.executor(ThreadPool.Names.SNAPSHOT_DELETION).execute(ActionRunnable.wrap(listener, l -> { try { // filtering files for which remote store lock release and cleanup succeeded, // remaining files for which it failed will be retried in next snapshot delete run. @@ -1390,7 +1395,7 @@ private void writeUpdatedShardMetaDataAndComputeDeletes( ActionListener> onAllShardsCompleted ) { - final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT); + final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT_DELETION); final List indices = oldRepositoryData.indicesToUpdateAfterRemovingSnapshot(snapshotIds); if (indices.isEmpty()) { @@ -1578,7 +1583,7 @@ private void cleanupStaleBlobs( listener.onResponse(deleteResult); }, listener::onFailure), 2); - final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT); + final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT_DELETION); final List staleRootBlobs = staleRootBlobs(newRepoData, rootBlobs.keySet()); if (staleRootBlobs.isEmpty()) { groupedListener.onResponse(DeleteResult.ZERO); @@ -1781,7 +1786,7 @@ void cleanupStaleIndices( // Start as many workers as fit into the snapshot pool at once at the most final int workers = Math.min( - threadPool.info(ThreadPool.Names.SNAPSHOT).getMax(), + threadPool.info(ThreadPool.Names.SNAPSHOT_DELETION).getMax(), foundIndices.size() - survivingIndexIds.size() ); for (int i = 0; i < workers; ++i) { @@ -1833,7 +1838,7 @@ private void executeOneStaleIndexDelete( return; } final String indexSnId = indexEntry.getKey(); - threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(ActionRunnable.supply(listener, () -> { + threadPool.executor(ThreadPool.Names.SNAPSHOT_DELETION).execute(ActionRunnable.supply(listener, () -> { try { logger.debug("[{}] Found stale index [{}]. Cleaning it up", metadata.name(), indexSnId); List matchingShardPaths = findMatchingShardPaths(indexSnId, snapshotShardPaths); @@ -2097,8 +2102,7 @@ public void finalizeSnapshot( stateTransformer, repositoryUpdatePriority, ActionListener.wrap(newRepoData -> { - cleanupOldShardGens(existingRepositoryData, updatedRepositoryData); - listener.onResponse(newRepoData); + cleanupOldShardGens(existingRepositoryData, updatedRepositoryData, newRepoData, listener); }, onUpdateFailure) ); }, onUpdateFailure), 2 + indices.size()); @@ -2254,7 +2258,12 @@ private void logShardPathsOperationWarning(IndexId indexId, SnapshotId snapshotI } // Delete all old shard gen blobs that aren't referenced any longer as a result from moving to updated repository data - private void cleanupOldShardGens(RepositoryData existingRepositoryData, RepositoryData updatedRepositoryData) { + private void cleanupOldShardGens( + RepositoryData existingRepositoryData, + RepositoryData updatedRepositoryData, + RepositoryData newRepositoryData, + ActionListener listener + ) { final List toDelete = new ArrayList<>(); updatedRepositoryData.shardGenerations() .obsoleteShardGenerations(existingRepositoryData.shardGenerations()) @@ -2263,10 +2272,62 @@ private void cleanupOldShardGens(RepositoryData existingRepositoryData, Reposito (shardId, oldGen) -> toDelete.add(shardPath(indexId, shardId).buildAsString() + INDEX_FILE_PREFIX + oldGen) ) ); + if (toDelete.isEmpty()) { + listener.onResponse(newRepositoryData); + return; + } try { - deleteFromContainer(rootBlobContainer(), toDelete); + AtomicInteger counter = new AtomicInteger(); + Collection> subList = toDelete.stream() + .collect(Collectors.groupingBy(it -> counter.getAndIncrement() / maxShardBlobDeleteBatch)) + .values(); + final BlockingQueue> staleFilesToDeleteInBatch = new LinkedBlockingQueue<>(subList); + logger.info( + "[{}] cleanupOldShardGens toDeleteSize={} groupSize={}", + metadata.name(), + toDelete.size(), + staleFilesToDeleteInBatch.size() + ); + final GroupedActionListener groupedListener = new GroupedActionListener<>(ActionListener.wrap(r -> { + logger.info("[{}] completed cleanupOldShardGens", metadata.name()); + listener.onResponse(newRepositoryData); + }, ex -> { + logger.error(new ParameterizedMessage("[{}] exception in cleanupOldShardGens", metadata.name()), ex); + listener.onResponse(newRepositoryData); + }), staleFilesToDeleteInBatch.size()); + + // Start as many workers as fit into the snapshot pool at once at the most + final int workers = Math.min(threadPool.info(ThreadPool.Names.SNAPSHOT_DELETION).getMax(), staleFilesToDeleteInBatch.size()); + for (int i = 0; i < workers; ++i) { + executeOldShardGensCleanup(staleFilesToDeleteInBatch, groupedListener); + } } catch (Exception e) { - logger.warn("Failed to clean up old shard generation blobs", e); + logger.warn(new ParameterizedMessage(" [{}] Failed to clean up old shard generation blobs", metadata.name()), e); + listener.onResponse(newRepositoryData); + } + } + + private void executeOldShardGensCleanup(BlockingQueue> staleFilesToDeleteInBatch, GroupedActionListener listener) + throws InterruptedException { + List filesToDelete = staleFilesToDeleteInBatch.poll(0L, TimeUnit.MILLISECONDS); + if (filesToDelete != null) { + threadPool.executor(ThreadPool.Names.SNAPSHOT_DELETION).execute(ActionRunnable.wrap(listener, l -> { + try { + deleteFromContainer(rootBlobContainer(), filesToDelete); + l.onResponse(null); + } catch (Exception e) { + logger.warn( + () -> new ParameterizedMessage( + "[{}] Failed to delete following blobs during cleanupOldFiles : {}", + metadata.name(), + filesToDelete + ), + e + ); + l.onFailure(e); + } + executeOldShardGensCleanup(staleFilesToDeleteInBatch, listener); + })); } } @@ -2383,10 +2444,11 @@ public long getRemoteDownloadThrottleTimeInNanos() { } protected void assertSnapshotOrGenericThread() { - assert Thread.currentThread().getName().contains('[' + ThreadPool.Names.SNAPSHOT + ']') + assert Thread.currentThread().getName().contains('[' + ThreadPool.Names.SNAPSHOT_DELETION + ']') + || Thread.currentThread().getName().contains('[' + ThreadPool.Names.SNAPSHOT + ']') || Thread.currentThread().getName().contains('[' + ThreadPool.Names.GENERIC + ']') : "Expected current thread [" + Thread.currentThread() - + "] to be the snapshot or generic thread."; + + "] to be the snapshot_deletion or snapshot or generic thread."; } @Override diff --git a/server/src/main/java/org/opensearch/threadpool/ThreadPool.java b/server/src/main/java/org/opensearch/threadpool/ThreadPool.java index 056ef0fac0153..81220ab171b34 100644 --- a/server/src/main/java/org/opensearch/threadpool/ThreadPool.java +++ b/server/src/main/java/org/opensearch/threadpool/ThreadPool.java @@ -105,6 +105,7 @@ public static class Names { public static final String REFRESH = "refresh"; public static final String WARMER = "warmer"; public static final String SNAPSHOT = "snapshot"; + public static final String SNAPSHOT_DELETION = "snapshot_deletion"; public static final String FORCE_MERGE = "force_merge"; public static final String FETCH_SHARD_STARTED = "fetch_shard_started"; public static final String FETCH_SHARD_STORE = "fetch_shard_store"; @@ -176,6 +177,7 @@ public static ThreadPoolType fromType(String type) { map.put(Names.REFRESH, ThreadPoolType.SCALING); map.put(Names.WARMER, ThreadPoolType.SCALING); map.put(Names.SNAPSHOT, ThreadPoolType.SCALING); + map.put(Names.SNAPSHOT_DELETION, ThreadPoolType.SCALING); map.put(Names.FORCE_MERGE, ThreadPoolType.FIXED); map.put(Names.FETCH_SHARD_STARTED, ThreadPoolType.SCALING); map.put(Names.FETCH_SHARD_STORE, ThreadPoolType.SCALING); @@ -234,6 +236,7 @@ public ThreadPool( final int halfProcMaxAt5 = halfAllocatedProcessorsMaxFive(allocatedProcessors); final int halfProcMaxAt10 = halfAllocatedProcessorsMaxTen(allocatedProcessors); final int genericThreadPoolMax = boundedBy(4 * allocatedProcessors, 128, 512); + final int snapshotDeletionPoolMax = boundedBy(4 * allocatedProcessors, 64, 256); builders.put(Names.GENERIC, new ScalingExecutorBuilder(Names.GENERIC, 4, genericThreadPoolMax, TimeValue.timeValueSeconds(30))); builders.put(Names.WRITE, new FixedExecutorBuilder(settings, Names.WRITE, allocatedProcessors, 10000)); builders.put(Names.GET, new FixedExecutorBuilder(settings, Names.GET, allocatedProcessors, 1000)); @@ -251,6 +254,10 @@ public ThreadPool( builders.put(Names.REFRESH, new ScalingExecutorBuilder(Names.REFRESH, 1, halfProcMaxAt10, TimeValue.timeValueMinutes(5))); builders.put(Names.WARMER, new ScalingExecutorBuilder(Names.WARMER, 1, halfProcMaxAt5, TimeValue.timeValueMinutes(5))); builders.put(Names.SNAPSHOT, new ScalingExecutorBuilder(Names.SNAPSHOT, 1, halfProcMaxAt5, TimeValue.timeValueMinutes(5))); + builders.put( + Names.SNAPSHOT_DELETION, + new ScalingExecutorBuilder(Names.SNAPSHOT_DELETION, 1, snapshotDeletionPoolMax, TimeValue.timeValueMinutes(5)) + ); builders.put( Names.FETCH_SHARD_STARTED, new ScalingExecutorBuilder(Names.FETCH_SHARD_STARTED, 1, 2 * allocatedProcessors, TimeValue.timeValueMinutes(5)) diff --git a/server/src/test/java/org/opensearch/threadpool/ScalingThreadPoolTests.java b/server/src/test/java/org/opensearch/threadpool/ScalingThreadPoolTests.java index d8f04a11fe494..4f59f9688fb7e 100644 --- a/server/src/test/java/org/opensearch/threadpool/ScalingThreadPoolTests.java +++ b/server/src/test/java/org/opensearch/threadpool/ScalingThreadPoolTests.java @@ -148,6 +148,7 @@ private int expectedSize(final String threadPoolName, final int numberOfProcesso sizes.put(ThreadPool.Names.REFRESH, ThreadPool::halfAllocatedProcessorsMaxTen); sizes.put(ThreadPool.Names.WARMER, ThreadPool::halfAllocatedProcessorsMaxFive); sizes.put(ThreadPool.Names.SNAPSHOT, ThreadPool::halfAllocatedProcessorsMaxFive); + sizes.put(ThreadPool.Names.SNAPSHOT_DELETION, n -> ThreadPool.boundedBy(4 * n, 64, 256)); sizes.put(ThreadPool.Names.FETCH_SHARD_STARTED, ThreadPool::twiceAllocatedProcessors); sizes.put(ThreadPool.Names.FETCH_SHARD_STORE, ThreadPool::twiceAllocatedProcessors); sizes.put(ThreadPool.Names.TRANSLOG_TRANSFER, ThreadPool::halfAllocatedProcessors); From a60b668ad98a52ee8e42ace52de3f68caff36276 Mon Sep 17 00:00:00 2001 From: Lakshya Taragi <157457166+ltaragi@users.noreply.github.com> Date: Wed, 4 Sep 2024 00:54:44 +0530 Subject: [PATCH 7/7] Remove breaking changes from #15409 (#15624) Signed-off-by: Lakshya Taragi --- .../client/SnapshotRequestConvertersTests.java | 2 +- .../upgrades/MultiVersionRepositoryAccessIT.java | 2 +- .../snapshots/status/SnapshotsStatusRequest.java | 12 ++++++++++++ 3 files changed, 14 insertions(+), 2 deletions(-) diff --git a/client/rest-high-level/src/test/java/org/opensearch/client/SnapshotRequestConvertersTests.java b/client/rest-high-level/src/test/java/org/opensearch/client/SnapshotRequestConvertersTests.java index e9ca03262bc4f..8b6910ffebe4a 100644 --- a/client/rest-high-level/src/test/java/org/opensearch/client/SnapshotRequestConvertersTests.java +++ b/client/rest-high-level/src/test/java/org/opensearch/client/SnapshotRequestConvertersTests.java @@ -238,7 +238,7 @@ public void testSnapshotsStatus() { boolean ignoreUnavailable = randomBoolean(); String endpoint = "/_snapshot/" + repository + "/" + snapshotNames.toString() + "/_status"; - SnapshotsStatusRequest snapshotsStatusRequest = new SnapshotsStatusRequest(repository, snapshots, indices); + SnapshotsStatusRequest snapshotsStatusRequest = (new SnapshotsStatusRequest(repository, snapshots)).indices(indices); RequestConvertersTests.setRandomClusterManagerTimeout(snapshotsStatusRequest, expectedParams); snapshotsStatusRequest.ignoreUnavailable(ignoreUnavailable); expectedParams.put("ignore_unavailable", Boolean.toString(ignoreUnavailable)); diff --git a/qa/repository-multi-version/src/test/java/org/opensearch/upgrades/MultiVersionRepositoryAccessIT.java b/qa/repository-multi-version/src/test/java/org/opensearch/upgrades/MultiVersionRepositoryAccessIT.java index 5c6dcdd142995..c47e0065b708b 100644 --- a/qa/repository-multi-version/src/test/java/org/opensearch/upgrades/MultiVersionRepositoryAccessIT.java +++ b/qa/repository-multi-version/src/test/java/org/opensearch/upgrades/MultiVersionRepositoryAccessIT.java @@ -243,7 +243,7 @@ public void testUpgradeMovesRepoToNewMetaVersion() throws IOException { private static void assertSnapshotStatusSuccessful(RestHighLevelClient client, String repoName, String[] snapshots, String[] indices) throws IOException { final SnapshotsStatusResponse statusResponse = client.snapshot() - .status(new SnapshotsStatusRequest(repoName, snapshots, indices), RequestOptions.DEFAULT); + .status((new SnapshotsStatusRequest(repoName, snapshots)).indices(indices), RequestOptions.DEFAULT); for (SnapshotStatus status : statusResponse.getSnapshots()) { assertThat(status.getShardsStats().getFailedShards(), is(0)); } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequest.java index 6c3438fbe58f6..3d7fb5b6beb56 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequest.java @@ -67,6 +67,18 @@ public SnapshotsStatusRequest() {} * @param repository repository name * @param snapshots list of snapshots */ + public SnapshotsStatusRequest(String repository, String[] snapshots) { + this.repository = repository; + this.snapshots = snapshots; + } + + /** + * Constructs a new get snapshots request with given repository name and list of snapshots + * + * @param repository repository name + * @param snapshots list of snapshots + * @param indices list of indices + */ public SnapshotsStatusRequest(String repository, String[] snapshots, String[] indices) { this.repository = repository; this.snapshots = snapshots;