From 0788afa1e52f8438ca4dd3577bf0b2571779f84a Mon Sep 17 00:00:00 2001 From: shailendra0811 <167273922+shailendra0811@users.noreply.github.com> Date: Mon, 22 Jul 2024 23:56:53 +0530 Subject: [PATCH] Add integration tests for RemoteRoutingTable Service. (#14631) Signed-off-by: Shailendra Singh Signed-off-by: Arpit Bandejiya --- .../RemoteClusterStateCleanupManagerIT.java | 138 ++++++++ .../remote/RemoteRoutingTableServiceIT.java | 297 ++++++++++++++++++ .../RemoteStoreBaseIntegTestCase.java | 17 + .../coordination/PersistedStateStats.java | 4 + .../PersistedStateStatsTests.java | 62 ++++ .../test/OpenSearchIntegTestCase.java | 121 +++++++ 6 files changed, 639 insertions(+) create mode 100644 server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteRoutingTableServiceIT.java create mode 100644 server/src/test/java/org/opensearch/cluster/coordination/PersistedStateStatsTests.java diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java index 5074971ab1a1f..93e634a0e37e3 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java @@ -8,9 +8,17 @@ package org.opensearch.gateway.remote; +import org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest; +import org.opensearch.action.admin.cluster.node.stats.NodesStatsResponse; import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; +import org.opensearch.cluster.coordination.PersistedStateStats; +import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.settings.Settings; +import org.opensearch.discovery.DiscoveryStats; +import org.opensearch.gateway.remote.model.RemoteRoutingTableBlobStore; +import org.opensearch.index.remote.RemoteStoreEnums; +import org.opensearch.index.remote.RemoteStorePathStrategy; import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.blobstore.BlobStoreRepository; @@ -18,21 +26,29 @@ import org.junit.Before; import java.nio.charset.StandardCharsets; +import java.nio.file.Path; +import java.util.ArrayList; import java.util.Base64; +import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import static org.opensearch.common.util.FeatureFlags.REMOTE_PUBLICATION_EXPERIMENTAL; import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT; import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING; import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.RETAINED_MANIFESTS; import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.SKIP_CLEANUP_STATE_CHANGES; import static org.opensearch.gateway.remote.RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING; +import static org.opensearch.gateway.remote.routingtable.RemoteIndexRoutingTable.INDEX_ROUTING_TABLE; import static org.opensearch.indices.IndicesService.CLUSTER_DEFAULT_INDEX_REFRESH_INTERVAL_SETTING; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY; @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) public class RemoteClusterStateCleanupManagerIT extends RemoteStoreBaseIntegTestCase { private static final String INDEX_NAME = "test-index"; + private final RemoteStoreEnums.PathType pathType = RemoteStoreEnums.PathType.HASHED_PREFIX; @Before public void setup() { @@ -52,6 +68,11 @@ private Map initialTestSetup(int shardCount, int replicaCount, int return indexStats; } + private void initialTestSetup(int shardCount, int replicaCount, int dataNodeCount, int clusterManagerNodeCount, Settings settings) { + prepareCluster(clusterManagerNodeCount, dataNodeCount, INDEX_NAME, replicaCount, shardCount, settings); + ensureGreen(INDEX_NAME); + } + public void testRemoteCleanupTaskUpdated() { int shardCount = randomIntBetween(1, 2); int replicaCount = 1; @@ -144,6 +165,102 @@ public void testRemoteCleanupDeleteStale() throws Exception { assertTrue(response.isAcknowledged()); } + public void testRemoteCleanupDeleteStaleIndexRoutingFiles() throws Exception { + clusterSettingsSuppliedByTest = true; + Path segmentRepoPath = randomRepoPath(); + Path translogRepoPath = randomRepoPath(); + Path remoteRoutingTableRepoPath = randomRepoPath(); + Settings.Builder settingsBuilder = Settings.builder(); + settingsBuilder.put( + buildRemoteStoreNodeAttributes( + REPOSITORY_NAME, + segmentRepoPath, + REPOSITORY_2_NAME, + translogRepoPath, + REMOTE_ROUTING_TABLE_REPO, + remoteRoutingTableRepoPath, + false + ) + ); + settingsBuilder.put( + RemoteRoutingTableBlobStore.REMOTE_ROUTING_TABLE_PATH_TYPE_SETTING.getKey(), + RemoteStoreEnums.PathType.HASHED_PREFIX.toString() + ) + .put("node.attr." + REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY, REMOTE_ROUTING_TABLE_REPO) + .put(REMOTE_PUBLICATION_EXPERIMENTAL, true); + + int shardCount = randomIntBetween(1, 2); + int replicaCount = 1; + int dataNodeCount = shardCount * (replicaCount + 1); + int clusterManagerNodeCount = 1; + initialTestSetup(shardCount, replicaCount, dataNodeCount, clusterManagerNodeCount, settingsBuilder.build()); + + // update cluster state 21 times to ensure that clean up has run after this will upload 42 manifest files + // to repository, if manifest files are less than that it means clean up has run + updateClusterStateNTimes(RETAINED_MANIFESTS + SKIP_CLEANUP_STATE_CHANGES + 1); + + RepositoriesService repositoriesService = internalCluster().getClusterManagerNodeInstance(RepositoriesService.class); + BlobStoreRepository repository = (BlobStoreRepository) repositoriesService.repository(REPOSITORY_NAME); + BlobPath baseMetadataPath = getBaseMetadataPath(repository); + + BlobStoreRepository routingTableRepository = (BlobStoreRepository) repositoriesService.repository(REMOTE_ROUTING_TABLE_REPO); + List indexRoutingTables = new ArrayList<>(getClusterState().routingTable().indicesRouting().values()); + BlobPath indexRoutingPath = getIndexRoutingPath(baseMetadataPath, indexRoutingTables.get(0).getIndex().getUUID()); + assertBusy(() -> { + // There would be >=3 files as shards will transition from UNASSIGNED -> INIT -> STARTED state + assertTrue(routingTableRepository.blobStore().blobContainer(indexRoutingPath).listBlobs().size() >= 3); + }); + + RemoteClusterStateCleanupManager remoteClusterStateCleanupManager = internalCluster().getClusterManagerNodeInstance( + RemoteClusterStateCleanupManager.class + ); + + // set cleanup interval to 100 ms to make the test faster + ClusterUpdateSettingsResponse response = client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING.getKey(), "100ms")) + .get(); + + assertTrue(response.isAcknowledged()); + assertBusy(() -> assertEquals(100, remoteClusterStateCleanupManager.getStaleFileDeletionTask().getInterval().getMillis())); + + String clusterManagerNode = internalCluster().getClusterManagerName(); + NodesStatsResponse nodesStatsResponse = client().admin() + .cluster() + .prepareNodesStats(clusterManagerNode) + .addMetric(NodesStatsRequest.Metric.DISCOVERY.metricName()) + .get(); + verifyIndexRoutingFilesDeletion(routingTableRepository, indexRoutingPath, nodesStatsResponse); + + // disable the clean up to avoid race condition during shutdown + response = client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING.getKey(), "-1")) + .get(); + assertTrue(response.isAcknowledged()); + } + + private void verifyIndexRoutingFilesDeletion( + BlobStoreRepository routingTableRepository, + BlobPath indexRoutingPath, + NodesStatsResponse nodesStatsResponse + ) throws Exception { + assertBusy(() -> { assertEquals(1, routingTableRepository.blobStore().blobContainer(indexRoutingPath).listBlobs().size()); }); + + // Verify index routing files delete stats + DiscoveryStats discoveryStats = nodesStatsResponse.getNodes().get(0).getDiscoveryStats(); + assertNotNull(discoveryStats.getClusterStateStats()); + for (PersistedStateStats persistedStateStats : discoveryStats.getClusterStateStats().getPersistenceStats()) { + Map extendedFields = persistedStateStats.getExtendedFields(); + assertTrue(extendedFields.containsKey(RemotePersistenceStats.INDEX_ROUTING_FILES_CLEANUP_ATTEMPT_FAILED_COUNT)); + long cleanupAttemptFailedCount = extendedFields.get(RemotePersistenceStats.INDEX_ROUTING_FILES_CLEANUP_ATTEMPT_FAILED_COUNT) + .get(); + assertEquals(0, cleanupAttemptFailedCount); + } + } + private void updateClusterStateNTimes(int n) { int newReplicaCount = randomIntBetween(0, 3); for (int i = n; i > 0; i--) { @@ -155,4 +272,25 @@ private void updateClusterStateNTimes(int n) { assertTrue(response.isAcknowledged()); } } + + private BlobPath getBaseMetadataPath(BlobStoreRepository repository) { + return repository.basePath() + .add( + Base64.getUrlEncoder() + .withoutPadding() + .encodeToString(getClusterState().getClusterName().value().getBytes(StandardCharsets.UTF_8)) + ) + .add("cluster-state") + .add(getClusterState().metadata().clusterUUID()); + } + + private BlobPath getIndexRoutingPath(BlobPath baseMetadataPath, String indexUUID) { + return pathType.path( + RemoteStorePathStrategy.BasePathInput.builder() + .basePath(baseMetadataPath.add(INDEX_ROUTING_TABLE)) + .indexUUID(indexUUID) + .build(), + RemoteStoreEnums.PathHashAlgorithm.FNV_1A_BASE64 + ); + } } diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteRoutingTableServiceIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteRoutingTableServiceIT.java new file mode 100644 index 0000000000000..ef10a93989a53 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteRoutingTableServiceIT.java @@ -0,0 +1,297 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway.remote; + +import org.opensearch.action.admin.cluster.state.ClusterStateRequest; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.settings.Settings; +import org.opensearch.gateway.remote.model.RemoteRoutingTableBlobStore; +import org.opensearch.index.remote.RemoteStoreEnums; +import org.opensearch.index.remote.RemoteStorePathStrategy; +import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchIntegTestCase; +import org.junit.Before; + +import java.nio.charset.StandardCharsets; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Base64; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.opensearch.common.util.FeatureFlags.REMOTE_PUBLICATION_EXPERIMENTAL; +import static org.opensearch.gateway.remote.RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING; +import static org.opensearch.gateway.remote.routingtable.RemoteIndexRoutingTable.INDEX_ROUTING_TABLE; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY; + +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) +public class RemoteRoutingTableServiceIT extends RemoteStoreBaseIntegTestCase { + private static final String INDEX_NAME = "test-index"; + BlobPath indexRoutingPath; + AtomicInteger indexRoutingFiles = new AtomicInteger(); + private final RemoteStoreEnums.PathType pathType = RemoteStoreEnums.PathType.HASHED_PREFIX; + + @Before + public void setup() { + asyncUploadMockFsRepo = false; + } + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal)) + .put(REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true) + .put( + RemoteRoutingTableBlobStore.REMOTE_ROUTING_TABLE_PATH_TYPE_SETTING.getKey(), + RemoteStoreEnums.PathType.HASHED_PREFIX.toString() + ) + .put("node.attr." + REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY, REMOTE_ROUTING_TABLE_REPO) + .put(REMOTE_PUBLICATION_EXPERIMENTAL, true) + .build(); + } + + public void testRemoteRoutingTableIndexLifecycle() throws Exception { + BlobStoreRepository repository = prepareClusterAndVerifyRepository(); + + RemoteClusterStateService remoteClusterStateService = internalCluster().getClusterManagerNodeInstance( + RemoteClusterStateService.class + ); + RemoteManifestManager remoteManifestManager = remoteClusterStateService.getRemoteManifestManager(); + verifyUpdatesInManifestFile(remoteManifestManager); + + List routingTableVersions = getRoutingTableFromAllNodes(); + assertTrue(areRoutingTablesSame(routingTableVersions)); + + // Update index settings + updateIndexSettings(INDEX_NAME, IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 2); + ensureGreen(INDEX_NAME); + assertBusy(() -> { + int indexRoutingFilesAfterUpdate = repository.blobStore().blobContainer(indexRoutingPath).listBlobs().size(); + // At-least 3 new index routing files will be created as shards will transition from INIT -> UNASSIGNED -> STARTED state + assertTrue(indexRoutingFilesAfterUpdate >= indexRoutingFiles.get() + 3); + }); + + verifyUpdatesInManifestFile(remoteManifestManager); + + routingTableVersions = getRoutingTableFromAllNodes(); + assertTrue(areRoutingTablesSame(routingTableVersions)); + + // Delete the index and assert its deletion + deleteIndexAndVerify(remoteManifestManager); + + routingTableVersions = getRoutingTableFromAllNodes(); + assertTrue(areRoutingTablesSame(routingTableVersions)); + } + + public void testRemoteRoutingTableIndexNodeRestart() throws Exception { + BlobStoreRepository repository = prepareClusterAndVerifyRepository(); + + List routingTableVersions = getRoutingTableFromAllNodes(); + assertTrue(areRoutingTablesSame(routingTableVersions)); + + // Ensure node comes healthy after restart + Set dataNodes = internalCluster().getDataNodeNames(); + internalCluster().restartNode(randomFrom(dataNodes)); + ensureGreen(); + ensureGreen(INDEX_NAME); + + // ensure restarted node joins and the cluster is stable + assertEquals(3, internalCluster().clusterService().state().nodes().getDataNodes().size()); + ensureStableCluster(4); + assertRemoteStoreRepositoryOnAllNodes(REMOTE_ROUTING_TABLE_REPO); + + assertBusy(() -> { + int indexRoutingFilesAfterNodeDrop = repository.blobStore().blobContainer(indexRoutingPath).listBlobs().size(); + assertTrue(indexRoutingFilesAfterNodeDrop > indexRoutingFiles.get()); + }); + + RemoteClusterStateService remoteClusterStateService = internalCluster().getClusterManagerNodeInstance( + RemoteClusterStateService.class + ); + RemoteManifestManager remoteManifestManager = remoteClusterStateService.getRemoteManifestManager(); + verifyUpdatesInManifestFile(remoteManifestManager); + } + + public void testRemoteRoutingTableIndexMasterRestart1() throws Exception { + BlobStoreRepository repository = prepareClusterAndVerifyRepository(); + + List routingTableVersions = getRoutingTableFromAllNodes(); + assertTrue(areRoutingTablesSame(routingTableVersions)); + + // Ensure node comes healthy after restart + String clusterManagerName = internalCluster().getClusterManagerName(); + internalCluster().restartNode(clusterManagerName); + ensureGreen(); + ensureGreen(INDEX_NAME); + + // ensure master is elected and the cluster is stable + assertNotNull(internalCluster().clusterService().state().nodes().getClusterManagerNode()); + ensureStableCluster(4); + assertRemoteStoreRepositoryOnAllNodes(REMOTE_ROUTING_TABLE_REPO); + + assertBusy(() -> { + int indexRoutingFilesAfterNodeDrop = repository.blobStore().blobContainer(indexRoutingPath).listBlobs().size(); + assertTrue(indexRoutingFilesAfterNodeDrop > indexRoutingFiles.get()); + }); + + RemoteClusterStateService remoteClusterStateService = internalCluster().getClusterManagerNodeInstance( + RemoteClusterStateService.class + ); + RemoteManifestManager remoteManifestManager = remoteClusterStateService.getRemoteManifestManager(); + verifyUpdatesInManifestFile(remoteManifestManager); + } + + private BlobStoreRepository prepareClusterAndVerifyRepository() throws Exception { + clusterSettingsSuppliedByTest = true; + Path segmentRepoPath = randomRepoPath(); + Path translogRepoPath = randomRepoPath(); + Path remoteRoutingTableRepoPath = randomRepoPath(); + Settings settings = buildRemoteStoreNodeAttributes( + REPOSITORY_NAME, + segmentRepoPath, + REPOSITORY_2_NAME, + translogRepoPath, + REMOTE_ROUTING_TABLE_REPO, + remoteRoutingTableRepoPath, + false + ); + prepareCluster(1, 3, INDEX_NAME, 1, 5, settings); + ensureGreen(INDEX_NAME); + + RepositoriesService repositoriesService = internalCluster().getClusterManagerNodeInstance(RepositoriesService.class); + BlobStoreRepository repository = (BlobStoreRepository) repositoriesService.repository(REMOTE_ROUTING_TABLE_REPO); + + BlobPath baseMetadataPath = getBaseMetadataPath(repository); + List indexRoutingTables = new ArrayList<>(getClusterState().routingTable().indicesRouting().values()); + indexRoutingPath = getIndexRoutingPath(baseMetadataPath.add(INDEX_ROUTING_TABLE), indexRoutingTables.get(0).getIndex().getUUID()); + + assertBusy(() -> { + indexRoutingFiles.set(repository.blobStore().blobContainer(indexRoutingPath).listBlobs().size()); + // There would be >=3 files as shards will transition from UNASSIGNED -> INIT -> STARTED state + assertTrue(indexRoutingFiles.get() >= 3); + }); + assertRemoteStoreRepositoryOnAllNodes(REMOTE_ROUTING_TABLE_REPO); + return repository; + } + + private BlobPath getBaseMetadataPath(BlobStoreRepository repository) { + return repository.basePath() + .add( + Base64.getUrlEncoder() + .withoutPadding() + .encodeToString(getClusterState().getClusterName().value().getBytes(StandardCharsets.UTF_8)) + ) + .add("cluster-state") + .add(getClusterState().metadata().clusterUUID()); + } + + private BlobPath getIndexRoutingPath(BlobPath indexRoutingPath, String indexUUID) { + RemoteStoreEnums.PathHashAlgorithm pathHashAlgo = RemoteStoreEnums.PathHashAlgorithm.FNV_1A_BASE64; + return pathType.path( + RemoteStorePathStrategy.BasePathInput.builder().basePath(indexRoutingPath).indexUUID(indexUUID).build(), + pathHashAlgo + ); + } + + private void verifyUpdatesInManifestFile(RemoteManifestManager remoteManifestManager) { + Optional latestManifest = remoteManifestManager.getLatestClusterMetadataManifest( + getClusterState().getClusterName().value(), + getClusterState().getMetadata().clusterUUID() + ); + assertTrue(latestManifest.isPresent()); + ClusterMetadataManifest manifest = latestManifest.get(); + assertTrue(manifest.getDiffManifest().getIndicesRoutingUpdated().contains(INDEX_NAME)); + assertTrue(manifest.getDiffManifest().getIndicesDeleted().isEmpty()); + assertFalse(manifest.getIndicesRouting().isEmpty()); + assertEquals(1, manifest.getIndicesRouting().size()); + assertTrue(manifest.getIndicesRouting().get(0).getUploadedFilename().contains(indexRoutingPath.buildAsString())); + } + + private List getRoutingTableFromAllNodes() throws ExecutionException, InterruptedException { + String[] allNodes = internalCluster().getNodeNames(); + List routingTables = new ArrayList<>(); + for (String node : allNodes) { + RoutingTable routingTable = internalCluster().client(node) + .admin() + .cluster() + .state(new ClusterStateRequest().local(true)) + .get() + .getState() + .routingTable(); + routingTables.add(routingTable); + } + return routingTables; + } + + private boolean areRoutingTablesSame(List routingTables) { + if (routingTables == null || routingTables.isEmpty()) { + return false; + } + + RoutingTable firstRoutingTable = routingTables.get(0); + for (RoutingTable routingTable : routingTables) { + if (!compareRoutingTables(firstRoutingTable, routingTable)) { + logger.info("Responses are not the same: {} {}", firstRoutingTable, routingTable); + return false; + } + } + return true; + } + + private boolean compareRoutingTables(RoutingTable a, RoutingTable b) { + if (a == b) return true; + if (b == null || a.getClass() != b.getClass()) return false; + if (a.version() != b.version()) return false; + if (a.indicesRouting().size() != b.indicesRouting().size()) return false; + + for (Map.Entry entry : a.indicesRouting().entrySet()) { + IndexRoutingTable thisIndexRoutingTable = entry.getValue(); + IndexRoutingTable thatIndexRoutingTable = b.indicesRouting().get(entry.getKey()); + if (!thatIndexRoutingTable.equals(thatIndexRoutingTable)) { + return false; + } + } + return true; + } + + private void updateIndexSettings(String indexName, String settingKey, int settingValue) { + client().admin() + .indices() + .prepareUpdateSettings(indexName) + .setSettings(Settings.builder().put(settingKey, settingValue)) + .execute() + .actionGet(); + } + + private void deleteIndexAndVerify(RemoteManifestManager remoteManifestManager) { + client().admin().indices().prepareDelete(INDEX_NAME).execute().actionGet(); + assertFalse(client().admin().indices().prepareExists(INDEX_NAME).get().isExists()); + + // Verify index is marked deleted in manifest + Optional latestManifest = remoteManifestManager.getLatestClusterMetadataManifest( + getClusterState().getClusterName().value(), + getClusterState().getMetadata().clusterUUID() + ); + assertTrue(latestManifest.isPresent()); + ClusterMetadataManifest manifest = latestManifest.get(); + assertTrue(manifest.getDiffManifest().getIndicesRoutingUpdated().isEmpty()); + assertTrue(manifest.getDiffManifest().getIndicesDeleted().contains(INDEX_NAME)); + assertTrue(manifest.getIndicesRouting().isEmpty()); + } + +} diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java index 64efcee6ef1b5..63a9451a27a12 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java @@ -69,6 +69,7 @@ public class RemoteStoreBaseIntegTestCase extends OpenSearchIntegTestCase { protected static final String REPOSITORY_NAME = "test-remote-store-repo"; protected static final String REPOSITORY_2_NAME = "test-remote-store-repo-2"; + protected static final String REMOTE_ROUTING_TABLE_REPO = "remote-routing-table-repo"; protected static final int SHARD_COUNT = 1; protected static int REPLICA_COUNT = 1; protected static final String TOTAL_OPERATIONS = "total-operations"; @@ -360,4 +361,20 @@ protected void prepareCluster(int numClusterManagerNodes, int numDataOnlyNodes, ensureGreen(index); } } + + protected void prepareCluster( + int numClusterManagerNodes, + int numDataOnlyNodes, + String indices, + int replicaCount, + int shardCount, + Settings settings + ) { + internalCluster().startClusterManagerOnlyNodes(numClusterManagerNodes, settings); + internalCluster().startDataOnlyNodes(numDataOnlyNodes, settings); + for (String index : indices.split(",")) { + createIndex(index, remoteStoreIndexSettings(replicaCount, shardCount)); + ensureGreen(index); + } + } } diff --git a/server/src/main/java/org/opensearch/cluster/coordination/PersistedStateStats.java b/server/src/main/java/org/opensearch/cluster/coordination/PersistedStateStats.java index 0b7ed4fee5775..023c2db1a574a 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/PersistedStateStats.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/PersistedStateStats.java @@ -117,6 +117,10 @@ protected void addToExtendedFields(String extendedField, AtomicLong extendedFiel this.extendedFields.put(extendedField, extendedFieldValue); } + public Map getExtendedFields() { + return extendedFields; + } + public String getStatsName() { return statsName; } diff --git a/server/src/test/java/org/opensearch/cluster/coordination/PersistedStateStatsTests.java b/server/src/test/java/org/opensearch/cluster/coordination/PersistedStateStatsTests.java new file mode 100644 index 0000000000000..15c7d3ea206ef --- /dev/null +++ b/server/src/test/java/org/opensearch/cluster/coordination/PersistedStateStatsTests.java @@ -0,0 +1,62 @@ +/* + * 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.cluster.coordination; + +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; + +import java.util.concurrent.atomic.AtomicLong; + +public class PersistedStateStatsTests extends OpenSearchTestCase { + private PersistedStateStats persistedStateStats; + + @Before + public void setup() { + persistedStateStats = new PersistedStateStats("testStats"); + } + + public void testAddToExtendedFieldsNewField() { + String fieldName = "testField"; + AtomicLong fieldValue = new AtomicLong(42); + + persistedStateStats.addToExtendedFields(fieldName, fieldValue); + + assertTrue(persistedStateStats.getExtendedFields().containsKey(fieldName)); + assertEquals(42, persistedStateStats.getExtendedFields().get(fieldName).get()); + } + + public void testAddToExtendedFieldsExistingField() { + String fieldName = "testField"; + AtomicLong initialValue = new AtomicLong(42); + persistedStateStats.addToExtendedFields(fieldName, initialValue); + + AtomicLong newValue = new AtomicLong(84); + persistedStateStats.addToExtendedFields(fieldName, newValue); + + assertTrue(persistedStateStats.getExtendedFields().containsKey(fieldName)); + assertEquals(84, persistedStateStats.getExtendedFields().get(fieldName).get()); + } + + public void testAddMultipleFields() { + String fieldName1 = "testField1"; + AtomicLong fieldValue1 = new AtomicLong(42); + + String fieldName2 = "testField2"; + AtomicLong fieldValue2 = new AtomicLong(84); + + persistedStateStats.addToExtendedFields(fieldName1, fieldValue1); + persistedStateStats.addToExtendedFields(fieldName2, fieldValue2); + + assertTrue(persistedStateStats.getExtendedFields().containsKey(fieldName1)); + assertTrue(persistedStateStats.getExtendedFields().containsKey(fieldName2)); + + assertEquals(42, persistedStateStats.getExtendedFields().get(fieldName1).get()); + assertEquals(84, persistedStateStats.getExtendedFields().get(fieldName2).get()); + } +} diff --git a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java index 7effafeb6ee9c..371781723bc8e 100644 --- a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java @@ -159,6 +159,7 @@ import org.opensearch.plugins.NetworkPlugin; import org.opensearch.plugins.Plugin; import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.repositories.fs.FsRepository; import org.opensearch.repositories.fs.ReloadableFsRepository; import org.opensearch.script.MockScriptService; import org.opensearch.script.ScriptMetadata; @@ -227,6 +228,7 @@ import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY; import static org.opensearch.node.remotestore.RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING; @@ -2599,6 +2601,35 @@ public static Settings remoteStoreClusterSettings( return settingsBuilder.build(); } + public static Settings remoteStoreClusterSettings( + String segmentRepoName, + Path segmentRepoPath, + String segmentRepoType, + String translogRepoName, + Path translogRepoPath, + String translogRepoType, + String routingTableRepoName, + Path routingTableRepoPath, + String routingTableRepoType + ) { + Settings.Builder settingsBuilder = Settings.builder(); + settingsBuilder.put( + buildRemoteStoreNodeAttributes( + segmentRepoName, + segmentRepoPath, + segmentRepoType, + translogRepoName, + translogRepoPath, + translogRepoType, + routingTableRepoName, + routingTableRepoPath, + routingTableRepoType, + false + ) + ); + return settingsBuilder.build(); + } + public static Settings remoteStoreClusterSettings( String segmentRepoName, Path segmentRepoPath, @@ -2610,6 +2641,29 @@ public static Settings remoteStoreClusterSettings( return settingsBuilder.build(); } + public static Settings remoteStoreClusterSettings( + String segmentRepoName, + Path segmentRepoPath, + String translogRepoName, + Path translogRepoPath, + String remoteRoutingTableRepoName, + Path remoteRoutingTableRepoPath + ) { + Settings.Builder settingsBuilder = Settings.builder(); + settingsBuilder.put( + buildRemoteStoreNodeAttributes( + segmentRepoName, + segmentRepoPath, + translogRepoName, + translogRepoPath, + remoteRoutingTableRepoName, + remoteRoutingTableRepoPath, + false + ) + ); + return settingsBuilder.build(); + } + public static Settings buildRemoteStoreNodeAttributes( String segmentRepoName, Path segmentRepoPath, @@ -2628,6 +2682,29 @@ public static Settings buildRemoteStoreNodeAttributes( ); } + public static Settings buildRemoteStoreNodeAttributes( + String segmentRepoName, + Path segmentRepoPath, + String translogRepoName, + Path translogRepoPath, + String remoteRoutingTableRepoName, + Path remoteRoutingTableRepoPath, + boolean withRateLimiterAttributes + ) { + return buildRemoteStoreNodeAttributes( + segmentRepoName, + segmentRepoPath, + ReloadableFsRepository.TYPE, + translogRepoName, + translogRepoPath, + ReloadableFsRepository.TYPE, + remoteRoutingTableRepoName, + remoteRoutingTableRepoPath, + FsRepository.TYPE, + withRateLimiterAttributes + ); + } + private static Settings buildRemoteStoreNodeAttributes( String segmentRepoName, Path segmentRepoPath, @@ -2636,6 +2713,32 @@ private static Settings buildRemoteStoreNodeAttributes( Path translogRepoPath, String translogRepoType, boolean withRateLimiterAttributes + ) { + return buildRemoteStoreNodeAttributes( + segmentRepoName, + segmentRepoPath, + segmentRepoType, + translogRepoName, + translogRepoPath, + translogRepoType, + null, + null, + null, + withRateLimiterAttributes + ); + } + + private static Settings buildRemoteStoreNodeAttributes( + String segmentRepoName, + Path segmentRepoPath, + String segmentRepoType, + String translogRepoName, + Path translogRepoPath, + String translogRepoType, + String routingTableRepoName, + Path routingTableRepoPath, + String routingTableRepoType, + boolean withRateLimiterAttributes ) { String segmentRepoTypeAttributeKey = String.format( Locale.getDefault(), @@ -2667,6 +2770,19 @@ private static Settings buildRemoteStoreNodeAttributes( "node.attr." + REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX, segmentRepoName ); + String routingTableRepoAttributeKey = null, routingTableRepoSettingsAttributeKeyPrefix = null; + if (routingTableRepoName != null) { + routingTableRepoAttributeKey = String.format( + Locale.getDefault(), + "node.attr." + REMOTE_STORE_REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT, + routingTableRepoName + ); + routingTableRepoSettingsAttributeKeyPrefix = String.format( + Locale.getDefault(), + "node.attr." + REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX, + routingTableRepoName + ); + } String prefixModeVerificationSuffix = BlobStoreRepository.PREFIX_MODE_VERIFICATION_SETTING.getKey(); @@ -2683,6 +2799,11 @@ private static Settings buildRemoteStoreNodeAttributes( .put(stateRepoTypeAttributeKey, segmentRepoType) .put(stateRepoSettingsAttributeKeyPrefix + "location", segmentRepoPath) .put(stateRepoSettingsAttributeKeyPrefix + prefixModeVerificationSuffix, prefixModeVerificationEnable); + if (routingTableRepoName != null) { + settings.put("node.attr." + REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY, routingTableRepoName) + .put(routingTableRepoAttributeKey, routingTableRepoType) + .put(routingTableRepoSettingsAttributeKeyPrefix + "location", routingTableRepoPath); + } if (withRateLimiterAttributes) { settings.put(segmentRepoSettingsAttributeKeyPrefix + "compress", randomBoolean())