From ed800083ce4e19fffaeddadcd3dad1d57a4ecf33 Mon Sep 17 00:00:00 2001 From: "opensearch-trigger-bot[bot]" <98922864+opensearch-trigger-bot[bot]@users.noreply.github.com> Date: Wed, 24 Jan 2024 11:47:43 -0800 Subject: [PATCH] Support index level allocation filtering for searchable snapshot index (#11563) (#11927) * support index level allocation filtering * run spotless * fix IT * fix not statement and add change log --------- (cherry picked from commit 2f81c5761ae04921b5ecca2299c466998865d7ad) Signed-off-by: panguixin Signed-off-by: github-actions[bot] Co-authored-by: github-actions[bot] --- CHANGELOG.md | 1 + .../snapshots/SearchableSnapshotIT.java | 71 +++++++++- .../put/TransportUpdateSettingsAction.java | 2 +- .../cluster/routing/RoutingNodes.java | 17 --- .../allocator/RemoteShardsBalancer.java | 129 ++++++++++-------- .../cluster/routing/RoutingNodesTests.java | 38 ------ .../RemoteShardsMoveShardsTests.java | 9 +- 7 files changed, 151 insertions(+), 116 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 3bfc84e0f459c..bc49552d81b63 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -44,6 +44,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Introduce new feature flag "WRITEABLE_REMOTE_INDEX" to gate the writeable remote index functionality ([#11717](https://github.com/opensearch-project/OpenSearch/pull/11170)) - Bump OpenTelemetry from 1.32.0 to 1.34.1 ([#11891](https://github.com/opensearch-project/OpenSearch/pull/11891)) - Add `org.opensearch.rest.MethodHandlers` and `RestController#getAllHandlers` ([11876](https://github.com/opensearch-project/OpenSearch/pull/11876)) +- Support index level allocation filtering for searchable snapshot index ([#11563](https://github.com/opensearch-project/OpenSearch/pull/11563)) ### Dependencies - Bumps jetty version to 9.4.52.v20230823 to fix GMS-2023-1857 ([#9822](https://github.com/opensearch-project/OpenSearch/pull/9822)) diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/SearchableSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/SearchableSnapshotIT.java index b4bf6b6ccae29..21554a8e4fb15 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/SearchableSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/SearchableSnapshotIT.java @@ -7,6 +7,7 @@ import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; import com.carrotsearch.randomizedtesting.generators.RandomPicks; +import org.opensearch.action.admin.cluster.health.ClusterHealthResponse; import org.opensearch.action.admin.cluster.node.stats.NodeStats; import org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest; import org.opensearch.action.admin.cluster.node.stats.NodesStatsResponse; @@ -25,11 +26,14 @@ import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.block.ClusterBlockException; import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.routing.GroupShardsIterator; import org.opensearch.cluster.routing.ShardIterator; import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.common.Priority; import org.opensearch.common.io.PathUtils; import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; import org.opensearch.core.common.unit.ByteSizeUnit; import org.opensearch.core.index.Index; import org.opensearch.index.IndexModule; @@ -47,6 +51,8 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.Stream; import java.util.stream.StreamSupport; @@ -235,6 +241,62 @@ public void testSearchableSnapshotAllocationForLocalAndRemoteShardsOnSameNode() assertDocCount(indexName, 100L); } + public void testSearchableSnapshotAllocationFilterSettings() throws Exception { + final int numShardsIndex = randomIntBetween(3, 6); + final String indexName = "test-idx"; + final String restoredIndexName = indexName + "-copy"; + final String repoName = "test-repo"; + final String snapshotName = "test-snap"; + final Client client = client(); + + internalCluster().ensureAtLeastNumSearchAndDataNodes(numShardsIndex); + createIndexWithDocsAndEnsureGreen(numShardsIndex, 1, 100, indexName); + createRepositoryWithSettings(null, repoName); + takeSnapshot(client, snapshotName, repoName, indexName); + + restoreSnapshotAndEnsureGreen(client, snapshotName, repoName); + assertRemoteSnapshotIndexSettings(client, restoredIndexName); + final Set searchNodes = StreamSupport.stream(clusterService().state().getNodes().spliterator(), false) + .filter(DiscoveryNode::isSearchNode) + .map(DiscoveryNode::getId) + .collect(Collectors.toSet()); + + for (int i = searchNodes.size(); i > 2; --i) { + String pickedNode = randomFrom(searchNodes); + searchNodes.remove(pickedNode); + assertIndexAssignedToNodeOrNot(restoredIndexName, pickedNode, true); + assertTrue( + client.admin() + .indices() + .prepareUpdateSettings(restoredIndexName) + .setSettings(Settings.builder().put("index.routing.allocation.exclude._id", pickedNode)) + .execute() + .actionGet() + .isAcknowledged() + ); + ClusterHealthResponse clusterHealthResponse = client.admin() + .cluster() + .prepareHealth() + .setWaitForEvents(Priority.LANGUID) + .setWaitForNoRelocatingShards(true) + .setTimeout(new TimeValue(5, TimeUnit.MINUTES)) + .execute() + .actionGet(); + assertThat(clusterHealthResponse.isTimedOut(), equalTo(false)); + assertIndexAssignedToNodeOrNot(restoredIndexName, pickedNode, false); + assertIndexAssignedToNodeOrNot(indexName, pickedNode, true); + } + } + + private void assertIndexAssignedToNodeOrNot(String index, String node, boolean assigned) { + final ClusterState state = clusterService().state(); + if (assigned) { + assertTrue(state.getRoutingTable().allShards(index).stream().anyMatch(shard -> shard.currentNodeId().equals(node))); + } else { + assertTrue(state.getRoutingTable().allShards(index).stream().noneMatch(shard -> shard.currentNodeId().equals(node))); + } + } + /** * Tests the functionality of remote shard allocation to * ensure it can handle node drops for failover scenarios and the cluster gets back to a healthy state when @@ -342,11 +404,16 @@ public void testDeleteSearchableSnapshotBackingIndex() throws Exception { } private void createIndexWithDocsAndEnsureGreen(int numReplicasIndex, int numOfDocs, String indexName) throws InterruptedException { + createIndexWithDocsAndEnsureGreen(1, numReplicasIndex, numOfDocs, indexName); + } + + private void createIndexWithDocsAndEnsureGreen(int numShardsIndex, int numReplicasIndex, int numOfDocs, String indexName) + throws InterruptedException { createIndex( indexName, Settings.builder() - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, Integer.toString(numReplicasIndex)) - .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, "1") + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, numReplicasIndex) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, numShardsIndex) .put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), IndexModule.Type.FS.getSettingsKey()) .build() ); diff --git a/server/src/main/java/org/opensearch/action/admin/indices/settings/put/TransportUpdateSettingsAction.java b/server/src/main/java/org/opensearch/action/admin/indices/settings/put/TransportUpdateSettingsAction.java index 14c985f1d3427..9265c6ae60678 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/settings/put/TransportUpdateSettingsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/settings/put/TransportUpdateSettingsAction.java @@ -82,7 +82,7 @@ public class TransportUpdateSettingsAction extends TransportClusterManagerNodeAc "index.number_of_replicas" ); - private final static String[] ALLOWLIST_REMOTE_SNAPSHOT_SETTINGS_PREFIXES = { "index.search.slowlog" }; + private final static String[] ALLOWLIST_REMOTE_SNAPSHOT_SETTINGS_PREFIXES = { "index.search.slowlog", "index.routing.allocation" }; private final MetadataUpdateSettingsService updateSettingsService; diff --git a/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java b/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java index bc28cd2521cd9..35752027e46ea 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java @@ -729,23 +729,6 @@ assert node(failedShard.currentNodeId()).getByShardId(failedShard.shardId()) == + " was matched but wasn't removed"; } - public void swapPrimaryWithReplica( - Logger logger, - ShardRouting primaryShard, - ShardRouting replicaShard, - RoutingChangesObserver changes - ) { - assert primaryShard.primary() : "Invalid primary shard provided"; - assert !replicaShard.primary() : "Invalid Replica shard provided"; - - ShardRouting newPrimary = primaryShard.moveActivePrimaryToReplica(); - ShardRouting newReplica = replicaShard.moveActiveReplicaToPrimary(); - updateAssigned(primaryShard, newPrimary); - updateAssigned(replicaShard, newReplica); - logger.info("Swap relocation performed for shard [{}]", newPrimary.shortSummary()); - changes.replicaPromoted(newPrimary); - } - private void unassignPrimaryAndPromoteActiveReplicaIfExists( ShardRouting failedShard, UnassignedInfo unassignedInfo, diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/RemoteShardsBalancer.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/RemoteShardsBalancer.java index 8a14ce3f1a288..e2f24e5f503df 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/RemoteShardsBalancer.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/RemoteShardsBalancer.java @@ -84,24 +84,39 @@ void moveShards() { Queue excludedNodes = new ArrayDeque<>(); classifyNodesForShardMovement(eligibleNodes, excludedNodes); - if (excludedNodes.isEmpty()) { - logger.debug("No excluded nodes found. Returning..."); - return; - } - - while (!eligibleNodes.isEmpty() && !excludedNodes.isEmpty()) { - RoutingNode sourceNode = excludedNodes.poll(); - for (ShardRouting ineligibleShard : sourceNode) { - if (ineligibleShard.started() == false) { + // move shards that cannot remain on eligible nodes + final List forceMoveShards = new ArrayList<>(); + eligibleNodes.forEach(sourceNode -> { + for (final ShardRouting shardRouting : sourceNode) { + if (ineligibleForMove(shardRouting)) { continue; } - if (!RoutingPool.REMOTE_CAPABLE.equals(RoutingPool.getShardPool(ineligibleShard, allocation))) { + if (allocation.deciders().canRemain(shardRouting, sourceNode, allocation) == Decision.NO) { + forceMoveShards.add(shardRouting); + } + } + }); + for (final ShardRouting shard : forceMoveShards) { + if (eligibleNodes.isEmpty()) { + logger.trace("there are no eligible nodes available, return"); + return; + } + + tryShardMovementToEligibleNode(eligibleNodes, shard); + } + + // move shards that are currently assigned on excluded nodes + while (eligibleNodes.isEmpty() == false && excludedNodes.isEmpty() == false) { + RoutingNode sourceNode = excludedNodes.poll(); + for (final ShardRouting ineligibleShard : sourceNode) { + if (ineligibleForMove(ineligibleShard)) { continue; } if (eligibleNodes.isEmpty()) { - break; + logger.trace("there are no eligible nodes available, return"); + return; } tryShardMovementToEligibleNode(eligibleNodes, ineligibleShard); @@ -109,6 +124,10 @@ void moveShards() { } } + private boolean ineligibleForMove(ShardRouting shard) { + return shard.started() == false || RoutingPool.REMOTE_CAPABLE.equals(RoutingPool.getShardPool(shard, allocation)) == false; + } + /** * Classifies the nodes into eligible and excluded depending on whether node is able or unable for shard assignment * @param eligibleNodes contains the list of classified nodes eligible to accept shards @@ -145,10 +164,23 @@ private void classifyNodesForShardMovement(Queue eligibleNodes, Que * @param shard the ineligible shard to be moved */ private void tryShardMovementToEligibleNode(Queue eligibleNodes, ShardRouting shard) { - Set nodesCheckedForShard = new HashSet<>(); - while (!eligibleNodes.isEmpty()) { - RoutingNode targetNode = eligibleNodes.poll(); - Decision currentShardDecision = allocation.deciders().canAllocate(shard, targetNode, allocation); + final Set nodesCheckedForShard = new HashSet<>(); + int numNodesToCheck = eligibleNodes.size(); + while (eligibleNodes.isEmpty() == false) { + assert numNodesToCheck > 0; + final RoutingNode targetNode = eligibleNodes.poll(); + --numNodesToCheck; + // skip the node that the target shard is currently allocated on + if (targetNode.nodeId().equals(shard.currentNodeId())) { + assert nodesCheckedForShard.add(targetNode.nodeId()); + eligibleNodes.offer(targetNode); + if (numNodesToCheck == 0) { + return; + } + continue; + } + + final Decision currentShardDecision = allocation.deciders().canAllocate(shard, targetNode, allocation); if (currentShardDecision.type() == Decision.Type.YES) { if (logger.isDebugEnabled()) { @@ -166,7 +198,7 @@ private void tryShardMovementToEligibleNode(Queue eligibleNodes, Sh allocation.changes() ); eligibleNodes.offer(targetNode); - break; + return; } else { if (logger.isTraceEnabled()) { logger.trace( @@ -177,18 +209,19 @@ private void tryShardMovementToEligibleNode(Queue eligibleNodes, Sh ); } - Decision nodeLevelDecision = allocation.deciders().canAllocateAnyShardToNode(targetNode, allocation); + final Decision nodeLevelDecision = allocation.deciders().canAllocateAnyShardToNode(targetNode, allocation); if (nodeLevelDecision.type() == Decision.Type.YES) { logger.debug("Node: [{}] can still accept shards. Adding it back to the queue.", targetNode.nodeId()); eligibleNodes.offer(targetNode); - nodesCheckedForShard.add(targetNode.nodeId()); + assert nodesCheckedForShard.add(targetNode.nodeId()); } else { logger.debug("Node: [{}] cannot accept any more shards. Removing it from queue.", targetNode.nodeId()); } - // Break out if all nodes in the queue have been checked for this shard - if (eligibleNodes.stream().allMatch(rn -> nodesCheckedForShard.contains(rn.nodeId()))) { - break; + // Break out if all eligible nodes have been examined + if (numNodesToCheck == 0) { + assert eligibleNodes.stream().allMatch(rn -> nodesCheckedForShard.contains(rn.nodeId())); + return; } } } @@ -225,7 +258,7 @@ void balance() { } } - while (!sourceNodes.isEmpty() && !targetNodes.isEmpty()) { + while (sourceNodes.isEmpty() == false && targetNodes.isEmpty() == false) { RoutingNode sourceNode = sourceNodes.poll(); tryRebalanceNode(sourceNode, targetNodes, avgPrimaryPerNode, nodePrimaryShardCount); } @@ -275,11 +308,11 @@ public Map groupUnassignedShardsByIndex() { HashMap unassignedShardMap = new HashMap<>(); for (ShardRouting shard : routingNodes.unassigned().drain()) { String index = shard.getIndexName(); - if (!RoutingPool.REMOTE_CAPABLE.equals(RoutingPool.getShardPool(shard, allocation))) { + if (RoutingPool.REMOTE_CAPABLE.equals(RoutingPool.getShardPool(shard, allocation)) == false) { routingNodes.unassigned().add(shard); continue; } - if (!unassignedShardMap.containsKey(index)) { + if (unassignedShardMap.containsKey(index) == false) { unassignedShardMap.put(index, new UnassignedIndexShards()); } unassignedShardMap.get(index).addShard(shard); @@ -296,13 +329,15 @@ private void unassignIgnoredRemoteShards(RoutingAllocation routingAllocation) { RoutingNodes.UnassignedShards unassignedShards = routingAllocation.routingNodes().unassigned(); for (ShardRouting shard : unassignedShards.drainIgnored()) { RoutingPool pool = RoutingPool.getShardPool(shard, routingAllocation); - if (pool == RoutingPool.REMOTE_CAPABLE && shard.unassigned() && (shard.primary() || !shard.unassignedInfo().isDelayed())) { + if (pool == RoutingPool.REMOTE_CAPABLE + && shard.unassigned() + && (shard.primary() || shard.unassignedInfo().isDelayed() == false)) { ShardRouting unassignedShard = shard; // Shard when moved to an unassigned state updates the recovery source to be ExistingStoreRecoverySource // Remote shards do not have an existing store to recover from and can be recovered from an empty source // to re-fetch any shard blocks from the repository. if (shard.primary()) { - if (!RecoverySource.Type.SNAPSHOT.equals(shard.recoverySource().getType())) { + if (RecoverySource.Type.SNAPSHOT.equals(shard.recoverySource().getType()) == false) { unassignedShard = shard.updateUnassigned(shard.unassignedInfo(), RecoverySource.EmptyStoreRecoverySource.INSTANCE); } } @@ -353,7 +388,7 @@ private void allocateUnassignedShards( } logger.debug("Allocating shards for index: [{}]", index); - while (!shardsToAllocate.isEmpty() && !nodeQueue.isEmpty()) { + while (shardsToAllocate.isEmpty() == false && nodeQueue.isEmpty() == false) { ShardRouting shard = shardsToAllocate.poll(); if (shard.assignedToNode()) { if (logger.isDebugEnabled()) { @@ -390,7 +425,7 @@ private void tryAllocateUnassignedShard(Queue nodeQueue, ShardRouti boolean allocated = false; boolean throttled = false; Set nodesCheckedForShard = new HashSet<>(); - while (!nodeQueue.isEmpty()) { + while (nodeQueue.isEmpty() == false) { RoutingNode node = nodeQueue.poll(); Decision allocateDecision = allocation.deciders().canAllocate(shard, node, allocation); nodesCheckedForShard.add(node.nodeId()); @@ -449,7 +484,7 @@ private void tryAllocateUnassignedShard(Queue nodeQueue, ShardRouti } } - if (!allocated) { + if (allocated == false) { UnassignedInfo.AllocationStatus status = throttled ? UnassignedInfo.AllocationStatus.DECIDERS_THROTTLED : UnassignedInfo.AllocationStatus.DECIDERS_NO; @@ -470,14 +505,16 @@ private void tryRebalanceNode( // Try to relocate the valid shards on the sourceNode, one at a time; // until either sourceNode is balanced OR no more active primary shard available OR all the target nodes are exhausted - while (shardsToBalance > 0 && shardIterator.hasNext() && !targetNodes.isEmpty()) { + while (shardsToBalance > 0 && shardIterator.hasNext() && targetNodes.isEmpty() == false) { // Find an active primary shard to relocate ShardRouting shard = shardIterator.next(); - if (!shard.started() || !shard.primary() || !RoutingPool.REMOTE_CAPABLE.equals(RoutingPool.getShardPool(shard, allocation))) { + if (shard.started() == false + || shard.primary() == false + || RoutingPool.REMOTE_CAPABLE.equals(RoutingPool.getShardPool(shard, allocation)) == false) { continue; } - while (!targetNodes.isEmpty()) { + while (targetNodes.isEmpty() == false) { // Find a valid target node that can accommodate the current shard relocation RoutingNode targetNode = targetNodes.poll(); if (primaryCount.get(targetNode.nodeId()) >= avgPrimary) { @@ -485,6 +522,10 @@ private void tryRebalanceNode( continue; } + if (targetNode.getByShardId(shard.shardId()) != null) { + continue; + } + // Try relocate the shard on the target node Decision rebalanceDecision = tryRelocateShard(shard, targetNode); @@ -522,21 +563,10 @@ private void tryRebalanceNode( } /** - * For every primary shard for which this method is invoked, - * swap is attempted with the destination node in case replica shard is present. - * In case replica is not present, relocation of the shard id performed. + * For every primary shard for which this method is invoked, relocation of the shard id performed. */ private Decision tryRelocateShard(ShardRouting shard, RoutingNode destinationNode) { - // Check if there is already a replica for the shard on the destination node. - // Then we can directly swap the replica with the primary shards. - // Invariant: We only allow swap relocation on remote shards. - ShardRouting replicaShard = destinationNode.getByShardId(shard.shardId()); - if (replicaShard != null) { - assert !replicaShard.primary() : "Primary Shard found while expected Replica during shard rebalance"; - return executeSwapShard(shard, replicaShard, allocation); - } - - // Since no replica present on the destinationNode; try relocating the shard to the destination node + assert destinationNode.getByShardId(shard.shardId()) == null; Decision allocationDecision = allocation.deciders().canAllocate(shard, destinationNode, allocation); Decision rebalanceDecision = allocation.deciders().canRebalance(shard, allocation); logger.trace( @@ -566,15 +596,6 @@ private Decision tryRelocateShard(ShardRouting shard, RoutingNode destinationNod return Decision.NO; } - private Decision executeSwapShard(ShardRouting primaryShard, ShardRouting replicaShard, RoutingAllocation allocation) { - if (!replicaShard.started()) { - return new Decision.Single(Decision.Type.NO); - } - - allocation.routingNodes().swapPrimaryWithReplica(logger, primaryShard, replicaShard, allocation.changes()); - return new Decision.Single(Decision.Type.YES); - } - private void failUnattemptedShards() { RoutingNodes.UnassignedShards.UnassignedIterator unassignedIterator = routingNodes.unassigned().iterator(); while (unassignedIterator.hasNext()) { diff --git a/server/src/test/java/org/opensearch/cluster/routing/RoutingNodesTests.java b/server/src/test/java/org/opensearch/cluster/routing/RoutingNodesTests.java index 780d041c25d04..7a0fd76b0fbd9 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/RoutingNodesTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/RoutingNodesTests.java @@ -46,8 +46,6 @@ import java.util.Iterator; import java.util.List; -import org.mockito.Mockito; - import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.not; @@ -206,40 +204,4 @@ public void testInterleavedShardIteratorReplicaFirst() { } assertEquals(shardCount, this.totalNumberOfShards); } - - public void testSwapPrimaryWithReplica() { - // Initialize all the shards for test index 1 and 2 - initPrimaries(); - startInitializingShards(TEST_INDEX_1); - startInitializingShards(TEST_INDEX_1); - startInitializingShards(TEST_INDEX_2); - startInitializingShards(TEST_INDEX_2); - - // Create primary shard count imbalance between two nodes - final RoutingNodes routingNodes = this.clusterState.getRoutingNodes(); - final RoutingNode node0 = routingNodes.node("node0"); - final RoutingNode node1 = routingNodes.node("node1"); - final List shardRoutingList = node0.shardsWithState(TEST_INDEX_1, ShardRoutingState.STARTED); - final RoutingChangesObserver routingChangesObserver = Mockito.mock(RoutingChangesObserver.class); - int swaps = 0; - - for (ShardRouting routing : shardRoutingList) { - if (routing.primary()) { - ShardRouting swap = node1.getByShardId(routing.shardId()); - routingNodes.swapPrimaryWithReplica(logger, routing, swap, routingChangesObserver); - swaps++; - } - } - Mockito.verify(routingChangesObserver, Mockito.times(swaps)).replicaPromoted(Mockito.any()); - - final List shards = node1.shardsWithState(TEST_INDEX_1, ShardRoutingState.STARTED); - int shardCount = 0; - for (ShardRouting shard : shards) { - if (shard.primary()) { - shardCount++; - } - } - - assertTrue(shardCount >= swaps); - } } diff --git a/server/src/test/java/org/opensearch/cluster/routing/allocation/RemoteShardsMoveShardsTests.java b/server/src/test/java/org/opensearch/cluster/routing/allocation/RemoteShardsMoveShardsTests.java index f2e79b319d0dd..b840b78eff448 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/allocation/RemoteShardsMoveShardsTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/allocation/RemoteShardsMoveShardsTests.java @@ -47,7 +47,7 @@ public void testExcludeNodeIdMoveBlocked() { /** * Test move operations for index level allocation settings. - * Supported for local indices, not supported for remote indices. + * Supported for local indices and remote indices. */ public void testIndexLevelExclusions() throws InterruptedException { int localOnlyNodes = 7; @@ -102,8 +102,9 @@ public void testIndexLevelExclusions() throws InterruptedException { // No shard of updated local index should be on excluded local capable node assertTrue(routingTable.allShards(localIndex).stream().noneMatch(shard -> shard.currentNodeId().equals(excludedLocalOnlyNode))); - // Since remote index shards are untouched, at least one shard should - // continue to stay on the excluded remote capable node - assertTrue(routingTable.allShards(remoteIndex).stream().anyMatch(shard -> shard.currentNodeId().equals(excludedRemoteCapableNode))); + // No shard of updated remote index should be on excluded remote capable node + assertTrue( + routingTable.allShards(remoteIndex).stream().noneMatch(shard -> shard.currentNodeId().equals(excludedRemoteCapableNode)) + ); } }