From 3a7ed961f6e3dfcc171517d25fd1a9752785245c Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Mon, 15 Apr 2024 17:31:48 +0530 Subject: [PATCH 01/24] Initial commit Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../RemoteMigrationIndexMetadataChangeIT.java | 344 ++++++++++++++++++ .../action/shard/ShardStateAction.java | 200 +++++++++- .../cluster/metadata/IndexMetadata.java | 1 + .../metadata/MetadataCreateIndexService.java | 6 + .../cluster/routing/IndexRoutingTable.java | 12 + .../org/opensearch/index/IndexSettings.java | 1 - .../index/remote/RemoteStoreUtils.java | 30 ++ ...dStartedClusterStateTaskExecutorTests.java | 10 + .../test/OpenSearchIntegTestCase.java | 6 + 9 files changed, 608 insertions(+), 2 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataChangeIT.java diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataChangeIT.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataChangeIT.java new file mode 100644 index 0000000000000..097283e9a363f --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataChangeIT.java @@ -0,0 +1,344 @@ +/* + * 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.remotemigration; + +import org.opensearch.action.admin.indices.settings.get.GetSettingsResponse; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.routing.allocation.command.MoveAllocationCommand; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; +import org.opensearch.index.remote.RemoteStoreEnums; +import org.opensearch.plugins.Plugin; +import org.opensearch.remotestore.multipart.mocks.MockFsRepositoryPlugin; +import org.opensearch.test.InternalSettingsPlugin; +import org.opensearch.test.OpenSearchIntegTestCase; +import org.opensearch.test.junit.annotations.TestLogging; +import org.opensearch.test.transport.MockTransportService; + +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.opensearch.cluster.metadata.IndexMetadata.REMOTE_STORE_CUSTOM_KEY; +import static org.opensearch.cluster.metadata.IndexMetadata.REMOTE_STORE_SEEDED_SHARDS_KEY; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; +import static org.opensearch.cluster.routing.allocation.decider.EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; + +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) +public class RemoteMigrationIndexMetadataChangeIT extends MigrationBaseTestCase { + @Override + protected Collection> nodePlugins() { + return Stream.concat( + super.nodePlugins().stream(), + Stream.of(InternalSettingsPlugin.class, MockFsRepositoryPlugin.class, MockTransportService.TestPlugin.class) + ).collect(Collectors.toList()); + } + + public void testIndexSettingsUpdateAfterIndexMovedToRemoteThroughAllocationExclude() throws Exception { + internalCluster().startClusterManagerOnlyNode(); + + addRemote = false; + internalCluster().startDataOnlyNodes(2, Settings.builder().put("node.attr._type", "docrep").build()); + internalCluster().validateClusterFormed(); + + String indexName = "migration-index-allocation-exclude"; + Settings oneReplica = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .build(); + createIndexAndAssertDocrepProperties(indexName, oneReplica); + String replicationType; + GetSettingsResponse response; + String remoteStoreEnabled; + + initDocRepToRemoteMigration(); + addRemote = true; + internalCluster().startDataOnlyNodes(2, Settings.builder().put("node.attr._type", "remote").build()); + internalCluster().validateClusterFormed(); + + excludeNodeSet("type", "docrep"); + waitForRelocation(); + waitNoPendingTasksOnAll(); + + response = internalCluster().client().admin().indices().prepareGetSettings(indexName).get(); + remoteStoreEnabled = response.getSetting(indexName, SETTING_REMOTE_STORE_ENABLED); + replicationType = response.getSetting(indexName, SETTING_REPLICATION_TYPE); + assertEquals(remoteStoreEnabled, "true"); + assertEquals(replicationType, "SEGMENT"); + } + + public void testIndexSettingsUpdateAfterIndexMovedToRemoteThroughManualReroute() throws Exception { + internalCluster().startClusterManagerOnlyNode(); + List docrepNodeNames = internalCluster().startDataOnlyNodes(2); + internalCluster().validateClusterFormed(); + + String indexName1 = "migration-index-manual-reroute-1"; + String indexName2 = "migration-index-manual-reroute-2"; + Settings oneReplica = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .build(); + createIndexAndAssertDocrepProperties(indexName1, oneReplica); + createIndexAndAssertDocrepProperties(indexName2, oneReplica); + + initDocRepToRemoteMigration(); + stopShardRebalancing(); + + addRemote = true; + List remoteNodeNames = internalCluster().startDataOnlyNodes(2); + internalCluster().validateClusterFormed(); + + String primaryNode = primaryNodeName(indexName1); + String replicaNode = docrepNodeNames.stream() + .filter(nodeName -> nodeName.equals(primaryNodeName(indexName1)) == false) + .collect(Collectors.toList()) + .get(0); + + assertAcked( + client().admin() + .cluster() + .prepareReroute() + .add(new MoveAllocationCommand(indexName1, 0, primaryNode, remoteNodeNames.get(0))) + .execute() + .actionGet() + ); + waitForRelocation(); + + assertAcked( + client().admin() + .cluster() + .prepareReroute() + .add(new MoveAllocationCommand(indexName1, 0, replicaNode, remoteNodeNames.get(1))) + .execute() + .actionGet() + ); + waitForRelocation(); + + assertAcked( + client().admin() + .cluster() + .prepareReroute() + .add(new MoveAllocationCommand(indexName2, 0, primaryNodeName(indexName2), remoteNodeNames.get(0))) + .execute() + .actionGet() + ); + waitForRelocation(); + waitNoPendingTasksOnAll(); + + assertRemoteProperties(indexName1); + assertDocrepProperties(indexName2); + } + + public void testIndexSettingsUpdatedOnlyForMigratingIndex() throws Exception { + internalCluster().startClusterManagerOnlyNode(); + + addRemote = false; + internalCluster().startDataOnlyNodes(2, Settings.builder().put("node.attr._type", "docrep").build()); + internalCluster().validateClusterFormed(); + + String indexName = "migration-index"; + createIndex( + indexName, + Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1).build() + ); + indexBulk(indexName, 100); + refresh(indexName); + ensureGreen(indexName); + + assertDocrepProperties(indexName); + long initalMetadataVersion = internalCluster().client() + .admin() + .cluster() + .prepareState() + .get() + .getState() + .metadata() + .index(indexName) + .getVersion(); + + initDocRepToRemoteMigration(); + addRemote = true; + internalCluster().startDataOnlyNodes(2, Settings.builder().put("node.attr._type", "remote").build()); + internalCluster().validateClusterFormed(); + + excludeNodeSet("type", "docrep"); + + waitForRelocation(); + waitNoPendingTasksOnAll(); + assertRemoteProperties(indexName); + assertTrue( + initalMetadataVersion < internalCluster().client() + .admin() + .cluster() + .prepareState() + .get() + .getState() + .metadata() + .index(indexName) + .getVersion() + ); + + String secondIndex = "remote-index"; + createIndex( + secondIndex, + Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1).build() + ); + indexBulk(secondIndex, 100); + refresh(secondIndex); + ensureGreen(secondIndex); + + waitNoPendingTasksOnAll(); + + assertRemoteProperties(secondIndex); + initalMetadataVersion = internalCluster().client() + .admin() + .cluster() + .prepareState() + .get() + .getState() + .metadata() + .index(secondIndex) + .getVersion(); + + assertEquals( + initalMetadataVersion, + internalCluster().client().admin().cluster().prepareState().get().getState().metadata().index(secondIndex).getVersion() + ); + } + + @TestLogging(reason = "", value = "org.opensearch.cluster.metadata:TRACE,org.opensearch.cluster.action.shard:TRACE") + public void testCustomSeedingMetadata() throws Exception { + String indexName = "custom-seeding-metadata-index"; + internalCluster().startClusterManagerOnlyNode(); + List docRepNodeNames = internalCluster().startNodes(2, Settings.builder().put("node.attr._type", "docrep").build()); + + // create index with 2 primaries + Settings zeroReplica = Settings.builder().put("number_of_replicas", 0).put("number_of_shards", 4).build(); + createIndex(indexName, zeroReplica); + indexBulk(indexName, 100); + ensureGreen(indexName); + + stopShardRebalancing(); + initDocRepToRemoteMigration(); + // add remote node in mixed mode cluster + addRemote = true; + List remoteNodeNames = internalCluster().startNodes(2, Settings.builder().put("node.attr._type", "remote").build()); + internalCluster().validateClusterFormed(); + + assertEquals( + internalCluster().client() + .admin() + .cluster() + .prepareGetRepositories(REPOSITORY_NAME, REPOSITORY_2_NAME) + .get() + .repositories() + .size(), + 2 + ); + + assertAcked( + client().admin() + .cluster() + .prepareReroute() + .add(new MoveAllocationCommand(indexName, 0, primaryNodeName(indexName, 0), remoteNodeNames.get(0))) + .add(new MoveAllocationCommand(indexName, 1, primaryNodeName(indexName, 1), remoteNodeNames.get(0))) + .add(new MoveAllocationCommand(indexName, 2, primaryNodeName(indexName, 2), remoteNodeNames.get(1))) + .execute() + .actionGet() + ); + waitForRelocation(); + ensureGreen(indexName); + waitNoPendingTasksOnAll(); + + ClusterState clusterState = internalCluster().client().admin().cluster().prepareState().get().getState(); + IndexMetadata indexMetadata = clusterState.metadata().index(indexName); + HashSet seededShards = new HashSet<>( + Strings.commaDelimitedListToSet( + indexMetadata.getCustomData(REMOTE_STORE_CUSTOM_KEY).get(IndexMetadata.REMOTE_STORE_SEEDED_SHARDS_KEY) + ) + ); + HashSet expectedSeededShards = new HashSet<>(); + for (int i = 0; i < 3; i++) { + expectedSeededShards.add("[" + indexName + "][" + i + "]"); + } + assertEquals(expectedSeededShards, seededShards); + + assertAcked( + client().admin() + .cluster() + .prepareReroute() + .add(new MoveAllocationCommand(indexName, 3, primaryNodeName(indexName, 3), remoteNodeNames.get(1))) + .execute() + .actionGet() + ); + waitForRelocation(); + ensureGreen(indexName); + waitNoPendingTasksOnAll(); + + clusterState = internalCluster().client().admin().cluster().prepareState().get().getState(); + indexMetadata = clusterState.metadata().index(indexName); + Map currentCustomData = indexMetadata.getCustomData(REMOTE_STORE_CUSTOM_KEY); + assertFalse(currentCustomData.containsKey(REMOTE_STORE_SEEDED_SHARDS_KEY)); + assertTrue( + currentCustomData.containsKey(RemoteStoreEnums.PathType.NAME) + && currentCustomData.containsKey(RemoteStoreEnums.PathHashAlgorithm.NAME) + ); + } + + private void createIndexAndAssertDocrepProperties(String index, Settings settings) throws Exception { + createIndex(index, settings); + indexBulk(index, 100); + refresh(index); + ensureGreen(index); + assertDocrepProperties(index); + } + + private void assertDocrepProperties(String index) { + GetSettingsResponse response = internalCluster().client().admin().indices().prepareGetSettings(index).get(); + String remoteStoreEnabled = response.getSetting(index, SETTING_REMOTE_STORE_ENABLED); + String replicationType = response.getSetting(index, SETTING_REPLICATION_TYPE); + assertNull(remoteStoreEnabled); + assertEquals(replicationType, "DOCUMENT"); + } + + private void assertRemoteProperties(String index) { + GetSettingsResponse response = internalCluster().client().admin().indices().prepareGetSettings(index).get(); + String remoteStoreEnabled = response.getSetting(index, SETTING_REMOTE_STORE_ENABLED); + String replicationType = response.getSetting(index, SETTING_REPLICATION_TYPE); + assertEquals(remoteStoreEnabled, "true"); + assertEquals(replicationType, "SEGMENT"); + } + + private void excludeNodeSet(String attr, String value) { + assertAcked( + internalCluster().client() + .admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().put("cluster.routing.allocation.exclude._" + attr, value)) + .get() + ); + } + + private void stopShardRebalancing() { + assertAcked( + client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), "none").build()) + .get() + ); + } +} diff --git a/server/src/main/java/org/opensearch/cluster/action/shard/ShardStateAction.java b/server/src/main/java/org/opensearch/cluster/action/shard/ShardStateAction.java index cb5749a91d448..4c5578261df2e 100644 --- a/server/src/main/java/org/opensearch/cluster/action/shard/ShardStateAction.java +++ b/server/src/main/java/org/opensearch/cluster/action/shard/ShardStateAction.java @@ -37,6 +37,7 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchException; +import org.opensearch.Version; import org.opensearch.cluster.ClusterChangedEvent; import org.opensearch.cluster.ClusterManagerNodeChangePredicate; import org.opensearch.cluster.ClusterState; @@ -47,7 +48,11 @@ import org.opensearch.cluster.NotClusterManagerException; import org.opensearch.cluster.coordination.FailedToCommitClusterStateException; import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.metadata.MetadataCreateIndexService; import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.cluster.routing.RerouteService; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.AllocationService; @@ -56,14 +61,22 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Nullable; import org.opensearch.common.Priority; +import org.opensearch.common.collect.Tuple; import org.opensearch.common.inject.Inject; import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.transport.TransportResponse; +import org.opensearch.index.remote.RemoteStoreEnums; +import org.opensearch.index.remote.RemoteStorePathStrategy; +import org.opensearch.index.remote.RemoteStorePathStrategyResolver; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.node.NodeClosedException; import org.opensearch.tasks.Task; import org.opensearch.threadpool.ThreadPool; @@ -79,13 +92,23 @@ import java.io.IOException; 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.Set; import java.util.function.Predicate; import java.util.function.Supplier; +import java.util.stream.Collectors; + +import static org.opensearch.cluster.metadata.IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING; +import static org.opensearch.cluster.metadata.IndexMetadata.REMOTE_STORE_CUSTOM_KEY; +import static org.opensearch.cluster.metadata.IndexMetadata.REMOTE_STORE_SEEDED_SHARDS_KEY; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; +import static org.opensearch.index.remote.RemoteStoreUtils.ongoingDocrepToRemoteMigration; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteDataAttributePresent; /** * Transport action for retrieving the shard state @@ -137,6 +160,9 @@ private static Priority parseReroutePriority(String priorityString) { // we keep track of these shards in order to avoid sending duplicate failed shard requests for a single failing shard. private final TransportRequestDeduplicator remoteFailedShardsDeduplicator = new TransportRequestDeduplicator<>(); + @Nullable + private static RemoteStorePathStrategyResolver remoteStorePathStrategyResolver; + @Inject public ShardStateAction( ClusterService clusterService, @@ -148,7 +174,10 @@ public ShardStateAction( this.transportService = transportService; this.clusterService = clusterService; this.threadPool = threadPool; - + Supplier minNodeVersionSupplier = () -> clusterService.state().nodes().getMinNodeVersion(); + remoteStorePathStrategyResolver = isRemoteDataAttributePresent(clusterService.getSettings()) + ? new RemoteStorePathStrategyResolver(clusterService.getClusterSettings(), minNodeVersionSupplier) + : null; followUpRerouteTaskPriority = FOLLOW_UP_REROUTE_PRIORITY_SETTING.get(clusterService.getSettings()); clusterService.getClusterSettings() .addSettingsUpdateConsumer(FOLLOW_UP_REROUTE_PRIORITY_SETTING, this::setFollowUpRerouteTaskPriority); @@ -804,8 +833,20 @@ public ClusterTasksResult execute(ClusterState currentState, assert tasksToBeApplied.size() >= shardRoutingsToBeApplied.size(); ClusterState maybeUpdatedState = currentState; + try { + // Let allocation service mark the incoming `RELOCATING` shard copies as `STARTED` maybeUpdatedState = allocationService.applyStartedShards(currentState, shardRoutingsToBeApplied); + // Run remote store migration based tasks + if (ongoingDocrepToRemoteMigration(currentState.metadata().settings())) { + DiscoveryNodes discoveryNodes = currentState.getNodes(); + if (discoveryNodes.getNodes().values().stream().noneMatch(DiscoveryNode::isRemoteStoreNode)) { + logger.debug("Cluster is in mixed mode but does not have any remote enabled nodes. No-Op"); + } else { + maybeUpdatedState = maybeAppendShardIdToSeededSet(maybeUpdatedState, seenShardRoutings, discoveryNodes); + maybeUpdatedState = maybeAddRemoteIndexSettings(maybeUpdatedState, seenShardRoutings, discoveryNodes); + } + } builder.successes(tasksToBeApplied); } catch (Exception e) { logger.warn(() -> new ParameterizedMessage("failed to apply started shards {}", shardRoutingsToBeApplied), e); @@ -835,6 +876,163 @@ public void clusterStatePublished(ClusterChangedEvent clusterChangedEvent) { ) ); } + + /** + * @param state + * @param seenShardRoutings + * @param discoveryNodes + * @return + */ + private ClusterState maybeAddRemoteIndexSettings( + ClusterState state, + Set seenShardRoutings, + DiscoveryNodes discoveryNodes + ) { + final Metadata.Builder metadataBuilder = Metadata.builder(state.metadata()); + HashSet indexNames = seenShardRoutings.stream() + .map(ShardRouting::getIndexName) + .collect(Collectors.toCollection(HashSet::new)); + logger.debug( + "Cluster is going through a docrep to remote store migration. Checking if any index has completely moved over to remote nodes" + ); + Tuple remoteStoreRepoNames = RemoteStoreUtils.getRemoteStoreRepositoryNames(discoveryNodes); + for (String index : indexNames) { + if (needsRemoteIndexSettingsUpdate(state, index, discoveryNodes, state.metadata().index(index))) { + logger.info( + "Index {} does not have remote store based index settings but all shards have moved to remote enabled nodes. Applying remote store settings to the index", + index + ); + final IndexMetadata indexMetadata = metadataBuilder.get(index); + Settings.Builder indexSettingsBuilder = Settings.builder().put(indexMetadata.getSettings()); + indexSettingsBuilder.put(SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT); + MetadataCreateIndexService.updateRemoteStoreSettings( + indexSettingsBuilder, + remoteStoreRepoNames.v1(), + remoteStoreRepoNames.v2() + ); + + // Overriding all existing customs with only path and path has algorithm based settings + Map newCustomRemoteStoreMetadata = new HashMap<>(); + if (remoteStorePathStrategyResolver != null) { + RemoteStorePathStrategy newPathStrategy = remoteStorePathStrategyResolver.get(); + newCustomRemoteStoreMetadata.put(RemoteStoreEnums.PathType.NAME, newPathStrategy.getType().name()); + if (Objects.nonNull(newPathStrategy.getHashAlgorithm())) { + newCustomRemoteStoreMetadata.put( + RemoteStoreEnums.PathHashAlgorithm.NAME, + newPathStrategy.getHashAlgorithm().name() + ); + } + } + + final IndexMetadata.Builder builder = IndexMetadata.builder(indexMetadata) + .settings(indexSettingsBuilder) + .putCustom(REMOTE_STORE_CUSTOM_KEY, newCustomRemoteStoreMetadata); + builder.settingsVersion(1 + indexMetadata.getVersion()); + metadataBuilder.put(builder); + } + } + return ClusterState.builder(state).metadata(metadataBuilder).build(); + } + + /** + * @param state + * @param seenShardRoutings + * @param discoveryNodes + * @return + */ + private ClusterState maybeAppendShardIdToSeededSet( + ClusterState state, + Set seenShardRoutings, + DiscoveryNodes discoveryNodes + ) { + final Metadata.Builder metadataBuilder = Metadata.builder(state.metadata()); + + for (ShardRouting shardRouting : seenShardRoutings) { + if (shardRouting.primary()) { + IndexMetadata currentIndexMetadata = metadataBuilder.get(shardRouting.getIndexName()); + Map existingCustomData = currentIndexMetadata.getCustomData(REMOTE_STORE_CUSTOM_KEY); + final IndexMetadata.Builder newIndexMetadataBuilder = IndexMetadata.builder(currentIndexMetadata); + if (existingCustomData == null) { + logger.debug("Index {} does not have any custom metadata. Initializing", shardRouting.getIndexName()); + buildCustomMetadataForShardSeeded(discoveryNodes, shardRouting, new HashMap<>(), newIndexMetadataBuilder); + metadataBuilder.put(newIndexMetadataBuilder); + } else if (INDEX_REMOTE_STORE_ENABLED_SETTING.get(currentIndexMetadata.getSettings()) == false + && existingCustomData.containsKey(REMOTE_STORE_SEEDED_SHARDS_KEY)) { + logger.debug("Index {} already has custom metadata {}", shardRouting.getIndexName(), existingCustomData); + buildCustomMetadataForShardSeeded( + discoveryNodes, + shardRouting, + new HashMap<>(existingCustomData), + newIndexMetadataBuilder + ); + metadataBuilder.put(newIndexMetadataBuilder); + } + } else { + logger.debug("No-Op for a replica shard copy"); + } + } + return ClusterState.builder(state).metadata(metadataBuilder).build(); + } + + /** + * + * @param discoveryNodes + * @param shardRouting + * @param existingCustomData + * @param newIndexMetadataBuilder + */ + private void buildCustomMetadataForShardSeeded( + DiscoveryNodes discoveryNodes, + ShardRouting shardRouting, + Map existingCustomData, + IndexMetadata.Builder newIndexMetadataBuilder + ) { + if (discoveryNodes.getNodes().get(shardRouting.currentNodeId()).isRemoteStoreNode()) { + Set seededShardIds = Strings.commaDelimitedListToSet(existingCustomData.get(REMOTE_STORE_SEEDED_SHARDS_KEY)); + if (seededShardIds.contains(shardRouting.shardId().toString()) == false) { + logger.info( + "Found primary shard copy {} seeded to remote node. Adding custom metadata", + shardRouting.shardId().toString() + ); + seededShardIds.add(shardRouting.shardId().toString()); + existingCustomData.put(REMOTE_STORE_SEEDED_SHARDS_KEY, Strings.collectionToCommaDelimitedString(seededShardIds)); + newIndexMetadataBuilder.putCustom(REMOTE_STORE_CUSTOM_KEY, existingCustomData); + } + } else { + Set seededShardIds = Strings.commaDelimitedListToSet(existingCustomData.get(REMOTE_STORE_SEEDED_SHARDS_KEY)); + if (seededShardIds.contains(shardRouting.shardId().toString()) == true) { + logger.info( + "Found remote seeded primary shard copy {} on docrep node. Removing custom metadata", + shardRouting.shardId().toString() + ); + seededShardIds.remove(shardRouting.shardId().toString()); + existingCustomData.put(REMOTE_STORE_SEEDED_SHARDS_KEY, Strings.collectionToCommaDelimitedString(seededShardIds)); + newIndexMetadataBuilder.putCustom(REMOTE_STORE_CUSTOM_KEY, existingCustomData); + } + } + } + + /** + * @param state + * @param index + * @param discoveryNodes + * @param indexMetadata + * @return + */ + private boolean needsRemoteIndexSettingsUpdate( + ClusterState state, + String index, + DiscoveryNodes discoveryNodes, + IndexMetadata indexMetadata + ) { + assert indexMetadata != null : "IndexMetadata for a STARTED shard cannot be null"; + IndexRoutingTable indexRoutingTable = state.routingTable().indicesRouting().get(index); + boolean allStartedShardsOnRemote = indexRoutingTable.shardsMatchingPredicateCount(ShardRouting::started) == indexRoutingTable + .shardsMatchingPredicateCount(shardRouting -> discoveryNodes.get(shardRouting.currentNodeId()).isRemoteStoreNode()); + return indexRoutingTable.allShardsStarted() + && allStartedShardsOnRemote + && IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING.get(indexMetadata.getSettings()) == false; + } } /** diff --git a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java index 80b78cfe154f1..ae11cdee87541 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java @@ -636,6 +636,7 @@ public static APIBlock readFrom(StreamInput input) throws IOException { static final String KEY_SYSTEM = "system"; public static final String KEY_PRIMARY_TERMS = "primary_terms"; public static final String REMOTE_STORE_CUSTOM_KEY = "remote_store"; + public static final String REMOTE_STORE_SEEDED_SHARDS_KEY = "remote_store_seeded_shards"; public static final String INDEX_STATE_FILE_PREFIX = "state-"; diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java index 0eba4d241f0fd..ea78e12ee33fe 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -1030,6 +1030,12 @@ public static void updateReplicationStrategy( settingsBuilder.put(SETTING_REPLICATION_TYPE, indexReplicationType); } + public static void updateRemoteStoreSettings(Settings.Builder settingsBuilder, String segmentRepository, String translogRepository) { + settingsBuilder.put(SETTING_REMOTE_STORE_ENABLED, true) + .put(SETTING_REMOTE_SEGMENT_STORE_REPOSITORY, segmentRepository) + .put(SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, translogRepository); + } + /** * Updates index settings to enable remote store by default based on node attributes * @param settingsBuilder index settings builder to be updated with relevant settings diff --git a/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java b/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java index faadc3f7583fb..e10bd0f014555 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java +++ b/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java @@ -266,6 +266,18 @@ public int primaryShardsActive() { return counter; } + /** + * Returns true if all shards for the index are in {@link ShardRoutingState#STARTED} state. Otherwise false. + */ + public boolean allShardsStarted() { + for (IndexShardRoutingTable shardRoutingTable : this) { + if (shardRoutingTable.allShardsStarted() == false) { + return false; + } + } + return true; + } + /** * Returns true if all primary shards are in * {@link ShardRoutingState#UNASSIGNED} state. Otherwise false. diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index 388de65ca58a1..b9a784a5b4abe 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -1913,7 +1913,6 @@ public void setDocIdFuzzySetFalsePositiveProbability(double docIdFuzzySetFalsePo private RemoteStorePathStrategy determineRemoteStorePathStrategy() { Map remoteCustomData = indexMetadata.getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY); - assert remoteCustomData == null || remoteCustomData.containsKey(PathType.NAME); if (remoteCustomData != null && remoteCustomData.containsKey(PathType.NAME)) { PathType pathType = PathType.parseString(remoteCustomData.get(PathType.NAME)); String hashAlgoStr = remoteCustomData.get(PathHashAlgorithm.NAME); diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java index 4d1d98334c3c4..9320dcae4d321 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java @@ -8,7 +8,14 @@ package org.opensearch.index.remote; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.common.collect.Tuple; +import org.opensearch.common.settings.Settings; +import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; +import org.opensearch.node.remotestore.RemoteStoreNodeService; import java.nio.ByteBuffer; import java.util.Arrays; @@ -17,6 +24,7 @@ import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Optional; import java.util.function.Function; /** @@ -26,6 +34,7 @@ */ public class RemoteStoreUtils { public static final int LONG_MAX_LENGTH = String.valueOf(Long.MAX_VALUE).length(); + private static final Logger logger = LogManager.getLogger(RemoteStoreUtils.class); /** * URL safe base 64 character set. This must not be changed as this is used in deriving the base64 equivalent of binary. @@ -146,4 +155,25 @@ static String longToCompositeBase64AndBinaryEncoding(long value, int len) { assert base64DecimalValue >= 0 && base64DecimalValue < 64; return URL_BASE64_CHARSET[base64DecimalValue] + binaryPart; } + + public static boolean ongoingDocrepToRemoteMigration(Settings settings) { + return RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING.get( + settings + ) == RemoteStoreNodeService.CompatibilityMode.MIXED + && RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING.get(settings) == RemoteStoreNodeService.Direction.REMOTE_STORE; + } + + public static Tuple getRemoteStoreRepositoryNames(DiscoveryNodes discoveryNodes) { + Optional remoteNode = discoveryNodes.getNodes() + .values() + .stream() + .filter(DiscoveryNode::isRemoteStoreNode) + .findFirst(); + assert remoteNode.isPresent() : "Cannot fetch remote store repository names as no remote nodes are present in the cluster"; + Map remoteNodeAttributes = remoteNode.get().getAttributes(); + return new Tuple<>( + remoteNodeAttributes.get(RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY), + remoteNodeAttributes.get(RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY) + ); + } } diff --git a/server/src/test/java/org/opensearch/cluster/action/shard/ShardStartedClusterStateTaskExecutorTests.java b/server/src/test/java/org/opensearch/cluster/action/shard/ShardStartedClusterStateTaskExecutorTests.java index 3dee3507ae71a..902b76c2a4c3a 100644 --- a/server/src/test/java/org/opensearch/cluster/action/shard/ShardStartedClusterStateTaskExecutorTests.java +++ b/server/src/test/java/org/opensearch/cluster/action/shard/ShardStartedClusterStateTaskExecutorTests.java @@ -46,6 +46,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.node.remotestore.RemoteStoreNodeService; import java.util.ArrayList; import java.util.Collections; @@ -283,6 +284,15 @@ public void testPrimaryTermsMismatch() throws Exception { } } + public void testAddRemoteIndexSettingsDoesNotExecuteWithoutMixedModeSettings() throws Exception { + final String indexName = "test-remote-migration"; + final ClusterState clusterState = state(indexName, randomBoolean(), ShardRoutingState.INITIALIZING, ShardRoutingState.INITIALIZING); + assertSame( + RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING.get(clusterState.getMetadata().settings()), + RemoteStoreNodeService.CompatibilityMode.STRICT + ); + } + private ClusterStateTaskExecutor.ClusterTasksResult executeTasks(final ClusterState state, final List tasks) throws Exception { final ClusterStateTaskExecutor.ClusterTasksResult result = executor.execute(state, tasks); 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 286f0a1d91b4c..7a96dd059508c 100644 --- a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java @@ -2405,6 +2405,12 @@ protected String primaryNodeName(String indexName) { return clusterState.getRoutingNodes().node(nodeId).node().getName(); } + protected String primaryNodeName(String indexName, int shardId) { + ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); + String nodeId = clusterState.getRoutingTable().index(indexName).shard(shardId).primaryShard().currentNodeId(); + return clusterState.getRoutingNodes().node(nodeId).node().getName(); + } + protected String replicaNodeName(String indexName) { ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); String nodeId = clusterState.getRoutingTable().index(indexName).shard(0).replicaShards().get(0).currentNodeId(); From 614d3d35b696ae44fe36bfbd42aad6a450483ab4 Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Wed, 17 Apr 2024 00:43:53 +0530 Subject: [PATCH 02/24] Adding UTs and removing seeding metadata change Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../RemoteMigrationIndexMetadataChangeIT.java | 132 ++++---------- .../action/shard/ShardStateAction.java | 143 +++------------ .../metadata/MetadataCreateIndexService.java | 1 + .../org/opensearch/index/IndexSettings.java | 1 + .../index/remote/RemoteStoreUtils.java | 25 ++- ...dStartedClusterStateTaskExecutorTests.java | 171 +++++++++++++++++- .../ClusterStateCreationUtils.java | 119 ++++++++++-- .../index/shard/IndexShardTestUtils.java | 5 +- 8 files changed, 361 insertions(+), 236 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataChangeIT.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataChangeIT.java index 097283e9a363f..4fc91ba114dd3 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataChangeIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataChangeIT.java @@ -9,29 +9,23 @@ package org.opensearch.remotemigration; import org.opensearch.action.admin.indices.settings.get.GetSettingsResponse; -import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.routing.allocation.command.MoveAllocationCommand; import org.opensearch.common.settings.Settings; -import org.opensearch.core.common.Strings; -import org.opensearch.index.remote.RemoteStoreEnums; import org.opensearch.plugins.Plugin; import org.opensearch.remotestore.multipart.mocks.MockFsRepositoryPlugin; import org.opensearch.test.InternalSettingsPlugin; import org.opensearch.test.OpenSearchIntegTestCase; -import org.opensearch.test.junit.annotations.TestLogging; import org.opensearch.test.transport.MockTransportService; import java.util.Collection; -import java.util.HashSet; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.opensearch.cluster.metadata.IndexMetadata.REMOTE_STORE_CUSTOM_KEY; -import static org.opensearch.cluster.metadata.IndexMetadata.REMOTE_STORE_SEEDED_SHARDS_KEY; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; import static org.opensearch.cluster.routing.allocation.decider.EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; @@ -59,6 +53,10 @@ public void testIndexSettingsUpdateAfterIndexMovedToRemoteThroughAllocationExclu .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) .build(); createIndexAndAssertDocrepProperties(indexName, oneReplica); + + AsyncIndexingService asyncIndexingService = new AsyncIndexingService(indexName); + asyncIndexingService.startIndexing(); + String replicationType; GetSettingsResponse response; String remoteStoreEnabled; @@ -71,6 +69,7 @@ public void testIndexSettingsUpdateAfterIndexMovedToRemoteThroughAllocationExclu excludeNodeSet("type", "docrep"); waitForRelocation(); waitNoPendingTasksOnAll(); + asyncIndexingService.stopIndexing(); response = internalCluster().client().admin().indices().prepareGetSettings(indexName).get(); remoteStoreEnabled = response.getSetting(indexName, SETTING_REMOTE_STORE_ENABLED); @@ -93,6 +92,12 @@ public void testIndexSettingsUpdateAfterIndexMovedToRemoteThroughManualReroute() createIndexAndAssertDocrepProperties(indexName1, oneReplica); createIndexAndAssertDocrepProperties(indexName2, oneReplica); + AsyncIndexingService indexOne = new AsyncIndexingService(indexName1); + indexOne.startIndexing(); + + AsyncIndexingService indexTwo = new AsyncIndexingService(indexName2); + indexTwo.startIndexing(); + initDocRepToRemoteMigration(); stopShardRebalancing(); @@ -137,6 +142,9 @@ public void testIndexSettingsUpdateAfterIndexMovedToRemoteThroughManualReroute() waitForRelocation(); waitNoPendingTasksOnAll(); + indexOne.stopIndexing(); + indexTwo.stopIndexing(); + assertRemoteProperties(indexName1); assertDocrepProperties(indexName2); } @@ -149,15 +157,15 @@ public void testIndexSettingsUpdatedOnlyForMigratingIndex() throws Exception { internalCluster().validateClusterFormed(); String indexName = "migration-index"; - createIndex( - indexName, - Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1).build() - ); - indexBulk(indexName, 100); - refresh(indexName); - ensureGreen(indexName); + Settings oneReplica = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .build(); + createIndexAndAssertDocrepProperties(indexName, oneReplica); + + AsyncIndexingService indexingService = new AsyncIndexingService(indexName); + indexingService.startIndexing(); - assertDocrepProperties(indexName); long initalMetadataVersion = internalCluster().client() .admin() .cluster() @@ -177,6 +185,8 @@ public void testIndexSettingsUpdatedOnlyForMigratingIndex() throws Exception { waitForRelocation(); waitNoPendingTasksOnAll(); + indexingService.stopIndexing(); + assertRemoteProperties(indexName); assertTrue( initalMetadataVersion < internalCluster().client() @@ -218,88 +228,8 @@ initalMetadataVersion < internalCluster().client() ); } - @TestLogging(reason = "", value = "org.opensearch.cluster.metadata:TRACE,org.opensearch.cluster.action.shard:TRACE") - public void testCustomSeedingMetadata() throws Exception { - String indexName = "custom-seeding-metadata-index"; - internalCluster().startClusterManagerOnlyNode(); - List docRepNodeNames = internalCluster().startNodes(2, Settings.builder().put("node.attr._type", "docrep").build()); - - // create index with 2 primaries - Settings zeroReplica = Settings.builder().put("number_of_replicas", 0).put("number_of_shards", 4).build(); - createIndex(indexName, zeroReplica); - indexBulk(indexName, 100); - ensureGreen(indexName); - - stopShardRebalancing(); - initDocRepToRemoteMigration(); - // add remote node in mixed mode cluster - addRemote = true; - List remoteNodeNames = internalCluster().startNodes(2, Settings.builder().put("node.attr._type", "remote").build()); - internalCluster().validateClusterFormed(); - - assertEquals( - internalCluster().client() - .admin() - .cluster() - .prepareGetRepositories(REPOSITORY_NAME, REPOSITORY_2_NAME) - .get() - .repositories() - .size(), - 2 - ); - - assertAcked( - client().admin() - .cluster() - .prepareReroute() - .add(new MoveAllocationCommand(indexName, 0, primaryNodeName(indexName, 0), remoteNodeNames.get(0))) - .add(new MoveAllocationCommand(indexName, 1, primaryNodeName(indexName, 1), remoteNodeNames.get(0))) - .add(new MoveAllocationCommand(indexName, 2, primaryNodeName(indexName, 2), remoteNodeNames.get(1))) - .execute() - .actionGet() - ); - waitForRelocation(); - ensureGreen(indexName); - waitNoPendingTasksOnAll(); - - ClusterState clusterState = internalCluster().client().admin().cluster().prepareState().get().getState(); - IndexMetadata indexMetadata = clusterState.metadata().index(indexName); - HashSet seededShards = new HashSet<>( - Strings.commaDelimitedListToSet( - indexMetadata.getCustomData(REMOTE_STORE_CUSTOM_KEY).get(IndexMetadata.REMOTE_STORE_SEEDED_SHARDS_KEY) - ) - ); - HashSet expectedSeededShards = new HashSet<>(); - for (int i = 0; i < 3; i++) { - expectedSeededShards.add("[" + indexName + "][" + i + "]"); - } - assertEquals(expectedSeededShards, seededShards); - - assertAcked( - client().admin() - .cluster() - .prepareReroute() - .add(new MoveAllocationCommand(indexName, 3, primaryNodeName(indexName, 3), remoteNodeNames.get(1))) - .execute() - .actionGet() - ); - waitForRelocation(); - ensureGreen(indexName); - waitNoPendingTasksOnAll(); - - clusterState = internalCluster().client().admin().cluster().prepareState().get().getState(); - indexMetadata = clusterState.metadata().index(indexName); - Map currentCustomData = indexMetadata.getCustomData(REMOTE_STORE_CUSTOM_KEY); - assertFalse(currentCustomData.containsKey(REMOTE_STORE_SEEDED_SHARDS_KEY)); - assertTrue( - currentCustomData.containsKey(RemoteStoreEnums.PathType.NAME) - && currentCustomData.containsKey(RemoteStoreEnums.PathHashAlgorithm.NAME) - ); - } - - private void createIndexAndAssertDocrepProperties(String index, Settings settings) throws Exception { + private void createIndexAndAssertDocrepProperties(String index, Settings settings) { createIndex(index, settings); - indexBulk(index, 100); refresh(index); ensureGreen(index); assertDocrepProperties(index); @@ -309,7 +239,11 @@ private void assertDocrepProperties(String index) { GetSettingsResponse response = internalCluster().client().admin().indices().prepareGetSettings(index).get(); String remoteStoreEnabled = response.getSetting(index, SETTING_REMOTE_STORE_ENABLED); String replicationType = response.getSetting(index, SETTING_REPLICATION_TYPE); + String segmentRepo = response.getSetting(index, SETTING_REMOTE_SEGMENT_STORE_REPOSITORY); + String translogRepo = response.getSetting(index, SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY); assertNull(remoteStoreEnabled); + assertNull(segmentRepo); + assertNull(translogRepo); assertEquals(replicationType, "DOCUMENT"); } @@ -317,7 +251,11 @@ private void assertRemoteProperties(String index) { GetSettingsResponse response = internalCluster().client().admin().indices().prepareGetSettings(index).get(); String remoteStoreEnabled = response.getSetting(index, SETTING_REMOTE_STORE_ENABLED); String replicationType = response.getSetting(index, SETTING_REPLICATION_TYPE); + String segmentRepo = response.getSetting(index, SETTING_REMOTE_SEGMENT_STORE_REPOSITORY); + String translogRepo = response.getSetting(index, SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY); assertEquals(remoteStoreEnabled, "true"); + assertNotNull(segmentRepo); + assertNotNull(translogRepo); assertEquals(replicationType, "SEGMENT"); } diff --git a/server/src/main/java/org/opensearch/cluster/action/shard/ShardStateAction.java b/server/src/main/java/org/opensearch/cluster/action/shard/ShardStateAction.java index 4c5578261df2e..78be68a9ea2a5 100644 --- a/server/src/main/java/org/opensearch/cluster/action/shard/ShardStateAction.java +++ b/server/src/main/java/org/opensearch/cluster/action/shard/ShardStateAction.java @@ -67,7 +67,6 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.core.action.ActionListener; -import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.index.shard.ShardId; @@ -76,7 +75,6 @@ import org.opensearch.index.remote.RemoteStorePathStrategy; import org.opensearch.index.remote.RemoteStorePathStrategyResolver; import org.opensearch.index.remote.RemoteStoreUtils; -import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.node.NodeClosedException; import org.opensearch.tasks.Task; import org.opensearch.threadpool.ThreadPool; @@ -103,10 +101,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; -import static org.opensearch.cluster.metadata.IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING; import static org.opensearch.cluster.metadata.IndexMetadata.REMOTE_STORE_CUSTOM_KEY; -import static org.opensearch.cluster.metadata.IndexMetadata.REMOTE_STORE_SEEDED_SHARDS_KEY; -import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; import static org.opensearch.index.remote.RemoteStoreUtils.ongoingDocrepToRemoteMigration; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteDataAttributePresent; @@ -839,13 +834,7 @@ public ClusterTasksResult execute(ClusterState currentState, maybeUpdatedState = allocationService.applyStartedShards(currentState, shardRoutingsToBeApplied); // Run remote store migration based tasks if (ongoingDocrepToRemoteMigration(currentState.metadata().settings())) { - DiscoveryNodes discoveryNodes = currentState.getNodes(); - if (discoveryNodes.getNodes().values().stream().noneMatch(DiscoveryNode::isRemoteStoreNode)) { - logger.debug("Cluster is in mixed mode but does not have any remote enabled nodes. No-Op"); - } else { - maybeUpdatedState = maybeAppendShardIdToSeededSet(maybeUpdatedState, seenShardRoutings, discoveryNodes); - maybeUpdatedState = maybeAddRemoteIndexSettings(maybeUpdatedState, seenShardRoutings, discoveryNodes); - } + maybeUpdatedState = maybeAddRemoteIndexSettings(maybeUpdatedState, seenShardRoutings); } builder.successes(tasksToBeApplied); } catch (Exception e) { @@ -878,16 +867,16 @@ public void clusterStatePublished(ClusterChangedEvent clusterChangedEvent) { } /** - * @param state - * @param seenShardRoutings - * @param discoveryNodes - * @return + * During docrep to remote store migration, applies the following remote store based index settings + * once all shards of an index have moved over to remote store enabled nodes + *
+ * Also appends the requisite Remote Store Path based custom metadata to the existing index metadata + * + * @param state updated state from allocation service + * @param seenShardRoutings {@link ShardRouting} set being processed in the task executor + * @return Mutated {@link ClusterState} with the remote store based settings applied */ - private ClusterState maybeAddRemoteIndexSettings( - ClusterState state, - Set seenShardRoutings, - DiscoveryNodes discoveryNodes - ) { + ClusterState maybeAddRemoteIndexSettings(ClusterState state, Set seenShardRoutings) { final Metadata.Builder metadataBuilder = Metadata.builder(state.metadata()); HashSet indexNames = seenShardRoutings.stream() .map(ShardRouting::getIndexName) @@ -895,23 +884,27 @@ private ClusterState maybeAddRemoteIndexSettings( logger.debug( "Cluster is going through a docrep to remote store migration. Checking if any index has completely moved over to remote nodes" ); + DiscoveryNodes discoveryNodes = state.getNodes(); Tuple remoteStoreRepoNames = RemoteStoreUtils.getRemoteStoreRepositoryNames(discoveryNodes); for (String index : indexNames) { - if (needsRemoteIndexSettingsUpdate(state, index, discoveryNodes, state.metadata().index(index))) { + if (needsRemoteIndexSettingsUpdate( + state.routingTable().indicesRouting().get(index), + discoveryNodes, + state.metadata().index(index) + )) { logger.info( "Index {} does not have remote store based index settings but all shards have moved to remote enabled nodes. Applying remote store settings to the index", index ); final IndexMetadata indexMetadata = metadataBuilder.get(index); Settings.Builder indexSettingsBuilder = Settings.builder().put(indexMetadata.getSettings()); - indexSettingsBuilder.put(SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT); MetadataCreateIndexService.updateRemoteStoreSettings( indexSettingsBuilder, remoteStoreRepoNames.v1(), remoteStoreRepoNames.v2() ); - // Overriding all existing customs with only path and path has algorithm based settings + // Overriding all existing customs with only path and path hash algorithm based settings Map newCustomRemoteStoreMetadata = new HashMap<>(); if (remoteStorePathStrategyResolver != null) { RemoteStorePathStrategy newPathStrategy = remoteStorePathStrategyResolver.get(); @@ -935,103 +928,25 @@ private ClusterState maybeAddRemoteIndexSettings( } /** - * @param state - * @param seenShardRoutings - * @param discoveryNodes - * @return - */ - private ClusterState maybeAppendShardIdToSeededSet( - ClusterState state, - Set seenShardRoutings, - DiscoveryNodes discoveryNodes - ) { - final Metadata.Builder metadataBuilder = Metadata.builder(state.metadata()); - - for (ShardRouting shardRouting : seenShardRoutings) { - if (shardRouting.primary()) { - IndexMetadata currentIndexMetadata = metadataBuilder.get(shardRouting.getIndexName()); - Map existingCustomData = currentIndexMetadata.getCustomData(REMOTE_STORE_CUSTOM_KEY); - final IndexMetadata.Builder newIndexMetadataBuilder = IndexMetadata.builder(currentIndexMetadata); - if (existingCustomData == null) { - logger.debug("Index {} does not have any custom metadata. Initializing", shardRouting.getIndexName()); - buildCustomMetadataForShardSeeded(discoveryNodes, shardRouting, new HashMap<>(), newIndexMetadataBuilder); - metadataBuilder.put(newIndexMetadataBuilder); - } else if (INDEX_REMOTE_STORE_ENABLED_SETTING.get(currentIndexMetadata.getSettings()) == false - && existingCustomData.containsKey(REMOTE_STORE_SEEDED_SHARDS_KEY)) { - logger.debug("Index {} already has custom metadata {}", shardRouting.getIndexName(), existingCustomData); - buildCustomMetadataForShardSeeded( - discoveryNodes, - shardRouting, - new HashMap<>(existingCustomData), - newIndexMetadataBuilder - ); - metadataBuilder.put(newIndexMetadataBuilder); - } - } else { - logger.debug("No-Op for a replica shard copy"); - } - } - return ClusterState.builder(state).metadata(metadataBuilder).build(); - } - - /** + * Returns true iff all shards for the index are in `STARTED` state + * all those `STARTED` shard copies are present in remote store enabled nodes * - * @param discoveryNodes - * @param shardRouting - * @param existingCustomData - * @param newIndexMetadataBuilder - */ - private void buildCustomMetadataForShardSeeded( - DiscoveryNodes discoveryNodes, - ShardRouting shardRouting, - Map existingCustomData, - IndexMetadata.Builder newIndexMetadataBuilder - ) { - if (discoveryNodes.getNodes().get(shardRouting.currentNodeId()).isRemoteStoreNode()) { - Set seededShardIds = Strings.commaDelimitedListToSet(existingCustomData.get(REMOTE_STORE_SEEDED_SHARDS_KEY)); - if (seededShardIds.contains(shardRouting.shardId().toString()) == false) { - logger.info( - "Found primary shard copy {} seeded to remote node. Adding custom metadata", - shardRouting.shardId().toString() - ); - seededShardIds.add(shardRouting.shardId().toString()); - existingCustomData.put(REMOTE_STORE_SEEDED_SHARDS_KEY, Strings.collectionToCommaDelimitedString(seededShardIds)); - newIndexMetadataBuilder.putCustom(REMOTE_STORE_CUSTOM_KEY, existingCustomData); - } - } else { - Set seededShardIds = Strings.commaDelimitedListToSet(existingCustomData.get(REMOTE_STORE_SEEDED_SHARDS_KEY)); - if (seededShardIds.contains(shardRouting.shardId().toString()) == true) { - logger.info( - "Found remote seeded primary shard copy {} on docrep node. Removing custom metadata", - shardRouting.shardId().toString() - ); - seededShardIds.remove(shardRouting.shardId().toString()); - existingCustomData.put(REMOTE_STORE_SEEDED_SHARDS_KEY, Strings.collectionToCommaDelimitedString(seededShardIds)); - newIndexMetadataBuilder.putCustom(REMOTE_STORE_CUSTOM_KEY, existingCustomData); - } - } - } - - /** - * @param state - * @param index - * @param discoveryNodes - * @param indexMetadata - * @return + * @param indexRoutingTable current {@link IndexRoutingTable} from cluster state + * @param discoveryNodes set of discovery nodes from cluster state + * @param indexMetadata current {@link IndexMetadata} from cluster state + * @return true or false depending on the met conditions */ private boolean needsRemoteIndexSettingsUpdate( - ClusterState state, - String index, + IndexRoutingTable indexRoutingTable, DiscoveryNodes discoveryNodes, IndexMetadata indexMetadata ) { assert indexMetadata != null : "IndexMetadata for a STARTED shard cannot be null"; - IndexRoutingTable indexRoutingTable = state.routingTable().indicesRouting().get(index); - boolean allStartedShardsOnRemote = indexRoutingTable.shardsMatchingPredicateCount(ShardRouting::started) == indexRoutingTable - .shardsMatchingPredicateCount(shardRouting -> discoveryNodes.get(shardRouting.currentNodeId()).isRemoteStoreNode()); - return indexRoutingTable.allShardsStarted() - && allStartedShardsOnRemote - && IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING.get(indexMetadata.getSettings()) == false; + boolean allStartedShardsOnRemote = indexRoutingTable.allShardsStarted() + && (indexRoutingTable.shardsMatchingPredicateCount(ShardRouting::started) == indexRoutingTable.shardsMatchingPredicateCount( + shardRouting -> discoveryNodes.get(shardRouting.currentNodeId()).isRemoteStoreNode() + )); + return allStartedShardsOnRemote && IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING.get(indexMetadata.getSettings()) == false; } } diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java index ea78e12ee33fe..d8d5456f79ada 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -1032,6 +1032,7 @@ public static void updateReplicationStrategy( public static void updateRemoteStoreSettings(Settings.Builder settingsBuilder, String segmentRepository, String translogRepository) { settingsBuilder.put(SETTING_REMOTE_STORE_ENABLED, true) + .put(SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) .put(SETTING_REMOTE_SEGMENT_STORE_REPOSITORY, segmentRepository) .put(SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, translogRepository); } diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index b9a784a5b4abe..388de65ca58a1 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -1913,6 +1913,7 @@ public void setDocIdFuzzySetFalsePositiveProbability(double docIdFuzzySetFalsePo private RemoteStorePathStrategy determineRemoteStorePathStrategy() { Map remoteCustomData = indexMetadata.getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY); + assert remoteCustomData == null || remoteCustomData.containsKey(PathType.NAME); if (remoteCustomData != null && remoteCustomData.containsKey(PathType.NAME)) { PathType pathType = PathType.parseString(remoteCustomData.get(PathType.NAME)); String hashAlgoStr = remoteCustomData.get(PathHashAlgorithm.NAME); diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java index 9320dcae4d321..12105fce6a1b8 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java @@ -8,8 +8,6 @@ package org.opensearch.index.remote; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.common.collect.Tuple; @@ -34,7 +32,11 @@ */ public class RemoteStoreUtils { public static final int LONG_MAX_LENGTH = String.valueOf(Long.MAX_VALUE).length(); - private static final Logger logger = LogManager.getLogger(RemoteStoreUtils.class); + + /** + * URL safe base 64 character set. This must not be changed as this is used in deriving the base64 equivalent of binary. + */ + static final char[] URL_BASE64_CHARSET = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789-_".toCharArray(); /** * URL safe base 64 character set. This must not be changed as this is used in deriving the base64 equivalent of binary. @@ -156,6 +158,18 @@ static String longToCompositeBase64AndBinaryEncoding(long value, int len) { return URL_BASE64_CHARSET[base64DecimalValue] + binaryPart; } + /** + * Returns true iff current cluster settings have: + *
+ * - remote_store.compatibility_mode set to mixed + *
+ * - migration.direction set to remote_store + *
+ * false otherwise + * + * @param settings Current Cluster Settings + * @return true or false depending upon the above-mentioned condition + */ public static boolean ongoingDocrepToRemoteMigration(Settings settings) { return RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING.get( settings @@ -163,6 +177,11 @@ public static boolean ongoingDocrepToRemoteMigration(Settings settings) { && RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING.get(settings) == RemoteStoreNodeService.Direction.REMOTE_STORE; } + /** + * Fetches segment and translog repository names from remote store node attributes + * @param discoveryNodes Current set of {@link DiscoveryNodes} in the cluster + * @return {@link Tuple} with segment repository name as first element and translog repository name as second element + */ public static Tuple getRemoteStoreRepositoryNames(DiscoveryNodes discoveryNodes) { Optional remoteNode = discoveryNodes.getNodes() .values() diff --git a/server/src/test/java/org/opensearch/cluster/action/shard/ShardStartedClusterStateTaskExecutorTests.java b/server/src/test/java/org/opensearch/cluster/action/shard/ShardStartedClusterStateTaskExecutorTests.java index 902b76c2a4c3a..0cf83055bab6d 100644 --- a/server/src/test/java/org/opensearch/cluster/action/shard/ShardStartedClusterStateTaskExecutorTests.java +++ b/server/src/test/java/org/opensearch/cluster/action/shard/ShardStartedClusterStateTaskExecutorTests.java @@ -32,25 +32,34 @@ package org.opensearch.cluster.action.shard; +import org.apache.logging.log4j.Logger; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.ClusterStateTaskExecutor; import org.opensearch.cluster.OpenSearchAllocationTestCase; import org.opensearch.cluster.action.shard.ShardStateAction.StartedShardEntry; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.cluster.routing.IndexShardRoutingTable; +import org.opensearch.cluster.routing.RerouteService; +import org.opensearch.cluster.routing.RoutingTable; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.ShardRoutingState; +import org.opensearch.cluster.routing.TestShardRouting; import org.opensearch.cluster.routing.allocation.AllocationService; import org.opensearch.common.Priority; import org.opensearch.common.settings.Settings; import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.shard.ShardId; -import org.opensearch.node.remotestore.RemoteStoreNodeService; +import org.opensearch.index.shard.IndexShardTestUtils; import java.util.ArrayList; import java.util.Collections; +import java.util.HashSet; import java.util.List; +import java.util.UUID; +import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; @@ -59,11 +68,17 @@ import static org.opensearch.action.support.replication.ClusterStateCreationUtils.state; import static org.opensearch.action.support.replication.ClusterStateCreationUtils.stateWithActivePrimary; import static org.opensearch.action.support.replication.ClusterStateCreationUtils.stateWithAssignedPrimariesAndReplicas; +import static org.opensearch.action.support.replication.ClusterStateCreationUtils.stateWithMixedNodes; import static org.opensearch.action.support.replication.ClusterStateCreationUtils.stateWithNoShard; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; import static org.opensearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES_SETTING; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; +import static org.mockito.Mockito.mock; public class ShardStartedClusterStateTaskExecutorTests extends OpenSearchAllocationTestCase { @@ -284,13 +299,109 @@ public void testPrimaryTermsMismatch() throws Exception { } } - public void testAddRemoteIndexSettingsDoesNotExecuteWithoutMixedModeSettings() throws Exception { - final String indexName = "test-remote-migration"; - final ClusterState clusterState = state(indexName, randomBoolean(), ShardRoutingState.INITIALIZING, ShardRoutingState.INITIALIZING); - assertSame( - RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING.get(clusterState.getMetadata().settings()), - RemoteStoreNodeService.CompatibilityMode.STRICT - ); + public void testMaybeAddRemoteIndexSettings() { + final String indexName = "test-add-remote-index-settings"; + final ShardId shardId = new ShardId(indexName, UUID.randomUUID().toString(), 0); + final int numberOfNodes = 3; + final int numberOfShards = 1; + final int numberOfReplicas = 2; + final ShardStateAction.ShardStartedClusterStateTaskExecutor mockShardStartedClusterStateTaskExecutor = + new ShardStateAction.ShardStartedClusterStateTaskExecutor( + mock(AllocationService.class), + mock(RerouteService.class), + mock(Supplier.class), + mock(Logger.class) + ); + // Index Settings are not updated when all shard copies are in docrep nodes + { + ClusterState initialClusterState = stateWithMixedNodes( + numberOfNodes, + numberOfNodes, + true, + List.of(indexName).toArray(String[]::new), + numberOfShards, + numberOfReplicas, + ShardRoutingState.STARTED + ); + assertRemoteStoreIndexSettingsAreAbsent(initialClusterState, indexName); + List docrepNodeSet = initialClusterState.getNodes() + .getDataNodes() + .values() + .stream() + .filter(discoveryNode -> discoveryNode.isRemoteStoreNode() == false) + .collect(Collectors.toList()); + List seenShardRoutings = generateMockShardRoutings(shardId, docrepNodeSet); + ClusterState resultingState = mockShardStartedClusterStateTaskExecutor.maybeAddRemoteIndexSettings( + loadShardRoutingsIntoClusterState(shardId, initialClusterState, seenShardRoutings), + new HashSet<>(seenShardRoutings) + ); + assertRemoteStoreIndexSettingsAreAbsent(resultingState, indexName); + } + // Index Settings are updated when all shard copies are in remote nodes and existing metadata + // does not have remote enabled settings + { + ClusterState initialClusterState = stateWithMixedNodes( + numberOfNodes, + numberOfNodes, + true, + List.of(indexName).toArray(String[]::new), + numberOfShards, + numberOfReplicas, + ShardRoutingState.STARTED + ); + assertRemoteStoreIndexSettingsAreAbsent(initialClusterState, indexName); + List remoteNodeSet = initialClusterState.getNodes() + .getDataNodes() + .values() + .stream() + .filter(DiscoveryNode::isRemoteStoreNode) + .collect(Collectors.toList()); + List seenShardRoutings = generateMockShardRoutings(shardId, remoteNodeSet); + + ClusterState resultingState = mockShardStartedClusterStateTaskExecutor.maybeAddRemoteIndexSettings( + loadShardRoutingsIntoClusterState(shardId, initialClusterState, seenShardRoutings), + new HashSet<>(seenShardRoutings) + ); + assertRemoteStoreIndexSettingsArePresent(resultingState, indexName); + } + // Index Settings are updated when some shard copies are in remote nodes while others are still on docrep + { + ClusterState initialClusterState = stateWithMixedNodes( + numberOfNodes, + numberOfNodes, + true, + List.of(indexName).toArray(String[]::new), + numberOfShards, + numberOfReplicas, + ShardRoutingState.STARTED + ); + assertRemoteStoreIndexSettingsAreAbsent(initialClusterState, indexName); + List mixedNodeSet = new ArrayList<>(); + mixedNodeSet.addAll( + initialClusterState.getNodes() + .getDataNodes() + .values() + .stream() + .filter(DiscoveryNode::isRemoteStoreNode) + .limit(2) + .collect(Collectors.toList()) + ); + mixedNodeSet.addAll( + initialClusterState.getNodes() + .getDataNodes() + .values() + .stream() + .filter(discoveryNode -> discoveryNode.isRemoteStoreNode() == false) + .limit(1) + .collect(Collectors.toList()) + ); + List seenShardRoutings = generateMockShardRoutings(shardId, mixedNodeSet); + ClusterState resultingState = mockShardStartedClusterStateTaskExecutor.maybeAddRemoteIndexSettings( + loadShardRoutingsIntoClusterState(shardId, initialClusterState, seenShardRoutings), + new HashSet<>(seenShardRoutings) + ); + assertRemoteStoreIndexSettingsAreAbsent(resultingState, indexName); + } } private ClusterStateTaskExecutor.ClusterTasksResult executeTasks(final ClusterState state, final List tasks) @@ -299,4 +410,48 @@ private ClusterStateTaskExecutor.ClusterTasksResult executeTasks(final ClusterSt assertThat(result, notNullValue()); return result; } + + private static void assertRemoteStoreIndexSettingsArePresent(ClusterState resultingState, String indexName) { + Settings resultingIndexSettings = resultingState.metadata().index(indexName).getSettings(); + assertEquals("true", resultingIndexSettings.get(SETTING_REMOTE_STORE_ENABLED)); + assertEquals("SEGMENT", resultingIndexSettings.get(SETTING_REPLICATION_TYPE)); + assertEquals(IndexShardTestUtils.MOCK_SEGMENT_REPO_NAME, resultingIndexSettings.get(SETTING_REMOTE_SEGMENT_STORE_REPOSITORY)); + assertEquals(IndexShardTestUtils.MOCK_TLOG_REPO_NAME, resultingIndexSettings.get(SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY)); + } + + private static void assertRemoteStoreIndexSettingsAreAbsent(ClusterState initialClusterState, String indexName) { + Settings initialIndexSettings = initialClusterState.metadata().index(indexName).getSettings(); + assertNull(initialIndexSettings.get(SETTING_REMOTE_STORE_ENABLED)); + assertNull(initialIndexSettings.get(SETTING_REPLICATION_TYPE)); + assertNull(initialIndexSettings.get(SETTING_REMOTE_SEGMENT_STORE_REPOSITORY)); + assertNull(initialIndexSettings.get(SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY)); + } + + private List generateMockShardRoutings(ShardId shardId, List nodeSet) { + List seenShardRoutings = new ArrayList<>(nodeSet.size()); + for (DiscoveryNode node : nodeSet) { + if (node.getId().contains("node_0")) { + seenShardRoutings.add(TestShardRouting.newShardRouting(shardId, node.getId(), true, ShardRoutingState.STARTED)); + } else { + seenShardRoutings.add(TestShardRouting.newShardRouting(shardId, node.getId(), false, ShardRoutingState.STARTED)); + } + } + return seenShardRoutings; + } + + private ClusterState loadShardRoutingsIntoClusterState(ShardId shardId, ClusterState currentState, List shardRoutings) { + ClusterState.Builder clusterStateBuilder = ClusterState.builder(currentState); + RoutingTable.Builder routingTableBuilder = RoutingTable.builder(currentState.routingTable()); + IndexRoutingTable.Builder indexRoutingTableBuilder = IndexRoutingTable.builder( + currentState.metadata().index(shardId.getIndexName()).getIndex() + ); + IndexShardRoutingTable.Builder indexShardRoutingBuilder = new IndexShardRoutingTable.Builder(shardId); + + for (ShardRouting shardRouting : shardRoutings) { + indexShardRoutingBuilder.addShard(shardRouting); + } + return clusterStateBuilder.routingTable( + routingTableBuilder.add(indexRoutingTableBuilder.addIndexShard(indexShardRoutingBuilder.build()).build()).build() + ).build(); + } } diff --git a/test/framework/src/main/java/org/opensearch/action/support/replication/ClusterStateCreationUtils.java b/test/framework/src/main/java/org/opensearch/action/support/replication/ClusterStateCreationUtils.java index 182b2c9288a3d..4a575d4de7679 100644 --- a/test/framework/src/main/java/org/opensearch/action/support/replication/ClusterStateCreationUtils.java +++ b/test/framework/src/main/java/org/opensearch/action/support/replication/ClusterStateCreationUtils.java @@ -51,6 +51,8 @@ import org.opensearch.cluster.routing.UnassignedInfo; import org.opensearch.common.settings.Settings; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.shard.IndexShardTestUtils; +import org.opensearch.node.remotestore.RemoteStoreNodeService; import org.opensearch.test.OpenSearchTestCase; import java.util.ArrayList; @@ -242,7 +244,7 @@ public static ClusterState state(final int numberOfNodes, final String[] indices discoBuilder.localNodeId(newNode(0).getId()); discoBuilder.clusterManagerNodeId(newNode(0).getId()); Metadata.Builder metadata = Metadata.builder(); - RoutingTable.Builder routingTable = RoutingTable.builder(); + Builder routingTable = RoutingTable.builder(); List nodesList = new ArrayList<>(nodes); int currentNodeToAssign = 0; for (String index : indices) { @@ -324,20 +326,57 @@ public static ClusterState stateWithAssignedPrimariesAndOneReplica(String index, * Creates cluster state with several indexes, shards and replicas and all shards STARTED. */ public static ClusterState stateWithAssignedPrimariesAndReplicas(String[] indices, int numberOfShards, int numberOfReplicas) { + return stateWithDiscoveryNodesAndAssignedPrimariesAndReplicas(null, indices, numberOfShards, numberOfReplicas); + } - int numberOfDataNodes = numberOfReplicas + 1; - DiscoveryNodes.Builder discoBuilder = DiscoveryNodes.builder(); - for (int i = 0; i < numberOfDataNodes + 1; i++) { - final DiscoveryNode node = newNode(i); - discoBuilder = discoBuilder.add(node); - } - discoBuilder.localNodeId(newNode(0).getId()); - discoBuilder.clusterManagerNodeId(newNode(numberOfDataNodes + 1).getId()); + /** + * Creates cluster state with provided {@link DiscoveryNodes}, several indexes, shards and replicas and all shards STARTED. + * Generates {@link DiscoveryNodes} if none are provided + */ + public static ClusterState stateWithDiscoveryNodesAndAssignedPrimariesAndReplicas( + DiscoveryNodes discoveryNodes, + String[] indices, + int numberOfShards, + int numberOfReplicas + ) { + return stateWithDiscoveryNodesAndPrimariesAndReplicas( + discoveryNodes, + indices, + numberOfShards, + numberOfReplicas, + ShardRoutingState.STARTED + ); + } + + /** + * Creates cluster state with provided {@link DiscoveryNodes}, several indexes, shards and replicas with specified {@link ShardRoutingState}. + * Generates {@link DiscoveryNodes} if none are provided + */ + public static ClusterState stateWithDiscoveryNodesAndPrimariesAndReplicas( + DiscoveryNodes discoveryNodes, + String[] indices, + int numberOfShards, + int numberOfReplicas, + ShardRoutingState shardRoutingState + ) { ClusterState.Builder state = ClusterState.builder(new ClusterName("test")); - state.nodes(discoBuilder); + // Generate fake nodes if none are provided + if (discoveryNodes == null) { + int numberOfDataNodes = numberOfReplicas + 1; + DiscoveryNodes.Builder discoBuilder = DiscoveryNodes.builder(); + for (int i = 0; i < numberOfDataNodes + 1; i++) { + final DiscoveryNode node = newNode(i); + discoBuilder = discoBuilder.add(node); + } + discoBuilder.localNodeId(newNode(0).getId()); + discoBuilder.clusterManagerNodeId(newNode(numberOfDataNodes + 1).getId()); + state.nodes(discoBuilder); + } else { + state.nodes(discoveryNodes); + } Builder routingTableBuilder = RoutingTable.builder(); - org.opensearch.cluster.metadata.Metadata.Builder metadataBuilder = Metadata.builder(); + Metadata.Builder metadataBuilder = Metadata.builder(); for (String index : indices) { IndexMetadata indexMetadata = IndexMetadata.builder(index) @@ -355,11 +394,11 @@ public static ClusterState stateWithAssignedPrimariesAndReplicas(String[] indice final ShardId shardId = new ShardId(index, "_na_", i); IndexShardRoutingTable.Builder indexShardRoutingBuilder = new IndexShardRoutingTable.Builder(shardId); indexShardRoutingBuilder.addShard( - TestShardRouting.newShardRouting(index, i, newNode(0).getId(), null, true, ShardRoutingState.STARTED) + TestShardRouting.newShardRouting(index, i, newNode(0).getId(), null, true, shardRoutingState) ); for (int replica = 0; replica < numberOfReplicas; replica++) { indexShardRoutingBuilder.addShard( - TestShardRouting.newShardRouting(index, i, newNode(replica + 1).getId(), null, false, ShardRoutingState.STARTED) + TestShardRouting.newShardRouting(index, i, newNode(replica + 1).getId(), null, false, shardRoutingState) ); } indexRoutingTableBuilder.addIndexShard(indexShardRoutingBuilder.build()); @@ -449,6 +488,52 @@ public static ClusterState state(DiscoveryNode localNode, DiscoveryNode clusterM return state.build(); } + public static ClusterState stateWithMixedNodes( + int numberOfRemoteDataNodes, + int numberOfDocrepDataNodes, + boolean applyMixedModeSetting, + String[] indices, + int numberOfShards, + int numberOfReplicas, + ShardRoutingState shardRoutingState + ) { + assert numberOfRemoteDataNodes > 1 && numberOfDocrepDataNodes > 1 : "Need at-least 2 nodes to create state for remote nodes"; + DiscoveryNodes.Builder mixedNodes = new DiscoveryNodes.Builder(); + for (int i = 0; i < numberOfRemoteDataNodes; i++) { + mixedNodes.add(newRemoteNodeWithName("remote_" + i)); + } + for (int i = 0; i < numberOfDocrepDataNodes; i++) { + mixedNodes.add(newNode(i)); + } + mixedNodes.localNodeId(newRemoteNode(numberOfRemoteDataNodes + numberOfDocrepDataNodes).getId()); + mixedNodes.clusterManagerNodeId(newRemoteNode(numberOfRemoteDataNodes + numberOfDocrepDataNodes + 1).getId()); + ClusterState initialState = stateWithDiscoveryNodesAndPrimariesAndReplicas( + mixedNodes.build(), + indices, + numberOfShards, + numberOfReplicas, + shardRoutingState + ); + if (applyMixedModeSetting) { + ClusterState.Builder finalState = ClusterState.builder(initialState); + Metadata.Builder finalMetadata = Metadata.builder(initialState.metadata()); + finalMetadata.persistentSettings( + Settings.builder() + .put(RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), "mixed") + .put(RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING.getKey(), "remote_store") + .build() + ); + finalMetadata.transientSettings( + Settings.builder() + .put(RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), "mixed") + .put(RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING.getKey(), "remote_store") + .build() + ); + return finalState.metadata(finalMetadata).build(); + } + return initialState; + } + private static DiscoveryNode newNode(int nodeId) { return new DiscoveryNode( "node_" + nodeId, @@ -459,6 +544,14 @@ private static DiscoveryNode newNode(int nodeId) { ); } + private static DiscoveryNode newRemoteNode(int nodeId) { + return newRemoteNodeWithName(String.valueOf(nodeId)); + } + + private static DiscoveryNode newRemoteNodeWithName(String nodeId) { + return IndexShardTestUtils.getFakeRemoteEnabledNode("node_" + nodeId); + } + private static String selectAndRemove(Set strings) { String selection = randomFrom(strings.toArray(new String[0])); strings.remove(selection); diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestUtils.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestUtils.java index d3a4a95c3bdef..13b93f52debfb 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestUtils.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestUtils.java @@ -21,6 +21,8 @@ import java.util.Map; public class IndexShardTestUtils { + public static final String MOCK_SEGMENT_REPO_NAME = "segment-test-repo"; + public static final String MOCK_TLOG_REPO_NAME = "tlog-test-repo"; public static DiscoveryNode getFakeDiscoNode(String id) { return new DiscoveryNode( id, @@ -34,7 +36,8 @@ public static DiscoveryNode getFakeDiscoNode(String id) { public static DiscoveryNode getFakeRemoteEnabledNode(String id) { Map remoteNodeAttributes = new HashMap(); - remoteNodeAttributes.put(RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY, "test-repo"); + remoteNodeAttributes.put(RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY, MOCK_SEGMENT_REPO_NAME); + remoteNodeAttributes.put(RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY, MOCK_TLOG_REPO_NAME); return new DiscoveryNode( id, id, From b160699b73c09019d17279bcf4e99a6da22a0300 Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Wed, 17 Apr 2024 11:51:27 +0530 Subject: [PATCH 03/24] Applying spotless on test classes Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../java/org/opensearch/index/shard/IndexShardTestUtils.java | 1 + 1 file changed, 1 insertion(+) diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestUtils.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestUtils.java index 13b93f52debfb..abf8f2a4da6c1 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestUtils.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestUtils.java @@ -23,6 +23,7 @@ public class IndexShardTestUtils { public static final String MOCK_SEGMENT_REPO_NAME = "segment-test-repo"; public static final String MOCK_TLOG_REPO_NAME = "tlog-test-repo"; + public static DiscoveryNode getFakeDiscoNode(String id) { return new DiscoveryNode( id, From 745099ea934750a97e4d51f26b1e0fc64511ddd6 Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Wed, 17 Apr 2024 12:15:19 +0530 Subject: [PATCH 04/24] Removing reverted changes from RemoteStoreUtils Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../index/remote/RemoteStoreUtils.java | 36 ++----------------- 1 file changed, 3 insertions(+), 33 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java index 12105fce6a1b8..369890e4e1311 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java @@ -20,7 +20,6 @@ import java.util.Base64; import java.util.HashMap; import java.util.List; -import java.util.Locale; import java.util.Map; import java.util.Optional; import java.util.function.Function; @@ -38,16 +37,10 @@ public class RemoteStoreUtils { */ static final char[] URL_BASE64_CHARSET = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789-_".toCharArray(); - /** - * URL safe base 64 character set. This must not be changed as this is used in deriving the base64 equivalent of binary. - */ - static final char[] URL_BASE64_CHARSET = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789-_".toCharArray(); - /** * This method subtracts given numbers from Long.MAX_VALUE and returns a string representation of the result. * The resultant string is guaranteed to be of the same length that of Long.MAX_VALUE. If shorter, we add left padding * of 0s to the string. - * * @param num number to get the inverted long string for * @return String value of Long.MAX_VALUE - num */ @@ -64,7 +57,6 @@ public static String invertLong(long num) { /** * This method converts the given string into long and subtracts it from Long.MAX_VALUE - * * @param str long in string format to be inverted * @return long value of the invert result */ @@ -78,7 +70,6 @@ public static long invertLong(String str) { /** * Extracts the segment name from the provided segment file name - * * @param filename Segment file name to parse * @return Name of the segment that the segment file belongs to */ @@ -99,9 +90,10 @@ public static String getSegmentName(String filename) { } /** + * * @param mdFiles List of segment/translog metadata files - * @param fn Function to extract PrimaryTerm_Generation and Node Id from metadata file name . - * fn returns null if node id is not part of the file name + * @param fn Function to extract PrimaryTerm_Generation and Node Id from metadata file name . + * fn returns null if node id is not part of the file name */ public static void verifyNoMultipleWriters(List mdFiles, Function> fn) { Map nodesByPrimaryTermAndGen = new HashMap<>(); @@ -136,28 +128,6 @@ static String longToUrlBase64(long value) { return base64Str.substring(0, base64Str.length() - 1); } - static long urlBase64ToLong(String base64Str) { - byte[] hashBytes = Base64.getUrlDecoder().decode(base64Str); - return ByteBuffer.wrap(hashBytes).getLong(); - } - - /** - * Converts an input hash which occupies 64 bits of memory into a composite encoded string. The string will have 2 parts - - * 1. Base 64 string and 2. Binary String. We will use the first 6 bits for creating the base 64 string. - * For the second part, the rest of the bits (of length {@code len}-6) will be used as is in string form. - */ - static String longToCompositeBase64AndBinaryEncoding(long value, int len) { - if (len < 7 || len > 64) { - throw new IllegalArgumentException("In longToCompositeBase64AndBinaryEncoding, len must be between 7 and 64 (both inclusive)"); - } - String binaryEncoding = String.format(Locale.ROOT, "%64s", Long.toBinaryString(value)).replace(' ', '0'); - String base64Part = binaryEncoding.substring(0, 6); - String binaryPart = binaryEncoding.substring(6, len); - int base64DecimalValue = Integer.valueOf(base64Part, 2); - assert base64DecimalValue >= 0 && base64DecimalValue < 64; - return URL_BASE64_CHARSET[base64DecimalValue] + binaryPart; - } - /** * Returns true iff current cluster settings have: *
From c35581b69f127a3c51232ec0df0996adb156ded7 Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Sat, 20 Apr 2024 21:08:10 +0530 Subject: [PATCH 05/24] Moving over metadata mutate logic to IndexMetadataUpdater Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../MigrationBaseTestCase.java | 22 + .../RemoteDualReplicationIT.java | 57 +- .../RemoteMigrationIndexMetadataChangeIT.java | 282 --------- .../RemoteMigrationIndexMetadataUpdateIT.java | 533 ++++++++++++++++++ .../TransportClusterUpdateSettingsAction.java | 34 ++ .../action/shard/ShardStateAction.java | 115 +--- .../cluster/node/DiscoveryNode.java | 16 + .../cluster/routing/IndexRoutingTable.java | 38 +- .../routing/IndexShardRoutingTable.java | 16 + .../allocation/IndexMetadataUpdater.java | 62 +- .../routing/allocation/RoutingAllocation.java | 7 +- .../org/opensearch/index/IndexService.java | 11 + .../index/remote/RemoteStoreUtils.java | 255 ++++++++- .../remotestore/RemoteStoreNodeService.java | 10 +- ...dStartedClusterStateTaskExecutorTests.java | 165 ------ 15 files changed, 996 insertions(+), 627 deletions(-) delete mode 100644 server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataChangeIT.java create mode 100644 server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java index 0c35f91121059..b2e24a15fcf04 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java @@ -29,6 +29,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; +import static org.opensearch.cluster.routing.allocation.decider.EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING; import static org.opensearch.node.remotestore.RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING; import static org.opensearch.node.remotestore.RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING; import static org.opensearch.repositories.fs.ReloadableFsRepository.REPOSITORIES_FAILRATE_SETTING; @@ -157,4 +158,25 @@ private Thread getIndexingThread() { }); } } + + public void excludeNodeSet(String attr, String value) { + assertAcked( + internalCluster().client() + .admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().put("cluster.routing.allocation.exclude._" + attr, value)) + .get() + ); + } + + public void stopShardRebalancing() { + assertAcked( + client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), "none").build()) + .get() + ); + } } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteDualReplicationIT.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteDualReplicationIT.java index 18f07910403d4..baa2b7503387b 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteDualReplicationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteDualReplicationIT.java @@ -132,8 +132,8 @@ public void testRemotePrimaryDocRepReplica() throws Exception { /* Scenario: - - Starts 1 docrep backed data node - - Creates an index with 0 replica + - Starts 2 docrep backed data node + - Creates an index with 1 replica - Starts 1 remote backed data node - Index some docs - Move primary copy from docrep to remote through _cluster/reroute @@ -145,14 +145,14 @@ public void testRemotePrimaryDocRepReplica() throws Exception { public void testRemotePrimaryDocRepAndRemoteReplica() throws Exception { internalCluster().startClusterManagerOnlyNode(); - logger.info("---> Starting 1 docrep data nodes"); - String docrepNodeName = internalCluster().startDataOnlyNode(); + logger.info("---> Starting 2 docrep data nodes"); + internalCluster().startDataOnlyNodes(2); internalCluster().validateClusterFormed(); assertEquals(internalCluster().client().admin().cluster().prepareGetRepositories().get().repositories().size(), 0); - logger.info("---> Creating index with 0 replica"); + logger.info("---> Creating index with 1 replica"); Settings zeroReplicas = Settings.builder() - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) .put(IndexService.RETENTION_LEASE_SYNC_INTERVAL_SETTING.getKey(), "1s") .put(IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "1s") .build(); @@ -245,14 +245,26 @@ RLs on remote enabled copies are brought up to (GlobalCkp + 1) upon a flush requ pollAndCheckRetentionLeases(REMOTE_PRI_DOCREP_REMOTE_REP); } + /* + Scenario: + - Starts 2 docrep backed data node + - Creates an index with 1 replica + - Starts 1 remote backed data node + - Index some docs + - Move primary copy from docrep to remote through _cluster/reroute + - Starts another remote backed data node + - Expands index to 2 replicas. One replica copy lies in remote backed node and other in docrep backed node + - Index some more docs + - Assert retention lease consistency + */ public void testMissingRetentionLeaseCreatedOnFailedOverRemoteReplica() throws Exception { internalCluster().startClusterManagerOnlyNode(); - logger.info("---> Starting docrep data node"); - internalCluster().startDataOnlyNode(); + logger.info("---> Starting 2 docrep data nodes"); + internalCluster().startDataOnlyNodes(2); Settings zeroReplicasAndOverridenSyncIntervals = Settings.builder() - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) .put(IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "100ms") .put(IndexService.RETENTION_LEASE_SYNC_INTERVAL_SETTING.getKey(), "100ms") .build(); @@ -323,11 +335,10 @@ private void pollAndCheckRetentionLeases(String indexName) throws Exception { /* Scenario: - - Starts 1 docrep backed data node - - Creates an index with 0 replica + - Starts 2 docrep backed data node + - Creates an index with 1 replica - Starts 1 remote backed data node - Move primary copy from docrep to remote through _cluster/reroute - - Expands index to 1 replica - Stops remote enabled node - Ensure doc count is same after failover - Index some more docs to ensure working of failed-over primary @@ -335,13 +346,13 @@ private void pollAndCheckRetentionLeases(String indexName) throws Exception { public void testFailoverRemotePrimaryToDocrepReplica() throws Exception { internalCluster().startClusterManagerOnlyNode(); - logger.info("---> Starting 1 docrep data nodes"); - String docrepNodeName = internalCluster().startDataOnlyNode(); + logger.info("---> Starting 2 docrep data nodes"); + internalCluster().startDataOnlyNodes(2); internalCluster().validateClusterFormed(); assertEquals(internalCluster().client().admin().cluster().prepareGetRepositories().get().repositories().size(), 0); logger.info("---> Creating index with 0 replica"); - Settings excludeRemoteNode = Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0).build(); + Settings excludeRemoteNode = Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).build(); createIndex(FAILOVER_REMOTE_TO_DOCREP, excludeRemoteNode); ensureGreen(FAILOVER_REMOTE_TO_DOCREP); initDocRepToRemoteMigration(); @@ -376,8 +387,8 @@ public void testFailoverRemotePrimaryToDocrepReplica() throws Exception { ); ensureGreen(FAILOVER_REMOTE_TO_DOCREP); - logger.info("---> Expanding index to 1 replica copy"); - Settings twoReplicas = Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).build(); + logger.info("---> Expanding index to 2 replica copies"); + Settings twoReplicas = Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 2).build(); assertAcked( internalCluster().client() .admin() @@ -412,7 +423,7 @@ public void testFailoverRemotePrimaryToDocrepReplica() throws Exception { logger.info("---> Stop remote store enabled node"); internalCluster().stopRandomNode(InternalTestCluster.nameFilter(remoteNodeName)); - ensureStableCluster(2); + ensureStableCluster(3); ensureYellow(FAILOVER_REMOTE_TO_DOCREP); shardStatsMap = internalCluster().client().admin().indices().prepareStats(FAILOVER_REMOTE_TO_DOCREP).setDocs(true).get().asMap(); @@ -433,7 +444,7 @@ public void testFailoverRemotePrimaryToDocrepReplica() throws Exception { refreshAndWaitForReplication(FAILOVER_REMOTE_TO_DOCREP); shardStatsMap = internalCluster().client().admin().indices().prepareStats(FAILOVER_REMOTE_TO_DOCREP).setDocs(true).get().asMap(); - assertEquals(1, shardStatsMap.size()); + assertEquals(2, shardStatsMap.size()); shardStatsMap.forEach( (shardRouting, shardStats) -> { assertEquals(firstBatch + secondBatch, shardStats.getStats().getDocs().getCount()); } ); @@ -441,8 +452,8 @@ public void testFailoverRemotePrimaryToDocrepReplica() throws Exception { /* Scenario: - - Starts 1 docrep backed data node - - Creates an index with 0 replica + - Starts 2 docrep backed data node + - Creates an index with 1 replica - Starts 1 remote backed data node - Moves primary copy from docrep to remote through _cluster/reroute - Starts 1 more remote backed data node @@ -580,7 +591,6 @@ public void testFailoverRemotePrimaryToRemoteReplica() throws Exception { - Creates an index with 0 replica - Starts 1 remote backed data node - Move primary copy from docrep to remote through _cluster/reroute - - Expands index to 1 replica - Stops remote enabled node - Ensure doc count is same after failover - Index some more docs to ensure working of failed-over primary @@ -664,7 +674,8 @@ private void assertReplicaAndPrimaryConsistency(String indexName, int firstBatch RemoteSegmentStats remoteSegmentStats = shardStats.getSegments().getRemoteSegmentStats(); assertTrue(remoteSegmentStats.getUploadBytesSucceeded() > 0); assertTrue(remoteSegmentStats.getTotalUploadTime() > 0); - } else { + } + if (shardRouting.unassigned() == false && shardRouting.primary() == false) { boolean remoteNode = nodes.get(shardRouting.currentNodeId()).isRemoteStoreNode(); assertEquals( "Mismatched doc count. Is this on remote node ? " + remoteNode, diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataChangeIT.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataChangeIT.java deleted file mode 100644 index 4fc91ba114dd3..0000000000000 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataChangeIT.java +++ /dev/null @@ -1,282 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -package org.opensearch.remotemigration; - -import org.opensearch.action.admin.indices.settings.get.GetSettingsResponse; -import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.cluster.routing.allocation.command.MoveAllocationCommand; -import org.opensearch.common.settings.Settings; -import org.opensearch.plugins.Plugin; -import org.opensearch.remotestore.multipart.mocks.MockFsRepositoryPlugin; -import org.opensearch.test.InternalSettingsPlugin; -import org.opensearch.test.OpenSearchIntegTestCase; -import org.opensearch.test.transport.MockTransportService; - -import java.util.Collection; -import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY; -import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; -import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY; -import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; -import static org.opensearch.cluster.routing.allocation.decider.EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING; -import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; - -@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) -public class RemoteMigrationIndexMetadataChangeIT extends MigrationBaseTestCase { - @Override - protected Collection> nodePlugins() { - return Stream.concat( - super.nodePlugins().stream(), - Stream.of(InternalSettingsPlugin.class, MockFsRepositoryPlugin.class, MockTransportService.TestPlugin.class) - ).collect(Collectors.toList()); - } - - public void testIndexSettingsUpdateAfterIndexMovedToRemoteThroughAllocationExclude() throws Exception { - internalCluster().startClusterManagerOnlyNode(); - - addRemote = false; - internalCluster().startDataOnlyNodes(2, Settings.builder().put("node.attr._type", "docrep").build()); - internalCluster().validateClusterFormed(); - - String indexName = "migration-index-allocation-exclude"; - Settings oneReplica = Settings.builder() - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .build(); - createIndexAndAssertDocrepProperties(indexName, oneReplica); - - AsyncIndexingService asyncIndexingService = new AsyncIndexingService(indexName); - asyncIndexingService.startIndexing(); - - String replicationType; - GetSettingsResponse response; - String remoteStoreEnabled; - - initDocRepToRemoteMigration(); - addRemote = true; - internalCluster().startDataOnlyNodes(2, Settings.builder().put("node.attr._type", "remote").build()); - internalCluster().validateClusterFormed(); - - excludeNodeSet("type", "docrep"); - waitForRelocation(); - waitNoPendingTasksOnAll(); - asyncIndexingService.stopIndexing(); - - response = internalCluster().client().admin().indices().prepareGetSettings(indexName).get(); - remoteStoreEnabled = response.getSetting(indexName, SETTING_REMOTE_STORE_ENABLED); - replicationType = response.getSetting(indexName, SETTING_REPLICATION_TYPE); - assertEquals(remoteStoreEnabled, "true"); - assertEquals(replicationType, "SEGMENT"); - } - - public void testIndexSettingsUpdateAfterIndexMovedToRemoteThroughManualReroute() throws Exception { - internalCluster().startClusterManagerOnlyNode(); - List docrepNodeNames = internalCluster().startDataOnlyNodes(2); - internalCluster().validateClusterFormed(); - - String indexName1 = "migration-index-manual-reroute-1"; - String indexName2 = "migration-index-manual-reroute-2"; - Settings oneReplica = Settings.builder() - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .build(); - createIndexAndAssertDocrepProperties(indexName1, oneReplica); - createIndexAndAssertDocrepProperties(indexName2, oneReplica); - - AsyncIndexingService indexOne = new AsyncIndexingService(indexName1); - indexOne.startIndexing(); - - AsyncIndexingService indexTwo = new AsyncIndexingService(indexName2); - indexTwo.startIndexing(); - - initDocRepToRemoteMigration(); - stopShardRebalancing(); - - addRemote = true; - List remoteNodeNames = internalCluster().startDataOnlyNodes(2); - internalCluster().validateClusterFormed(); - - String primaryNode = primaryNodeName(indexName1); - String replicaNode = docrepNodeNames.stream() - .filter(nodeName -> nodeName.equals(primaryNodeName(indexName1)) == false) - .collect(Collectors.toList()) - .get(0); - - assertAcked( - client().admin() - .cluster() - .prepareReroute() - .add(new MoveAllocationCommand(indexName1, 0, primaryNode, remoteNodeNames.get(0))) - .execute() - .actionGet() - ); - waitForRelocation(); - - assertAcked( - client().admin() - .cluster() - .prepareReroute() - .add(new MoveAllocationCommand(indexName1, 0, replicaNode, remoteNodeNames.get(1))) - .execute() - .actionGet() - ); - waitForRelocation(); - - assertAcked( - client().admin() - .cluster() - .prepareReroute() - .add(new MoveAllocationCommand(indexName2, 0, primaryNodeName(indexName2), remoteNodeNames.get(0))) - .execute() - .actionGet() - ); - waitForRelocation(); - waitNoPendingTasksOnAll(); - - indexOne.stopIndexing(); - indexTwo.stopIndexing(); - - assertRemoteProperties(indexName1); - assertDocrepProperties(indexName2); - } - - public void testIndexSettingsUpdatedOnlyForMigratingIndex() throws Exception { - internalCluster().startClusterManagerOnlyNode(); - - addRemote = false; - internalCluster().startDataOnlyNodes(2, Settings.builder().put("node.attr._type", "docrep").build()); - internalCluster().validateClusterFormed(); - - String indexName = "migration-index"; - Settings oneReplica = Settings.builder() - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .build(); - createIndexAndAssertDocrepProperties(indexName, oneReplica); - - AsyncIndexingService indexingService = new AsyncIndexingService(indexName); - indexingService.startIndexing(); - - long initalMetadataVersion = internalCluster().client() - .admin() - .cluster() - .prepareState() - .get() - .getState() - .metadata() - .index(indexName) - .getVersion(); - - initDocRepToRemoteMigration(); - addRemote = true; - internalCluster().startDataOnlyNodes(2, Settings.builder().put("node.attr._type", "remote").build()); - internalCluster().validateClusterFormed(); - - excludeNodeSet("type", "docrep"); - - waitForRelocation(); - waitNoPendingTasksOnAll(); - indexingService.stopIndexing(); - - assertRemoteProperties(indexName); - assertTrue( - initalMetadataVersion < internalCluster().client() - .admin() - .cluster() - .prepareState() - .get() - .getState() - .metadata() - .index(indexName) - .getVersion() - ); - - String secondIndex = "remote-index"; - createIndex( - secondIndex, - Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1).build() - ); - indexBulk(secondIndex, 100); - refresh(secondIndex); - ensureGreen(secondIndex); - - waitNoPendingTasksOnAll(); - - assertRemoteProperties(secondIndex); - initalMetadataVersion = internalCluster().client() - .admin() - .cluster() - .prepareState() - .get() - .getState() - .metadata() - .index(secondIndex) - .getVersion(); - - assertEquals( - initalMetadataVersion, - internalCluster().client().admin().cluster().prepareState().get().getState().metadata().index(secondIndex).getVersion() - ); - } - - private void createIndexAndAssertDocrepProperties(String index, Settings settings) { - createIndex(index, settings); - refresh(index); - ensureGreen(index); - assertDocrepProperties(index); - } - - private void assertDocrepProperties(String index) { - GetSettingsResponse response = internalCluster().client().admin().indices().prepareGetSettings(index).get(); - String remoteStoreEnabled = response.getSetting(index, SETTING_REMOTE_STORE_ENABLED); - String replicationType = response.getSetting(index, SETTING_REPLICATION_TYPE); - String segmentRepo = response.getSetting(index, SETTING_REMOTE_SEGMENT_STORE_REPOSITORY); - String translogRepo = response.getSetting(index, SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY); - assertNull(remoteStoreEnabled); - assertNull(segmentRepo); - assertNull(translogRepo); - assertEquals(replicationType, "DOCUMENT"); - } - - private void assertRemoteProperties(String index) { - GetSettingsResponse response = internalCluster().client().admin().indices().prepareGetSettings(index).get(); - String remoteStoreEnabled = response.getSetting(index, SETTING_REMOTE_STORE_ENABLED); - String replicationType = response.getSetting(index, SETTING_REPLICATION_TYPE); - String segmentRepo = response.getSetting(index, SETTING_REMOTE_SEGMENT_STORE_REPOSITORY); - String translogRepo = response.getSetting(index, SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY); - assertEquals(remoteStoreEnabled, "true"); - assertNotNull(segmentRepo); - assertNotNull(translogRepo); - assertEquals(replicationType, "SEGMENT"); - } - - private void excludeNodeSet(String attr, String value) { - assertAcked( - internalCluster().client() - .admin() - .cluster() - .prepareUpdateSettings() - .setTransientSettings(Settings.builder().put("cluster.routing.allocation.exclude._" + attr, value)) - .get() - ); - } - - private void stopShardRebalancing() { - assertAcked( - client().admin() - .cluster() - .prepareUpdateSettings() - .setPersistentSettings(Settings.builder().put(CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), "none").build()) - .get() - ); - } -} diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java new file mode 100644 index 0000000000000..5d33c488d04eb --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java @@ -0,0 +1,533 @@ +/* + * 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.remotemigration; + +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.health.ClusterHealthStatus; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.allocation.command.MoveAllocationCommand; +import org.opensearch.common.settings.Settings; +import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.test.InternalTestCluster; +import org.opensearch.test.OpenSearchIntegTestCase; + +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; + +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) +public class RemoteMigrationIndexMetadataUpdateIT extends MigrationBaseTestCase { + /** + * Scenario: + * Performs a blue/green type migration from docrep to remote enabled cluster. + * Asserts that remote based index settings are applied after all shards move over + */ + public void testIndexSettingsUpdateAfterIndexMovedToRemoteThroughAllocationExclude() throws Exception { + internalCluster().startClusterManagerOnlyNode(); + + logger.info("---> Starting 2 docrep nodes"); + addRemote = false; + internalCluster().startDataOnlyNodes(2, Settings.builder().put("node.attr._type", "docrep").build()); + internalCluster().validateClusterFormed(); + + logger.info("---> Creates an index with 1 primary and 1 replica"); + String indexName = "migration-index-allocation-exclude"; + Settings oneReplica = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .build(); + logger.info("---> Asserts index still has docrep index settings"); + createIndexAndAssertDocrepProperties(indexName, oneReplica); + + logger.info("---> Start indexing in parallel thread"); + AsyncIndexingService asyncIndexingService = new AsyncIndexingService(indexName); + asyncIndexingService.startIndexing(); + initDocRepToRemoteMigration(); + + logger.info("---> Adding 2 remote enabled nodes to the cluster"); + addRemote = true; + internalCluster().startDataOnlyNodes(2, Settings.builder().put("node.attr._type", "remote").build()); + internalCluster().validateClusterFormed(); + + logger.info("---> Excluding docrep nodes from allocation"); + excludeNodeSet("type", "docrep"); + waitForRelocation(); + waitNoPendingTasksOnAll(); + + logger.info("---> Stop indexing and assert remote enabled index settings have been applied"); + asyncIndexingService.stopIndexing(); + assertRemoteProperties(indexName); + } + + /** + * Scenario: + * Performs a manual _cluster/reroute to move shards from docrep to remote enabled nodes. + * Asserts that remote based index settings are only applied for indices whose shards + * have completely moved over to remote enabled nodes + */ + public void testIndexSettingsUpdateAfterIndexMovedToRemoteThroughManualReroute() throws Exception { + internalCluster().startClusterManagerOnlyNode(); + + logger.info("---> Starting 2 docrep nodes"); + List docrepNodeNames = internalCluster().startDataOnlyNodes(2); + internalCluster().validateClusterFormed(); + + logger.info("---> Creating 2 indices with 1 primary and 1 replica"); + String indexName1 = "migration-index-manual-reroute-1"; + String indexName2 = "migration-index-manual-reroute-2"; + Settings oneReplica = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .build(); + createIndexAndAssertDocrepProperties(indexName1, oneReplica); + createIndexAndAssertDocrepProperties(indexName2, oneReplica); + + logger.info("---> Starting parallel indexing on both indices"); + AsyncIndexingService indexOne = new AsyncIndexingService(indexName1); + indexOne.startIndexing(); + + AsyncIndexingService indexTwo = new AsyncIndexingService(indexName2); + indexTwo.startIndexing(); + + logger.info( + "---> Stopping shard rebalancing to ensure shards do not automatically move over to newer nodes after they are launched" + ); + stopShardRebalancing(); + + logger.info("---> Starting 2 remote store enabled nodes"); + initDocRepToRemoteMigration(); + addRemote = true; + List remoteNodeNames = internalCluster().startDataOnlyNodes(2); + internalCluster().validateClusterFormed(); + + String primaryNode = primaryNodeName(indexName1); + String replicaNode = docrepNodeNames.stream() + .filter(nodeName -> nodeName.equals(primaryNodeName(indexName1)) == false) + .collect(Collectors.toList()) + .get(0); + + logger.info("---> Moving over both shard copies for the first index to remote enabled nodes"); + assertAcked( + client().admin() + .cluster() + .prepareReroute() + .add(new MoveAllocationCommand(indexName1, 0, primaryNode, remoteNodeNames.get(0))) + .execute() + .actionGet() + ); + waitForRelocation(); + + assertAcked( + client().admin() + .cluster() + .prepareReroute() + .add(new MoveAllocationCommand(indexName1, 0, replicaNode, remoteNodeNames.get(1))) + .execute() + .actionGet() + ); + waitForRelocation(); + + logger.info("---> Moving only primary for the second index to remote enabled nodes"); + assertAcked( + client().admin() + .cluster() + .prepareReroute() + .add(new MoveAllocationCommand(indexName2, 0, primaryNodeName(indexName2), remoteNodeNames.get(0))) + .execute() + .actionGet() + ); + waitForRelocation(); + waitNoPendingTasksOnAll(); + + logger.info("---> Stopping indexing"); + indexOne.stopIndexing(); + indexTwo.stopIndexing(); + + logger.info("---> Assert remote settings are applied for index one but not for index two"); + assertRemoteProperties(indexName1); + assertDocrepProperties(indexName2); + } + + /** + * Scenario: + * Creates a mixed mode cluster. One index gets created before remote nodes are introduced, + * while the other one is created after remote nodes are added. + *

+ * For the first index, asserts docrep settings at first, excludes docrep nodes from + * allocation and asserts that remote index settings are applied after all shards + * have been relocated. + *

+ * For the second index, asserts that it already has remote enabled settings. + * Indexes some more docs and asserts that the index metadata version does not increment + */ + public void testIndexSettingsUpdatedOnlyForMigratingIndex() throws Exception { + internalCluster().startClusterManagerOnlyNode(); + + logger.info("---> Starting 2 docrep nodes"); + addRemote = false; + internalCluster().startDataOnlyNodes(2, Settings.builder().put("node.attr._type", "docrep").build()); + internalCluster().validateClusterFormed(); + + logger.info("---> Creating the first index with 1 primary and 1 replica"); + String indexName = "migration-index"; + Settings oneReplica = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .build(); + createIndexAndAssertDocrepProperties(indexName, oneReplica); + + logger.info("---> Starting indexing in parallel"); + AsyncIndexingService indexingService = new AsyncIndexingService(indexName); + indexingService.startIndexing(); + + logger.info("---> Storing current index metadata version"); + long initalMetadataVersion = internalCluster().client() + .admin() + .cluster() + .prepareState() + .get() + .getState() + .metadata() + .index(indexName) + .getVersion(); + + logger.info("---> Adding 2 remote enabled nodes to the cluster"); + initDocRepToRemoteMigration(); + addRemote = true; + internalCluster().startDataOnlyNodes(2, Settings.builder().put("node.attr._type", "remote").build()); + internalCluster().validateClusterFormed(); + + logger.info("---> Excluding docrep nodes from allocation"); + excludeNodeSet("type", "docrep"); + + waitForRelocation(); + waitNoPendingTasksOnAll(); + indexingService.stopIndexing(); + + logger.info("---> Assert remote settings are applied"); + assertRemoteProperties(indexName); + assertTrue( + initalMetadataVersion < internalCluster().client() + .admin() + .cluster() + .prepareState() + .get() + .getState() + .metadata() + .index(indexName) + .getVersion() + ); + + logger.info("---> Creating a new index on remote enabled nodes"); + String secondIndex = "remote-index"; + createIndex( + secondIndex, + Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1).build() + ); + indexBulk(secondIndex, 100); + initalMetadataVersion = internalCluster().client() + .admin() + .cluster() + .prepareState() + .get() + .getState() + .metadata() + .index(secondIndex) + .getVersion(); + refresh(secondIndex); + ensureGreen(secondIndex); + + waitNoPendingTasksOnAll(); + + assertRemoteProperties(secondIndex); + + logger.info("---> Assert metadata version is not changed"); + assertEquals( + initalMetadataVersion, + internalCluster().client().admin().cluster().prepareState().get().getState().metadata().index(secondIndex).getVersion() + ); + } + + /** + * Scenario: + * Creates an index with 1 primary, 2 replicas on 2 docrep nodes. Since the replica + * configuration is incorrect, the index stays YELLOW. + * Starts 2 more remote nodes and initiates shard relocation through allocation exclusion. + * After shard relocation completes, shuts down the docrep nodes and asserts remote + * index settings are applied even when the index is in YELLOW state + */ + public void testIndexSettingsUpdatedEvenForMisconfiguredReplicas() throws Exception { + internalCluster().startClusterManagerOnlyNode(); + + logger.info("---> Starting 2 docrep nodes"); + addRemote = false; + List docrepNodes = internalCluster().startDataOnlyNodes(2, Settings.builder().put("node.attr._type", "docrep").build()); + internalCluster().validateClusterFormed(); + + logger.info("---> Creating index with 1 primary and 2 replicas"); + String indexName = "migration-index-allocation-exclude"; + Settings oneReplica = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 2) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .build(); + createIndexAssertHealthAndDocrepProperties(indexName, oneReplica, this::ensureYellowAndNoInitializingShards); + + logger.info("---> Starting indexing in parallel"); + AsyncIndexingService asyncIndexingService = new AsyncIndexingService(indexName); + asyncIndexingService.startIndexing(); + + logger.info("---> Starts 2 remote enabled nodes"); + initDocRepToRemoteMigration(); + addRemote = true; + internalCluster().startDataOnlyNodes(2, Settings.builder().put("node.attr._type", "remote").build()); + internalCluster().validateClusterFormed(); + + logger.info("---> Excluding docrep nodes from allocation"); + excludeNodeSet("type", "docrep"); + waitForRelocation(); + waitNoPendingTasksOnAll(); + asyncIndexingService.stopIndexing(); + + logger.info("---> Assert cluster has turned green since more nodes are added to the cluster"); + ensureGreen(indexName); + + logger.info("---> Assert index still has dcorep settings since replica copies are still on docrep nodes"); + assertDocrepProperties(indexName); + + logger.info("---> Stopping docrep nodes"); + for (String node : docrepNodes) { + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(node)); + } + waitNoPendingTasksOnAll(); + ensureYellowAndNoInitializingShards(indexName); + + logger.info("---> Assert remote settings are applied"); + assertRemoteProperties(indexName); + } + + /** + * Scenario: + * Creates an index with 1 primary, 2 replicas on 2 docrep nodes. + * Starts 2 more remote nodes and initiates shard relocation through allocation exclusion. + * After shard relocation completes, restarts the docrep node holding extra replica shard copy + * and asserts remote index settings are applied as soon as the docrep replica copy is unassigned + */ + public void testIndexSettingsUpdatedWhenDocrepNodeIsRestarted() throws Exception { + internalCluster().startClusterManagerOnlyNode(); + + logger.info("---> Starting 2 docrep nodes"); + addRemote = false; + List docrepNodes = internalCluster().startDataOnlyNodes(2, Settings.builder().put("node.attr._type", "docrep").build()); + internalCluster().validateClusterFormed(); + + logger.info("---> Creating index with 1 primary and 2 replicas"); + String indexName = "migration-index-allocation-exclude"; + Settings oneReplica = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 2) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .build(); + createIndexAssertHealthAndDocrepProperties(indexName, oneReplica, this::ensureYellowAndNoInitializingShards); + + logger.info("---> Starting indexing in parallel"); + AsyncIndexingService asyncIndexingService = new AsyncIndexingService(indexName); + asyncIndexingService.startIndexing(); + + logger.info("---> Starts 2 remote enabled nodes"); + initDocRepToRemoteMigration(); + addRemote = true; + internalCluster().startDataOnlyNodes(2, Settings.builder().put("node.attr._type", "remote").build()); + internalCluster().validateClusterFormed(); + + logger.info("---> Excluding docrep nodes from allocation"); + excludeNodeSet("type", "docrep"); + waitForRelocation(); + waitNoPendingTasksOnAll(); + asyncIndexingService.stopIndexing(); + + logger.info("---> Assert cluster has turned green since more nodes are added to the cluster"); + ensureGreen(indexName); + + logger.info("---> Assert index still has dcorep settings since replica copies are still on docrep nodes"); + assertDocrepProperties(indexName); + + ClusterState clusterState = internalCluster().client().admin().cluster().prepareState().get().getState(); + DiscoveryNodes nodes = clusterState.nodes(); + + String docrepReplicaNodeName = ""; + for (ShardRouting shardRouting : clusterState.routingTable().index(indexName).shard(0).getShards()) { + if (nodes.get(shardRouting.currentNodeId()).isRemoteStoreNode() == false) { + docrepReplicaNodeName = nodes.get(shardRouting.currentNodeId()).getName(); + break; + } + } + excludeNodeSet("type", null); + + logger.info("---> Stopping docrep node holding the replica copy"); + internalCluster().restartNode(docrepReplicaNodeName); + ensureStableCluster(5); + waitNoPendingTasksOnAll(); + + logger.info("---> Assert remote index settings have been applied"); + assertRemoteProperties(indexName); + logger.info("---> Assert cluster is yellow since remote index settings have been applied"); + ensureYellowAndNoInitializingShards(indexName); + } + + /** + * Scenario: + * Creates a docrep cluster with 3 nodes and an index with 1 primary and 2 replicas. + * Adds 3 more remote nodes to the cluster and moves over the primary copy from docrep + * to remote through _cluster/reroute. Asserts that the remote store path based metadata + * have been applied to the index. + * Moves over the first replica copy and asserts that the remote store based settings has not been applied + * Excludes docrep nodes from allocation to force migration of the 3rd replica copy and asserts remote + * store settings has been applied as all shards have moved over + */ + public void testRemotePathMetadataAddedWithFirstPrimaryMovingToRemote() throws Exception { + String indexName = "index-1"; + internalCluster().startClusterManagerOnlyNode(); + + logger.info("---> Starting 3 docrep nodes"); + internalCluster().startDataOnlyNodes(3, Settings.builder().put("node.attr._type", "docrep").build()); + internalCluster().validateClusterFormed(); + + logger.info("---> Creating index with 1 primary and 2 replicas"); + Settings oneReplica = Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 2).build(); + createIndexAndAssertDocrepProperties(indexName, oneReplica); + + logger.info("---> Adding 3 remote enabled nodes"); + initDocRepToRemoteMigration(); + addRemote = true; + List remoteEnabledNodes = internalCluster().startDataOnlyNodes( + 3, + Settings.builder().put("node.attr._type", "remote").build() + ); + + logger.info("---> Moving primary copy to remote enabled node"); + String primaryNodeName = primaryNodeName(indexName); + assertAcked( + client().admin() + .cluster() + .prepareReroute() + .add(new MoveAllocationCommand(indexName, 0, primaryNodeName, remoteEnabledNodes.get(0))) + .execute() + .actionGet() + ); + waitForRelocation(); + waitNoPendingTasksOnAll(); + + logger.info("---> Assert custom remote path based metadata is applied"); + assertCustomIndexMetadata(indexName); + + logger.info("---> Moving over one replica copy to remote enabled node"); + String replicaNodeName = replicaNodeName(indexName); + assertAcked( + client().admin() + .cluster() + .prepareReroute() + .add(new MoveAllocationCommand(indexName, 0, replicaNodeName, remoteEnabledNodes.get(1))) + .execute() + .actionGet() + ); + waitForRelocation(); + waitNoPendingTasksOnAll(); + + logger.info("---> Assert index still has docrep settings"); + assertDocrepProperties(indexName); + + logger.info("---> Excluding docrep nodes from allocation"); + excludeNodeSet("type", "docrep"); + waitForRelocation(); + waitNoPendingTasksOnAll(); + + logger.info("---> Assert index has remote store settings"); + assertRemoteProperties(indexName); + } + + private void createIndexAndAssertDocrepProperties(String index, Settings settings) { + createIndexAssertHealthAndDocrepProperties(index, settings, this::ensureGreen); + } + + private void createIndexAssertHealthAndDocrepProperties( + String index, + Settings settings, + Function ensureState + ) { + createIndex(index, settings); + refresh(index); + ensureState.apply(index); + assertDocrepProperties(index); + } + + /** + * Assert index settings have: + *

  • + * index.remote_store.enabled is false + *
  • + *
  • + * index.remote_store.segment.repository is null + *
  • + *
  • + * index.remote_store.translog.repository is null + *
  • + *
  • + * index.replication.type is DOCUMENT + *
  • + */ + private void assertDocrepProperties(String index) { + logger.info("---> Asserting docrep index settings"); + IndexMetadata iMd = internalCluster().client().admin().cluster().prepareState().get().getState().metadata().index(index); + Settings settings = iMd.getSettings(); + assertFalse(IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING.get(settings)); + assertFalse(IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.exists(settings)); + assertFalse(IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING.exists(settings)); + assertEquals(ReplicationType.DOCUMENT, IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.get(settings)); + } + + /** + * Assert index settings have: + *
  • + * index.remote_store.enabled is true + *
  • + *
  • + * index.remote_store.segment.repository is not null + *
  • + *
  • + * index.remote_store.translog.repository is not null + *
  • + *
  • + * index.replication.type is SEGMENT + *
  • + *
    + * Also asserts index metadata customs has the remote_store key + */ + private void assertRemoteProperties(String index) { + logger.info("---> Asserting remote index settings"); + IndexMetadata iMd = internalCluster().client().admin().cluster().prepareState().get().getState().metadata().index(index); + Settings settings = iMd.getSettings(); + assertTrue(IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING.get(settings)); + assertTrue(IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.exists(settings)); + assertTrue(IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING.exists(settings)); + assertEquals(ReplicationType.SEGMENT, IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.get(settings)); + assertNotNull(iMd.getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY)); + } + + /** + * Asserts index metadata customs has the remote_store key + */ + private void assertCustomIndexMetadata(String index) { + logger.info("---> Asserting custom index metadata"); + IndexMetadata iMd = internalCluster().client().admin().cluster().prepareState().get().getState().metadata().index(index); + assertNotNull(iMd.getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY)); + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java index e6c149216da09..f478b8e96dff6 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java @@ -42,6 +42,7 @@ import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.block.ClusterBlockException; import org.opensearch.cluster.block.ClusterBlockLevel; +import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.IndexNameExpressionResolver; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.node.DiscoveryNode; @@ -58,6 +59,7 @@ import org.opensearch.common.settings.SettingsException; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.node.remotestore.RemoteStoreNodeService; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; @@ -67,6 +69,8 @@ import java.util.Set; import java.util.stream.Collectors; +import static org.opensearch.index.remote.RemoteStoreUtils.RemoteMigrationClusterStateUtils.indexHasAllRemoteStoreRelatedMetadata; + /** * Transport action for updating cluster settings * @@ -284,6 +288,7 @@ public void validateCompatibilityModeSettingRequest(ClusterUpdateSettingsRequest validateAllNodesOfSameVersion(clusterState.nodes()); if (value.equals(RemoteStoreNodeService.CompatibilityMode.STRICT.mode)) { validateAllNodesOfSameType(clusterState.nodes()); + validateIndexSettings(clusterState); } } } @@ -317,4 +322,33 @@ private void validateAllNodesOfSameType(DiscoveryNodes discoveryNodes) { } } + /** + * Verifies that while trying to switch to STRICT compatibility mode, + * all indices in the cluster have {@link RemoteStoreUtils.RemoteMigrationClusterStateUtils#indexHasAllRemoteStoreRelatedMetadata(IndexMetadata)} as true. + * If not, throws {@link RemoteIndexSettingsNotUpdatedException} error + * @param clusterState current cluster state + */ + private void validateIndexSettings(ClusterState clusterState) { + if (clusterState.metadata() + .indices() + .values() + .stream() + .allMatch(indexMetadata -> indexHasAllRemoteStoreRelatedMetadata(indexMetadata) == false)) { + throw new RemoteIndexSettingsNotUpdatedException( + "can not switch to STRICT compatibility mode since all indices in the cluster does not have remote store based index settings" + ); + } + } + + /** + * Exception raised when all shards in remote store enabled nodes + * does not have remote store based index settings + * + * @opensearch.internal + */ + static class RemoteIndexSettingsNotUpdatedException extends SettingsException { + public RemoteIndexSettingsNotUpdatedException(String message) { + super(message); + } + } } diff --git a/server/src/main/java/org/opensearch/cluster/action/shard/ShardStateAction.java b/server/src/main/java/org/opensearch/cluster/action/shard/ShardStateAction.java index 78be68a9ea2a5..cb5749a91d448 100644 --- a/server/src/main/java/org/opensearch/cluster/action/shard/ShardStateAction.java +++ b/server/src/main/java/org/opensearch/cluster/action/shard/ShardStateAction.java @@ -37,7 +37,6 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchException; -import org.opensearch.Version; import org.opensearch.cluster.ClusterChangedEvent; import org.opensearch.cluster.ClusterManagerNodeChangePredicate; import org.opensearch.cluster.ClusterState; @@ -48,11 +47,7 @@ import org.opensearch.cluster.NotClusterManagerException; import org.opensearch.cluster.coordination.FailedToCommitClusterStateException; import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.cluster.metadata.Metadata; -import org.opensearch.cluster.metadata.MetadataCreateIndexService; import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.cluster.routing.RerouteService; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.AllocationService; @@ -61,20 +56,14 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Nullable; import org.opensearch.common.Priority; -import org.opensearch.common.collect.Tuple; import org.opensearch.common.inject.Inject; import org.opensearch.common.settings.Setting; -import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.transport.TransportResponse; -import org.opensearch.index.remote.RemoteStoreEnums; -import org.opensearch.index.remote.RemoteStorePathStrategy; -import org.opensearch.index.remote.RemoteStorePathStrategyResolver; -import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.node.NodeClosedException; import org.opensearch.tasks.Task; import org.opensearch.threadpool.ThreadPool; @@ -90,20 +79,13 @@ import java.io.IOException; 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.Set; import java.util.function.Predicate; import java.util.function.Supplier; -import java.util.stream.Collectors; - -import static org.opensearch.cluster.metadata.IndexMetadata.REMOTE_STORE_CUSTOM_KEY; -import static org.opensearch.index.remote.RemoteStoreUtils.ongoingDocrepToRemoteMigration; -import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteDataAttributePresent; /** * Transport action for retrieving the shard state @@ -155,9 +137,6 @@ private static Priority parseReroutePriority(String priorityString) { // we keep track of these shards in order to avoid sending duplicate failed shard requests for a single failing shard. private final TransportRequestDeduplicator remoteFailedShardsDeduplicator = new TransportRequestDeduplicator<>(); - @Nullable - private static RemoteStorePathStrategyResolver remoteStorePathStrategyResolver; - @Inject public ShardStateAction( ClusterService clusterService, @@ -169,10 +148,7 @@ public ShardStateAction( this.transportService = transportService; this.clusterService = clusterService; this.threadPool = threadPool; - Supplier minNodeVersionSupplier = () -> clusterService.state().nodes().getMinNodeVersion(); - remoteStorePathStrategyResolver = isRemoteDataAttributePresent(clusterService.getSettings()) - ? new RemoteStorePathStrategyResolver(clusterService.getClusterSettings(), minNodeVersionSupplier) - : null; + followUpRerouteTaskPriority = FOLLOW_UP_REROUTE_PRIORITY_SETTING.get(clusterService.getSettings()); clusterService.getClusterSettings() .addSettingsUpdateConsumer(FOLLOW_UP_REROUTE_PRIORITY_SETTING, this::setFollowUpRerouteTaskPriority); @@ -828,14 +804,8 @@ public ClusterTasksResult execute(ClusterState currentState, assert tasksToBeApplied.size() >= shardRoutingsToBeApplied.size(); ClusterState maybeUpdatedState = currentState; - try { - // Let allocation service mark the incoming `RELOCATING` shard copies as `STARTED` maybeUpdatedState = allocationService.applyStartedShards(currentState, shardRoutingsToBeApplied); - // Run remote store migration based tasks - if (ongoingDocrepToRemoteMigration(currentState.metadata().settings())) { - maybeUpdatedState = maybeAddRemoteIndexSettings(maybeUpdatedState, seenShardRoutings); - } builder.successes(tasksToBeApplied); } catch (Exception e) { logger.warn(() -> new ParameterizedMessage("failed to apply started shards {}", shardRoutingsToBeApplied), e); @@ -865,89 +835,6 @@ public void clusterStatePublished(ClusterChangedEvent clusterChangedEvent) { ) ); } - - /** - * During docrep to remote store migration, applies the following remote store based index settings - * once all shards of an index have moved over to remote store enabled nodes - *
    - * Also appends the requisite Remote Store Path based custom metadata to the existing index metadata - * - * @param state updated state from allocation service - * @param seenShardRoutings {@link ShardRouting} set being processed in the task executor - * @return Mutated {@link ClusterState} with the remote store based settings applied - */ - ClusterState maybeAddRemoteIndexSettings(ClusterState state, Set seenShardRoutings) { - final Metadata.Builder metadataBuilder = Metadata.builder(state.metadata()); - HashSet indexNames = seenShardRoutings.stream() - .map(ShardRouting::getIndexName) - .collect(Collectors.toCollection(HashSet::new)); - logger.debug( - "Cluster is going through a docrep to remote store migration. Checking if any index has completely moved over to remote nodes" - ); - DiscoveryNodes discoveryNodes = state.getNodes(); - Tuple remoteStoreRepoNames = RemoteStoreUtils.getRemoteStoreRepositoryNames(discoveryNodes); - for (String index : indexNames) { - if (needsRemoteIndexSettingsUpdate( - state.routingTable().indicesRouting().get(index), - discoveryNodes, - state.metadata().index(index) - )) { - logger.info( - "Index {} does not have remote store based index settings but all shards have moved to remote enabled nodes. Applying remote store settings to the index", - index - ); - final IndexMetadata indexMetadata = metadataBuilder.get(index); - Settings.Builder indexSettingsBuilder = Settings.builder().put(indexMetadata.getSettings()); - MetadataCreateIndexService.updateRemoteStoreSettings( - indexSettingsBuilder, - remoteStoreRepoNames.v1(), - remoteStoreRepoNames.v2() - ); - - // Overriding all existing customs with only path and path hash algorithm based settings - Map newCustomRemoteStoreMetadata = new HashMap<>(); - if (remoteStorePathStrategyResolver != null) { - RemoteStorePathStrategy newPathStrategy = remoteStorePathStrategyResolver.get(); - newCustomRemoteStoreMetadata.put(RemoteStoreEnums.PathType.NAME, newPathStrategy.getType().name()); - if (Objects.nonNull(newPathStrategy.getHashAlgorithm())) { - newCustomRemoteStoreMetadata.put( - RemoteStoreEnums.PathHashAlgorithm.NAME, - newPathStrategy.getHashAlgorithm().name() - ); - } - } - - final IndexMetadata.Builder builder = IndexMetadata.builder(indexMetadata) - .settings(indexSettingsBuilder) - .putCustom(REMOTE_STORE_CUSTOM_KEY, newCustomRemoteStoreMetadata); - builder.settingsVersion(1 + indexMetadata.getVersion()); - metadataBuilder.put(builder); - } - } - return ClusterState.builder(state).metadata(metadataBuilder).build(); - } - - /** - * Returns true iff all shards for the index are in `STARTED` state - * all those `STARTED` shard copies are present in remote store enabled nodes - * - * @param indexRoutingTable current {@link IndexRoutingTable} from cluster state - * @param discoveryNodes set of discovery nodes from cluster state - * @param indexMetadata current {@link IndexMetadata} from cluster state - * @return true or false depending on the met conditions - */ - private boolean needsRemoteIndexSettingsUpdate( - IndexRoutingTable indexRoutingTable, - DiscoveryNodes discoveryNodes, - IndexMetadata indexMetadata - ) { - assert indexMetadata != null : "IndexMetadata for a STARTED shard cannot be null"; - boolean allStartedShardsOnRemote = indexRoutingTable.allShardsStarted() - && (indexRoutingTable.shardsMatchingPredicateCount(ShardRouting::started) == indexRoutingTable.shardsMatchingPredicateCount( - shardRouting -> discoveryNodes.get(shardRouting.currentNodeId()).isRemoteStoreNode() - )); - return allStartedShardsOnRemote && IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING.get(indexMetadata.getSettings()) == false; - } } /** 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 5226e9570ac14..c9fa925a6a314 100644 --- a/server/src/main/java/org/opensearch/cluster/node/DiscoveryNode.java +++ b/server/src/main/java/org/opensearch/cluster/node/DiscoveryNode.java @@ -44,6 +44,7 @@ import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.node.Node; +import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; import java.io.IOException; import java.util.Collections; @@ -412,6 +413,21 @@ public Map getAttributes() { return this.attributes; } + /** + * Remote Store segment and translog repository names + * + * @return {@link Map} of the segment and translog repository with their names + */ + public Map getRemoteStoreRepoNames() { + assert isRemoteStoreNode(); + return Map.of( + RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY, + attributes.get(RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY), + RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY, + attributes.get(RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY) + ); + } + /** * Should this node hold data (shards) or not. */ diff --git a/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java b/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java index e10bd0f014555..89a7310a2b935 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java +++ b/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java @@ -266,18 +266,6 @@ public int primaryShardsActive() { return counter; } - /** - * Returns true if all shards for the index are in {@link ShardRoutingState#STARTED} state. Otherwise false. - */ - public boolean allShardsStarted() { - for (IndexShardRoutingTable shardRoutingTable : this) { - if (shardRoutingTable.allShardsStarted() == false) { - return false; - } - } - return true; - } - /** * Returns true if all primary shards are in * {@link ShardRoutingState#UNASSIGNED} state. Otherwise false. @@ -314,6 +302,32 @@ public List shardsWithState(ShardRoutingState state) { return shards; } + /** + * Returns a {@link List} of shards that match the provided {@link Predicate} + * + * @param predicate {@link Predicate} to apply + * @return a {@link List} of shards that match one of the given {@link Predicate} + */ + public List shardsMatchingPredicate(Predicate predicate) { + List shards = new ArrayList<>(); + for (IndexShardRoutingTable shardRoutingTable : this) { + shards.addAll(shardRoutingTable.shardsMatchingPredicate(predicate)); + } + return shards; + } + + /** + * Returns true iff all shards for the index are started otherwise false + */ + public boolean allShardsStarted() { + for (IndexShardRoutingTable shardRoutingTable : this) { + if (shardRoutingTable.allShardsStarted() == false) { + return false; + } + } + return true; + } + public int shardsMatchingPredicateCount(Predicate predicate) { int count = 0; for (IndexShardRoutingTable shardRoutingTable : this) { diff --git a/server/src/main/java/org/opensearch/cluster/routing/IndexShardRoutingTable.java b/server/src/main/java/org/opensearch/cluster/routing/IndexShardRoutingTable.java index 2c250f6a5d86e..fd8cbea42c12f 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/IndexShardRoutingTable.java +++ b/server/src/main/java/org/opensearch/cluster/routing/IndexShardRoutingTable.java @@ -904,6 +904,22 @@ public List shardsWithState(ShardRoutingState state) { return shards; } + /** + * Returns a {@link List} of shards that match the provided {@link Predicate} + * + * @param predicate {@link Predicate} to apply + * @return a {@link List} of shards that match one of the given {@link Predicate} + */ + public List shardsMatchingPredicate(Predicate predicate) { + List shards = new ArrayList<>(); + for (ShardRouting shardEntry : this) { + if (predicate.test(shardEntry)) { + shards.add(shardEntry); + } + } + return shards; + } + public int shardsMatchingPredicateCount(Predicate predicate) { int count = 0; for (ShardRouting shardEntry : this) { diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java index 7fc78b05880f3..6af92b89cf93b 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java @@ -36,6 +36,7 @@ import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.cluster.routing.IndexShardRoutingTable; import org.opensearch.cluster.routing.RecoverySource; import org.opensearch.cluster.routing.RoutingChangesObserver; @@ -56,6 +57,12 @@ import java.util.Set; import java.util.stream.Collectors; +import static org.opensearch.index.remote.RemoteStoreUtils.RemoteMigrationClusterStateUtils.maybeAddRemoteIndexSettings; +import static org.opensearch.index.remote.RemoteStoreUtils.RemoteMigrationClusterStateUtils.maybeUpdateRemoteStorePathStrategy; +import static org.opensearch.index.remote.RemoteStoreUtils.getRemoteStoreRepoName; +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; + /** * Observer that tracks changes made to RoutingNodes in order to update the primary terms and in-sync allocation ids in * {@link IndexMetadata} once the allocation round has completed. @@ -68,13 +75,13 @@ */ public class IndexMetadataUpdater extends RoutingChangesObserver.AbstractRoutingChangesObserver { private final Map shardChanges = new HashMap<>(); + private boolean ongoingRemoteStoreMigration = false; @Override public void shardInitialized(ShardRouting unassignedShard, ShardRouting initializedShard) { assert initializedShard.isRelocationTarget() == false : "shardInitialized is not called on relocation target: " + initializedShard; if (initializedShard.primary()) { increasePrimaryTerm(initializedShard.shardId()); - Updates updates = changes(initializedShard.shardId()); assert updates.initializedPrimary == null : "Primary cannot be initialized more than once in same allocation round: " + "(previous: " @@ -113,6 +120,12 @@ public void shardFailed(ShardRouting failedShard, UnassignedInfo unassignedInfo) } increasePrimaryTerm(failedShard.shardId()); } + + // Track change through shardChanges Map regardless of above-mentioned conditions + // To be used to update index metadata while computing new cluster state + if (ongoingRemoteStoreMigration) { + changes(failedShard.shardId()); + } } @Override @@ -120,20 +133,40 @@ public void relocationCompleted(ShardRouting removedRelocationSource) { removeAllocationId(removedRelocationSource); } + /** + * Adds the target {@link ShardRouting} to the tracking updates set. + * Used to track started relocations while applying changes to the new {@link ClusterState} + */ + @Override + public void relocationStarted(ShardRouting startedShard, ShardRouting targetRelocatingShard) { + // Store change in shardChanges Map regardless of above-mentioned conditions + // To be used to update index metadata while computing new cluster state + if (ongoingRemoteStoreMigration) { + changes(targetRelocatingShard.shardId()); + } else { + super.relocationStarted(startedShard, targetRelocatingShard); + } + } + /** * Updates the current {@link Metadata} based on the changes of this RoutingChangesObserver. Specifically * we update {@link IndexMetadata#getInSyncAllocationIds()} and {@link IndexMetadata#primaryTerm(int)} based on * the changes made during this allocation. + *
    + * Manipulates index settings or index metadata during an ongoing remote store migration * * @param oldMetadata {@link Metadata} object from before the routing nodes was changed. * @param newRoutingTable {@link RoutingTable} object after routing changes were applied. * @return adapted {@link Metadata}, potentially the original one if no change was needed. */ - public Metadata applyChanges(Metadata oldMetadata, RoutingTable newRoutingTable) { + public Metadata applyChanges(Metadata oldMetadata, RoutingTable newRoutingTable, DiscoveryNodes discoveryNodes) { Map>> changesGroupedByIndex = shardChanges.entrySet() .stream() .collect(Collectors.groupingBy(e -> e.getKey().getIndex())); - + Map remoteRepoNames = new HashMap<>(); + if (ongoingRemoteStoreMigration) { + remoteRepoNames = getRemoteStoreRepoName(discoveryNodes); + } Metadata.Builder metadataBuilder = null; for (Map.Entry>> indexChanges : changesGroupedByIndex.entrySet()) { Index index = indexChanges.getKey(); @@ -144,6 +177,25 @@ public Metadata applyChanges(Metadata oldMetadata, RoutingTable newRoutingTable) Updates updates = shardEntry.getValue(); indexMetadataBuilder = updateInSyncAllocations(newRoutingTable, oldIndexMetadata, indexMetadataBuilder, shardId, updates); indexMetadataBuilder = updatePrimaryTerm(oldIndexMetadata, indexMetadataBuilder, shardId, updates); + if (ongoingRemoteStoreMigration) { + assert remoteRepoNames.isEmpty() == false : "Remote repo names cannot be empty during remote store migration"; + maybeUpdateRemoteStorePathStrategy( + oldIndexMetadata, + indexMetadataBuilder, + index.getName(), + discoveryNodes, + oldMetadata.settings() + ); + maybeAddRemoteIndexSettings( + oldIndexMetadata, + indexMetadataBuilder, + newRoutingTable, + index.getName(), + discoveryNodes, + remoteRepoNames.get(REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY), + remoteRepoNames.get(REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY) + ); + } } if (indexMetadataBuilder != null) { @@ -369,6 +421,10 @@ private void increasePrimaryTerm(ShardId shardId) { changes(shardId).increaseTerm = true; } + public void setOngoingRemoteStoreMigration(boolean ongoingRemoteStoreMigration) { + this.ongoingRemoteStoreMigration = ongoingRemoteStoreMigration; + } + private static class Updates { private boolean increaseTerm; // whether primary term should be increased private Set addedAllocationIds = new HashSet<>(); // allocation ids that should be added to the in-sync set diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/RoutingAllocation.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/RoutingAllocation.java index bf2db57128517..e583fdefacbd9 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/RoutingAllocation.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/RoutingAllocation.java @@ -55,6 +55,8 @@ import static java.util.Collections.emptySet; import static java.util.Collections.unmodifiableSet; +import static org.opensearch.index.remote.RemoteStoreUtils.hasAtLeastOneRemoteNode; +import static org.opensearch.node.remotestore.RemoteStoreNodeService.isMigratingToRemoteStore; /** * The {@link RoutingAllocation} keep the state of the current allocation @@ -125,6 +127,9 @@ public RoutingAllocation( this.clusterInfo = clusterInfo; this.shardSizeInfo = shardSizeInfo; this.currentNanoTime = currentNanoTime; + if (isMigratingToRemoteStore(metadata) && hasAtLeastOneRemoteNode(clusterState.getNodes())) { + indexMetadataUpdater.setOngoingRemoteStoreMigration(true); + } } /** returns the nano time captured at the beginning of the allocation. used to make sure all time based decisions are aligned */ @@ -267,7 +272,7 @@ public RoutingChangesObserver changes() { * Returns updated {@link Metadata} based on the changes that were made to the routing nodes */ public Metadata updateMetadataWithRoutingChanges(RoutingTable newRoutingTable) { - return indexMetadataUpdater.applyChanges(metadata, newRoutingTable); + return indexMetadataUpdater.applyChanges(metadata, newRoutingTable, nodes()); } /** diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index a7b29314210df..f14228a6ffb36 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -132,6 +132,7 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.unmodifiableMap; import static org.opensearch.common.collect.MapBuilder.newMapBuilder; +import static org.opensearch.index.remote.RemoteStoreUtils.RemoteMigrationClusterStateUtils.indexHasRemoteStoreSettings; /** * The main OpenSearch index service @@ -515,6 +516,16 @@ public synchronized IndexShard createShard( ); } remoteStore = new Store(shardId, this.indexSettings, remoteDirectory, lock, Store.OnClose.EMPTY, path); + } else { + // Disallow shards with remote store based settings to be created on non-remote store enabled nodes + // Even though we have ample deciders in place to prevent something like this from happening at the allocation level, + // keeping this defensive check in place + // TODO: Remove this once remote to docrep migration is supported + if (indexHasRemoteStoreSettings(indexSettings)) { + throw new IllegalStateException( + "[{" + routing.shardId() + "}] Cannot initialize shards with remote store index settings on non-remote store nodes" + ); + } } Directory directory = directoryFactory.newDirectory(this.indexSettings, path); diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java index 369890e4e1311..54872ba3e739b 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java @@ -8,28 +8,44 @@ package org.opensearch.index.remote; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.Version; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.MetadataCreateIndexService; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.ShardRoutingState; import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.Settings; -import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; -import org.opensearch.node.remotestore.RemoteStoreNodeService; +import org.opensearch.indices.replication.common.ReplicationType; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Base64; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.function.Function; +import static org.opensearch.cluster.metadata.IndexMetadata.REMOTE_STORE_CUSTOM_KEY; +import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING; +import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING; + /** * Utils for remote store * * @opensearch.internal */ public class RemoteStoreUtils { + private static final Logger logger = LogManager.getLogger(RemoteStoreUtils.class); public static final int LONG_MAX_LENGTH = String.valueOf(Long.MAX_VALUE).length(); /** @@ -41,6 +57,7 @@ public class RemoteStoreUtils { * This method subtracts given numbers from Long.MAX_VALUE and returns a string representation of the result. * The resultant string is guaranteed to be of the same length that of Long.MAX_VALUE. If shorter, we add left padding * of 0s to the string. + * * @param num number to get the inverted long string for * @return String value of Long.MAX_VALUE - num */ @@ -57,6 +74,7 @@ public static String invertLong(long num) { /** * This method converts the given string into long and subtracts it from Long.MAX_VALUE + * * @param str long in string format to be inverted * @return long value of the invert result */ @@ -70,6 +88,7 @@ public static long invertLong(String str) { /** * Extracts the segment name from the provided segment file name + * * @param filename Segment file name to parse * @return Name of the segment that the segment file belongs to */ @@ -90,10 +109,9 @@ public static String getSegmentName(String filename) { } /** - * * @param mdFiles List of segment/translog metadata files - * @param fn Function to extract PrimaryTerm_Generation and Node Id from metadata file name . - * fn returns null if node id is not part of the file name + * @param fn Function to extract PrimaryTerm_Generation and Node Id from metadata file name . + * fn returns null if node id is not part of the file name */ public static void verifyNoMultipleWriters(List mdFiles, Function> fn) { Map nodesByPrimaryTermAndGen = new HashMap<>(); @@ -128,23 +146,26 @@ static String longToUrlBase64(long value) { return base64Str.substring(0, base64Str.length() - 1); } + static long urlBase64ToLong(String base64Str) { + byte[] hashBytes = Base64.getUrlDecoder().decode(base64Str); + return ByteBuffer.wrap(hashBytes).getLong(); + } + /** - * Returns true iff current cluster settings have: - *
    - * - remote_store.compatibility_mode set to mixed - *
    - * - migration.direction set to remote_store - *
    - * false otherwise - * - * @param settings Current Cluster Settings - * @return true or false depending upon the above-mentioned condition + * Converts an input hash which occupies 64 bits of memory into a composite encoded string. The string will have 2 parts - + * 1. Base 64 string and 2. Binary String. We will use the first 6 bits for creating the base 64 string. + * For the second part, the rest of the bits (of length {@code len}-6) will be used as is in string form. */ - public static boolean ongoingDocrepToRemoteMigration(Settings settings) { - return RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING.get( - settings - ) == RemoteStoreNodeService.CompatibilityMode.MIXED - && RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING.get(settings) == RemoteStoreNodeService.Direction.REMOTE_STORE; + static String longToCompositeBase64AndBinaryEncoding(long value, int len) { + if (len < 7 || len > 64) { + throw new IllegalArgumentException("In longToCompositeBase64AndBinaryEncoding, len must be between 7 and 64 (both inclusive)"); + } + String binaryEncoding = String.format(Locale.ROOT, "%64s", Long.toBinaryString(value)).replace(' ', '0'); + String base64Part = binaryEncoding.substring(0, 6); + String binaryPart = binaryEncoding.substring(6, len); + int base64DecimalValue = Integer.valueOf(base64Part, 2); + assert base64DecimalValue >= 0 && base64DecimalValue < 64; + return URL_BASE64_CHARSET[base64DecimalValue] + binaryPart; } /** @@ -152,17 +173,199 @@ public static boolean ongoingDocrepToRemoteMigration(Settings settings) { * @param discoveryNodes Current set of {@link DiscoveryNodes} in the cluster * @return {@link Tuple} with segment repository name as first element and translog repository name as second element */ - public static Tuple getRemoteStoreRepositoryNames(DiscoveryNodes discoveryNodes) { + public static Map getRemoteStoreRepoName(DiscoveryNodes discoveryNodes) { Optional remoteNode = discoveryNodes.getNodes() .values() .stream() .filter(DiscoveryNode::isRemoteStoreNode) .findFirst(); assert remoteNode.isPresent() : "Cannot fetch remote store repository names as no remote nodes are present in the cluster"; - Map remoteNodeAttributes = remoteNode.get().getAttributes(); - return new Tuple<>( - remoteNodeAttributes.get(RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY), - remoteNodeAttributes.get(RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY) - ); + return remoteNode.get().getRemoteStoreRepoNames(); + } + + public static boolean hasAtLeastOneRemoteNode(DiscoveryNodes discoveryNodes) { + return discoveryNodes.getNodes().values().stream().anyMatch(DiscoveryNode::isRemoteStoreNode); + } + + /** + * Utils for checking and mutating cluster state during remote migration + * + * @opensearch.internal + */ + public static class RemoteMigrationClusterStateUtils { + /** + * During docrep to remote store migration, applies the following remote store based index settings + * once all shards of an index have moved over to remote store enabled nodes + *
    + * Also appends the requisite Remote Store Path based custom metadata to the existing index metadata + * + * @return Mutated {@link ClusterState} with the remote store based settings applied + */ + public static void maybeAddRemoteIndexSettings( + IndexMetadata indexMetadata, + IndexMetadata.Builder indexMetadataBuilder, + RoutingTable routingTable, + String index, + DiscoveryNodes discoveryNodes, + String segmentRepoName, + String tlogRepoName + ) { + Settings currentIndexSettings = indexMetadata.getSettings(); + if (needsRemoteIndexSettingsUpdate(routingTable.indicesRouting().get(index), discoveryNodes, currentIndexSettings)) { + logger.info( + "Index {} does not have remote store based index settings but all primary shards and STARTED replica shards have moved to remote enabled nodes. Applying remote store settings to the index", + index + ); + Settings.Builder indexSettingsBuilder = Settings.builder().put(currentIndexSettings); + MetadataCreateIndexService.updateRemoteStoreSettings(indexSettingsBuilder, segmentRepoName, tlogRepoName); + indexMetadataBuilder.settings(indexSettingsBuilder); + indexMetadataBuilder.settingsVersion(1 + indexMetadata.getVersion()); + } else { + logger.debug("Index does not satisfy criteria for applying remote store settings"); + } + } + + /** + * Returns true iff all the below conditions are true: + *
  • + * All primary shards are in {@link ShardRoutingState#STARTED} state and are in remote store enabled nodes + *
  • + *
  • + * No replica shard in {@link ShardRoutingState#INITIALIZING} or {@link ShardRoutingState#RELOCATING} state + *
  • + *
  • + * All {@link ShardRoutingState#STARTED} replica shards are in remote store enabled nodes + *
  • + * + * + * @param indexRoutingTable current {@link IndexRoutingTable} from cluster state + * @param discoveryNodes set of discovery nodes from cluster state + * @param currentIndexSettings current {@link IndexMetadata} from cluster state + * @return true or false depending on the met conditions + */ + public static boolean needsRemoteIndexSettingsUpdate( + IndexRoutingTable indexRoutingTable, + DiscoveryNodes discoveryNodes, + Settings currentIndexSettings + ) { + assert currentIndexSettings != null : "IndexMetadata for a shard cannot be null"; + if (indexHasRemoteStoreSettings(currentIndexSettings) == false) { + boolean allPrimariesStartedAndOnRemote = indexRoutingTable.shardsMatchingPredicate(ShardRouting::primary) + .stream() + .allMatch( + shardRouting -> shardRouting.started() && discoveryNodes.get(shardRouting.currentNodeId()).isRemoteStoreNode() + ); + List replicaShards = indexRoutingTable.shardsMatchingPredicate( + shardRouting -> shardRouting.primary() == false + ); + boolean noRelocatingReplicas = replicaShards.stream().noneMatch(ShardRouting::relocating); + boolean allStartedReplicasOnRemote = replicaShards.stream() + .filter(ShardRouting::started) + .allMatch(shardRouting -> discoveryNodes.get(shardRouting.currentNodeId()).isRemoteStoreNode()); + return allPrimariesStartedAndOnRemote && noRelocatingReplicas && allStartedReplicasOnRemote; + } + return false; + } + + /** + * Updates the remote store path strategy metadata for the index when it is migrating to remote. + * This should be run only when the first primary copy moves over from docrep to remote. + * Checks are in place to make this execution no-op if the index metadata is already present + * + * @param indexMetadata Current {@link IndexMetadata} + * @param indexMetadataBuilder Mutated {@link IndexMetadata.Builder} having the previous state updates + * @param index index name + * @param discoveryNodes Current {@link DiscoveryNodes} from the cluster state + * @param settings current cluster settings from {@link ClusterState} + */ + public static void maybeUpdateRemoteStorePathStrategy( + IndexMetadata indexMetadata, + IndexMetadata.Builder indexMetadataBuilder, + String index, + DiscoveryNodes discoveryNodes, + Settings settings + ) { + if (indexHasRemotePathMetadata(indexMetadata) == false) { + logger.info("Adding remote store path strategy for index [{}] during migration", index); + indexMetadataBuilder.putCustom(REMOTE_STORE_CUSTOM_KEY, createRemoteStorePathTypeMetadata(settings, discoveryNodes)); + } else { + logger.debug("Does not match criteria to update remote store path type for index {}", index); + } + } + + /** + * Generates the remote store path type information to be added to custom data of index metadata. + * + * @param settings Current Cluster settings from {@link ClusterState} + * @param discoveryNodes Current {@link DiscoveryNodes} from the cluster state + * @return {@link Map} to be added as custom data in index metadata + */ + public static Map createRemoteStorePathTypeMetadata(Settings settings, DiscoveryNodes discoveryNodes) { + Version minNodeVersion = discoveryNodes.getMinNodeVersion(); + RemoteStoreEnums.PathType pathType = Version.CURRENT.compareTo(minNodeVersion) <= 0 + ? CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.get(settings) + : RemoteStoreEnums.PathType.FIXED; + RemoteStoreEnums.PathHashAlgorithm pathHashAlgorithm = pathType == RemoteStoreEnums.PathType.FIXED + ? null + : CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.get(settings); + Map remoteCustomData = new HashMap<>(); + remoteCustomData.put(RemoteStoreEnums.PathType.NAME, pathType.name()); + if (Objects.nonNull(pathHashAlgorithm)) { + remoteCustomData.put(RemoteStoreEnums.PathHashAlgorithm.NAME, pathHashAlgorithm.name()); + } + return remoteCustomData; + } + + public static boolean indexHasAllRemoteStoreRelatedMetadata(IndexMetadata indexMetadata) { + return indexHasRemoteStoreSettings(indexMetadata.getSettings()) && indexHasRemotePathMetadata(indexMetadata); + } + + /** + * Assert current index settings have: + *
  • + * index.remote_store.enabled is true + *
  • + *
  • + * index.remote_store.segment.repository is not null + *
  • + *
  • + * index.remote_store.translog.repository is not null + *
  • + *
  • + * index.replication.type is SEGMENT + *
  • + * + * @param indexSettings Current index settings + * @return true if all above conditions match. false otherwise + */ + public static boolean indexHasRemoteStoreSettings(Settings indexSettings) { + return IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING.exists(indexSettings) + && IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.exists(indexSettings) + && IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING.exists(indexSettings) + && IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.get(indexSettings) == ReplicationType.SEGMENT; + } + + /** + * Asserts current index metadata customs has the {@link IndexMetadata#REMOTE_STORE_CUSTOM_KEY} key. + * If it does, checks if the following sub-keys are present + *
  • + * {@link RemoteStoreEnums.PathType#NAME)} + *
  • + *
  • + * {@link RemoteStoreEnums.PathHashAlgorithm#NAME)} + *
  • + * + * @param indexMetadata Current index metadata + * @return true if all above conditions match. false otherwise + */ + public static boolean indexHasRemotePathMetadata(IndexMetadata indexMetadata) { + Map customMetadata = indexMetadata.getCustomData(REMOTE_STORE_CUSTOM_KEY); + if (Objects.nonNull(customMetadata)) { + return Objects.nonNull(customMetadata.get(RemoteStoreEnums.PathType.NAME)) + && Objects.nonNull(customMetadata.get(RemoteStoreEnums.PathHashAlgorithm.NAME)); + } + ; + return false; + } } } diff --git a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeService.java b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeService.java index adfb751421db7..0d29fd36e97c6 100644 --- a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeService.java +++ b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeService.java @@ -17,6 +17,7 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Settings; import org.opensearch.common.util.FeatureFlags; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; @@ -227,7 +228,14 @@ public RepositoriesMetadata updateRepositoriesMetadata(DiscoveryNode joiningNode } /** - * To check if the cluster is undergoing remote store migration + * Returns true iff current cluster settings have: + *
    + * - remote_store.compatibility_mode set to mixed + *
    + * - migration.direction set to remote_store + *
    + * false otherwise + * * @param clusterSettings cluster level settings * @return * true For REMOTE_STORE migration direction and MIXED compatibility mode, diff --git a/server/src/test/java/org/opensearch/cluster/action/shard/ShardStartedClusterStateTaskExecutorTests.java b/server/src/test/java/org/opensearch/cluster/action/shard/ShardStartedClusterStateTaskExecutorTests.java index 0cf83055bab6d..3dee3507ae71a 100644 --- a/server/src/test/java/org/opensearch/cluster/action/shard/ShardStartedClusterStateTaskExecutorTests.java +++ b/server/src/test/java/org/opensearch/cluster/action/shard/ShardStartedClusterStateTaskExecutorTests.java @@ -32,34 +32,24 @@ package org.opensearch.cluster.action.shard; -import org.apache.logging.log4j.Logger; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.ClusterStateTaskExecutor; import org.opensearch.cluster.OpenSearchAllocationTestCase; import org.opensearch.cluster.action.shard.ShardStateAction.StartedShardEntry; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.Metadata; -import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.cluster.routing.IndexShardRoutingTable; -import org.opensearch.cluster.routing.RerouteService; -import org.opensearch.cluster.routing.RoutingTable; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.ShardRoutingState; -import org.opensearch.cluster.routing.TestShardRouting; import org.opensearch.cluster.routing.allocation.AllocationService; import org.opensearch.common.Priority; import org.opensearch.common.settings.Settings; import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.shard.ShardId; -import org.opensearch.index.shard.IndexShardTestUtils; import java.util.ArrayList; import java.util.Collections; -import java.util.HashSet; import java.util.List; -import java.util.UUID; -import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; @@ -68,17 +58,11 @@ import static org.opensearch.action.support.replication.ClusterStateCreationUtils.state; import static org.opensearch.action.support.replication.ClusterStateCreationUtils.stateWithActivePrimary; import static org.opensearch.action.support.replication.ClusterStateCreationUtils.stateWithAssignedPrimariesAndReplicas; -import static org.opensearch.action.support.replication.ClusterStateCreationUtils.stateWithMixedNodes; import static org.opensearch.action.support.replication.ClusterStateCreationUtils.stateWithNoShard; -import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY; -import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; -import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY; -import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; import static org.opensearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES_SETTING; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; -import static org.mockito.Mockito.mock; public class ShardStartedClusterStateTaskExecutorTests extends OpenSearchAllocationTestCase { @@ -299,159 +283,10 @@ public void testPrimaryTermsMismatch() throws Exception { } } - public void testMaybeAddRemoteIndexSettings() { - final String indexName = "test-add-remote-index-settings"; - final ShardId shardId = new ShardId(indexName, UUID.randomUUID().toString(), 0); - final int numberOfNodes = 3; - final int numberOfShards = 1; - final int numberOfReplicas = 2; - final ShardStateAction.ShardStartedClusterStateTaskExecutor mockShardStartedClusterStateTaskExecutor = - new ShardStateAction.ShardStartedClusterStateTaskExecutor( - mock(AllocationService.class), - mock(RerouteService.class), - mock(Supplier.class), - mock(Logger.class) - ); - // Index Settings are not updated when all shard copies are in docrep nodes - { - ClusterState initialClusterState = stateWithMixedNodes( - numberOfNodes, - numberOfNodes, - true, - List.of(indexName).toArray(String[]::new), - numberOfShards, - numberOfReplicas, - ShardRoutingState.STARTED - ); - assertRemoteStoreIndexSettingsAreAbsent(initialClusterState, indexName); - List docrepNodeSet = initialClusterState.getNodes() - .getDataNodes() - .values() - .stream() - .filter(discoveryNode -> discoveryNode.isRemoteStoreNode() == false) - .collect(Collectors.toList()); - List seenShardRoutings = generateMockShardRoutings(shardId, docrepNodeSet); - ClusterState resultingState = mockShardStartedClusterStateTaskExecutor.maybeAddRemoteIndexSettings( - loadShardRoutingsIntoClusterState(shardId, initialClusterState, seenShardRoutings), - new HashSet<>(seenShardRoutings) - ); - assertRemoteStoreIndexSettingsAreAbsent(resultingState, indexName); - } - // Index Settings are updated when all shard copies are in remote nodes and existing metadata - // does not have remote enabled settings - { - ClusterState initialClusterState = stateWithMixedNodes( - numberOfNodes, - numberOfNodes, - true, - List.of(indexName).toArray(String[]::new), - numberOfShards, - numberOfReplicas, - ShardRoutingState.STARTED - ); - assertRemoteStoreIndexSettingsAreAbsent(initialClusterState, indexName); - List remoteNodeSet = initialClusterState.getNodes() - .getDataNodes() - .values() - .stream() - .filter(DiscoveryNode::isRemoteStoreNode) - .collect(Collectors.toList()); - List seenShardRoutings = generateMockShardRoutings(shardId, remoteNodeSet); - - ClusterState resultingState = mockShardStartedClusterStateTaskExecutor.maybeAddRemoteIndexSettings( - loadShardRoutingsIntoClusterState(shardId, initialClusterState, seenShardRoutings), - new HashSet<>(seenShardRoutings) - ); - assertRemoteStoreIndexSettingsArePresent(resultingState, indexName); - } - // Index Settings are updated when some shard copies are in remote nodes while others are still on docrep - { - ClusterState initialClusterState = stateWithMixedNodes( - numberOfNodes, - numberOfNodes, - true, - List.of(indexName).toArray(String[]::new), - numberOfShards, - numberOfReplicas, - ShardRoutingState.STARTED - ); - assertRemoteStoreIndexSettingsAreAbsent(initialClusterState, indexName); - List mixedNodeSet = new ArrayList<>(); - mixedNodeSet.addAll( - initialClusterState.getNodes() - .getDataNodes() - .values() - .stream() - .filter(DiscoveryNode::isRemoteStoreNode) - .limit(2) - .collect(Collectors.toList()) - ); - mixedNodeSet.addAll( - initialClusterState.getNodes() - .getDataNodes() - .values() - .stream() - .filter(discoveryNode -> discoveryNode.isRemoteStoreNode() == false) - .limit(1) - .collect(Collectors.toList()) - ); - List seenShardRoutings = generateMockShardRoutings(shardId, mixedNodeSet); - ClusterState resultingState = mockShardStartedClusterStateTaskExecutor.maybeAddRemoteIndexSettings( - loadShardRoutingsIntoClusterState(shardId, initialClusterState, seenShardRoutings), - new HashSet<>(seenShardRoutings) - ); - assertRemoteStoreIndexSettingsAreAbsent(resultingState, indexName); - } - } - private ClusterStateTaskExecutor.ClusterTasksResult executeTasks(final ClusterState state, final List tasks) throws Exception { final ClusterStateTaskExecutor.ClusterTasksResult result = executor.execute(state, tasks); assertThat(result, notNullValue()); return result; } - - private static void assertRemoteStoreIndexSettingsArePresent(ClusterState resultingState, String indexName) { - Settings resultingIndexSettings = resultingState.metadata().index(indexName).getSettings(); - assertEquals("true", resultingIndexSettings.get(SETTING_REMOTE_STORE_ENABLED)); - assertEquals("SEGMENT", resultingIndexSettings.get(SETTING_REPLICATION_TYPE)); - assertEquals(IndexShardTestUtils.MOCK_SEGMENT_REPO_NAME, resultingIndexSettings.get(SETTING_REMOTE_SEGMENT_STORE_REPOSITORY)); - assertEquals(IndexShardTestUtils.MOCK_TLOG_REPO_NAME, resultingIndexSettings.get(SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY)); - } - - private static void assertRemoteStoreIndexSettingsAreAbsent(ClusterState initialClusterState, String indexName) { - Settings initialIndexSettings = initialClusterState.metadata().index(indexName).getSettings(); - assertNull(initialIndexSettings.get(SETTING_REMOTE_STORE_ENABLED)); - assertNull(initialIndexSettings.get(SETTING_REPLICATION_TYPE)); - assertNull(initialIndexSettings.get(SETTING_REMOTE_SEGMENT_STORE_REPOSITORY)); - assertNull(initialIndexSettings.get(SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY)); - } - - private List generateMockShardRoutings(ShardId shardId, List nodeSet) { - List seenShardRoutings = new ArrayList<>(nodeSet.size()); - for (DiscoveryNode node : nodeSet) { - if (node.getId().contains("node_0")) { - seenShardRoutings.add(TestShardRouting.newShardRouting(shardId, node.getId(), true, ShardRoutingState.STARTED)); - } else { - seenShardRoutings.add(TestShardRouting.newShardRouting(shardId, node.getId(), false, ShardRoutingState.STARTED)); - } - } - return seenShardRoutings; - } - - private ClusterState loadShardRoutingsIntoClusterState(ShardId shardId, ClusterState currentState, List shardRoutings) { - ClusterState.Builder clusterStateBuilder = ClusterState.builder(currentState); - RoutingTable.Builder routingTableBuilder = RoutingTable.builder(currentState.routingTable()); - IndexRoutingTable.Builder indexRoutingTableBuilder = IndexRoutingTable.builder( - currentState.metadata().index(shardId.getIndexName()).getIndex() - ); - IndexShardRoutingTable.Builder indexShardRoutingBuilder = new IndexShardRoutingTable.Builder(shardId); - - for (ShardRouting shardRouting : shardRoutings) { - indexShardRoutingBuilder.addShard(shardRouting); - } - return clusterStateBuilder.routingTable( - routingTableBuilder.add(indexRoutingTableBuilder.addIndexShard(indexShardRoutingBuilder.build()).build()).build() - ).build(); - } } From 55f0c621412f226f51eb7d4b37676e264f33312d Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Sat, 20 Apr 2024 22:05:33 +0530 Subject: [PATCH 06/24] Fixing javadocs Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../cluster/routing/IndexRoutingTable.java | 12 -- .../index/remote/RemoteStoreUtils.java | 39 ++---- .../ClusterStateCreationUtils.java | 119 ++---------------- 3 files changed, 22 insertions(+), 148 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java b/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java index 89a7310a2b935..7c179f6d4d8fd 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java +++ b/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java @@ -316,18 +316,6 @@ public List shardsMatchingPredicate(Predicate predic return shards; } - /** - * Returns true iff all shards for the index are started otherwise false - */ - public boolean allShardsStarted() { - for (IndexShardRoutingTable shardRoutingTable : this) { - if (shardRoutingTable.allShardsStarted() == false) { - return false; - } - } - return true; - } - public int shardsMatchingPredicateCount(Predicate predicate) { int count = 0; for (IndexShardRoutingTable shardRoutingTable : this) { diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java index 54872ba3e739b..5c3c4bd46dd1a 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java @@ -198,8 +198,6 @@ public static class RemoteMigrationClusterStateUtils { * once all shards of an index have moved over to remote store enabled nodes *
    * Also appends the requisite Remote Store Path based custom metadata to the existing index metadata - * - * @return Mutated {@link ClusterState} with the remote store based settings applied */ public static void maybeAddRemoteIndexSettings( IndexMetadata indexMetadata, @@ -227,15 +225,9 @@ public static void maybeAddRemoteIndexSettings( /** * Returns true iff all the below conditions are true: - *
  • - * All primary shards are in {@link ShardRoutingState#STARTED} state and are in remote store enabled nodes - *
  • - *
  • - * No replica shard in {@link ShardRoutingState#INITIALIZING} or {@link ShardRoutingState#RELOCATING} state - *
  • - *
  • - * All {@link ShardRoutingState#STARTED} replica shards are in remote store enabled nodes - *
  • + * - All primary shards are in {@link ShardRoutingState#STARTED} state and are in remote store enabled nodes + * - No replica shard in {@link ShardRoutingState#INITIALIZING} or {@link ShardRoutingState#RELOCATING} state + * - All {@link ShardRoutingState#STARTED} replica shards are in remote store enabled nodes * * * @param indexRoutingTable current {@link IndexRoutingTable} from cluster state @@ -322,18 +314,10 @@ public static boolean indexHasAllRemoteStoreRelatedMetadata(IndexMetadata indexM /** * Assert current index settings have: - *
  • - * index.remote_store.enabled is true - *
  • - *
  • - * index.remote_store.segment.repository is not null - *
  • - *
  • - * index.remote_store.translog.repository is not null - *
  • - *
  • - * index.replication.type is SEGMENT - *
  • + * - index.remote_store.enabled == true + * - index.remote_store.segment.repository != null + * - index.remote_store.translog.repository != null + * - index.replication.type == SEGMENT * * @param indexSettings Current index settings * @return true if all above conditions match. false otherwise @@ -348,12 +332,8 @@ public static boolean indexHasRemoteStoreSettings(Settings indexSettings) { /** * Asserts current index metadata customs has the {@link IndexMetadata#REMOTE_STORE_CUSTOM_KEY} key. * If it does, checks if the following sub-keys are present - *
  • - * {@link RemoteStoreEnums.PathType#NAME)} - *
  • - *
  • - * {@link RemoteStoreEnums.PathHashAlgorithm#NAME)} - *
  • + * - path_type + * - path_hash_algorithm * * @param indexMetadata Current index metadata * @return true if all above conditions match. false otherwise @@ -364,7 +344,6 @@ public static boolean indexHasRemotePathMetadata(IndexMetadata indexMetadata) { return Objects.nonNull(customMetadata.get(RemoteStoreEnums.PathType.NAME)) && Objects.nonNull(customMetadata.get(RemoteStoreEnums.PathHashAlgorithm.NAME)); } - ; return false; } } diff --git a/test/framework/src/main/java/org/opensearch/action/support/replication/ClusterStateCreationUtils.java b/test/framework/src/main/java/org/opensearch/action/support/replication/ClusterStateCreationUtils.java index 4a575d4de7679..182b2c9288a3d 100644 --- a/test/framework/src/main/java/org/opensearch/action/support/replication/ClusterStateCreationUtils.java +++ b/test/framework/src/main/java/org/opensearch/action/support/replication/ClusterStateCreationUtils.java @@ -51,8 +51,6 @@ import org.opensearch.cluster.routing.UnassignedInfo; import org.opensearch.common.settings.Settings; import org.opensearch.core.index.shard.ShardId; -import org.opensearch.index.shard.IndexShardTestUtils; -import org.opensearch.node.remotestore.RemoteStoreNodeService; import org.opensearch.test.OpenSearchTestCase; import java.util.ArrayList; @@ -244,7 +242,7 @@ public static ClusterState state(final int numberOfNodes, final String[] indices discoBuilder.localNodeId(newNode(0).getId()); discoBuilder.clusterManagerNodeId(newNode(0).getId()); Metadata.Builder metadata = Metadata.builder(); - Builder routingTable = RoutingTable.builder(); + RoutingTable.Builder routingTable = RoutingTable.builder(); List nodesList = new ArrayList<>(nodes); int currentNodeToAssign = 0; for (String index : indices) { @@ -326,57 +324,20 @@ public static ClusterState stateWithAssignedPrimariesAndOneReplica(String index, * Creates cluster state with several indexes, shards and replicas and all shards STARTED. */ public static ClusterState stateWithAssignedPrimariesAndReplicas(String[] indices, int numberOfShards, int numberOfReplicas) { - return stateWithDiscoveryNodesAndAssignedPrimariesAndReplicas(null, indices, numberOfShards, numberOfReplicas); - } - /** - * Creates cluster state with provided {@link DiscoveryNodes}, several indexes, shards and replicas and all shards STARTED. - * Generates {@link DiscoveryNodes} if none are provided - */ - public static ClusterState stateWithDiscoveryNodesAndAssignedPrimariesAndReplicas( - DiscoveryNodes discoveryNodes, - String[] indices, - int numberOfShards, - int numberOfReplicas - ) { - return stateWithDiscoveryNodesAndPrimariesAndReplicas( - discoveryNodes, - indices, - numberOfShards, - numberOfReplicas, - ShardRoutingState.STARTED - ); - } - - /** - * Creates cluster state with provided {@link DiscoveryNodes}, several indexes, shards and replicas with specified {@link ShardRoutingState}. - * Generates {@link DiscoveryNodes} if none are provided - */ - public static ClusterState stateWithDiscoveryNodesAndPrimariesAndReplicas( - DiscoveryNodes discoveryNodes, - String[] indices, - int numberOfShards, - int numberOfReplicas, - ShardRoutingState shardRoutingState - ) { - ClusterState.Builder state = ClusterState.builder(new ClusterName("test")); - // Generate fake nodes if none are provided - if (discoveryNodes == null) { - int numberOfDataNodes = numberOfReplicas + 1; - DiscoveryNodes.Builder discoBuilder = DiscoveryNodes.builder(); - for (int i = 0; i < numberOfDataNodes + 1; i++) { - final DiscoveryNode node = newNode(i); - discoBuilder = discoBuilder.add(node); - } - discoBuilder.localNodeId(newNode(0).getId()); - discoBuilder.clusterManagerNodeId(newNode(numberOfDataNodes + 1).getId()); - state.nodes(discoBuilder); - } else { - state.nodes(discoveryNodes); + int numberOfDataNodes = numberOfReplicas + 1; + DiscoveryNodes.Builder discoBuilder = DiscoveryNodes.builder(); + for (int i = 0; i < numberOfDataNodes + 1; i++) { + final DiscoveryNode node = newNode(i); + discoBuilder = discoBuilder.add(node); } + discoBuilder.localNodeId(newNode(0).getId()); + discoBuilder.clusterManagerNodeId(newNode(numberOfDataNodes + 1).getId()); + ClusterState.Builder state = ClusterState.builder(new ClusterName("test")); + state.nodes(discoBuilder); Builder routingTableBuilder = RoutingTable.builder(); - Metadata.Builder metadataBuilder = Metadata.builder(); + org.opensearch.cluster.metadata.Metadata.Builder metadataBuilder = Metadata.builder(); for (String index : indices) { IndexMetadata indexMetadata = IndexMetadata.builder(index) @@ -394,11 +355,11 @@ public static ClusterState stateWithDiscoveryNodesAndPrimariesAndReplicas( final ShardId shardId = new ShardId(index, "_na_", i); IndexShardRoutingTable.Builder indexShardRoutingBuilder = new IndexShardRoutingTable.Builder(shardId); indexShardRoutingBuilder.addShard( - TestShardRouting.newShardRouting(index, i, newNode(0).getId(), null, true, shardRoutingState) + TestShardRouting.newShardRouting(index, i, newNode(0).getId(), null, true, ShardRoutingState.STARTED) ); for (int replica = 0; replica < numberOfReplicas; replica++) { indexShardRoutingBuilder.addShard( - TestShardRouting.newShardRouting(index, i, newNode(replica + 1).getId(), null, false, shardRoutingState) + TestShardRouting.newShardRouting(index, i, newNode(replica + 1).getId(), null, false, ShardRoutingState.STARTED) ); } indexRoutingTableBuilder.addIndexShard(indexShardRoutingBuilder.build()); @@ -488,52 +449,6 @@ public static ClusterState state(DiscoveryNode localNode, DiscoveryNode clusterM return state.build(); } - public static ClusterState stateWithMixedNodes( - int numberOfRemoteDataNodes, - int numberOfDocrepDataNodes, - boolean applyMixedModeSetting, - String[] indices, - int numberOfShards, - int numberOfReplicas, - ShardRoutingState shardRoutingState - ) { - assert numberOfRemoteDataNodes > 1 && numberOfDocrepDataNodes > 1 : "Need at-least 2 nodes to create state for remote nodes"; - DiscoveryNodes.Builder mixedNodes = new DiscoveryNodes.Builder(); - for (int i = 0; i < numberOfRemoteDataNodes; i++) { - mixedNodes.add(newRemoteNodeWithName("remote_" + i)); - } - for (int i = 0; i < numberOfDocrepDataNodes; i++) { - mixedNodes.add(newNode(i)); - } - mixedNodes.localNodeId(newRemoteNode(numberOfRemoteDataNodes + numberOfDocrepDataNodes).getId()); - mixedNodes.clusterManagerNodeId(newRemoteNode(numberOfRemoteDataNodes + numberOfDocrepDataNodes + 1).getId()); - ClusterState initialState = stateWithDiscoveryNodesAndPrimariesAndReplicas( - mixedNodes.build(), - indices, - numberOfShards, - numberOfReplicas, - shardRoutingState - ); - if (applyMixedModeSetting) { - ClusterState.Builder finalState = ClusterState.builder(initialState); - Metadata.Builder finalMetadata = Metadata.builder(initialState.metadata()); - finalMetadata.persistentSettings( - Settings.builder() - .put(RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), "mixed") - .put(RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING.getKey(), "remote_store") - .build() - ); - finalMetadata.transientSettings( - Settings.builder() - .put(RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), "mixed") - .put(RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING.getKey(), "remote_store") - .build() - ); - return finalState.metadata(finalMetadata).build(); - } - return initialState; - } - private static DiscoveryNode newNode(int nodeId) { return new DiscoveryNode( "node_" + nodeId, @@ -544,14 +459,6 @@ private static DiscoveryNode newNode(int nodeId) { ); } - private static DiscoveryNode newRemoteNode(int nodeId) { - return newRemoteNodeWithName(String.valueOf(nodeId)); - } - - private static DiscoveryNode newRemoteNodeWithName(String nodeId) { - return IndexShardTestUtils.getFakeRemoteEnabledNode("node_" + nodeId); - } - private static String selectAndRemove(Set strings) { String selection = randomFrom(strings.toArray(new String[0])); strings.remove(selection); From 46d7270f618e91480725fb433e1102bec3225b9a Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Sat, 20 Apr 2024 22:14:50 +0530 Subject: [PATCH 07/24] Remving unnecessary changes Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../main/java/org/opensearch/cluster/metadata/IndexMetadata.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java index ae11cdee87541..80b78cfe154f1 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java @@ -636,7 +636,6 @@ public static APIBlock readFrom(StreamInput input) throws IOException { static final String KEY_SYSTEM = "system"; public static final String KEY_PRIMARY_TERMS = "primary_terms"; public static final String REMOTE_STORE_CUSTOM_KEY = "remote_store"; - public static final String REMOTE_STORE_SEEDED_SHARDS_KEY = "remote_store_seeded_shards"; public static final String INDEX_STATE_FILE_PREFIX = "state-"; From 58717ab5caa8a6005442d7f3e96ac6c9de0d25d4 Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Sat, 20 Apr 2024 22:18:51 +0530 Subject: [PATCH 08/24] Fixing javadocs for ITs Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../RemoteMigrationIndexMetadataUpdateIT.java | 39 ++++++------------- 1 file changed, 11 insertions(+), 28 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java index 5d33c488d04eb..45679598dc551 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java @@ -470,19 +470,11 @@ private void createIndexAssertHealthAndDocrepProperties( } /** - * Assert index settings have: - *
  • - * index.remote_store.enabled is false - *
  • - *
  • - * index.remote_store.segment.repository is null - *
  • - *
  • - * index.remote_store.translog.repository is null - *
  • - *
  • - * index.replication.type is DOCUMENT - *
  • + * Assert current index settings have: + * - index.remote_store.enabled == false + * - index.remote_store.segment.repository == null + * - index.remote_store.translog.repository == null + * - index.replication.type == DOCUMENT */ private void assertDocrepProperties(String index) { logger.info("---> Asserting docrep index settings"); @@ -495,21 +487,12 @@ private void assertDocrepProperties(String index) { } /** - * Assert index settings have: - *
  • - * index.remote_store.enabled is true - *
  • - *
  • - * index.remote_store.segment.repository is not null - *
  • - *
  • - * index.remote_store.translog.repository is not null - *
  • - *
  • - * index.replication.type is SEGMENT - *
  • - *
    - * Also asserts index metadata customs has the remote_store key + * Assert current index settings have: + * - index.remote_store.enabled == true + * - index.remote_store.segment.repository != null + * - index.remote_store.translog.repository != null + * - index.replication.type == SEGMENT + * Asserts index metadata customs has the remote_store key */ private void assertRemoteProperties(String index) { logger.info("---> Asserting remote index settings"); From 6706d0c7f0d27f0d1fb0610013c3aeccd07ad3b3 Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Sat, 20 Apr 2024 22:45:59 +0530 Subject: [PATCH 09/24] Removing custom exception Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../TransportClusterUpdateSettingsAction.java | 16 ++-------------- 1 file changed, 2 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java index f478b8e96dff6..d6b6c47fe4aa7 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java @@ -325,7 +325,7 @@ private void validateAllNodesOfSameType(DiscoveryNodes discoveryNodes) { /** * Verifies that while trying to switch to STRICT compatibility mode, * all indices in the cluster have {@link RemoteStoreUtils.RemoteMigrationClusterStateUtils#indexHasAllRemoteStoreRelatedMetadata(IndexMetadata)} as true. - * If not, throws {@link RemoteIndexSettingsNotUpdatedException} error + * If not, throws {@link SettingsException} * @param clusterState current cluster state */ private void validateIndexSettings(ClusterState clusterState) { @@ -334,21 +334,9 @@ private void validateIndexSettings(ClusterState clusterState) { .values() .stream() .allMatch(indexMetadata -> indexHasAllRemoteStoreRelatedMetadata(indexMetadata) == false)) { - throw new RemoteIndexSettingsNotUpdatedException( + throw new SettingsException( "can not switch to STRICT compatibility mode since all indices in the cluster does not have remote store based index settings" ); } } - - /** - * Exception raised when all shards in remote store enabled nodes - * does not have remote store based index settings - * - * @opensearch.internal - */ - static class RemoteIndexSettingsNotUpdatedException extends SettingsException { - public RemoteIndexSettingsNotUpdatedException(String message) { - super(message); - } - } } From 16298c970a52ec401706c0ffafbaab4e61b0e701 Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Sat, 20 Apr 2024 23:33:49 +0530 Subject: [PATCH 10/24] Fixing UTs for mode validations Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- ...ransportClusterManagerNodeActionTests.java | 103 +++++++++++++++++- 1 file changed, 100 insertions(+), 3 deletions(-) diff --git a/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeActionTests.java b/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeActionTests.java index b3eb2443fa940..e8d16e1302b50 100644 --- a/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeActionTests.java +++ b/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeActionTests.java @@ -31,6 +31,7 @@ import org.opensearch.cluster.block.ClusterBlockLevel; import org.opensearch.cluster.block.ClusterBlocks; import org.opensearch.cluster.coordination.FailedToCommitClusterStateException; +import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.IndexNameExpressionResolver; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.node.DiscoveryNode; @@ -55,6 +56,7 @@ import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.rest.RestStatus; import org.opensearch.discovery.ClusterManagerNotDiscoveredException; +import org.opensearch.index.remote.RemoteStoreEnums; import org.opensearch.node.NodeClosedException; import org.opensearch.node.remotestore.RemoteStoreNodeService; import org.opensearch.snapshots.EmptySnapshotsInfoService; @@ -85,6 +87,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import static org.opensearch.cluster.metadata.IndexMetadata.REMOTE_STORE_CUSTOM_KEY; import static org.opensearch.common.util.FeatureFlags.REMOTE_STORE_MIGRATION_EXPERIMENTAL; 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; @@ -791,7 +794,9 @@ public void testDontAllowSwitchingToStrictCompatibilityModeForMixedCluster() { .add(nonRemoteNode2) .localNodeId(nonRemoteNode2.getId()) .build(); - ClusterState sameTypeClusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).build(); + + metadata = createIndexMetadataWithRemoteStoreSettings("test-index"); + ClusterState sameTypeClusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).metadata(metadata).build(); transportClusterUpdateSettingsAction.validateCompatibilityModeSettingRequest(request, sameTypeClusterState); // cluster with only non-remote nodes @@ -801,10 +806,74 @@ public void testDontAllowSwitchingToStrictCompatibilityModeForMixedCluster() { .add(remoteNode2) .localNodeId(remoteNode2.getId()) .build(); - sameTypeClusterState = ClusterState.builder(sameTypeClusterState).nodes(discoveryNodes).build(); + sameTypeClusterState = ClusterState.builder(sameTypeClusterState).nodes(discoveryNodes).metadata(metadata).build(); transportClusterUpdateSettingsAction.validateCompatibilityModeSettingRequest(request, sameTypeClusterState); } + public void testDontAllowSwitchingToStrictCompatibilityModeWithoutRemoteIndexSettings() { + Settings nodeSettings = Settings.builder().put(REMOTE_STORE_MIGRATION_EXPERIMENTAL, "true").build(); + FeatureFlags.initializeFeatureFlags(nodeSettings); + Settings currentCompatibilityModeSettings = Settings.builder() + .put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), RemoteStoreNodeService.CompatibilityMode.MIXED) + .build(); + Settings intendedCompatibilityModeSettings = Settings.builder() + .put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), RemoteStoreNodeService.CompatibilityMode.STRICT) + .build(); + ClusterUpdateSettingsRequest request = new ClusterUpdateSettingsRequest(); + request.persistentSettings(intendedCompatibilityModeSettings); + DiscoveryNode remoteNode1 = new DiscoveryNode( + UUIDs.base64UUID(), + buildNewFakeTransportAddress(), + getRemoteStoreNodeAttributes(), + DiscoveryNodeRole.BUILT_IN_ROLES, + Version.CURRENT + ); + DiscoveryNode remoteNode2 = new DiscoveryNode( + UUIDs.base64UUID(), + buildNewFakeTransportAddress(), + getRemoteStoreNodeAttributes(), + DiscoveryNodeRole.BUILT_IN_ROLES, + Version.CURRENT + ); + DiscoveryNodes discoveryNodes = DiscoveryNodes.builder() + .add(remoteNode1) + .localNodeId(remoteNode1.getId()) + .add(remoteNode2) + .localNodeId(remoteNode2.getId()) + .build(); + AllocationService allocationService = new AllocationService( + new AllocationDeciders(Collections.singleton(new MaxRetryAllocationDecider())), + new TestGatewayAllocator(), + new BalancedShardsAllocator(Settings.EMPTY), + EmptyClusterInfoService.INSTANCE, + EmptySnapshotsInfoService.INSTANCE + ); + TransportClusterUpdateSettingsAction transportClusterUpdateSettingsAction = new TransportClusterUpdateSettingsAction( + transportService, + clusterService, + threadPool, + allocationService, + new ActionFilters(Collections.emptySet()), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), + clusterService.getClusterSettings() + ); + + Metadata nonRemoteIndexMd = Metadata.builder(createIndexMetadataWithDocrepSettings("test")).persistentSettings(currentCompatibilityModeSettings).build(); + final ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT).metadata(nonRemoteIndexMd).nodes(discoveryNodes).build(); + final SettingsException exception = expectThrows( + SettingsException.class, + () -> transportClusterUpdateSettingsAction.validateCompatibilityModeSettingRequest(request, clusterState) + ); + assertEquals( + "can not switch to STRICT compatibility mode since all indices in the cluster does not have remote store based index settings", + exception.getMessage() + ); + + Metadata remoteIndexMd = Metadata.builder(createIndexMetadataWithRemoteStoreSettings("test")).persistentSettings(currentCompatibilityModeSettings).build(); + ClusterState clusterStateWithRemoteIndices = ClusterState.builder(ClusterName.DEFAULT).metadata(remoteIndexMd).nodes(discoveryNodes).build(); + transportClusterUpdateSettingsAction.validateCompatibilityModeSettingRequest(request, clusterStateWithRemoteIndices); + } + public void testDontAllowSwitchingCompatibilityModeForClusterWithMultipleVersions() { Settings nodeSettings = Settings.builder().put(REMOTE_STORE_MIGRATION_EXPERIMENTAL, "true").build(); FeatureFlags.initializeFeatureFlags(nodeSettings); @@ -897,7 +966,7 @@ public void testDontAllowSwitchingCompatibilityModeForClusterWithMultipleVersion .localNodeId(discoveryNode2.getId()) .build(); - ClusterState sameVersionClusterState = ClusterState.builder(differentVersionClusterState).nodes(discoveryNodes).build(); + ClusterState sameVersionClusterState = ClusterState.builder(differentVersionClusterState).nodes(discoveryNodes).metadata(createIndexMetadataWithRemoteStoreSettings("test")).build(); transportClusterUpdateSettingsAction.validateCompatibilityModeSettingRequest(request, sameVersionClusterState); } @@ -907,4 +976,32 @@ private Map getRemoteStoreNodeAttributes() { remoteStoreNodeAttributes.put(REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY, "my-translog-repo-1"); return remoteStoreNodeAttributes; } + + private Metadata createIndexMetadataWithRemoteStoreSettings(String indexName) { + IndexMetadata.Builder indexMetadata = IndexMetadata.builder(indexName); + indexMetadata.settings(Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING.getKey(), true) + .put(IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.getKey(), "dummy-tlog-repo") + .put(IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING.getKey(), "dummy-segment-repo") + .put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.getKey(), "SEGMENT") + .build()) + .putCustom(REMOTE_STORE_CUSTOM_KEY, Map.of(RemoteStoreEnums.PathType.NAME, "dummy", RemoteStoreEnums.PathHashAlgorithm.NAME, "dummy")) + .build(); + return Metadata.builder().put(indexMetadata).build(); + } + + private Metadata createIndexMetadataWithDocrepSettings(String indexName) { + IndexMetadata.Builder indexMetadata = IndexMetadata.builder(indexName); + indexMetadata.settings(Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.getKey(), "DOCUMENT") + .build()) + .build(); + return Metadata.builder().put(indexMetadata).build(); + } } From 98f567da0127265a4e18668a912d2808951d691c Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Mon, 22 Apr 2024 11:24:55 +0530 Subject: [PATCH 11/24] Addressing comments Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../TransportClusterUpdateSettingsAction.java | 6 +- .../allocation/IndexMetadataUpdater.java | 8 +- .../org/opensearch/index/IndexService.java | 5 +- .../RemoteMigrationClusterStateUtils.java | 191 ++++++++++++++++++ .../index/remote/RemoteStoreUtils.java | 181 +---------------- ...ransportClusterManagerNodeActionTests.java | 121 ++++++----- 6 files changed, 277 insertions(+), 235 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/remote/RemoteMigrationClusterStateUtils.java diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java index d6b6c47fe4aa7..b068392b7a739 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java @@ -59,7 +59,7 @@ import org.opensearch.common.settings.SettingsException; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.io.stream.StreamInput; -import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.index.remote.RemoteMigrationClusterStateUtils; import org.opensearch.node.remotestore.RemoteStoreNodeService; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; @@ -69,7 +69,7 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.opensearch.index.remote.RemoteStoreUtils.RemoteMigrationClusterStateUtils.indexHasAllRemoteStoreRelatedMetadata; +import static org.opensearch.index.remote.RemoteMigrationClusterStateUtils.indexHasAllRemoteStoreRelatedMetadata; /** * Transport action for updating cluster settings @@ -324,7 +324,7 @@ private void validateAllNodesOfSameType(DiscoveryNodes discoveryNodes) { /** * Verifies that while trying to switch to STRICT compatibility mode, - * all indices in the cluster have {@link RemoteStoreUtils.RemoteMigrationClusterStateUtils#indexHasAllRemoteStoreRelatedMetadata(IndexMetadata)} as true. + * all indices in the cluster have {@link RemoteMigrationClusterStateUtils#indexHasAllRemoteStoreRelatedMetadata(IndexMetadata)} as true. * If not, throws {@link SettingsException} * @param clusterState current cluster state */ diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java index 6af92b89cf93b..9d9397abecbe5 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java @@ -57,8 +57,8 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.opensearch.index.remote.RemoteStoreUtils.RemoteMigrationClusterStateUtils.maybeAddRemoteIndexSettings; -import static org.opensearch.index.remote.RemoteStoreUtils.RemoteMigrationClusterStateUtils.maybeUpdateRemoteStorePathStrategy; +import static org.opensearch.index.remote.RemoteMigrationClusterStateUtils.maybeAddRemoteIndexSettings; +import static org.opensearch.index.remote.RemoteMigrationClusterStateUtils.maybeUpdateRemoteStorePathStrategy; import static org.opensearch.index.remote.RemoteStoreUtils.getRemoteStoreRepoName; 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; @@ -143,9 +143,9 @@ public void relocationStarted(ShardRouting startedShard, ShardRouting targetRelo // To be used to update index metadata while computing new cluster state if (ongoingRemoteStoreMigration) { changes(targetRelocatingShard.shardId()); - } else { - super.relocationStarted(startedShard, targetRelocatingShard); } + // Does not need a call to the super method since + // AbstractRoutingChangesObserver#relocationStarted is always No-Op } /** diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index f14228a6ffb36..c59ef32d85ef5 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -132,7 +132,7 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.unmodifiableMap; import static org.opensearch.common.collect.MapBuilder.newMapBuilder; -import static org.opensearch.index.remote.RemoteStoreUtils.RemoteMigrationClusterStateUtils.indexHasRemoteStoreSettings; +import static org.opensearch.index.remote.RemoteMigrationClusterStateUtils.indexHasRemoteStoreSettings; /** * The main OpenSearch index service @@ -518,7 +518,8 @@ public synchronized IndexShard createShard( remoteStore = new Store(shardId, this.indexSettings, remoteDirectory, lock, Store.OnClose.EMPTY, path); } else { // Disallow shards with remote store based settings to be created on non-remote store enabled nodes - // Even though we have ample deciders in place to prevent something like this from happening at the allocation level, + // Even though we have `RemoteStoreMigrationAllocationDecider` in place to prevent something like this from happening at the + // allocation level, // keeping this defensive check in place // TODO: Remove this once remote to docrep migration is supported if (indexHasRemoteStoreSettings(indexSettings)) { diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteMigrationClusterStateUtils.java b/server/src/main/java/org/opensearch/index/remote/RemoteMigrationClusterStateUtils.java new file mode 100644 index 0000000000000..9f95a1bb537a2 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/remote/RemoteMigrationClusterStateUtils.java @@ -0,0 +1,191 @@ +/* + * 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.remote; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.Version; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.MetadataCreateIndexService; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.ShardRoutingState; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.remote.RemoteStoreEnums.PathHashAlgorithm; +import org.opensearch.index.remote.RemoteStoreEnums.PathType; +import org.opensearch.indices.replication.common.ReplicationType; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static org.opensearch.cluster.metadata.IndexMetadata.REMOTE_STORE_CUSTOM_KEY; +import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING; +import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING; + +/** + * Utils for checking and mutating cluster state during remote migration + * + * @opensearch.internal + */ +public class RemoteMigrationClusterStateUtils { + private static final Logger logger = LogManager.getLogger(RemoteMigrationClusterStateUtils.class); + + /** + * During docrep to remote store migration, applies the following remote store based index settings + * once all shards of an index have moved over to remote store enabled nodes + *
    + * Also appends the requisite Remote Store Path based custom metadata to the existing index metadata + */ + public static void maybeAddRemoteIndexSettings( + IndexMetadata indexMetadata, + IndexMetadata.Builder indexMetadataBuilder, + RoutingTable routingTable, + String index, + DiscoveryNodes discoveryNodes, + String segmentRepoName, + String tlogRepoName + ) { + Settings currentIndexSettings = indexMetadata.getSettings(); + if (needsRemoteIndexSettingsUpdate(routingTable.indicesRouting().get(index), discoveryNodes, currentIndexSettings)) { + logger.info( + "Index {} does not have remote store based index settings but all primary shards and STARTED replica shards have moved to remote enabled nodes. Applying remote store settings to the index", + index + ); + Settings.Builder indexSettingsBuilder = Settings.builder().put(currentIndexSettings); + MetadataCreateIndexService.updateRemoteStoreSettings(indexSettingsBuilder, segmentRepoName, tlogRepoName); + indexMetadataBuilder.settings(indexSettingsBuilder); + indexMetadataBuilder.settingsVersion(1 + indexMetadata.getVersion()); + } else { + logger.debug("Index {} does not satisfy criteria for applying remote store settings", index); + } + } + + /** + * Returns true iff all the below conditions are true: + * - All primary shards are in {@link ShardRoutingState#STARTED} state and are in remote store enabled nodes + * - No replica shard in {@link ShardRoutingState#RELOCATING} state + * - All {@link ShardRoutingState#STARTED} replica shards are in remote store enabled nodes + * + * @param indexRoutingTable current {@link IndexRoutingTable} from cluster state + * @param discoveryNodes set of discovery nodes from cluster state + * @param currentIndexSettings current {@link IndexMetadata} from cluster state + * @return true or false depending on the met conditions + */ + public static boolean needsRemoteIndexSettingsUpdate( + IndexRoutingTable indexRoutingTable, + DiscoveryNodes discoveryNodes, + Settings currentIndexSettings + ) { + assert currentIndexSettings != null : "IndexMetadata for a shard cannot be null"; + if (indexHasRemoteStoreSettings(currentIndexSettings) == false) { + boolean allPrimariesStartedAndOnRemote = indexRoutingTable.shardsMatchingPredicate(ShardRouting::primary) + .stream() + .allMatch(shardRouting -> shardRouting.started() && discoveryNodes.get(shardRouting.currentNodeId()).isRemoteStoreNode()); + List replicaShards = indexRoutingTable.shardsMatchingPredicate(shardRouting -> shardRouting.primary() == false); + boolean noRelocatingReplicas = replicaShards.stream().noneMatch(ShardRouting::relocating); + boolean allStartedReplicasOnRemote = replicaShards.stream() + .filter(ShardRouting::started) + .allMatch(shardRouting -> discoveryNodes.get(shardRouting.currentNodeId()).isRemoteStoreNode()); + return allPrimariesStartedAndOnRemote && noRelocatingReplicas && allStartedReplicasOnRemote; + } + return false; + } + + /** + * Updates the remote store path strategy metadata for the index when it is migrating to remote. + * This should be run only when the first primary copy moves over from docrep to remote. + * Checks are in place to make this execution no-op if the index metadata is already present + * + * @param indexMetadata Current {@link IndexMetadata} + * @param indexMetadataBuilder Mutated {@link IndexMetadata.Builder} having the previous state updates + * @param index index name + * @param discoveryNodes Current {@link DiscoveryNodes} from the cluster state + * @param settings current cluster settings from {@link ClusterState} + */ + public static void maybeUpdateRemoteStorePathStrategy( + IndexMetadata indexMetadata, + IndexMetadata.Builder indexMetadataBuilder, + String index, + DiscoveryNodes discoveryNodes, + Settings settings + ) { + if (indexHasRemotePathMetadata(indexMetadata) == false) { + logger.info("Adding remote store path strategy for index [{}] during migration", index); + indexMetadataBuilder.putCustom(REMOTE_STORE_CUSTOM_KEY, createRemoteStorePathTypeMetadata(settings, discoveryNodes)); + } else { + logger.debug("Does not match criteria to update remote store path type for index {}", index); + } + } + + /** + * Generates the remote store path type information to be added to custom data of index metadata. + * + * @param settings Current Cluster settings from {@link ClusterState} + * @param discoveryNodes Current {@link DiscoveryNodes} from the cluster state + * @return {@link Map} to be added as custom data in index metadata + */ + public static Map createRemoteStorePathTypeMetadata(Settings settings, DiscoveryNodes discoveryNodes) { + Version minNodeVersion = discoveryNodes.getMinNodeVersion(); + PathType pathType = Version.CURRENT.compareTo(minNodeVersion) <= 0 + ? CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.get(settings) + : PathType.FIXED; + PathHashAlgorithm pathHashAlgorithm = pathType == PathType.FIXED + ? null + : CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.get(settings); + Map remoteCustomData = new HashMap<>(); + remoteCustomData.put(PathType.NAME, pathType.name()); + if (Objects.nonNull(pathHashAlgorithm)) { + remoteCustomData.put(PathHashAlgorithm.NAME, pathHashAlgorithm.name()); + } + return remoteCustomData; + } + + public static boolean indexHasAllRemoteStoreRelatedMetadata(IndexMetadata indexMetadata) { + return indexHasRemoteStoreSettings(indexMetadata.getSettings()) && indexHasRemotePathMetadata(indexMetadata); + } + + /** + * Assert current index settings have: + * - index.remote_store.enabled == true + * - index.remote_store.segment.repository != null + * - index.remote_store.translog.repository != null + * - index.replication.type == SEGMENT + * + * @param indexSettings Current index settings + * @return true if all above conditions match. false otherwise + */ + public static boolean indexHasRemoteStoreSettings(Settings indexSettings) { + return IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING.exists(indexSettings) + && IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.exists(indexSettings) + && IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING.exists(indexSettings) + && IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.get(indexSettings) == ReplicationType.SEGMENT; + } + + /** + * Asserts current index metadata customs has the {@link IndexMetadata#REMOTE_STORE_CUSTOM_KEY} key. + * If it does, checks if the following sub-keys are present + * - path_type + * - path_hash_algorithm + * + * @param indexMetadata Current index metadata + * @return true if all above conditions match. false otherwise + */ + public static boolean indexHasRemotePathMetadata(IndexMetadata indexMetadata) { + Map customMetadata = indexMetadata.getCustomData(REMOTE_STORE_CUSTOM_KEY); + if (Objects.nonNull(customMetadata)) { + return Objects.nonNull(customMetadata.get(PathType.NAME)) && Objects.nonNull(customMetadata.get(PathHashAlgorithm.NAME)); + } + return false; + } +} diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java index 5c3c4bd46dd1a..9961876c3ad8c 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java @@ -10,19 +10,9 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.opensearch.Version; -import org.opensearch.cluster.ClusterState; -import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.cluster.metadata.MetadataCreateIndexService; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.cluster.routing.IndexRoutingTable; -import org.opensearch.cluster.routing.RoutingTable; -import org.opensearch.cluster.routing.ShardRouting; -import org.opensearch.cluster.routing.ShardRoutingState; import org.opensearch.common.collect.Tuple; -import org.opensearch.common.settings.Settings; -import org.opensearch.indices.replication.common.ReplicationType; import java.nio.ByteBuffer; import java.util.Arrays; @@ -31,14 +21,9 @@ import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.Objects; import java.util.Optional; import java.util.function.Function; -import static org.opensearch.cluster.metadata.IndexMetadata.REMOTE_STORE_CUSTOM_KEY; -import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING; -import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING; - /** * Utils for remote store * @@ -183,168 +168,14 @@ public static Map getRemoteStoreRepoName(DiscoveryNodes discover return remoteNode.get().getRemoteStoreRepoNames(); } - public static boolean hasAtLeastOneRemoteNode(DiscoveryNodes discoveryNodes) { - return discoveryNodes.getNodes().values().stream().anyMatch(DiscoveryNode::isRemoteStoreNode); - } - /** - * Utils for checking and mutating cluster state during remote migration + * Ensures that the current set of {@link DiscoveryNodes} contains at-least one node + * matching the {@link DiscoveryNode#isRemoteStoreNode()} predicate * - * @opensearch.internal + * @param discoveryNodes Current set of {@link DiscoveryNodes} + * @return true if there is at-least one remote store enabled node, false otherwise */ - public static class RemoteMigrationClusterStateUtils { - /** - * During docrep to remote store migration, applies the following remote store based index settings - * once all shards of an index have moved over to remote store enabled nodes - *
    - * Also appends the requisite Remote Store Path based custom metadata to the existing index metadata - */ - public static void maybeAddRemoteIndexSettings( - IndexMetadata indexMetadata, - IndexMetadata.Builder indexMetadataBuilder, - RoutingTable routingTable, - String index, - DiscoveryNodes discoveryNodes, - String segmentRepoName, - String tlogRepoName - ) { - Settings currentIndexSettings = indexMetadata.getSettings(); - if (needsRemoteIndexSettingsUpdate(routingTable.indicesRouting().get(index), discoveryNodes, currentIndexSettings)) { - logger.info( - "Index {} does not have remote store based index settings but all primary shards and STARTED replica shards have moved to remote enabled nodes. Applying remote store settings to the index", - index - ); - Settings.Builder indexSettingsBuilder = Settings.builder().put(currentIndexSettings); - MetadataCreateIndexService.updateRemoteStoreSettings(indexSettingsBuilder, segmentRepoName, tlogRepoName); - indexMetadataBuilder.settings(indexSettingsBuilder); - indexMetadataBuilder.settingsVersion(1 + indexMetadata.getVersion()); - } else { - logger.debug("Index does not satisfy criteria for applying remote store settings"); - } - } - - /** - * Returns true iff all the below conditions are true: - * - All primary shards are in {@link ShardRoutingState#STARTED} state and are in remote store enabled nodes - * - No replica shard in {@link ShardRoutingState#INITIALIZING} or {@link ShardRoutingState#RELOCATING} state - * - All {@link ShardRoutingState#STARTED} replica shards are in remote store enabled nodes - * - * - * @param indexRoutingTable current {@link IndexRoutingTable} from cluster state - * @param discoveryNodes set of discovery nodes from cluster state - * @param currentIndexSettings current {@link IndexMetadata} from cluster state - * @return true or false depending on the met conditions - */ - public static boolean needsRemoteIndexSettingsUpdate( - IndexRoutingTable indexRoutingTable, - DiscoveryNodes discoveryNodes, - Settings currentIndexSettings - ) { - assert currentIndexSettings != null : "IndexMetadata for a shard cannot be null"; - if (indexHasRemoteStoreSettings(currentIndexSettings) == false) { - boolean allPrimariesStartedAndOnRemote = indexRoutingTable.shardsMatchingPredicate(ShardRouting::primary) - .stream() - .allMatch( - shardRouting -> shardRouting.started() && discoveryNodes.get(shardRouting.currentNodeId()).isRemoteStoreNode() - ); - List replicaShards = indexRoutingTable.shardsMatchingPredicate( - shardRouting -> shardRouting.primary() == false - ); - boolean noRelocatingReplicas = replicaShards.stream().noneMatch(ShardRouting::relocating); - boolean allStartedReplicasOnRemote = replicaShards.stream() - .filter(ShardRouting::started) - .allMatch(shardRouting -> discoveryNodes.get(shardRouting.currentNodeId()).isRemoteStoreNode()); - return allPrimariesStartedAndOnRemote && noRelocatingReplicas && allStartedReplicasOnRemote; - } - return false; - } - - /** - * Updates the remote store path strategy metadata for the index when it is migrating to remote. - * This should be run only when the first primary copy moves over from docrep to remote. - * Checks are in place to make this execution no-op if the index metadata is already present - * - * @param indexMetadata Current {@link IndexMetadata} - * @param indexMetadataBuilder Mutated {@link IndexMetadata.Builder} having the previous state updates - * @param index index name - * @param discoveryNodes Current {@link DiscoveryNodes} from the cluster state - * @param settings current cluster settings from {@link ClusterState} - */ - public static void maybeUpdateRemoteStorePathStrategy( - IndexMetadata indexMetadata, - IndexMetadata.Builder indexMetadataBuilder, - String index, - DiscoveryNodes discoveryNodes, - Settings settings - ) { - if (indexHasRemotePathMetadata(indexMetadata) == false) { - logger.info("Adding remote store path strategy for index [{}] during migration", index); - indexMetadataBuilder.putCustom(REMOTE_STORE_CUSTOM_KEY, createRemoteStorePathTypeMetadata(settings, discoveryNodes)); - } else { - logger.debug("Does not match criteria to update remote store path type for index {}", index); - } - } - - /** - * Generates the remote store path type information to be added to custom data of index metadata. - * - * @param settings Current Cluster settings from {@link ClusterState} - * @param discoveryNodes Current {@link DiscoveryNodes} from the cluster state - * @return {@link Map} to be added as custom data in index metadata - */ - public static Map createRemoteStorePathTypeMetadata(Settings settings, DiscoveryNodes discoveryNodes) { - Version minNodeVersion = discoveryNodes.getMinNodeVersion(); - RemoteStoreEnums.PathType pathType = Version.CURRENT.compareTo(minNodeVersion) <= 0 - ? CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.get(settings) - : RemoteStoreEnums.PathType.FIXED; - RemoteStoreEnums.PathHashAlgorithm pathHashAlgorithm = pathType == RemoteStoreEnums.PathType.FIXED - ? null - : CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.get(settings); - Map remoteCustomData = new HashMap<>(); - remoteCustomData.put(RemoteStoreEnums.PathType.NAME, pathType.name()); - if (Objects.nonNull(pathHashAlgorithm)) { - remoteCustomData.put(RemoteStoreEnums.PathHashAlgorithm.NAME, pathHashAlgorithm.name()); - } - return remoteCustomData; - } - - public static boolean indexHasAllRemoteStoreRelatedMetadata(IndexMetadata indexMetadata) { - return indexHasRemoteStoreSettings(indexMetadata.getSettings()) && indexHasRemotePathMetadata(indexMetadata); - } - - /** - * Assert current index settings have: - * - index.remote_store.enabled == true - * - index.remote_store.segment.repository != null - * - index.remote_store.translog.repository != null - * - index.replication.type == SEGMENT - * - * @param indexSettings Current index settings - * @return true if all above conditions match. false otherwise - */ - public static boolean indexHasRemoteStoreSettings(Settings indexSettings) { - return IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING.exists(indexSettings) - && IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.exists(indexSettings) - && IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING.exists(indexSettings) - && IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.get(indexSettings) == ReplicationType.SEGMENT; - } - - /** - * Asserts current index metadata customs has the {@link IndexMetadata#REMOTE_STORE_CUSTOM_KEY} key. - * If it does, checks if the following sub-keys are present - * - path_type - * - path_hash_algorithm - * - * @param indexMetadata Current index metadata - * @return true if all above conditions match. false otherwise - */ - public static boolean indexHasRemotePathMetadata(IndexMetadata indexMetadata) { - Map customMetadata = indexMetadata.getCustomData(REMOTE_STORE_CUSTOM_KEY); - if (Objects.nonNull(customMetadata)) { - return Objects.nonNull(customMetadata.get(RemoteStoreEnums.PathType.NAME)) - && Objects.nonNull(customMetadata.get(RemoteStoreEnums.PathHashAlgorithm.NAME)); - } - return false; - } + public static boolean hasAtLeastOneRemoteNode(DiscoveryNodes discoveryNodes) { + return discoveryNodes.getNodes().values().stream().anyMatch(DiscoveryNode::isRemoteStoreNode); } } diff --git a/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeActionTests.java b/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeActionTests.java index e8d16e1302b50..df1eca90d484e 100644 --- a/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeActionTests.java +++ b/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeActionTests.java @@ -814,63 +814,73 @@ public void testDontAllowSwitchingToStrictCompatibilityModeWithoutRemoteIndexSet Settings nodeSettings = Settings.builder().put(REMOTE_STORE_MIGRATION_EXPERIMENTAL, "true").build(); FeatureFlags.initializeFeatureFlags(nodeSettings); Settings currentCompatibilityModeSettings = Settings.builder() - .put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), RemoteStoreNodeService.CompatibilityMode.MIXED) - .build(); + .put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), RemoteStoreNodeService.CompatibilityMode.MIXED) + .build(); Settings intendedCompatibilityModeSettings = Settings.builder() - .put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), RemoteStoreNodeService.CompatibilityMode.STRICT) - .build(); + .put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), RemoteStoreNodeService.CompatibilityMode.STRICT) + .build(); ClusterUpdateSettingsRequest request = new ClusterUpdateSettingsRequest(); request.persistentSettings(intendedCompatibilityModeSettings); DiscoveryNode remoteNode1 = new DiscoveryNode( - UUIDs.base64UUID(), - buildNewFakeTransportAddress(), - getRemoteStoreNodeAttributes(), - DiscoveryNodeRole.BUILT_IN_ROLES, - Version.CURRENT + UUIDs.base64UUID(), + buildNewFakeTransportAddress(), + getRemoteStoreNodeAttributes(), + DiscoveryNodeRole.BUILT_IN_ROLES, + Version.CURRENT ); DiscoveryNode remoteNode2 = new DiscoveryNode( - UUIDs.base64UUID(), - buildNewFakeTransportAddress(), - getRemoteStoreNodeAttributes(), - DiscoveryNodeRole.BUILT_IN_ROLES, - Version.CURRENT + UUIDs.base64UUID(), + buildNewFakeTransportAddress(), + getRemoteStoreNodeAttributes(), + DiscoveryNodeRole.BUILT_IN_ROLES, + Version.CURRENT ); DiscoveryNodes discoveryNodes = DiscoveryNodes.builder() - .add(remoteNode1) - .localNodeId(remoteNode1.getId()) - .add(remoteNode2) - .localNodeId(remoteNode2.getId()) - .build(); + .add(remoteNode1) + .localNodeId(remoteNode1.getId()) + .add(remoteNode2) + .localNodeId(remoteNode2.getId()) + .build(); AllocationService allocationService = new AllocationService( - new AllocationDeciders(Collections.singleton(new MaxRetryAllocationDecider())), - new TestGatewayAllocator(), - new BalancedShardsAllocator(Settings.EMPTY), - EmptyClusterInfoService.INSTANCE, - EmptySnapshotsInfoService.INSTANCE + new AllocationDeciders(Collections.singleton(new MaxRetryAllocationDecider())), + new TestGatewayAllocator(), + new BalancedShardsAllocator(Settings.EMPTY), + EmptyClusterInfoService.INSTANCE, + EmptySnapshotsInfoService.INSTANCE ); TransportClusterUpdateSettingsAction transportClusterUpdateSettingsAction = new TransportClusterUpdateSettingsAction( - transportService, - clusterService, - threadPool, - allocationService, - new ActionFilters(Collections.emptySet()), - new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), - clusterService.getClusterSettings() + transportService, + clusterService, + threadPool, + allocationService, + new ActionFilters(Collections.emptySet()), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), + clusterService.getClusterSettings() ); - Metadata nonRemoteIndexMd = Metadata.builder(createIndexMetadataWithDocrepSettings("test")).persistentSettings(currentCompatibilityModeSettings).build(); - final ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT).metadata(nonRemoteIndexMd).nodes(discoveryNodes).build(); + Metadata nonRemoteIndexMd = Metadata.builder(createIndexMetadataWithDocrepSettings("test")) + .persistentSettings(currentCompatibilityModeSettings) + .build(); + final ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT) + .metadata(nonRemoteIndexMd) + .nodes(discoveryNodes) + .build(); final SettingsException exception = expectThrows( - SettingsException.class, - () -> transportClusterUpdateSettingsAction.validateCompatibilityModeSettingRequest(request, clusterState) + SettingsException.class, + () -> transportClusterUpdateSettingsAction.validateCompatibilityModeSettingRequest(request, clusterState) ); assertEquals( - "can not switch to STRICT compatibility mode since all indices in the cluster does not have remote store based index settings", - exception.getMessage() + "can not switch to STRICT compatibility mode since all indices in the cluster does not have remote store based index settings", + exception.getMessage() ); - Metadata remoteIndexMd = Metadata.builder(createIndexMetadataWithRemoteStoreSettings("test")).persistentSettings(currentCompatibilityModeSettings).build(); - ClusterState clusterStateWithRemoteIndices = ClusterState.builder(ClusterName.DEFAULT).metadata(remoteIndexMd).nodes(discoveryNodes).build(); + Metadata remoteIndexMd = Metadata.builder(createIndexMetadataWithRemoteStoreSettings("test")) + .persistentSettings(currentCompatibilityModeSettings) + .build(); + ClusterState clusterStateWithRemoteIndices = ClusterState.builder(ClusterName.DEFAULT) + .metadata(remoteIndexMd) + .nodes(discoveryNodes) + .build(); transportClusterUpdateSettingsAction.validateCompatibilityModeSettingRequest(request, clusterStateWithRemoteIndices); } @@ -966,7 +976,10 @@ public void testDontAllowSwitchingCompatibilityModeForClusterWithMultipleVersion .localNodeId(discoveryNode2.getId()) .build(); - ClusterState sameVersionClusterState = ClusterState.builder(differentVersionClusterState).nodes(discoveryNodes).metadata(createIndexMetadataWithRemoteStoreSettings("test")).build(); + ClusterState sameVersionClusterState = ClusterState.builder(differentVersionClusterState) + .nodes(discoveryNodes) + .metadata(createIndexMetadataWithRemoteStoreSettings("test")) + .build(); transportClusterUpdateSettingsAction.validateCompatibilityModeSettingRequest(request, sameVersionClusterState); } @@ -979,7 +992,8 @@ private Map getRemoteStoreNodeAttributes() { private Metadata createIndexMetadataWithRemoteStoreSettings(String indexName) { IndexMetadata.Builder indexMetadata = IndexMetadata.builder(indexName); - indexMetadata.settings(Settings.builder() + indexMetadata.settings( + Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) @@ -987,21 +1001,26 @@ private Metadata createIndexMetadataWithRemoteStoreSettings(String indexName) { .put(IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.getKey(), "dummy-tlog-repo") .put(IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING.getKey(), "dummy-segment-repo") .put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.getKey(), "SEGMENT") - .build()) - .putCustom(REMOTE_STORE_CUSTOM_KEY, Map.of(RemoteStoreEnums.PathType.NAME, "dummy", RemoteStoreEnums.PathHashAlgorithm.NAME, "dummy")) - .build(); + .build() + ) + .putCustom( + REMOTE_STORE_CUSTOM_KEY, + Map.of(RemoteStoreEnums.PathType.NAME, "dummy", RemoteStoreEnums.PathHashAlgorithm.NAME, "dummy") + ) + .build(); return Metadata.builder().put(indexMetadata).build(); } private Metadata createIndexMetadataWithDocrepSettings(String indexName) { IndexMetadata.Builder indexMetadata = IndexMetadata.builder(indexName); - indexMetadata.settings(Settings.builder() - .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) - .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.getKey(), "DOCUMENT") - .build()) - .build(); + indexMetadata.settings( + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.getKey(), "DOCUMENT") + .build() + ).build(); return Metadata.builder().put(indexMetadata).build(); } } From 9dda73604a970e26d5a42662b80e70f7ace10351 Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Tue, 23 Apr 2024 15:04:01 +0530 Subject: [PATCH 12/24] Addressing comments and adding UTs Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../TransportClusterUpdateSettingsAction.java | 15 +- .../metadata/MetadataCreateIndexService.java | 7 - .../cluster/node/DiscoveryNode.java | 16 - .../allocation/IndexMetadataUpdater.java | 13 +- .../routing/allocation/RoutingAllocation.java | 3 +- .../org/opensearch/index/IndexService.java | 2 +- ... RemoteMigrationIndexMetadataUpdater.java} | 40 ++- .../index/remote/RemoteStoreUtils.java | 23 +- .../remotestore/RemoteStoreNodeAttribute.java | 30 ++ ...ransportClusterManagerNodeActionTests.java | 38 +-- .../routing/IndexShardRoutingTableTests.java | 45 +++ ...oteMigrationIndexMetadataUpdaterTests.java | 291 ++++++++++++++++++ 12 files changed, 419 insertions(+), 104 deletions(-) rename server/src/main/java/org/opensearch/index/remote/{RemoteMigrationClusterStateUtils.java => RemoteMigrationIndexMetadataUpdater.java} (82%) create mode 100644 server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java index b068392b7a739..339c9e347fcb7 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java @@ -59,17 +59,18 @@ import org.opensearch.common.settings.SettingsException; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.io.stream.StreamInput; -import org.opensearch.index.remote.RemoteMigrationClusterStateUtils; +import org.opensearch.index.remote.RemoteMigrationIndexMetadataUpdater; import org.opensearch.node.remotestore.RemoteStoreNodeService; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; import java.io.IOException; +import java.util.Collection; import java.util.Locale; import java.util.Set; import java.util.stream.Collectors; -import static org.opensearch.index.remote.RemoteMigrationClusterStateUtils.indexHasAllRemoteStoreRelatedMetadata; +import static org.opensearch.index.remote.RemoteMigrationIndexMetadataUpdater.indexHasAllRemoteStoreRelatedMetadata; /** * Transport action for updating cluster settings @@ -324,16 +325,14 @@ private void validateAllNodesOfSameType(DiscoveryNodes discoveryNodes) { /** * Verifies that while trying to switch to STRICT compatibility mode, - * all indices in the cluster have {@link RemoteMigrationClusterStateUtils#indexHasAllRemoteStoreRelatedMetadata(IndexMetadata)} as true. + * all indices in the cluster have {@link RemoteMigrationIndexMetadataUpdater#indexHasAllRemoteStoreRelatedMetadata(IndexMetadata)} as true. * If not, throws {@link SettingsException} * @param clusterState current cluster state */ private void validateIndexSettings(ClusterState clusterState) { - if (clusterState.metadata() - .indices() - .values() - .stream() - .allMatch(indexMetadata -> indexHasAllRemoteStoreRelatedMetadata(indexMetadata) == false)) { + Collection allIndicesMetadata = clusterState.metadata().indices().values(); + if (allIndicesMetadata.isEmpty() == false + && allIndicesMetadata.stream().allMatch(indexMetadata -> indexHasAllRemoteStoreRelatedMetadata(indexMetadata) == false)) { throw new SettingsException( "can not switch to STRICT compatibility mode since all indices in the cluster does not have remote store based index settings" ); diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java index d8d5456f79ada..0eba4d241f0fd 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -1030,13 +1030,6 @@ public static void updateReplicationStrategy( settingsBuilder.put(SETTING_REPLICATION_TYPE, indexReplicationType); } - public static void updateRemoteStoreSettings(Settings.Builder settingsBuilder, String segmentRepository, String translogRepository) { - settingsBuilder.put(SETTING_REMOTE_STORE_ENABLED, true) - .put(SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) - .put(SETTING_REMOTE_SEGMENT_STORE_REPOSITORY, segmentRepository) - .put(SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, translogRepository); - } - /** * Updates index settings to enable remote store by default based on node attributes * @param settingsBuilder index settings builder to be updated with relevant settings 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 c9fa925a6a314..5226e9570ac14 100644 --- a/server/src/main/java/org/opensearch/cluster/node/DiscoveryNode.java +++ b/server/src/main/java/org/opensearch/cluster/node/DiscoveryNode.java @@ -44,7 +44,6 @@ import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.node.Node; -import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; import java.io.IOException; import java.util.Collections; @@ -413,21 +412,6 @@ public Map getAttributes() { return this.attributes; } - /** - * Remote Store segment and translog repository names - * - * @return {@link Map} of the segment and translog repository with their names - */ - public Map getRemoteStoreRepoNames() { - assert isRemoteStoreNode(); - return Map.of( - RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY, - attributes.get(RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY), - RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY, - attributes.get(RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY) - ); - } - /** * Should this node hold data (shards) or not. */ diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java index 9d9397abecbe5..6bc7d3023198a 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java @@ -32,6 +32,7 @@ package org.opensearch.cluster.routing.allocation; +import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; @@ -46,6 +47,7 @@ import org.opensearch.common.util.set.Sets; import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.remote.RemoteMigrationIndexMetadataUpdater; import java.util.Collections; import java.util.Comparator; @@ -57,8 +59,6 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.opensearch.index.remote.RemoteMigrationClusterStateUtils.maybeAddRemoteIndexSettings; -import static org.opensearch.index.remote.RemoteMigrationClusterStateUtils.maybeUpdateRemoteStorePathStrategy; import static org.opensearch.index.remote.RemoteStoreUtils.getRemoteStoreRepoName; 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; @@ -74,6 +74,7 @@ * @opensearch.internal */ public class IndexMetadataUpdater extends RoutingChangesObserver.AbstractRoutingChangesObserver { + private final Logger logger = LogManager.getLogger(IndexMetadataUpdater.class); private final Map shardChanges = new HashMap<>(); private boolean ongoingRemoteStoreMigration = false; @@ -144,8 +145,6 @@ public void relocationStarted(ShardRouting startedShard, ShardRouting targetRelo if (ongoingRemoteStoreMigration) { changes(targetRelocatingShard.shardId()); } - // Does not need a call to the super method since - // AbstractRoutingChangesObserver#relocationStarted is always No-Op } /** @@ -178,15 +177,15 @@ public Metadata applyChanges(Metadata oldMetadata, RoutingTable newRoutingTable, indexMetadataBuilder = updateInSyncAllocations(newRoutingTable, oldIndexMetadata, indexMetadataBuilder, shardId, updates); indexMetadataBuilder = updatePrimaryTerm(oldIndexMetadata, indexMetadataBuilder, shardId, updates); if (ongoingRemoteStoreMigration) { - assert remoteRepoNames.isEmpty() == false : "Remote repo names cannot be empty during remote store migration"; - maybeUpdateRemoteStorePathStrategy( + RemoteMigrationIndexMetadataUpdater migrationImdUpdater = new RemoteMigrationIndexMetadataUpdater(logger); + migrationImdUpdater.maybeUpdateRemoteStorePathStrategy( oldIndexMetadata, indexMetadataBuilder, index.getName(), discoveryNodes, oldMetadata.settings() ); - maybeAddRemoteIndexSettings( + migrationImdUpdater.maybeAddRemoteIndexSettings( oldIndexMetadata, indexMetadataBuilder, newRoutingTable, diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/RoutingAllocation.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/RoutingAllocation.java index e583fdefacbd9..fd789774f6f4f 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/RoutingAllocation.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/RoutingAllocation.java @@ -55,7 +55,6 @@ import static java.util.Collections.emptySet; import static java.util.Collections.unmodifiableSet; -import static org.opensearch.index.remote.RemoteStoreUtils.hasAtLeastOneRemoteNode; import static org.opensearch.node.remotestore.RemoteStoreNodeService.isMigratingToRemoteStore; /** @@ -127,7 +126,7 @@ public RoutingAllocation( this.clusterInfo = clusterInfo; this.shardSizeInfo = shardSizeInfo; this.currentNanoTime = currentNanoTime; - if (isMigratingToRemoteStore(metadata) && hasAtLeastOneRemoteNode(clusterState.getNodes())) { + if (isMigratingToRemoteStore(metadata)) { indexMetadataUpdater.setOngoingRemoteStoreMigration(true); } } diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index c59ef32d85ef5..6c95c8ea50e72 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -132,7 +132,7 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.unmodifiableMap; import static org.opensearch.common.collect.MapBuilder.newMapBuilder; -import static org.opensearch.index.remote.RemoteMigrationClusterStateUtils.indexHasRemoteStoreSettings; +import static org.opensearch.index.remote.RemoteMigrationIndexMetadataUpdater.indexHasRemoteStoreSettings; /** * The main OpenSearch index service diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteMigrationClusterStateUtils.java b/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java similarity index 82% rename from server/src/main/java/org/opensearch/index/remote/RemoteMigrationClusterStateUtils.java rename to server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java index 9f95a1bb537a2..0c3bee9cab7c2 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteMigrationClusterStateUtils.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java @@ -8,12 +8,10 @@ package org.opensearch.index.remote; -import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.opensearch.Version; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.cluster.metadata.MetadataCreateIndexService; import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.cluster.routing.RoutingTable; @@ -30,6 +28,10 @@ import java.util.Objects; import static org.opensearch.cluster.metadata.IndexMetadata.REMOTE_STORE_CUSTOM_KEY; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING; import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING; @@ -38,8 +40,12 @@ * * @opensearch.internal */ -public class RemoteMigrationClusterStateUtils { - private static final Logger logger = LogManager.getLogger(RemoteMigrationClusterStateUtils.class); +public class RemoteMigrationIndexMetadataUpdater { + private final Logger logger; + + public RemoteMigrationIndexMetadataUpdater(Logger logger) { + this.logger = logger; + } /** * During docrep to remote store migration, applies the following remote store based index settings @@ -47,7 +53,7 @@ public class RemoteMigrationClusterStateUtils { *
    * Also appends the requisite Remote Store Path based custom metadata to the existing index metadata */ - public static void maybeAddRemoteIndexSettings( + public void maybeAddRemoteIndexSettings( IndexMetadata indexMetadata, IndexMetadata.Builder indexMetadataBuilder, RoutingTable routingTable, @@ -62,8 +68,9 @@ public static void maybeAddRemoteIndexSettings( "Index {} does not have remote store based index settings but all primary shards and STARTED replica shards have moved to remote enabled nodes. Applying remote store settings to the index", index ); + assert Objects.nonNull(segmentRepoName) && Objects.nonNull(tlogRepoName) : "Remote repo names cannot be null"; Settings.Builder indexSettingsBuilder = Settings.builder().put(currentIndexSettings); - MetadataCreateIndexService.updateRemoteStoreSettings(indexSettingsBuilder, segmentRepoName, tlogRepoName); + updateRemoteStoreSettings(indexSettingsBuilder, segmentRepoName, tlogRepoName); indexMetadataBuilder.settings(indexSettingsBuilder); indexMetadataBuilder.settingsVersion(1 + indexMetadata.getVersion()); } else { @@ -82,7 +89,7 @@ public static void maybeAddRemoteIndexSettings( * @param currentIndexSettings current {@link IndexMetadata} from cluster state * @return true or false depending on the met conditions */ - public static boolean needsRemoteIndexSettingsUpdate( + public boolean needsRemoteIndexSettingsUpdate( IndexRoutingTable indexRoutingTable, DiscoveryNodes discoveryNodes, Settings currentIndexSettings @@ -104,8 +111,10 @@ public static boolean needsRemoteIndexSettingsUpdate( /** * Updates the remote store path strategy metadata for the index when it is migrating to remote. - * This should be run only when the first primary copy moves over from docrep to remote. - * Checks are in place to make this execution no-op if the index metadata is already present + * This is run during state change of each shard copy when the cluster is in `MIXED` mode and the direction of migration is `REMOTE_STORE` + * Should not interfere with docrep functionality even if the index is in docrep nodes since this metadata + * is not used anywhere in the docrep flow + * Checks are in place to make this execution no-op if the index metadata is already present. * * @param indexMetadata Current {@link IndexMetadata} * @param indexMetadataBuilder Mutated {@link IndexMetadata.Builder} having the previous state updates @@ -113,7 +122,7 @@ public static boolean needsRemoteIndexSettingsUpdate( * @param discoveryNodes Current {@link DiscoveryNodes} from the cluster state * @param settings current cluster settings from {@link ClusterState} */ - public static void maybeUpdateRemoteStorePathStrategy( + public void maybeUpdateRemoteStorePathStrategy( IndexMetadata indexMetadata, IndexMetadata.Builder indexMetadataBuilder, String index, @@ -124,7 +133,7 @@ public static void maybeUpdateRemoteStorePathStrategy( logger.info("Adding remote store path strategy for index [{}] during migration", index); indexMetadataBuilder.putCustom(REMOTE_STORE_CUSTOM_KEY, createRemoteStorePathTypeMetadata(settings, discoveryNodes)); } else { - logger.debug("Does not match criteria to update remote store path type for index {}", index); + logger.debug("Index {} already has remote store path strategy", index); } } @@ -135,7 +144,7 @@ public static void maybeUpdateRemoteStorePathStrategy( * @param discoveryNodes Current {@link DiscoveryNodes} from the cluster state * @return {@link Map} to be added as custom data in index metadata */ - public static Map createRemoteStorePathTypeMetadata(Settings settings, DiscoveryNodes discoveryNodes) { + public Map createRemoteStorePathTypeMetadata(Settings settings, DiscoveryNodes discoveryNodes) { Version minNodeVersion = discoveryNodes.getMinNodeVersion(); PathType pathType = Version.CURRENT.compareTo(minNodeVersion) <= 0 ? CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.get(settings) @@ -188,4 +197,11 @@ public static boolean indexHasRemotePathMetadata(IndexMetadata indexMetadata) { } return false; } + + public static void updateRemoteStoreSettings(Settings.Builder settingsBuilder, String segmentRepository, String translogRepository) { + settingsBuilder.put(SETTING_REMOTE_STORE_ENABLED, true) + .put(SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .put(SETTING_REMOTE_SEGMENT_STORE_REPOSITORY, segmentRepository) + .put(SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, translogRepository); + } } diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java index 9961876c3ad8c..6c76e67b012b4 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java @@ -13,6 +13,7 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.common.collect.Tuple; +import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; import java.nio.ByteBuffer; import java.util.Arrays; @@ -154,9 +155,13 @@ static String longToCompositeBase64AndBinaryEncoding(long value, int len) { } /** - * Fetches segment and translog repository names from remote store node attributes + * Fetches segment and translog repository names from remote store node attributes. + * Returns a blank {@link HashMap} if the cluster does not contain any remote nodes. + *
    + * Caller need to handle null checks if {@link DiscoveryNodes} object does not have any remote nodes + * * @param discoveryNodes Current set of {@link DiscoveryNodes} in the cluster - * @return {@link Tuple} with segment repository name as first element and translog repository name as second element + * @return {@link Map} of data repository node attributes keys and their values */ public static Map getRemoteStoreRepoName(DiscoveryNodes discoveryNodes) { Optional remoteNode = discoveryNodes.getNodes() @@ -164,18 +169,6 @@ public static Map getRemoteStoreRepoName(DiscoveryNodes discover .stream() .filter(DiscoveryNode::isRemoteStoreNode) .findFirst(); - assert remoteNode.isPresent() : "Cannot fetch remote store repository names as no remote nodes are present in the cluster"; - return remoteNode.get().getRemoteStoreRepoNames(); - } - - /** - * Ensures that the current set of {@link DiscoveryNodes} contains at-least one node - * matching the {@link DiscoveryNode#isRemoteStoreNode()} predicate - * - * @param discoveryNodes Current set of {@link DiscoveryNodes} - * @return true if there is at-least one remote store enabled node, false otherwise - */ - public static boolean hasAtLeastOneRemoteNode(DiscoveryNodes discoveryNodes) { - return discoveryNodes.getNodes().values().stream().anyMatch(DiscoveryNode::isRemoteStoreNode); + return remoteNode.map(RemoteStoreNodeAttribute::getDataRepoNames).orElseGet(HashMap::new); } } diff --git a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java index a3bfe1195d8cc..b10ec0d99c3d5 100644 --- a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java +++ b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java @@ -18,6 +18,7 @@ import org.opensearch.repositories.blobstore.BlobStoreRepository; import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -47,6 +48,11 @@ public class RemoteStoreNodeAttribute { public static final String REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX = "remote_store.repository.%s.settings."; private final RepositoriesMetadata repositoriesMetadata; + public static List SUPPORTED_DATA_REPO_NAME_ATTRIBUTES = List.of( + REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY, + REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY + ); + /** * Creates a new {@link RemoteStoreNodeAttribute} */ @@ -185,6 +191,30 @@ public RepositoriesMetadata getRepositoriesMetadata() { return this.repositoriesMetadata; } + /** + * Return {@link Map} of all the supported data repo names listed on {@link RemoteStoreNodeAttribute#SUPPORTED_DATA_REPO_NAME_ATTRIBUTES} + * + * @param node Node to fetch attributes from + * @return {@link Map} of all remote store data repo attribute keys and their values + */ + public static Map getDataRepoNames(DiscoveryNode node) { + assert remoteDataAttributesPresent(node.getAttributes()); + Map dataRepoNames = new HashMap<>(); + for (String supportedRepoAttribute : SUPPORTED_DATA_REPO_NAME_ATTRIBUTES) { + dataRepoNames.put(supportedRepoAttribute, node.getAttributes().get(supportedRepoAttribute)); + } + return dataRepoNames; + } + + private static boolean remoteDataAttributesPresent(Map nodeAttrs) { + for (String supportedRepoAttributes : SUPPORTED_DATA_REPO_NAME_ATTRIBUTES) { + if (nodeAttrs.get(supportedRepoAttributes) == null || nodeAttrs.get(supportedRepoAttributes).isEmpty()) { + return false; + } + } + return true; + } + @Override public int hashCode() { // The hashCode is generated by computing the hash of all the repositoryMetadata present in diff --git a/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeActionTests.java b/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeActionTests.java index df1eca90d484e..35c5c5e605b4d 100644 --- a/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeActionTests.java +++ b/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeActionTests.java @@ -31,7 +31,6 @@ import org.opensearch.cluster.block.ClusterBlockLevel; import org.opensearch.cluster.block.ClusterBlocks; import org.opensearch.cluster.coordination.FailedToCommitClusterStateException; -import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.IndexNameExpressionResolver; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.node.DiscoveryNode; @@ -56,7 +55,6 @@ import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.rest.RestStatus; import org.opensearch.discovery.ClusterManagerNotDiscoveredException; -import org.opensearch.index.remote.RemoteStoreEnums; import org.opensearch.node.NodeClosedException; import org.opensearch.node.remotestore.RemoteStoreNodeService; import org.opensearch.snapshots.EmptySnapshotsInfoService; @@ -87,8 +85,9 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import static org.opensearch.cluster.metadata.IndexMetadata.REMOTE_STORE_CUSTOM_KEY; import static org.opensearch.common.util.FeatureFlags.REMOTE_STORE_MIGRATION_EXPERIMENTAL; +import static org.opensearch.index.remote.RemoteMigrationIndexMetadataUpdaterTests.createIndexMetadataWithDocrepSettings; +import static org.opensearch.index.remote.RemoteMigrationIndexMetadataUpdaterTests.createIndexMetadataWithRemoteStoreSettings; 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.REMOTE_STORE_COMPATIBILITY_MODE_SETTING; @@ -990,37 +989,4 @@ private Map getRemoteStoreNodeAttributes() { return remoteStoreNodeAttributes; } - private Metadata createIndexMetadataWithRemoteStoreSettings(String indexName) { - IndexMetadata.Builder indexMetadata = IndexMetadata.builder(indexName); - indexMetadata.settings( - Settings.builder() - .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) - .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING.getKey(), true) - .put(IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.getKey(), "dummy-tlog-repo") - .put(IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING.getKey(), "dummy-segment-repo") - .put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.getKey(), "SEGMENT") - .build() - ) - .putCustom( - REMOTE_STORE_CUSTOM_KEY, - Map.of(RemoteStoreEnums.PathType.NAME, "dummy", RemoteStoreEnums.PathHashAlgorithm.NAME, "dummy") - ) - .build(); - return Metadata.builder().put(indexMetadata).build(); - } - - private Metadata createIndexMetadataWithDocrepSettings(String indexName) { - IndexMetadata.Builder indexMetadata = IndexMetadata.builder(indexName); - indexMetadata.settings( - Settings.builder() - .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) - .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.getKey(), "DOCUMENT") - .build() - ).build(); - return Metadata.builder().put(indexMetadata).build(); - } } diff --git a/server/src/test/java/org/opensearch/cluster/routing/IndexShardRoutingTableTests.java b/server/src/test/java/org/opensearch/cluster/routing/IndexShardRoutingTableTests.java index ebb7529d3f733..e881016fb9305 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/IndexShardRoutingTableTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/IndexShardRoutingTableTests.java @@ -39,6 +39,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.UUID; public class IndexShardRoutingTableTests extends OpenSearchTestCase { public void testEqualsAttributesKey() { @@ -69,4 +70,48 @@ public void testEquals() { assertNotEquals(table1, s); assertNotEquals(table1, table3); } + + public void testShardsMatchingPredicate() { + ShardId shardId = new ShardId(new Index("a", UUID.randomUUID().toString()), 0); + ShardRouting primary = TestShardRouting.newShardRouting(shardId, "node-1", true, ShardRoutingState.STARTED); + ShardRouting replica = TestShardRouting.newShardRouting(shardId, "node-2", false, ShardRoutingState.STARTED); + ShardRouting unassignedReplica = ShardRouting.newUnassigned( + shardId, + false, + RecoverySource.PeerRecoverySource.INSTANCE, + new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null) + ); + ShardRouting relocatingReplica1 = TestShardRouting.newShardRouting( + shardId, + "node-3", + "node-4", + false, + ShardRoutingState.RELOCATING + ); + ShardRouting relocatingReplica2 = TestShardRouting.newShardRouting( + shardId, + "node-4", + "node-5", + false, + ShardRoutingState.RELOCATING + ); + + IndexShardRoutingTable table = new IndexShardRoutingTable( + shardId, + Arrays.asList(primary, replica, unassignedReplica, relocatingReplica1, relocatingReplica2) + ); + assertEquals(List.of(primary), table.shardsMatchingPredicate(ShardRouting::primary)); + assertEquals( + List.of(replica, unassignedReplica, relocatingReplica1, relocatingReplica2), + table.shardsMatchingPredicate(shardRouting -> !shardRouting.primary()) + ); + assertEquals( + List.of(unassignedReplica), + table.shardsMatchingPredicate(shardRouting -> !shardRouting.primary() && shardRouting.unassigned()) + ); + assertEquals( + Arrays.asList(relocatingReplica1, relocatingReplica2), + table.shardsMatchingPredicate(shardRouting -> !shardRouting.primary() && shardRouting.relocating()) + ); + } } diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java new file mode 100644 index 0000000000000..224020991d826 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java @@ -0,0 +1,291 @@ +/* + * 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.remote; + +import org.opensearch.Version; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.IndexShardRoutingTable; +import org.opensearch.cluster.routing.RecoverySource; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.ShardRoutingState; +import org.opensearch.cluster.routing.TestShardRouting; +import org.opensearch.cluster.routing.UnassignedInfo; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.index.Index; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.shard.IndexShardTestUtils; +import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; +import java.util.Map; +import java.util.UUID; + +import static org.opensearch.cluster.metadata.IndexMetadata.REMOTE_STORE_CUSTOM_KEY; + +public class RemoteMigrationIndexMetadataUpdaterTests extends OpenSearchTestCase { + private final String tlogRepoName = "test-tlog-repo"; + private final String segmentRepoName = "test-segment-repo"; + private final String indexName = "test-index"; + + public void testMaybeAddRemoteIndexSettingsAllPrimariesAndReplicasOnRemote() throws IOException { + Metadata metadata = createIndexMetadataWithDocrepSettings(indexName); + IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(metadata.index(indexName)); + long currentSettingsVersion = indexMetadataBuilder.settingsVersion(); + DiscoveryNode primaryNode = IndexShardTestUtils.getFakeRemoteEnabledNode("1"); + DiscoveryNode replicaNode = IndexShardTestUtils.getFakeRemoteEnabledNode("2"); + DiscoveryNodes allNodes = DiscoveryNodes.builder().add(primaryNode).add(replicaNode).build(); + RoutingTable routingTable = createRoutingTableAllShardsStarted(indexName, 1, 1, primaryNode, replicaNode); + RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater(logger); + migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings( + metadata.index(indexName), + indexMetadataBuilder, + routingTable, + indexName, + allNodes, + segmentRepoName, + tlogRepoName + ); + assertTrue(currentSettingsVersion < indexMetadataBuilder.settingsVersion()); + assertRemoteSettingsApplied(indexMetadataBuilder.build()); + } + + public void testMaybeAddRemoteIndexSettingsDoesNotRunWhenSettingsAlreadyPresent() throws IOException { + Metadata metadata = createIndexMetadataWithRemoteStoreSettings(indexName); + IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(metadata.index(indexName)); + long currentSettingsVersion = indexMetadataBuilder.settingsVersion(); + DiscoveryNode primaryNode = IndexShardTestUtils.getFakeRemoteEnabledNode("1"); + DiscoveryNode replicaNode = IndexShardTestUtils.getFakeRemoteEnabledNode("2"); + DiscoveryNodes allNodes = DiscoveryNodes.builder().add(primaryNode).add(replicaNode).build(); + RoutingTable routingTable = createRoutingTableAllShardsStarted(indexName, 1, 1, primaryNode, replicaNode); + RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater(logger); + migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings( + metadata.index(indexName), + indexMetadataBuilder, + routingTable, + indexName, + allNodes, + segmentRepoName, + tlogRepoName + ); + assertEquals(currentSettingsVersion, indexMetadataBuilder.settingsVersion()); + } + + public void testMaybeAddRemoteIndexSettingsDoesNotUpdateSettingsWhenAllShardsInDocrep() throws IOException { + Metadata metadata = createIndexMetadataWithDocrepSettings(indexName); + IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(metadata.index(indexName)); + long currentSettingsVersion = indexMetadataBuilder.settingsVersion(); + DiscoveryNode primaryNode = IndexShardTestUtils.getFakeDiscoNode("1"); + DiscoveryNode replicaNode = IndexShardTestUtils.getFakeDiscoNode("2"); + DiscoveryNodes allNodes = DiscoveryNodes.builder().add(primaryNode).add(replicaNode).build(); + RoutingTable routingTable = createRoutingTableAllShardsStarted(indexName, 1, 1, primaryNode, replicaNode); + RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater(logger); + migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings( + metadata.index(indexName), + indexMetadataBuilder, + routingTable, + indexName, + allNodes, + segmentRepoName, + tlogRepoName + ); + assertEquals(currentSettingsVersion, indexMetadataBuilder.settingsVersion()); + assertDocrepSettingsApplied(indexMetadataBuilder.build()); + } + + public void testMaybeAddRemoteIndexSettingsUpdatesIndexSettingsWithUnassignedReplicas() throws IOException { + Metadata metadata = createIndexMetadataWithDocrepSettings(indexName); + IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(metadata.index(indexName)); + long currentSettingsVersion = indexMetadataBuilder.settingsVersion(); + DiscoveryNode primaryNode = IndexShardTestUtils.getFakeRemoteEnabledNode("1"); + DiscoveryNode replicaNode = IndexShardTestUtils.getFakeDiscoNode("2"); + DiscoveryNodes allNodes = DiscoveryNodes.builder().add(primaryNode).add(replicaNode).build(); + RoutingTable routingTable = createRoutingTableReplicasUnassigned(indexName, 1, 1, primaryNode); + RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater(logger); + migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings( + metadata.index(indexName), + indexMetadataBuilder, + routingTable, + indexName, + allNodes, + segmentRepoName, + tlogRepoName + ); + assertTrue(currentSettingsVersion < indexMetadataBuilder.settingsVersion()); + assertRemoteSettingsApplied(indexMetadataBuilder.build()); + } + + public void testMaybeAddRemoteIndexSettingsDoesNotUpdateIndexSettingsWithRelocatingReplicas() throws IOException { + Metadata metadata = createIndexMetadataWithDocrepSettings(indexName); + IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(metadata.index(indexName)); + long currentSettingsVersion = indexMetadataBuilder.settingsVersion(); + DiscoveryNode primaryNode = IndexShardTestUtils.getFakeRemoteEnabledNode("1"); + DiscoveryNode replicaNode = IndexShardTestUtils.getFakeDiscoNode("2"); + DiscoveryNode replicaRelocatingNode = IndexShardTestUtils.getFakeDiscoNode("3"); + DiscoveryNodes allNodes = DiscoveryNodes.builder().add(primaryNode).add(replicaNode).build(); + RoutingTable routingTable = createRoutingTableReplicasRelocating(indexName, 1, 1, primaryNode, replicaNode, replicaRelocatingNode); + RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater(logger); + migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings( + metadata.index(indexName), + indexMetadataBuilder, + routingTable, + indexName, + allNodes, + segmentRepoName, + tlogRepoName + ); + assertEquals(currentSettingsVersion, indexMetadataBuilder.settingsVersion()); + assertDocrepSettingsApplied(indexMetadataBuilder.build()); + } + + private RoutingTable createRoutingTableAllShardsStarted( + String indexName, + int numberOfShards, + int numberOfReplicas, + DiscoveryNode primaryHostingNode, + DiscoveryNode replicaHostingNode + ) { + RoutingTable.Builder builder = RoutingTable.builder(); + Index index = new Index(indexName, UUID.randomUUID().toString()); + + IndexRoutingTable.Builder indexRoutingTableBuilder = IndexRoutingTable.builder(index); + for (int i = 0; i < numberOfShards; i++) { + ShardId shardId = new ShardId(index, i); + IndexShardRoutingTable.Builder indexShardRoutingTable = new IndexShardRoutingTable.Builder(shardId); + indexShardRoutingTable.addShard( + TestShardRouting.newShardRouting(shardId, primaryHostingNode.getId(), true, ShardRoutingState.STARTED) + ); + for (int j = 0; j < numberOfReplicas; j++) { + indexShardRoutingTable.addShard( + TestShardRouting.newShardRouting(shardId, replicaHostingNode.getId(), false, ShardRoutingState.STARTED) + ); + } + indexRoutingTableBuilder.addIndexShard(indexShardRoutingTable.build()); + } + return builder.add(indexRoutingTableBuilder.build()).build(); + } + + private RoutingTable createRoutingTableReplicasUnassigned( + String indexName, + int numberOfShards, + int numberOfReplicas, + DiscoveryNode primaryHostingNode + ) { + RoutingTable.Builder builder = RoutingTable.builder(); + Index index = new Index(indexName, UUID.randomUUID().toString()); + + IndexRoutingTable.Builder indexRoutingTableBuilder = IndexRoutingTable.builder(index); + for (int i = 0; i < numberOfShards; i++) { + ShardId shardId = new ShardId(index, i); + IndexShardRoutingTable.Builder indexShardRoutingTable = new IndexShardRoutingTable.Builder(shardId); + indexShardRoutingTable.addShard( + TestShardRouting.newShardRouting(shardId, primaryHostingNode.getId(), true, ShardRoutingState.STARTED) + ); + for (int j = 0; j < numberOfReplicas; j++) { + indexShardRoutingTable.addShard( + ShardRouting.newUnassigned( + shardId, + false, + RecoverySource.PeerRecoverySource.INSTANCE, + new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null) + ) + ); + } + indexRoutingTableBuilder.addIndexShard(indexShardRoutingTable.build()); + } + return builder.add(indexRoutingTableBuilder.build()).build(); + } + + private RoutingTable createRoutingTableReplicasRelocating( + String indexName, + int numberOfShards, + int numberOfReplicas, + DiscoveryNode primaryHostingNodes, + DiscoveryNode replicaHostingNode, + DiscoveryNode replicaRelocatingNode + ) { + RoutingTable.Builder builder = RoutingTable.builder(); + Index index = new Index(indexName, UUID.randomUUID().toString()); + + IndexRoutingTable.Builder indexRoutingTableBuilder = IndexRoutingTable.builder(index); + for (int i = 0; i < numberOfShards; i++) { + ShardId shardId = new ShardId(index, i); + IndexShardRoutingTable.Builder indexShardRoutingTable = new IndexShardRoutingTable.Builder(shardId); + indexShardRoutingTable.addShard( + TestShardRouting.newShardRouting(shardId, primaryHostingNodes.getId(), true, ShardRoutingState.STARTED) + ); + for (int j = 0; j < numberOfReplicas; j++) { + indexShardRoutingTable.addShard( + TestShardRouting.newShardRouting( + shardId, + replicaHostingNode.getId(), + replicaRelocatingNode.getId(), + false, + ShardRoutingState.RELOCATING + ) + ); + } + indexRoutingTableBuilder.addIndexShard(indexShardRoutingTable.build()); + } + return builder.add(indexRoutingTableBuilder.build()).build(); + } + + public static Metadata createIndexMetadataWithRemoteStoreSettings(String indexName) { + IndexMetadata.Builder indexMetadata = IndexMetadata.builder(indexName); + indexMetadata.settings( + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING.getKey(), true) + .put(IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.getKey(), "dummy-tlog-repo") + .put(IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING.getKey(), "dummy-segment-repo") + .put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.getKey(), "SEGMENT") + .build() + ) + .putCustom( + REMOTE_STORE_CUSTOM_KEY, + Map.of(RemoteStoreEnums.PathType.NAME, "dummy", RemoteStoreEnums.PathHashAlgorithm.NAME, "dummy") + ) + .build(); + return Metadata.builder().put(indexMetadata).build(); + } + + public static Metadata createIndexMetadataWithDocrepSettings(String indexName) { + IndexMetadata.Builder indexMetadata = IndexMetadata.builder(indexName); + indexMetadata.settings( + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.getKey(), "DOCUMENT") + .build() + ).build(); + return Metadata.builder().put(indexMetadata).build(); + } + + private void assertRemoteSettingsApplied(IndexMetadata indexMetadata) { + assertTrue(IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING.get(indexMetadata.getSettings())); + assertTrue(IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.exists(indexMetadata.getSettings())); + assertTrue(IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING.exists(indexMetadata.getSettings())); + assertEquals(ReplicationType.SEGMENT, IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.get(indexMetadata.getSettings())); + } + + private void assertDocrepSettingsApplied(IndexMetadata indexMetadata) { + assertFalse(IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING.get(indexMetadata.getSettings())); + assertFalse(IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.exists(indexMetadata.getSettings())); + assertFalse(IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING.exists(indexMetadata.getSettings())); + assertEquals(ReplicationType.DOCUMENT, IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.get(indexMetadata.getSettings())); + } +} From 9a915bd99023e5be3189dea2d1cbf4d2975654a9 Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Tue, 23 Apr 2024 15:32:59 +0530 Subject: [PATCH 13/24] Changing check for switching to strict mode Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../cluster/settings/TransportClusterUpdateSettingsAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java index 339c9e347fcb7..6292d32fee26d 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java @@ -332,7 +332,7 @@ private void validateAllNodesOfSameType(DiscoveryNodes discoveryNodes) { private void validateIndexSettings(ClusterState clusterState) { Collection allIndicesMetadata = clusterState.metadata().indices().values(); if (allIndicesMetadata.isEmpty() == false - && allIndicesMetadata.stream().allMatch(indexMetadata -> indexHasAllRemoteStoreRelatedMetadata(indexMetadata) == false)) { + && allIndicesMetadata.stream().anyMatch(indexMetadata -> indexHasAllRemoteStoreRelatedMetadata(indexMetadata) == false)) { throw new SettingsException( "can not switch to STRICT compatibility mode since all indices in the cluster does not have remote store based index settings" ); From 10c52479312fcf7f25c3b6de405c1482e7b3dde9 Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Tue, 23 Apr 2024 16:53:54 +0530 Subject: [PATCH 14/24] Adding more UTs Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- ...oteMigrationIndexMetadataUpdaterTests.java | 41 +++++++++++++++++++ .../index/remote/RemoteStoreUtilsTests.java | 18 ++++++++ 2 files changed, 59 insertions(+) diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java index 224020991d826..418b2f8916b1e 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java @@ -29,10 +29,13 @@ import org.opensearch.test.OpenSearchTestCase; import java.io.IOException; +import java.util.HashMap; import java.util.Map; import java.util.UUID; import static org.opensearch.cluster.metadata.IndexMetadata.REMOTE_STORE_CUSTOM_KEY; +import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING; +import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING; public class RemoteMigrationIndexMetadataUpdaterTests extends OpenSearchTestCase { private final String tlogRepoName = "test-tlog-repo"; @@ -149,6 +152,29 @@ public void testMaybeAddRemoteIndexSettingsDoesNotUpdateIndexSettingsWithRelocat assertDocrepSettingsApplied(indexMetadataBuilder.build()); } + public void testMaybeUpdateRemoteStorePathStrategyExecutes() { + Metadata currentMetadata = createIndexMetadataWithDocrepSettings(indexName); + IndexMetadata.Builder builder = IndexMetadata.builder(currentMetadata.index(indexName)); + RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater(logger); + DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().add(IndexShardTestUtils.getFakeRemoteEnabledNode("1")).build(); + migrationIndexMetadataUpdater.maybeUpdateRemoteStorePathStrategy(currentMetadata.index(indexName), builder, indexName, discoveryNodes, Settings.builder() + .put(CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.getKey(), RemoteStoreEnums.PathHashAlgorithm.FNV_1A_COMPOSITE_1.name()) + .put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), RemoteStoreEnums.PathType.HASHED_PREFIX.name()).build()); + assertCustomPathMetadataIsPresent(builder.build()); + } + + public void testMaybeUpdateRemoteStorePathStrategyDoesNotExecute() { + Metadata currentMetadata = createIndexMetadataWithRemoteStoreSettings(indexName); + IndexMetadata.Builder builder = IndexMetadata.builder(currentMetadata.index(indexName)); + RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater(logger); + DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().add(IndexShardTestUtils.getFakeRemoteEnabledNode("1")).build(); + migrationIndexMetadataUpdater.maybeUpdateRemoteStorePathStrategy(currentMetadata.index(indexName), builder, indexName, discoveryNodes, Settings.builder() + .put(CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.getKey(), RemoteStoreEnums.PathHashAlgorithm.FNV_1A_COMPOSITE_1.name()) + .put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), RemoteStoreEnums.PathType.HASHED_PREFIX.name()).build()); + + assertCustomPathMetadataIsPresent(builder.build()); + } + private RoutingTable createRoutingTableAllShardsStarted( String indexName, int numberOfShards, @@ -275,6 +301,15 @@ public static Metadata createIndexMetadataWithDocrepSettings(String indexName) { return Metadata.builder().put(indexMetadata).build(); } + private static IndexMetadata createIndexMetadataWithCustomRemotePath(String indexName) { + IndexMetadata.Builder indexMetadata = IndexMetadata.builder(indexName); + Map customRemotePathData = new HashMap<>(); + customRemotePathData.put(RemoteStoreEnums.PathType.NAME, RemoteStoreEnums.PathType.FIXED.name()); + customRemotePathData.put(RemoteStoreEnums.PathHashAlgorithm.NAME, RemoteStoreEnums.PathHashAlgorithm.FNV_1A_BASE64.name()); + indexMetadata.putCustom(REMOTE_STORE_CUSTOM_KEY, customRemotePathData); + return indexMetadata.build(); + } + private void assertRemoteSettingsApplied(IndexMetadata indexMetadata) { assertTrue(IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING.get(indexMetadata.getSettings())); assertTrue(IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.exists(indexMetadata.getSettings())); @@ -288,4 +323,10 @@ private void assertDocrepSettingsApplied(IndexMetadata indexMetadata) { assertFalse(IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING.exists(indexMetadata.getSettings())); assertEquals(ReplicationType.DOCUMENT, IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.get(indexMetadata.getSettings())); } + + private void assertCustomPathMetadataIsPresent(IndexMetadata indexMetadata) { + assertNotNull(indexMetadata.getCustomData(REMOTE_STORE_CUSTOM_KEY)); + assertNotNull(indexMetadata.getCustomData(REMOTE_STORE_CUSTOM_KEY).get(RemoteStoreEnums.PathType.NAME)); + assertNotNull(indexMetadata.getCustomData(REMOTE_STORE_CUSTOM_KEY).get(RemoteStoreEnums.PathHashAlgorithm.NAME)); + } } diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteStoreUtilsTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteStoreUtilsTests.java index 4d3e633848975..c1fc0cdaa0d3b 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteStoreUtilsTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteStoreUtilsTests.java @@ -8,10 +8,13 @@ package org.opensearch.index.remote; +import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.support.PlainBlobMetadata; +import org.opensearch.index.shard.IndexShardTestUtils; import org.opensearch.index.store.RemoteSegmentStoreDirectory; import org.opensearch.index.translog.transfer.TranslogTransferMetadata; +import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; import org.opensearch.test.OpenSearchTestCase; import java.math.BigInteger; @@ -28,6 +31,8 @@ import static org.opensearch.index.remote.RemoteStoreUtils.longToUrlBase64; import static org.opensearch.index.remote.RemoteStoreUtils.urlBase64ToLong; import static org.opensearch.index.remote.RemoteStoreUtils.verifyNoMultipleWriters; +import static org.opensearch.index.shard.IndexShardTestUtils.MOCK_SEGMENT_REPO_NAME; +import static org.opensearch.index.shard.IndexShardTestUtils.MOCK_TLOG_REPO_NAME; import static org.opensearch.index.store.RemoteSegmentStoreDirectory.MetadataFilenameUtils.METADATA_PREFIX; import static org.opensearch.index.store.RemoteSegmentStoreDirectory.MetadataFilenameUtils.SEPARATOR; import static org.opensearch.index.translog.transfer.TranslogTransferMetadata.METADATA_SEPARATOR; @@ -316,6 +321,19 @@ public void testLongToCompositeUrlBase64AndBinaryEncoding() { } } + public void testGetRemoteStoreRepoNameWithRemoteNodes() { + DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().add(IndexShardTestUtils.getFakeRemoteEnabledNode("1")).build(); + Map expected = new HashMap<>(); + expected.put(RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY, MOCK_SEGMENT_REPO_NAME); + expected.put(RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY, MOCK_TLOG_REPO_NAME); + assertEquals(expected, RemoteStoreUtils.getRemoteStoreRepoName(discoveryNodes)); + } + + public void testGetRemoteStoreRepoNameWithDocrepNdoes() { + DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().add(IndexShardTestUtils.getFakeDiscoNode("1")).build(); + assertTrue(RemoteStoreUtils.getRemoteStoreRepoName(discoveryNodes).isEmpty()); + } + static long compositeUrlBase64BinaryEncodingToLong(String encodedValue) { char ch = encodedValue.charAt(0); int base64BitsIntValue = BASE64_CHARSET_IDX_MAP.get(ch); From 8a519480e986fec0e8081d1c5ce06b85eeff9fb6 Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Tue, 23 Apr 2024 16:55:59 +0530 Subject: [PATCH 15/24] Applying Spotless Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../remotestore/RemoteStoreNodeService.java | 1 - ...oteMigrationIndexMetadataUpdaterTests.java | 32 +++++++++++++++---- 2 files changed, 26 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeService.java b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeService.java index 0d29fd36e97c6..874c9408de6c5 100644 --- a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeService.java +++ b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeService.java @@ -17,7 +17,6 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; -import org.opensearch.common.settings.Settings; import org.opensearch.common.util.FeatureFlags; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java index 418b2f8916b1e..0aaaa859a153f 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java @@ -157,9 +157,19 @@ public void testMaybeUpdateRemoteStorePathStrategyExecutes() { IndexMetadata.Builder builder = IndexMetadata.builder(currentMetadata.index(indexName)); RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater(logger); DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().add(IndexShardTestUtils.getFakeRemoteEnabledNode("1")).build(); - migrationIndexMetadataUpdater.maybeUpdateRemoteStorePathStrategy(currentMetadata.index(indexName), builder, indexName, discoveryNodes, Settings.builder() - .put(CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.getKey(), RemoteStoreEnums.PathHashAlgorithm.FNV_1A_COMPOSITE_1.name()) - .put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), RemoteStoreEnums.PathType.HASHED_PREFIX.name()).build()); + migrationIndexMetadataUpdater.maybeUpdateRemoteStorePathStrategy( + currentMetadata.index(indexName), + builder, + indexName, + discoveryNodes, + Settings.builder() + .put( + CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.getKey(), + RemoteStoreEnums.PathHashAlgorithm.FNV_1A_COMPOSITE_1.name() + ) + .put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), RemoteStoreEnums.PathType.HASHED_PREFIX.name()) + .build() + ); assertCustomPathMetadataIsPresent(builder.build()); } @@ -168,9 +178,19 @@ public void testMaybeUpdateRemoteStorePathStrategyDoesNotExecute() { IndexMetadata.Builder builder = IndexMetadata.builder(currentMetadata.index(indexName)); RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater(logger); DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().add(IndexShardTestUtils.getFakeRemoteEnabledNode("1")).build(); - migrationIndexMetadataUpdater.maybeUpdateRemoteStorePathStrategy(currentMetadata.index(indexName), builder, indexName, discoveryNodes, Settings.builder() - .put(CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.getKey(), RemoteStoreEnums.PathHashAlgorithm.FNV_1A_COMPOSITE_1.name()) - .put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), RemoteStoreEnums.PathType.HASHED_PREFIX.name()).build()); + migrationIndexMetadataUpdater.maybeUpdateRemoteStorePathStrategy( + currentMetadata.index(indexName), + builder, + indexName, + discoveryNodes, + Settings.builder() + .put( + CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.getKey(), + RemoteStoreEnums.PathHashAlgorithm.FNV_1A_COMPOSITE_1.name() + ) + .put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), RemoteStoreEnums.PathType.HASHED_PREFIX.name()) + .build() + ); assertCustomPathMetadataIsPresent(builder.build()); } From 3195b7acbc842e04cd80ad5dbfddfe9860e3ddc5 Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Tue, 23 Apr 2024 18:55:13 +0530 Subject: [PATCH 16/24] Fixing Dual Replication ITs after rebasing Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../RemoteDualReplicationIT.java | 29 ++++++++++++------- .../RemoteMigrationIndexMetadataUpdater.java | 9 ++---- 2 files changed, 21 insertions(+), 17 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteDualReplicationIT.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteDualReplicationIT.java index baa2b7503387b..d6e3b98fcb917 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteDualReplicationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteDualReplicationIT.java @@ -30,6 +30,7 @@ import org.opensearch.test.transport.MockTransportService; import java.util.Collection; +import java.util.List; import java.util.Map; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -452,7 +453,7 @@ public void testFailoverRemotePrimaryToDocrepReplica() throws Exception { /* Scenario: - - Starts 2 docrep backed data node + - Starts 2 docrep backed data nodes - Creates an index with 1 replica - Starts 1 remote backed data node - Moves primary copy from docrep to remote through _cluster/reroute @@ -466,13 +467,13 @@ public void testFailoverRemotePrimaryToDocrepReplica() throws Exception { public void testFailoverRemotePrimaryToRemoteReplica() throws Exception { internalCluster().startClusterManagerOnlyNode(); - logger.info("---> Starting 1 docrep data node"); - String docrepNodeName = internalCluster().startDataOnlyNode(); + logger.info("---> Starting 2 docrep data nodes"); + List docrepNodeNames = internalCluster().startDataOnlyNodes(2); internalCluster().validateClusterFormed(); assertEquals(internalCluster().client().admin().cluster().prepareGetRepositories().get().repositories().size(), 0); - logger.info("---> Creating index with 0 replica"); - createIndex(FAILOVER_REMOTE_TO_REMOTE, Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0).build()); + logger.info("---> Creating index with 1 replica"); + createIndex(FAILOVER_REMOTE_TO_REMOTE, Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).build()); ensureGreen(FAILOVER_REMOTE_TO_REMOTE); initDocRepToRemoteMigration(); @@ -495,15 +496,17 @@ public void testFailoverRemotePrimaryToRemoteReplica() throws Exception { AsyncIndexingService asyncIndexingService = new AsyncIndexingService(FAILOVER_REMOTE_TO_REMOTE); asyncIndexingService.startIndexing(); - logger.info("---> Moving primary copy from docrep node {} to remote enabled node {}", docrepNodeName, remoteNodeName1); + String primaryNodeName = primaryNodeName(FAILOVER_REMOTE_TO_REMOTE); + logger.info("---> Moving primary copy from docrep node {} to remote enabled node {}", primaryNodeName, remoteNodeName1); assertAcked( internalCluster().client() .admin() .cluster() .prepareReroute() - .add(new MoveAllocationCommand(FAILOVER_REMOTE_TO_REMOTE, 0, docrepNodeName, remoteNodeName1)) + .add(new MoveAllocationCommand(FAILOVER_REMOTE_TO_REMOTE, 0, primaryNodeName, remoteNodeName1)) .get() ); + waitForRelocation(); ensureGreen(FAILOVER_REMOTE_TO_REMOTE); assertEquals(primaryNodeName(FAILOVER_REMOTE_TO_REMOTE), remoteNodeName1); @@ -518,7 +521,13 @@ public void testFailoverRemotePrimaryToRemoteReplica() throws Exception { .indices() .prepareUpdateSettings() .setIndices(FAILOVER_REMOTE_TO_REMOTE) - .setSettings(Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 2).build()) + .setSettings( + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 2) + // prevent replica copy from being allocated to the extra docrep node + .put("index.routing.allocation.exclude._name", primaryNodeName) + .build() + ) .get() ); ensureGreen(FAILOVER_REMOTE_TO_REMOTE); @@ -547,8 +556,8 @@ public void testFailoverRemotePrimaryToRemoteReplica() throws Exception { logger.info("---> Stop remote store enabled node hosting the primary"); internalCluster().stopRandomNode(InternalTestCluster.nameFilter(remoteNodeName1)); - ensureStableCluster(3); - ensureYellow(FAILOVER_REMOTE_TO_REMOTE); + ensureStableCluster(4); + ensureYellowAndNoInitializingShards(FAILOVER_REMOTE_TO_REMOTE); DiscoveryNodes finalNodes = internalCluster().client().admin().cluster().prepareState().get().getState().getNodes(); waitUntil(() -> { diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java b/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java index 0c3bee9cab7c2..be052904f7d32 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java @@ -183,19 +183,14 @@ public static boolean indexHasRemoteStoreSettings(Settings indexSettings) { /** * Asserts current index metadata customs has the {@link IndexMetadata#REMOTE_STORE_CUSTOM_KEY} key. - * If it does, checks if the following sub-keys are present - * - path_type - * - path_hash_algorithm + * If it does, checks if the path_type sub-key is present * * @param indexMetadata Current index metadata * @return true if all above conditions match. false otherwise */ public static boolean indexHasRemotePathMetadata(IndexMetadata indexMetadata) { Map customMetadata = indexMetadata.getCustomData(REMOTE_STORE_CUSTOM_KEY); - if (Objects.nonNull(customMetadata)) { - return Objects.nonNull(customMetadata.get(PathType.NAME)) && Objects.nonNull(customMetadata.get(PathHashAlgorithm.NAME)); - } - return false; + return Objects.nonNull(customMetadata) && Objects.nonNull(customMetadata.get(PathType.NAME)); } public static void updateRemoteStoreSettings(Settings.Builder settingsBuilder, String segmentRepository, String translogRepository) { From 7e1d9dfa11e7c0d7163330715b811ec30eecde45 Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Tue, 23 Apr 2024 19:39:30 +0530 Subject: [PATCH 17/24] Fixing more UTs after rebasing Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../routing/allocation/FailedShardsRoutingTests.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/server/src/test/java/org/opensearch/cluster/routing/allocation/FailedShardsRoutingTests.java b/server/src/test/java/org/opensearch/cluster/routing/allocation/FailedShardsRoutingTests.java index 04e37e7d958d0..5e3b74ee138ab 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/allocation/FailedShardsRoutingTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/allocation/FailedShardsRoutingTests.java @@ -68,7 +68,8 @@ import static org.opensearch.cluster.routing.ShardRoutingState.STARTED; import static org.opensearch.cluster.routing.ShardRoutingState.UNASSIGNED; import static org.opensearch.common.util.FeatureFlags.REMOTE_STORE_MIGRATION_EXPERIMENTAL; -import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_CLUSTER_STATE_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; import static org.opensearch.node.remotestore.RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING; import static org.hamcrest.Matchers.anyOf; @@ -852,8 +853,10 @@ public void testPreferReplicaOnRemoteNodeForPrimaryPromotion() { // add a remote node and start primary shard Map remoteStoreNodeAttributes = Map.of( - REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY, - "REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_VALUE" + REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY, + "REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_VALUE", + REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY, + "REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_VALUE" ); DiscoveryNode remoteNode1 = new DiscoveryNode( UUIDs.base64UUID(), From 6f9f0c327eb1373370470c753bf30bd46403a2da Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Wed, 24 Apr 2024 21:23:26 +0530 Subject: [PATCH 18/24] Addressing comments Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../allocation/IndexMetadataUpdater.java | 15 +-- .../RemoteMigrationIndexMetadataUpdater.java | 35 ++--- ...oteMigrationIndexMetadataUpdaterTests.java | 125 ++++++++---------- 3 files changed, 82 insertions(+), 93 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java index 6bc7d3023198a..b5ee8d1f464c5 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java @@ -177,20 +177,17 @@ public Metadata applyChanges(Metadata oldMetadata, RoutingTable newRoutingTable, indexMetadataBuilder = updateInSyncAllocations(newRoutingTable, oldIndexMetadata, indexMetadataBuilder, shardId, updates); indexMetadataBuilder = updatePrimaryTerm(oldIndexMetadata, indexMetadataBuilder, shardId, updates); if (ongoingRemoteStoreMigration) { - RemoteMigrationIndexMetadataUpdater migrationImdUpdater = new RemoteMigrationIndexMetadataUpdater(logger); - migrationImdUpdater.maybeUpdateRemoteStorePathStrategy( - oldIndexMetadata, - indexMetadataBuilder, - index.getName(), + RemoteMigrationIndexMetadataUpdater migrationImdUpdater = new RemoteMigrationIndexMetadataUpdater( discoveryNodes, - oldMetadata.settings() + newRoutingTable, + oldIndexMetadata, + oldMetadata.settings(), + logger ); + migrationImdUpdater.maybeUpdateRemoteStorePathStrategy(indexMetadataBuilder, index.getName()); migrationImdUpdater.maybeAddRemoteIndexSettings( - oldIndexMetadata, indexMetadataBuilder, - newRoutingTable, index.getName(), - discoveryNodes, remoteRepoNames.get(REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY), remoteRepoNames.get(REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY) ); diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java b/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java index be052904f7d32..11d5316ac8da0 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java @@ -41,9 +41,24 @@ * @opensearch.internal */ public class RemoteMigrationIndexMetadataUpdater { + private final DiscoveryNodes discoveryNodes; + private final RoutingTable routingTable; + private final Settings settings; + private final IndexMetadata indexMetadata; private final Logger logger; - public RemoteMigrationIndexMetadataUpdater(Logger logger) { + public RemoteMigrationIndexMetadataUpdater( + DiscoveryNodes discoveryNodes, + RoutingTable routingTable, + IndexMetadata indexMetadata, + Settings settings, + Logger logger + + ) { + this.discoveryNodes = discoveryNodes; + this.routingTable = routingTable; + this.settings = settings; + this.indexMetadata = indexMetadata; this.logger = logger; } @@ -54,11 +69,8 @@ public RemoteMigrationIndexMetadataUpdater(Logger logger) { * Also appends the requisite Remote Store Path based custom metadata to the existing index metadata */ public void maybeAddRemoteIndexSettings( - IndexMetadata indexMetadata, IndexMetadata.Builder indexMetadataBuilder, - RoutingTable routingTable, String index, - DiscoveryNodes discoveryNodes, String segmentRepoName, String tlogRepoName ) { @@ -89,7 +101,7 @@ public void maybeAddRemoteIndexSettings( * @param currentIndexSettings current {@link IndexMetadata} from cluster state * @return true or false depending on the met conditions */ - public boolean needsRemoteIndexSettingsUpdate( + private boolean needsRemoteIndexSettingsUpdate( IndexRoutingTable indexRoutingTable, DiscoveryNodes discoveryNodes, Settings currentIndexSettings @@ -116,19 +128,10 @@ public boolean needsRemoteIndexSettingsUpdate( * is not used anywhere in the docrep flow * Checks are in place to make this execution no-op if the index metadata is already present. * - * @param indexMetadata Current {@link IndexMetadata} * @param indexMetadataBuilder Mutated {@link IndexMetadata.Builder} having the previous state updates * @param index index name - * @param discoveryNodes Current {@link DiscoveryNodes} from the cluster state - * @param settings current cluster settings from {@link ClusterState} */ - public void maybeUpdateRemoteStorePathStrategy( - IndexMetadata indexMetadata, - IndexMetadata.Builder indexMetadataBuilder, - String index, - DiscoveryNodes discoveryNodes, - Settings settings - ) { + public void maybeUpdateRemoteStorePathStrategy(IndexMetadata.Builder indexMetadataBuilder, String index) { if (indexHasRemotePathMetadata(indexMetadata) == false) { logger.info("Adding remote store path strategy for index [{}] during migration", index); indexMetadataBuilder.putCustom(REMOTE_STORE_CUSTOM_KEY, createRemoteStorePathTypeMetadata(settings, discoveryNodes)); @@ -144,7 +147,7 @@ public void maybeUpdateRemoteStorePathStrategy( * @param discoveryNodes Current {@link DiscoveryNodes} from the cluster state * @return {@link Map} to be added as custom data in index metadata */ - public Map createRemoteStorePathTypeMetadata(Settings settings, DiscoveryNodes discoveryNodes) { + private Map createRemoteStorePathTypeMetadata(Settings settings, DiscoveryNodes discoveryNodes) { Version minNodeVersion = discoveryNodes.getMinNodeVersion(); PathType pathType = Version.CURRENT.compareTo(minNodeVersion) <= 0 ? CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.get(settings) diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java index 0aaaa859a153f..ffc32b92b5cf4 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java @@ -29,13 +29,13 @@ import org.opensearch.test.OpenSearchTestCase; import java.io.IOException; -import java.util.HashMap; import java.util.Map; import java.util.UUID; import static org.opensearch.cluster.metadata.IndexMetadata.REMOTE_STORE_CUSTOM_KEY; import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING; import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING; +import static org.mockito.Mockito.mock; public class RemoteMigrationIndexMetadataUpdaterTests extends OpenSearchTestCase { private final String tlogRepoName = "test-tlog-repo"; @@ -44,154 +44,152 @@ public class RemoteMigrationIndexMetadataUpdaterTests extends OpenSearchTestCase public void testMaybeAddRemoteIndexSettingsAllPrimariesAndReplicasOnRemote() throws IOException { Metadata metadata = createIndexMetadataWithDocrepSettings(indexName); - IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(metadata.index(indexName)); + IndexMetadata existingIndexMetadata = metadata.index(indexName); + IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(existingIndexMetadata); long currentSettingsVersion = indexMetadataBuilder.settingsVersion(); DiscoveryNode primaryNode = IndexShardTestUtils.getFakeRemoteEnabledNode("1"); DiscoveryNode replicaNode = IndexShardTestUtils.getFakeRemoteEnabledNode("2"); DiscoveryNodes allNodes = DiscoveryNodes.builder().add(primaryNode).add(replicaNode).build(); RoutingTable routingTable = createRoutingTableAllShardsStarted(indexName, 1, 1, primaryNode, replicaNode); - RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater(logger); - migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings( - metadata.index(indexName), - indexMetadataBuilder, - routingTable, - indexName, + RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater( allNodes, - segmentRepoName, - tlogRepoName + routingTable, + existingIndexMetadata, + existingIndexMetadata.getSettings(), + logger ); + migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, indexName, segmentRepoName, tlogRepoName); assertTrue(currentSettingsVersion < indexMetadataBuilder.settingsVersion()); assertRemoteSettingsApplied(indexMetadataBuilder.build()); } public void testMaybeAddRemoteIndexSettingsDoesNotRunWhenSettingsAlreadyPresent() throws IOException { Metadata metadata = createIndexMetadataWithRemoteStoreSettings(indexName); - IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(metadata.index(indexName)); + IndexMetadata existingIndexMetadata = metadata.index(indexName); + IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(existingIndexMetadata); long currentSettingsVersion = indexMetadataBuilder.settingsVersion(); DiscoveryNode primaryNode = IndexShardTestUtils.getFakeRemoteEnabledNode("1"); DiscoveryNode replicaNode = IndexShardTestUtils.getFakeRemoteEnabledNode("2"); DiscoveryNodes allNodes = DiscoveryNodes.builder().add(primaryNode).add(replicaNode).build(); RoutingTable routingTable = createRoutingTableAllShardsStarted(indexName, 1, 1, primaryNode, replicaNode); - RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater(logger); - migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings( - metadata.index(indexName), - indexMetadataBuilder, - routingTable, - indexName, + RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater( allNodes, - segmentRepoName, - tlogRepoName + routingTable, + existingIndexMetadata, + existingIndexMetadata.getSettings(), + logger ); + migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, indexName, segmentRepoName, tlogRepoName); assertEquals(currentSettingsVersion, indexMetadataBuilder.settingsVersion()); } public void testMaybeAddRemoteIndexSettingsDoesNotUpdateSettingsWhenAllShardsInDocrep() throws IOException { Metadata metadata = createIndexMetadataWithDocrepSettings(indexName); - IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(metadata.index(indexName)); + IndexMetadata existingIndexMetadata = metadata.index(indexName); + IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(existingIndexMetadata); long currentSettingsVersion = indexMetadataBuilder.settingsVersion(); DiscoveryNode primaryNode = IndexShardTestUtils.getFakeDiscoNode("1"); DiscoveryNode replicaNode = IndexShardTestUtils.getFakeDiscoNode("2"); DiscoveryNodes allNodes = DiscoveryNodes.builder().add(primaryNode).add(replicaNode).build(); RoutingTable routingTable = createRoutingTableAllShardsStarted(indexName, 1, 1, primaryNode, replicaNode); - RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater(logger); - migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings( - metadata.index(indexName), - indexMetadataBuilder, - routingTable, - indexName, + RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater( allNodes, - segmentRepoName, - tlogRepoName + routingTable, + existingIndexMetadata, + existingIndexMetadata.getSettings(), + logger ); + migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, indexName, segmentRepoName, tlogRepoName); assertEquals(currentSettingsVersion, indexMetadataBuilder.settingsVersion()); assertDocrepSettingsApplied(indexMetadataBuilder.build()); } public void testMaybeAddRemoteIndexSettingsUpdatesIndexSettingsWithUnassignedReplicas() throws IOException { Metadata metadata = createIndexMetadataWithDocrepSettings(indexName); - IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(metadata.index(indexName)); + IndexMetadata existingIndexMetadata = metadata.index(indexName); + IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(existingIndexMetadata); long currentSettingsVersion = indexMetadataBuilder.settingsVersion(); DiscoveryNode primaryNode = IndexShardTestUtils.getFakeRemoteEnabledNode("1"); DiscoveryNode replicaNode = IndexShardTestUtils.getFakeDiscoNode("2"); DiscoveryNodes allNodes = DiscoveryNodes.builder().add(primaryNode).add(replicaNode).build(); RoutingTable routingTable = createRoutingTableReplicasUnassigned(indexName, 1, 1, primaryNode); - RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater(logger); - migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings( - metadata.index(indexName), - indexMetadataBuilder, - routingTable, - indexName, + RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater( allNodes, - segmentRepoName, - tlogRepoName + routingTable, + existingIndexMetadata, + existingIndexMetadata.getSettings(), + logger ); + migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, indexName, segmentRepoName, tlogRepoName); assertTrue(currentSettingsVersion < indexMetadataBuilder.settingsVersion()); assertRemoteSettingsApplied(indexMetadataBuilder.build()); } public void testMaybeAddRemoteIndexSettingsDoesNotUpdateIndexSettingsWithRelocatingReplicas() throws IOException { Metadata metadata = createIndexMetadataWithDocrepSettings(indexName); - IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(metadata.index(indexName)); + IndexMetadata existingIndexMetadata = metadata.index(indexName); + IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(existingIndexMetadata); long currentSettingsVersion = indexMetadataBuilder.settingsVersion(); DiscoveryNode primaryNode = IndexShardTestUtils.getFakeRemoteEnabledNode("1"); DiscoveryNode replicaNode = IndexShardTestUtils.getFakeDiscoNode("2"); DiscoveryNode replicaRelocatingNode = IndexShardTestUtils.getFakeDiscoNode("3"); DiscoveryNodes allNodes = DiscoveryNodes.builder().add(primaryNode).add(replicaNode).build(); RoutingTable routingTable = createRoutingTableReplicasRelocating(indexName, 1, 1, primaryNode, replicaNode, replicaRelocatingNode); - RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater(logger); - migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings( - metadata.index(indexName), - indexMetadataBuilder, - routingTable, - indexName, + RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater( allNodes, - segmentRepoName, - tlogRepoName + routingTable, + existingIndexMetadata, + existingIndexMetadata.getSettings(), + logger ); + migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, indexName, segmentRepoName, tlogRepoName); assertEquals(currentSettingsVersion, indexMetadataBuilder.settingsVersion()); assertDocrepSettingsApplied(indexMetadataBuilder.build()); } public void testMaybeUpdateRemoteStorePathStrategyExecutes() { Metadata currentMetadata = createIndexMetadataWithDocrepSettings(indexName); - IndexMetadata.Builder builder = IndexMetadata.builder(currentMetadata.index(indexName)); - RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater(logger); + IndexMetadata existingIndexMetadata = currentMetadata.index(indexName); + IndexMetadata.Builder builder = IndexMetadata.builder(existingIndexMetadata); DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().add(IndexShardTestUtils.getFakeRemoteEnabledNode("1")).build(); - migrationIndexMetadataUpdater.maybeUpdateRemoteStorePathStrategy( - currentMetadata.index(indexName), - builder, - indexName, + RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater( discoveryNodes, + mock(RoutingTable.class), + existingIndexMetadata, Settings.builder() .put( CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.getKey(), RemoteStoreEnums.PathHashAlgorithm.FNV_1A_COMPOSITE_1.name() ) .put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), RemoteStoreEnums.PathType.HASHED_PREFIX.name()) - .build() + .build(), + logger ); + migrationIndexMetadataUpdater.maybeUpdateRemoteStorePathStrategy(builder, indexName); assertCustomPathMetadataIsPresent(builder.build()); } public void testMaybeUpdateRemoteStorePathStrategyDoesNotExecute() { Metadata currentMetadata = createIndexMetadataWithRemoteStoreSettings(indexName); + IndexMetadata existingIndexMetadata = currentMetadata.index(indexName); IndexMetadata.Builder builder = IndexMetadata.builder(currentMetadata.index(indexName)); - RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater(logger); DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().add(IndexShardTestUtils.getFakeRemoteEnabledNode("1")).build(); - migrationIndexMetadataUpdater.maybeUpdateRemoteStorePathStrategy( - currentMetadata.index(indexName), - builder, - indexName, + RemoteMigrationIndexMetadataUpdater migrationIndexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater( discoveryNodes, + mock(RoutingTable.class), + existingIndexMetadata, Settings.builder() .put( CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.getKey(), RemoteStoreEnums.PathHashAlgorithm.FNV_1A_COMPOSITE_1.name() ) .put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), RemoteStoreEnums.PathType.HASHED_PREFIX.name()) - .build() + .build(), + logger ); + migrationIndexMetadataUpdater.maybeUpdateRemoteStorePathStrategy(builder, indexName); + assertCustomPathMetadataIsPresent(builder.build()); } @@ -321,15 +319,6 @@ public static Metadata createIndexMetadataWithDocrepSettings(String indexName) { return Metadata.builder().put(indexMetadata).build(); } - private static IndexMetadata createIndexMetadataWithCustomRemotePath(String indexName) { - IndexMetadata.Builder indexMetadata = IndexMetadata.builder(indexName); - Map customRemotePathData = new HashMap<>(); - customRemotePathData.put(RemoteStoreEnums.PathType.NAME, RemoteStoreEnums.PathType.FIXED.name()); - customRemotePathData.put(RemoteStoreEnums.PathHashAlgorithm.NAME, RemoteStoreEnums.PathHashAlgorithm.FNV_1A_BASE64.name()); - indexMetadata.putCustom(REMOTE_STORE_CUSTOM_KEY, customRemotePathData); - return indexMetadata.build(); - } - private void assertRemoteSettingsApplied(IndexMetadata indexMetadata) { assertTrue(IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING.get(indexMetadata.getSettings())); assertTrue(IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.exists(indexMetadata.getSettings())); From 5c8df2c17e00640e3ddf7deb39146793be3fd777 Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Wed, 24 Apr 2024 21:44:30 +0530 Subject: [PATCH 19/24] Changing variable name from settings to clusterSettings Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../RemoteMigrationIndexMetadataUpdater.java | 16 ++++++++-------- ...RemoteMigrationIndexMetadataUpdaterTests.java | 10 +++++----- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java b/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java index 11d5316ac8da0..3feb919f7506e 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java @@ -43,7 +43,7 @@ public class RemoteMigrationIndexMetadataUpdater { private final DiscoveryNodes discoveryNodes; private final RoutingTable routingTable; - private final Settings settings; + private final Settings clusterSettings; private final IndexMetadata indexMetadata; private final Logger logger; @@ -51,13 +51,13 @@ public RemoteMigrationIndexMetadataUpdater( DiscoveryNodes discoveryNodes, RoutingTable routingTable, IndexMetadata indexMetadata, - Settings settings, + Settings clusterSettings, Logger logger ) { this.discoveryNodes = discoveryNodes; this.routingTable = routingTable; - this.settings = settings; + this.clusterSettings = clusterSettings; this.indexMetadata = indexMetadata; this.logger = logger; } @@ -134,7 +134,7 @@ private boolean needsRemoteIndexSettingsUpdate( public void maybeUpdateRemoteStorePathStrategy(IndexMetadata.Builder indexMetadataBuilder, String index) { if (indexHasRemotePathMetadata(indexMetadata) == false) { logger.info("Adding remote store path strategy for index [{}] during migration", index); - indexMetadataBuilder.putCustom(REMOTE_STORE_CUSTOM_KEY, createRemoteStorePathTypeMetadata(settings, discoveryNodes)); + indexMetadataBuilder.putCustom(REMOTE_STORE_CUSTOM_KEY, createRemoteStorePathTypeMetadata(clusterSettings, discoveryNodes)); } else { logger.debug("Index {} already has remote store path strategy", index); } @@ -143,18 +143,18 @@ public void maybeUpdateRemoteStorePathStrategy(IndexMetadata.Builder indexMetada /** * Generates the remote store path type information to be added to custom data of index metadata. * - * @param settings Current Cluster settings from {@link ClusterState} + * @param clusterSettings Current Cluster settings from {@link ClusterState} * @param discoveryNodes Current {@link DiscoveryNodes} from the cluster state * @return {@link Map} to be added as custom data in index metadata */ - private Map createRemoteStorePathTypeMetadata(Settings settings, DiscoveryNodes discoveryNodes) { + private Map createRemoteStorePathTypeMetadata(Settings clusterSettings, DiscoveryNodes discoveryNodes) { Version minNodeVersion = discoveryNodes.getMinNodeVersion(); PathType pathType = Version.CURRENT.compareTo(minNodeVersion) <= 0 - ? CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.get(settings) + ? CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.get(clusterSettings) : PathType.FIXED; PathHashAlgorithm pathHashAlgorithm = pathType == PathType.FIXED ? null - : CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.get(settings); + : CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.get(clusterSettings); Map remoteCustomData = new HashMap<>(); remoteCustomData.put(PathType.NAME, pathType.name()); if (Objects.nonNull(pathHashAlgorithm)) { diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java index ffc32b92b5cf4..2f89e90a5bc10 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java @@ -55,7 +55,7 @@ public void testMaybeAddRemoteIndexSettingsAllPrimariesAndReplicasOnRemote() thr allNodes, routingTable, existingIndexMetadata, - existingIndexMetadata.getSettings(), + metadata.settings(), logger ); migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, indexName, segmentRepoName, tlogRepoName); @@ -76,7 +76,7 @@ public void testMaybeAddRemoteIndexSettingsDoesNotRunWhenSettingsAlreadyPresent( allNodes, routingTable, existingIndexMetadata, - existingIndexMetadata.getSettings(), + metadata.settings(), logger ); migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, indexName, segmentRepoName, tlogRepoName); @@ -96,7 +96,7 @@ public void testMaybeAddRemoteIndexSettingsDoesNotUpdateSettingsWhenAllShardsInD allNodes, routingTable, existingIndexMetadata, - existingIndexMetadata.getSettings(), + metadata.settings(), logger ); migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, indexName, segmentRepoName, tlogRepoName); @@ -117,7 +117,7 @@ public void testMaybeAddRemoteIndexSettingsUpdatesIndexSettingsWithUnassignedRep allNodes, routingTable, existingIndexMetadata, - existingIndexMetadata.getSettings(), + metadata.settings(), logger ); migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, indexName, segmentRepoName, tlogRepoName); @@ -139,7 +139,7 @@ public void testMaybeAddRemoteIndexSettingsDoesNotUpdateIndexSettingsWithRelocat allNodes, routingTable, existingIndexMetadata, - existingIndexMetadata.getSettings(), + metadata.settings(), logger ); migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, indexName, segmentRepoName, tlogRepoName); From 98004f824535b20e28f8ce3a5bcc376b93e08314 Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Thu, 25 Apr 2024 11:09:28 +0530 Subject: [PATCH 20/24] Addressing comments Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../routing/allocation/IndexMetadataUpdater.java | 9 +-------- .../remote/RemoteMigrationIndexMetadataUpdater.java | 13 +++++++------ .../RemoteMigrationIndexMetadataUpdaterTests.java | 12 +++++------- 3 files changed, 13 insertions(+), 21 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java index b5ee8d1f464c5..25be7f92fffb9 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java @@ -60,8 +60,6 @@ import java.util.stream.Collectors; import static org.opensearch.index.remote.RemoteStoreUtils.getRemoteStoreRepoName; -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; /** * Observer that tracks changes made to RoutingNodes in order to update the primary terms and in-sync allocation ids in @@ -185,12 +183,7 @@ public Metadata applyChanges(Metadata oldMetadata, RoutingTable newRoutingTable, logger ); migrationImdUpdater.maybeUpdateRemoteStorePathStrategy(indexMetadataBuilder, index.getName()); - migrationImdUpdater.maybeAddRemoteIndexSettings( - indexMetadataBuilder, - index.getName(), - remoteRepoNames.get(REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY), - remoteRepoNames.get(REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY) - ); + migrationImdUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, index.getName()); } } diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java b/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java index 3feb919f7506e..fd4dc42df28d6 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java @@ -32,8 +32,11 @@ import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; +import static org.opensearch.index.remote.RemoteStoreUtils.getRemoteStoreRepoName; import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING; import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING; +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; /** * Utils for checking and mutating cluster state during remote migration @@ -68,18 +71,16 @@ public RemoteMigrationIndexMetadataUpdater( *
    * Also appends the requisite Remote Store Path based custom metadata to the existing index metadata */ - public void maybeAddRemoteIndexSettings( - IndexMetadata.Builder indexMetadataBuilder, - String index, - String segmentRepoName, - String tlogRepoName - ) { + public void maybeAddRemoteIndexSettings(IndexMetadata.Builder indexMetadataBuilder, String index) { Settings currentIndexSettings = indexMetadata.getSettings(); if (needsRemoteIndexSettingsUpdate(routingTable.indicesRouting().get(index), discoveryNodes, currentIndexSettings)) { logger.info( "Index {} does not have remote store based index settings but all primary shards and STARTED replica shards have moved to remote enabled nodes. Applying remote store settings to the index", index ); + Map remoteRepoNames = getRemoteStoreRepoName(discoveryNodes); + String segmentRepoName = remoteRepoNames.get(REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY); + String tlogRepoName = remoteRepoNames.get(REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY); assert Objects.nonNull(segmentRepoName) && Objects.nonNull(tlogRepoName) : "Remote repo names cannot be null"; Settings.Builder indexSettingsBuilder = Settings.builder().put(currentIndexSettings); updateRemoteStoreSettings(indexSettingsBuilder, segmentRepoName, tlogRepoName); diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java index 2f89e90a5bc10..d8220c93e4eeb 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java @@ -38,8 +38,6 @@ import static org.mockito.Mockito.mock; public class RemoteMigrationIndexMetadataUpdaterTests extends OpenSearchTestCase { - private final String tlogRepoName = "test-tlog-repo"; - private final String segmentRepoName = "test-segment-repo"; private final String indexName = "test-index"; public void testMaybeAddRemoteIndexSettingsAllPrimariesAndReplicasOnRemote() throws IOException { @@ -58,7 +56,7 @@ public void testMaybeAddRemoteIndexSettingsAllPrimariesAndReplicasOnRemote() thr metadata.settings(), logger ); - migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, indexName, segmentRepoName, tlogRepoName); + migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, indexName); assertTrue(currentSettingsVersion < indexMetadataBuilder.settingsVersion()); assertRemoteSettingsApplied(indexMetadataBuilder.build()); } @@ -79,7 +77,7 @@ public void testMaybeAddRemoteIndexSettingsDoesNotRunWhenSettingsAlreadyPresent( metadata.settings(), logger ); - migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, indexName, segmentRepoName, tlogRepoName); + migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, indexName); assertEquals(currentSettingsVersion, indexMetadataBuilder.settingsVersion()); } @@ -99,7 +97,7 @@ public void testMaybeAddRemoteIndexSettingsDoesNotUpdateSettingsWhenAllShardsInD metadata.settings(), logger ); - migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, indexName, segmentRepoName, tlogRepoName); + migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, indexName); assertEquals(currentSettingsVersion, indexMetadataBuilder.settingsVersion()); assertDocrepSettingsApplied(indexMetadataBuilder.build()); } @@ -120,7 +118,7 @@ public void testMaybeAddRemoteIndexSettingsUpdatesIndexSettingsWithUnassignedRep metadata.settings(), logger ); - migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, indexName, segmentRepoName, tlogRepoName); + migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, indexName); assertTrue(currentSettingsVersion < indexMetadataBuilder.settingsVersion()); assertRemoteSettingsApplied(indexMetadataBuilder.build()); } @@ -142,7 +140,7 @@ public void testMaybeAddRemoteIndexSettingsDoesNotUpdateIndexSettingsWithRelocat metadata.settings(), logger ); - migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, indexName, segmentRepoName, tlogRepoName); + migrationIndexMetadataUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, indexName); assertEquals(currentSettingsVersion, indexMetadataBuilder.settingsVersion()); assertDocrepSettingsApplied(indexMetadataBuilder.build()); } From e4ce0879b88dbb66f7c9fd47adcef204954c4fb7 Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Tue, 30 Apr 2024 10:43:05 +0530 Subject: [PATCH 21/24] Moving path strategy method to Utils Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../RemoteMigrationIndexMetadataUpdater.java | 35 ++---------- .../index/remote/RemoteStoreUtils.java | 53 +++++++++++++++---- 2 files changed, 49 insertions(+), 39 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java b/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java index fd4dc42df28d6..761fa20ea64e5 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java @@ -9,8 +9,6 @@ package org.opensearch.index.remote; import org.apache.logging.log4j.Logger; -import org.opensearch.Version; -import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.cluster.routing.IndexRoutingTable; @@ -18,11 +16,9 @@ import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.ShardRoutingState; import org.opensearch.common.settings.Settings; -import org.opensearch.index.remote.RemoteStoreEnums.PathHashAlgorithm; import org.opensearch.index.remote.RemoteStoreEnums.PathType; import org.opensearch.indices.replication.common.ReplicationType; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -32,9 +28,8 @@ import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; +import static org.opensearch.index.remote.RemoteStoreUtils.determineRemoteStorePathStrategyDuringMigration; import static org.opensearch.index.remote.RemoteStoreUtils.getRemoteStoreRepoName; -import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING; -import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING; 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; @@ -135,35 +130,15 @@ private boolean needsRemoteIndexSettingsUpdate( public void maybeUpdateRemoteStorePathStrategy(IndexMetadata.Builder indexMetadataBuilder, String index) { if (indexHasRemotePathMetadata(indexMetadata) == false) { logger.info("Adding remote store path strategy for index [{}] during migration", index); - indexMetadataBuilder.putCustom(REMOTE_STORE_CUSTOM_KEY, createRemoteStorePathTypeMetadata(clusterSettings, discoveryNodes)); + indexMetadataBuilder.putCustom( + REMOTE_STORE_CUSTOM_KEY, + determineRemoteStorePathStrategyDuringMigration(clusterSettings, discoveryNodes) + ); } else { logger.debug("Index {} already has remote store path strategy", index); } } - /** - * Generates the remote store path type information to be added to custom data of index metadata. - * - * @param clusterSettings Current Cluster settings from {@link ClusterState} - * @param discoveryNodes Current {@link DiscoveryNodes} from the cluster state - * @return {@link Map} to be added as custom data in index metadata - */ - private Map createRemoteStorePathTypeMetadata(Settings clusterSettings, DiscoveryNodes discoveryNodes) { - Version minNodeVersion = discoveryNodes.getMinNodeVersion(); - PathType pathType = Version.CURRENT.compareTo(minNodeVersion) <= 0 - ? CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.get(clusterSettings) - : PathType.FIXED; - PathHashAlgorithm pathHashAlgorithm = pathType == PathType.FIXED - ? null - : CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.get(clusterSettings); - Map remoteCustomData = new HashMap<>(); - remoteCustomData.put(PathType.NAME, pathType.name()); - if (Objects.nonNull(pathHashAlgorithm)) { - remoteCustomData.put(PathHashAlgorithm.NAME, pathHashAlgorithm.name()); - } - return remoteCustomData; - } - public static boolean indexHasAllRemoteStoreRelatedMetadata(IndexMetadata indexMetadata) { return indexHasRemoteStoreSettings(indexMetadata.getSettings()) && indexHasRemotePathMetadata(indexMetadata); } diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java index 6b4d7adac4b6e..27b1b88034573 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java @@ -10,10 +10,13 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.opensearch.Version; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.collect.Tuple; +import org.opensearch.common.settings.Settings; import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; import java.nio.ByteBuffer; @@ -23,10 +26,13 @@ import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.Optional; import java.util.Objects; +import java.util.Optional; import java.util.function.Function; +import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING; +import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING; + /** * Utils for remote store * @@ -156,22 +162,51 @@ static String longToCompositeBase64AndBinaryEncoding(long value, int len) { return URL_BASE64_CHARSET[base64DecimalValue] + binaryPart; } + /** + * Determines the remote store path strategy by reading the custom data map in IndexMetadata class. + */ public static RemoteStorePathStrategy determineRemoteStorePathStrategy(IndexMetadata indexMetadata) { Map remoteCustomData = indexMetadata.getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY); assert remoteCustomData == null || remoteCustomData.containsKey(RemoteStoreEnums.PathType.NAME); if (remoteCustomData != null && remoteCustomData.containsKey(RemoteStoreEnums.PathType.NAME)) { RemoteStoreEnums.PathType pathType = RemoteStoreEnums.PathType.parseString( - remoteCustomData.get(RemoteStoreEnums.PathType.NAME) + remoteCustomData.get(RemoteStoreEnums.PathType.NAME) ); String hashAlgoStr = remoteCustomData.get(RemoteStoreEnums.PathHashAlgorithm.NAME); RemoteStoreEnums.PathHashAlgorithm hashAlgorithm = Objects.nonNull(hashAlgoStr) - ? RemoteStoreEnums.PathHashAlgorithm.parseString(hashAlgoStr) - : null; + ? RemoteStoreEnums.PathHashAlgorithm.parseString(hashAlgoStr) + : null; return new RemoteStorePathStrategy(pathType, hashAlgorithm); } return new RemoteStorePathStrategy(RemoteStoreEnums.PathType.FIXED); } + /** + * Generates the remote store path type information to be added to custom data of index metadata during migration + * + * @param clusterSettings Current Cluster settings from {@link ClusterState} + * @param discoveryNodes Current {@link DiscoveryNodes} from the cluster state + * @return {@link Map} to be added as custom data in index metadata + */ + public static Map determineRemoteStorePathStrategyDuringMigration( + Settings clusterSettings, + DiscoveryNodes discoveryNodes + ) { + Version minNodeVersion = discoveryNodes.getMinNodeVersion(); + RemoteStoreEnums.PathType pathType = Version.CURRENT.compareTo(minNodeVersion) <= 0 + ? CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.get(clusterSettings) + : RemoteStoreEnums.PathType.FIXED; + RemoteStoreEnums.PathHashAlgorithm pathHashAlgorithm = pathType == RemoteStoreEnums.PathType.FIXED + ? null + : CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.get(clusterSettings); + Map remoteCustomData = new HashMap<>(); + remoteCustomData.put(RemoteStoreEnums.PathType.NAME, pathType.name()); + if (Objects.nonNull(pathHashAlgorithm)) { + remoteCustomData.put(RemoteStoreEnums.PathHashAlgorithm.NAME, pathHashAlgorithm.name()); + } + return remoteCustomData; + } + /** * Fetches segment and translog repository names from remote store node attributes. * Returns a blank {@link HashMap} if the cluster does not contain any remote nodes. @@ -183,10 +218,10 @@ public static RemoteStorePathStrategy determineRemoteStorePathStrategy(IndexMeta */ public static Map getRemoteStoreRepoName(DiscoveryNodes discoveryNodes) { Optional remoteNode = discoveryNodes.getNodes() - .values() - .stream() - .filter(DiscoveryNode::isRemoteStoreNode) - .findFirst(); + .values() + .stream() + .filter(DiscoveryNode::isRemoteStoreNode) + .findFirst(); return remoteNode.map(RemoteStoreNodeAttribute::getDataRepoNames).orElseGet(HashMap::new); } } From 11790722db7e28a8a70b0b41ca3f8bbdee313869 Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Tue, 30 Apr 2024 11:14:29 +0530 Subject: [PATCH 22/24] Removing unused code Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../cluster/routing/allocation/IndexMetadataUpdater.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java index 25be7f92fffb9..ddcccd597e894 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java @@ -59,8 +59,6 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.opensearch.index.remote.RemoteStoreUtils.getRemoteStoreRepoName; - /** * Observer that tracks changes made to RoutingNodes in order to update the primary terms and in-sync allocation ids in * {@link IndexMetadata} once the allocation round has completed. @@ -160,10 +158,6 @@ public Metadata applyChanges(Metadata oldMetadata, RoutingTable newRoutingTable, Map>> changesGroupedByIndex = shardChanges.entrySet() .stream() .collect(Collectors.groupingBy(e -> e.getKey().getIndex())); - Map remoteRepoNames = new HashMap<>(); - if (ongoingRemoteStoreMigration) { - remoteRepoNames = getRemoteStoreRepoName(discoveryNodes); - } Metadata.Builder metadataBuilder = null; for (Map.Entry>> indexChanges : changesGroupedByIndex.entrySet()) { Index index = indexChanges.getKey(); From 4d37794308045c374b150a9717ce2e4785172235 Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Tue, 30 Apr 2024 12:23:00 +0530 Subject: [PATCH 23/24] Retrigger integs Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> From a19c5cd1bebf9df619558eee82723bdd42b3fe52 Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Tue, 30 Apr 2024 13:12:36 +0530 Subject: [PATCH 24/24] Retrigger integs Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com>