diff --git a/server/src/main/java/org/opensearch/cluster/ClusterModule.java b/server/src/main/java/org/opensearch/cluster/ClusterModule.java index d9bb87a517927..3924e8e0589a7 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterModule.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterModule.java @@ -53,6 +53,7 @@ import org.opensearch.cluster.metadata.ViewMetadata; import org.opensearch.cluster.metadata.WeightedRoutingMetadata; import org.opensearch.cluster.routing.DelayedAllocationService; +import org.opensearch.cluster.routing.RerouteService; import org.opensearch.cluster.routing.allocation.AllocationService; import org.opensearch.cluster.routing.allocation.ExistingShardsAllocator; import org.opensearch.cluster.routing.allocation.allocator.BalancedShardsAllocator; @@ -479,4 +480,7 @@ public void setExistingShardsAllocators(GatewayAllocator gatewayAllocator, Shard allocationService.setExistingShardsAllocators(existingShardsAllocators); } + public void setRerouteServiceForAllocator(RerouteService rerouteService) { + shardsAllocator.setRerouteService(rerouteService); + } } diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java index a5193ca602f04..785636fa7ff2a 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java @@ -35,6 +35,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.util.IntroSorter; +import org.opensearch.cluster.routing.RerouteService; import org.opensearch.cluster.routing.RoutingNode; import org.opensearch.cluster.routing.RoutingNodes; import org.opensearch.cluster.routing.ShardMovementStrategy; @@ -49,12 +50,14 @@ import org.opensearch.cluster.routing.allocation.RebalanceParameter; import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.cluster.routing.allocation.ShardAllocationDecision; +import org.opensearch.common.Priority; import org.opensearch.common.inject.Inject; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.action.ActionListener; import java.util.HashMap; import java.util.HashSet; @@ -202,6 +205,7 @@ public class BalancedShardsAllocator implements ShardsAllocator { private volatile boolean ignoreThrottleInRestore; private volatile TimeValue allocatorTimeout; private long startTime; + private RerouteService rerouteService; public BalancedShardsAllocator(Settings settings) { this(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); @@ -231,6 +235,12 @@ public BalancedShardsAllocator(Settings settings, ClusterSettings clusterSetting clusterSettings.addSettingsUpdateConsumer(ALLOCATOR_TIMEOUT_SETTING, this::setAllocatorTimeout); } + @Override + public void setRerouteService(RerouteService rerouteService) { + assert this.rerouteService == null : "RerouteService is already set"; + this.rerouteService = rerouteService; + } + /** * Changes in deprecated setting SHARD_MOVE_PRIMARY_FIRST_SETTING affect value of its replacement setting SHARD_MOVEMENT_STRATEGY_SETTING. */ @@ -342,6 +352,7 @@ public void allocate(RoutingAllocation allocation) { localShardsBalancer.allocateUnassigned(); localShardsBalancer.moveShards(); localShardsBalancer.balance(); + scheduleRerouteIfAllocatorTimedOut(); final ShardsBalancer remoteShardsBalancer = new RemoteShardsBalancer(logger, allocation); remoteShardsBalancer.allocateUnassigned(); @@ -404,6 +415,20 @@ private void failAllocationOfNewPrimaries(RoutingAllocation allocation) { } } + private void scheduleRerouteIfAllocatorTimedOut() { + if (allocatorTimedOut()) { + assert rerouteService != null : "RerouteService not set to schedule reroute after allocator time out"; + rerouteService.reroute( + "reroute after balanced shards allocator timed out", + Priority.HIGH, + ActionListener.wrap( + r -> logger.trace("reroute after balanced shards allocator timed out completed"), + e -> logger.debug("reroute after balanced shards allocator timed out failed", e) + ) + ); + } + } + /** * Returns the currently configured delta threshold */ diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/ShardsAllocator.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/ShardsAllocator.java index 29e9acca4e6c2..38aafff6ce3e8 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/ShardsAllocator.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/ShardsAllocator.java @@ -32,6 +32,7 @@ package org.opensearch.cluster.routing.allocation.allocator; +import org.opensearch.cluster.routing.RerouteService; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.opensearch.cluster.routing.allocation.MoveDecision; @@ -73,4 +74,6 @@ public interface ShardsAllocator { * the cluster explain API, then this method should throw a {@code UnsupportedOperationException}. */ ShardAllocationDecision decideShardAllocation(ShardRouting shard, RoutingAllocation allocation); + + default void setRerouteService(RerouteService rerouteService) {} } diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index d18304ea73ed0..5e2dcbcd70b40 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -184,11 +184,11 @@ public void cleanCaches() { // for tests protected ShardsBatchGatewayAllocator() { - this(DEFAULT_SHARD_BATCH_SIZE); + this(DEFAULT_SHARD_BATCH_SIZE, null); } - protected ShardsBatchGatewayAllocator(long batchSize) { - this.rerouteService = null; + protected ShardsBatchGatewayAllocator(long batchSize, RerouteService rerouteService) { + this.rerouteService = rerouteService; this.batchStartedAction = null; this.primaryShardBatchAllocator = null; this.batchStoreAction = null; @@ -297,6 +297,18 @@ public void run() { public void onComplete() { logger.trace("Triggering oncomplete after timeout for [{}] primary shards", timedOutPrimaryShardIds.size()); primaryBatchShardAllocator.allocateUnassignedBatchOnTimeout(timedOutPrimaryShardIds, allocation, true); + if (timedOutPrimaryShardIds.isEmpty() == false) { + logger.trace("scheduling reroute after existing shards allocator timed out for primary shards"); + assert rerouteService != null; + rerouteService.reroute( + "reroute after existing shards allocator timed out", + Priority.HIGH, + ActionListener.wrap( + r -> logger.trace("reroute after existing shards allocator timed out completed"), + e -> logger.debug("reroute after existing shards allocator timed out failed", e) + ) + ); + } } }; } else { @@ -320,6 +332,18 @@ public void run() { public void onComplete() { logger.trace("Triggering oncomplete after timeout for [{}] replica shards", timedOutReplicaShardIds.size()); replicaBatchShardAllocator.allocateUnassignedBatchOnTimeout(timedOutReplicaShardIds, allocation, false); + if (timedOutReplicaShardIds.isEmpty() == false) { + logger.trace("scheduling reroute after existing shards allocator timed out for replica shards"); + assert rerouteService != null; + rerouteService.reroute( + "reroute after existing shards allocator timed out", + Priority.HIGH, + ActionListener.wrap( + r -> logger.trace("reroute after existing shards allocator timed out completed"), + e -> logger.debug("reroute after existing shards allocator timed out failed", e) + ) + ); + } } }; } diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index b143333954338..34ed957542aff 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -871,6 +871,7 @@ protected Node( final RerouteService rerouteService = new BatchedRerouteService(clusterService, clusterModule.getAllocationService()::reroute); rerouteServiceReference.set(rerouteService); clusterService.setRerouteService(rerouteService); + clusterModule.setRerouteServiceForAllocator(rerouteService); final RecoverySettings recoverySettings = new RecoverySettings(settings, settingsModule.getClusterSettings()); diff --git a/server/src/test/java/org/opensearch/cluster/ClusterModuleTests.java b/server/src/test/java/org/opensearch/cluster/ClusterModuleTests.java index 97706927ba857..f8240e775cfa5 100644 --- a/server/src/test/java/org/opensearch/cluster/ClusterModuleTests.java +++ b/server/src/test/java/org/opensearch/cluster/ClusterModuleTests.java @@ -337,6 +337,19 @@ public void testQueryGroupMetadataRegister() { ); } + public void testRerouteServiceSetForBalancedShardsAllocator() { + ClusterModule clusterModule = new ClusterModule( + Settings.EMPTY, + clusterService, + Collections.emptyList(), + clusterInfoService, + null, + threadContext, + new ClusterManagerMetrics(NoopMetricsRegistry.INSTANCE) + ); + clusterModule.setRerouteServiceForAllocator((reason, priority, listener) -> listener.onResponse(clusterService.state())); + } + private static ClusterPlugin existingShardsAllocatorPlugin(final String allocatorName) { return new ClusterPlugin() { @Override diff --git a/server/src/test/java/org/opensearch/cluster/routing/allocation/allocator/TimeBoundBalancedShardsAllocatorTests.java b/server/src/test/java/org/opensearch/cluster/routing/allocation/allocator/TimeBoundBalancedShardsAllocatorTests.java index a10c305686638..45a0bd7b18afd 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/allocation/allocator/TimeBoundBalancedShardsAllocatorTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/allocation/allocator/TimeBoundBalancedShardsAllocatorTests.java @@ -8,6 +8,7 @@ package org.opensearch.cluster.routing.allocation.allocator; +import org.opensearch.OpenSearchException; import org.opensearch.Version; import org.opensearch.cluster.ClusterInfo; import org.opensearch.cluster.ClusterName; @@ -17,6 +18,7 @@ import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.RerouteService; import org.opensearch.cluster.routing.RoutingNodes; import org.opensearch.cluster.routing.RoutingTable; import org.opensearch.cluster.routing.ShardRouting; @@ -26,14 +28,20 @@ import org.opensearch.cluster.routing.allocation.decider.AllocationDeciders; import org.opensearch.cluster.routing.allocation.decider.Decision; import org.opensearch.cluster.routing.allocation.decider.SameShardAllocationDecider; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.Priority; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; +import org.opensearch.test.ClusterServiceUtils; +import org.opensearch.threadpool.TestThreadPool; +import org.junit.After; import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; import static org.opensearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.opensearch.cluster.routing.ShardRoutingState.STARTED; @@ -41,26 +49,49 @@ public class TimeBoundBalancedShardsAllocatorTests extends OpenSearchAllocationTestCase { + private TestThreadPool threadPool; + private ClusterService clusterService; + private ClusterState state; + private final DiscoveryNode node1 = newNode("node1", "node1", Collections.singletonMap("zone", "1a")); private final DiscoveryNode node2 = newNode("node2", "node2", Collections.singletonMap("zone", "1b")); private final DiscoveryNode node3 = newNode("node3", "node3", Collections.singletonMap("zone", "1c")); - public void testAllUnassignedShardsAllocatedWhenNoTimeOut() { + @After + @Override + public void tearDown() throws Exception { + super.tearDown(); + if (threadPool != null) { + final boolean terminated = terminate(threadPool); + assert terminated; + } + if (clusterService != null) { + clusterService.close(); + } + } + + public void setupStateAndService(Metadata metadata, RoutingTable routingTable) { + state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTable) + .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) + .build(); + threadPool = new TestThreadPool(getTestName()); + clusterService = ClusterServiceUtils.createClusterService(state, threadPool); + } + + public void testAllUnassignedShardsAllocatedWhenNoTimeOutAndRerouteNotScheduled() { int numberOfIndices = 2; int numberOfShards = 5; int numberOfReplicas = 1; int totalPrimaryCount = numberOfIndices * numberOfShards; int totalShardCount = numberOfIndices * (numberOfShards * (numberOfReplicas + 1)); Settings.Builder settings = Settings.builder(); - // passing total shard count for timed out latch such that no shard times out - BalancedShardsAllocator allocator = new TestBalancedShardsAllocator(settings.build(), new CountDownLatch(totalShardCount)); + // passing sufficiently high count for timeout latch to simulate no time out + BalancedShardsAllocator allocator = new TestBalancedShardsAllocator(settings.build(), new CountDownLatch(Integer.MAX_VALUE)); Metadata metadata = buildMetadata(Metadata.builder(), numberOfIndices, numberOfShards, numberOfReplicas); RoutingTable routingTable = buildRoutingTable(metadata); - ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) - .metadata(metadata) - .routingTable(routingTable) - .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) - .build(); + setupStateAndService(metadata, routingTable); RoutingAllocation allocation = new RoutingAllocation( yesAllocationDeciders(), new RoutingNodes(state, false), @@ -69,6 +100,18 @@ public void testAllUnassignedShardsAllocatedWhenNoTimeOut() { null, System.nanoTime() ); + AtomicBoolean rerouteScheduled = new AtomicBoolean(false); + final RerouteService rerouteService = (reason, priority, listener) -> { + if (randomBoolean()) { + listener.onFailure(new OpenSearchException("simulated")); + } else { + listener.onResponse(clusterService.state()); + } + assertEquals("reroute after balanced shards allocator timed out", reason); + assertEquals(Priority.HIGH, priority); + rerouteScheduled.compareAndSet(false, true); + }; + allocator.setRerouteService(rerouteService); allocator.allocate(allocation); List initializingShards = allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING); int node1Recoveries = allocation.routingNodes().getInitialPrimariesIncomingRecoveries(node1.getId()); @@ -77,9 +120,10 @@ public void testAllUnassignedShardsAllocatedWhenNoTimeOut() { assertEquals(totalShardCount, initializingShards.size()); assertEquals(0, allocation.routingNodes().unassigned().ignored().size()); assertEquals(totalPrimaryCount, node1Recoveries + node2Recoveries + node3Recoveries); + assertFalse(rerouteScheduled.get()); } - public void testAllUnassignedShardsIgnoredWhenTimedOut() { + public void testAllUnassignedShardsIgnoredWhenTimedOutAndRerouteScheduled() { int numberOfIndices = 2; int numberOfShards = 5; int numberOfReplicas = 1; @@ -89,11 +133,7 @@ public void testAllUnassignedShardsIgnoredWhenTimedOut() { BalancedShardsAllocator allocator = new TestBalancedShardsAllocator(settings.build(), new CountDownLatch(0)); Metadata metadata = buildMetadata(Metadata.builder(), numberOfIndices, numberOfShards, numberOfReplicas); RoutingTable routingTable = buildRoutingTable(metadata); - ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) - .metadata(metadata) - .routingTable(routingTable) - .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) - .build(); + setupStateAndService(metadata, routingTable); RoutingAllocation allocation = new RoutingAllocation( yesAllocationDeciders(), new RoutingNodes(state, false), @@ -102,6 +142,18 @@ public void testAllUnassignedShardsIgnoredWhenTimedOut() { null, System.nanoTime() ); + AtomicBoolean rerouteScheduled = new AtomicBoolean(false); + final RerouteService rerouteService = (reason, priority, listener) -> { + if (randomBoolean()) { + listener.onFailure(new OpenSearchException("simulated")); + } else { + listener.onResponse(clusterService.state()); + } + assertEquals("reroute after balanced shards allocator timed out", reason); + assertEquals(Priority.HIGH, priority); + rerouteScheduled.compareAndSet(false, true); + }; + allocator.setRerouteService(rerouteService); allocator.allocate(allocation); List initializingShards = allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING); int node1Recoveries = allocation.routingNodes().getInitialPrimariesIncomingRecoveries(node1.getId()); @@ -110,9 +162,10 @@ public void testAllUnassignedShardsIgnoredWhenTimedOut() { assertEquals(0, initializingShards.size()); assertEquals(totalShardCount, allocation.routingNodes().unassigned().ignored().size()); assertEquals(0, node1Recoveries + node2Recoveries + node3Recoveries); + assertTrue(rerouteScheduled.get()); } - public void testAllocatePartialPrimaryShardsUntilTimedOut() { + public void testAllocatePartialPrimaryShardsUntilTimedOutAndRerouteScheduled() { int numberOfIndices = 2; int numberOfShards = 5; int numberOfReplicas = 1; @@ -123,11 +176,7 @@ public void testAllocatePartialPrimaryShardsUntilTimedOut() { BalancedShardsAllocator allocator = new TestBalancedShardsAllocator(settings.build(), new CountDownLatch(shardsToAllocate)); Metadata metadata = buildMetadata(Metadata.builder(), numberOfIndices, numberOfShards, numberOfReplicas); RoutingTable routingTable = buildRoutingTable(metadata); - ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) - .metadata(metadata) - .routingTable(routingTable) - .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) - .build(); + setupStateAndService(metadata, routingTable); RoutingAllocation allocation = new RoutingAllocation( yesAllocationDeciders(), new RoutingNodes(state, false), @@ -136,6 +185,18 @@ public void testAllocatePartialPrimaryShardsUntilTimedOut() { null, System.nanoTime() ); + AtomicBoolean rerouteScheduled = new AtomicBoolean(false); + final RerouteService rerouteService = (reason, priority, listener) -> { + if (randomBoolean()) { + listener.onFailure(new OpenSearchException("simulated")); + } else { + listener.onResponse(clusterService.state()); + } + assertEquals("reroute after balanced shards allocator timed out", reason); + assertEquals(Priority.HIGH, priority); + rerouteScheduled.compareAndSet(false, true); + }; + allocator.setRerouteService(rerouteService); allocator.allocate(allocation); List initializingShards = allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING); int node1Recoveries = allocation.routingNodes().getInitialPrimariesIncomingRecoveries(node1.getId()); @@ -144,9 +205,10 @@ public void testAllocatePartialPrimaryShardsUntilTimedOut() { assertEquals(shardsToAllocate, initializingShards.size()); assertEquals(totalShardCount - shardsToAllocate, allocation.routingNodes().unassigned().ignored().size()); assertEquals(shardsToAllocate, node1Recoveries + node2Recoveries + node3Recoveries); + assertTrue(rerouteScheduled.get()); } - public void testAllocateAllPrimaryShardsAndPartialReplicaShardsUntilTimedOut() { + public void testAllocateAllPrimaryShardsAndPartialReplicaShardsUntilTimedOutAndRerouteScheduled() { int numberOfIndices = 2; int numberOfShards = 5; int numberOfReplicas = 1; @@ -158,11 +220,7 @@ public void testAllocateAllPrimaryShardsAndPartialReplicaShardsUntilTimedOut() { BalancedShardsAllocator allocator = new TestBalancedShardsAllocator(settings.build(), new CountDownLatch(shardsToAllocate)); Metadata metadata = buildMetadata(Metadata.builder(), numberOfIndices, numberOfShards, numberOfReplicas); RoutingTable routingTable = buildRoutingTable(metadata); - ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) - .metadata(metadata) - .routingTable(routingTable) - .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) - .build(); + setupStateAndService(metadata, routingTable); RoutingAllocation allocation = new RoutingAllocation( yesAllocationDeciders(), new RoutingNodes(state, false), @@ -171,6 +229,18 @@ public void testAllocateAllPrimaryShardsAndPartialReplicaShardsUntilTimedOut() { null, System.nanoTime() ); + AtomicBoolean rerouteScheduled = new AtomicBoolean(false); + final RerouteService rerouteService = (reason, priority, listener) -> { + if (randomBoolean()) { + listener.onFailure(new OpenSearchException("simulated")); + } else { + listener.onResponse(clusterService.state()); + } + assertEquals("reroute after balanced shards allocator timed out", reason); + assertEquals(Priority.HIGH, priority); + rerouteScheduled.compareAndSet(false, true); + }; + allocator.setRerouteService(rerouteService); allocator.allocate(allocation); List initializingShards = allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING); int node1Recoveries = allocation.routingNodes().getInitialPrimariesIncomingRecoveries(node1.getId()); @@ -179,20 +249,17 @@ public void testAllocateAllPrimaryShardsAndPartialReplicaShardsUntilTimedOut() { assertEquals(shardsToAllocate, initializingShards.size()); assertEquals(totalShardCount - shardsToAllocate, allocation.routingNodes().unassigned().ignored().size()); assertEquals(numberOfShards * numberOfIndices, node1Recoveries + node2Recoveries + node3Recoveries); + assertTrue(rerouteScheduled.get()); } - public void testAllShardsMoveWhenExcludedAndTimeoutNotBreached() { + public void testAllShardsMoveWhenExcludedAndTimeoutNotBreachedAndRerouteNotScheduled() { int numberOfIndices = 3; int numberOfShards = 5; int numberOfReplicas = 1; int totalShardCount = numberOfIndices * (numberOfShards * (numberOfReplicas + 1)); Metadata metadata = buildMetadata(Metadata.builder(), numberOfIndices, numberOfShards, numberOfReplicas); RoutingTable routingTable = buildRoutingTable(metadata); - ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) - .metadata(metadata) - .routingTable(routingTable) - .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) - .build(); + setupStateAndService(metadata, routingTable); MockAllocationService allocationService = createAllocationService(); state = applyStartedShardsUntilNoChange(state, allocationService); // check all shards allocated @@ -200,8 +267,7 @@ public void testAllShardsMoveWhenExcludedAndTimeoutNotBreached() { assertEquals(totalShardCount, state.getRoutingNodes().shardsWithState(STARTED).size()); int node1ShardCount = state.getRoutingNodes().node("node1").size(); Settings settings = Settings.builder().put("cluster.routing.allocation.exclude.zone", "1a").build(); - int shardsToMove = 10 + 1000; // such that time out is never breached - BalancedShardsAllocator allocator = new TestBalancedShardsAllocator(settings, new CountDownLatch(shardsToMove)); + BalancedShardsAllocator allocator = new TestBalancedShardsAllocator(settings, new CountDownLatch(Integer.MAX_VALUE)); RoutingAllocation allocation = new RoutingAllocation( allocationDecidersForExcludeAPI(settings), new RoutingNodes(state, false), @@ -210,30 +276,39 @@ public void testAllShardsMoveWhenExcludedAndTimeoutNotBreached() { null, System.nanoTime() ); + AtomicBoolean rerouteScheduled = new AtomicBoolean(false); + final RerouteService rerouteService = (reason, priority, listener) -> { + if (randomBoolean()) { + listener.onFailure(new OpenSearchException("simulated")); + } else { + listener.onResponse(clusterService.state()); + } + assertEquals("reroute after balanced shards allocator timed out", reason); + assertEquals(Priority.HIGH, priority); + rerouteScheduled.compareAndSet(false, true); + }; + allocator.setRerouteService(rerouteService); allocator.allocate(allocation); List relocatingShards = allocation.routingNodes().shardsWithState(ShardRoutingState.RELOCATING); assertEquals(node1ShardCount, relocatingShards.size()); + assertFalse(rerouteScheduled.get()); } - public void testNoShardsMoveWhenExcludedAndTimeoutBreached() { + public void testNoShardsMoveWhenExcludedAndTimeoutBreachedAndRerouteScheduled() { int numberOfIndices = 3; int numberOfShards = 5; int numberOfReplicas = 1; int totalShardCount = numberOfIndices * (numberOfShards * (numberOfReplicas + 1)); Metadata metadata = buildMetadata(Metadata.builder(), numberOfIndices, numberOfShards, numberOfReplicas); RoutingTable routingTable = buildRoutingTable(metadata); - ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) - .metadata(metadata) - .routingTable(routingTable) - .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) - .build(); + setupStateAndService(metadata, routingTable); MockAllocationService allocationService = createAllocationService(); state = applyStartedShardsUntilNoChange(state, allocationService); // check all shards allocated assertEquals(0, state.getRoutingNodes().shardsWithState(INITIALIZING).size()); assertEquals(totalShardCount, state.getRoutingNodes().shardsWithState(STARTED).size()); Settings settings = Settings.builder().put("cluster.routing.allocation.exclude.zone", "1a").build(); - int shardsToMove = 0; // such that time out is never breached + int shardsToMove = 0; // such that time out is breached BalancedShardsAllocator allocator = new TestBalancedShardsAllocator(settings, new CountDownLatch(shardsToMove)); RoutingAllocation allocation = new RoutingAllocation( allocationDecidersForExcludeAPI(settings), @@ -243,23 +318,32 @@ public void testNoShardsMoveWhenExcludedAndTimeoutBreached() { null, System.nanoTime() ); + AtomicBoolean rerouteScheduled = new AtomicBoolean(false); + final RerouteService rerouteService = (reason, priority, listener) -> { + if (randomBoolean()) { + listener.onFailure(new OpenSearchException("simulated")); + } else { + listener.onResponse(clusterService.state()); + } + assertEquals("reroute after balanced shards allocator timed out", reason); + assertEquals(Priority.HIGH, priority); + rerouteScheduled.compareAndSet(false, true); + }; + allocator.setRerouteService(rerouteService); allocator.allocate(allocation); List relocatingShards = allocation.routingNodes().shardsWithState(ShardRoutingState.RELOCATING); assertEquals(0, relocatingShards.size()); + assertTrue(rerouteScheduled.get()); } - public void testPartialShardsMoveWhenExcludedAndTimeoutBreached() { + public void testPartialShardsMoveWhenExcludedAndTimeoutBreachedAndRerouteScheduled() { int numberOfIndices = 3; int numberOfShards = 5; int numberOfReplicas = 1; int totalShardCount = numberOfIndices * (numberOfShards * (numberOfReplicas + 1)); Metadata metadata = buildMetadata(Metadata.builder(), numberOfIndices, numberOfShards, numberOfReplicas); RoutingTable routingTable = buildRoutingTable(metadata); - ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) - .metadata(metadata) - .routingTable(routingTable) - .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) - .build(); + setupStateAndService(metadata, routingTable); MockAllocationService allocationService = createAllocationService(); state = applyStartedShardsUntilNoChange(state, allocationService); // check all shards allocated @@ -279,23 +363,32 @@ public void testPartialShardsMoveWhenExcludedAndTimeoutBreached() { null, System.nanoTime() ); + AtomicBoolean rerouteScheduled = new AtomicBoolean(false); + final RerouteService rerouteService = (reason, priority, listener) -> { + if (randomBoolean()) { + listener.onFailure(new OpenSearchException("simulated")); + } else { + listener.onResponse(clusterService.state()); + } + assertEquals("reroute after balanced shards allocator timed out", reason); + assertEquals(Priority.HIGH, priority); + rerouteScheduled.compareAndSet(false, true); + }; + allocator.setRerouteService(rerouteService); allocator.allocate(allocation); List relocatingShards = allocation.routingNodes().shardsWithState(ShardRoutingState.RELOCATING); assertEquals(shardsToMove / 3, relocatingShards.size()); + assertTrue(rerouteScheduled.get()); } - public void testClusterRebalancedWhenNotTimedOut() { + public void testClusterRebalancedWhenNotTimedOutAndRerouteNotScheduled() { int numberOfIndices = 1; int numberOfShards = 15; int numberOfReplicas = 1; int totalShardCount = numberOfIndices * (numberOfShards * (numberOfReplicas + 1)); Metadata metadata = buildMetadata(Metadata.builder(), numberOfIndices, numberOfShards, numberOfReplicas); RoutingTable routingTable = buildRoutingTable(metadata); - ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) - .metadata(metadata) - .routingTable(routingTable) - .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) - .build(); + setupStateAndService(metadata, routingTable); MockAllocationService allocationService = createAllocationService( Settings.builder().put("cluster.routing.allocation.exclude.zone", "1a").build() ); // such that no shards are allocated to node1 @@ -306,8 +399,7 @@ public void testClusterRebalancedWhenNotTimedOut() { assertEquals(totalShardCount, state.getRoutingNodes().shardsWithState(STARTED).size()); assertEquals(0, node1ShardCount); Settings newSettings = Settings.builder().put("cluster.routing.allocation.exclude.zone", "").build(); - int shardsToMove = 1000; // such that time out is never breached - BalancedShardsAllocator allocator = new TestBalancedShardsAllocator(newSettings, new CountDownLatch(shardsToMove)); + BalancedShardsAllocator allocator = new TestBalancedShardsAllocator(newSettings, new CountDownLatch(Integer.MAX_VALUE)); RoutingAllocation allocation = new RoutingAllocation( allocationDecidersForExcludeAPI(newSettings), new RoutingNodes(state, false), @@ -316,23 +408,32 @@ public void testClusterRebalancedWhenNotTimedOut() { null, System.nanoTime() ); + AtomicBoolean rerouteScheduled = new AtomicBoolean(false); + final RerouteService rerouteService = (reason, priority, listener) -> { + if (randomBoolean()) { + listener.onFailure(new OpenSearchException("simulated")); + } else { + listener.onResponse(clusterService.state()); + } + assertEquals("reroute after balanced shards allocator timed out", reason); + assertEquals(Priority.HIGH, priority); + rerouteScheduled.compareAndSet(false, true); + }; + allocator.setRerouteService(rerouteService); allocator.allocate(allocation); List relocatingShards = allocation.routingNodes().shardsWithState(ShardRoutingState.RELOCATING); assertEquals(totalShardCount / 3, relocatingShards.size()); + assertFalse(rerouteScheduled.get()); } - public void testClusterNotRebalancedWhenTimedOut() { + public void testClusterNotRebalancedWhenTimedOutAndRerouteScheduled() { int numberOfIndices = 1; int numberOfShards = 15; int numberOfReplicas = 1; int totalShardCount = numberOfIndices * (numberOfShards * (numberOfReplicas + 1)); Metadata metadata = buildMetadata(Metadata.builder(), numberOfIndices, numberOfShards, numberOfReplicas); RoutingTable routingTable = buildRoutingTable(metadata); - ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) - .metadata(metadata) - .routingTable(routingTable) - .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) - .build(); + setupStateAndService(metadata, routingTable); MockAllocationService allocationService = createAllocationService( Settings.builder().put("cluster.routing.allocation.exclude.zone", "1a").build() ); // such that no shards are allocated to node1 @@ -353,23 +454,32 @@ public void testClusterNotRebalancedWhenTimedOut() { null, System.nanoTime() ); + AtomicBoolean rerouteScheduled = new AtomicBoolean(false); + final RerouteService rerouteService = (reason, priority, listener) -> { + if (randomBoolean()) { + listener.onFailure(new OpenSearchException("simulated")); + } else { + listener.onResponse(clusterService.state()); + } + assertEquals("reroute after balanced shards allocator timed out", reason); + assertEquals(Priority.HIGH, priority); + rerouteScheduled.compareAndSet(false, true); + }; + allocator.setRerouteService(rerouteService); allocator.allocate(allocation); List relocatingShards = allocation.routingNodes().shardsWithState(ShardRoutingState.RELOCATING); assertEquals(0, relocatingShards.size()); + assertTrue(rerouteScheduled.get()); } - public void testClusterPartialRebalancedWhenTimedOut() { + public void testClusterPartialRebalancedWhenTimedOutAndRerouteScheduled() { int numberOfIndices = 1; int numberOfShards = 15; int numberOfReplicas = 1; int totalShardCount = numberOfIndices * (numberOfShards * (numberOfReplicas + 1)); Metadata metadata = buildMetadata(Metadata.builder(), numberOfIndices, numberOfShards, numberOfReplicas); RoutingTable routingTable = buildRoutingTable(metadata); - ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) - .metadata(metadata) - .routingTable(routingTable) - .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) - .build(); + setupStateAndService(metadata, routingTable); MockAllocationService allocationService = createAllocationService( Settings.builder().put("cluster.routing.allocation.exclude.zone", "1a").build() ); // such that no shards are allocated to node1 @@ -404,9 +514,22 @@ public Decision canRebalance(ShardRouting shardRouting, RoutingAllocation alloca null, System.nanoTime() ); + AtomicBoolean rerouteScheduled = new AtomicBoolean(false); + final RerouteService rerouteService = (reason, priority, listener) -> { + if (randomBoolean()) { + listener.onFailure(new OpenSearchException("simulated")); + } else { + listener.onResponse(clusterService.state()); + } + assertEquals("reroute after balanced shards allocator timed out", reason); + assertEquals(Priority.HIGH, priority); + rerouteScheduled.compareAndSet(false, true); + }; + allocator.setRerouteService(rerouteService); allocator.allocate(allocation); List relocatingShards = allocation.routingNodes().shardsWithState(ShardRoutingState.RELOCATING); assertEquals(3, relocatingShards.size()); + assertTrue(rerouteScheduled.get()); } public void testAllocatorNeverTimedOutIfValueIsMinusOne() { diff --git a/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java index c7eae77d6deba..ebc2e59fa5a30 100644 --- a/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java @@ -10,6 +10,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.opensearch.OpenSearchException; import org.opensearch.Version; import org.opensearch.action.support.nodes.BaseNodeResponse; import org.opensearch.cluster.ClusterInfo; @@ -22,6 +23,7 @@ import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.cluster.routing.IndexShardRoutingTable; import org.opensearch.cluster.routing.RecoverySource; +import org.opensearch.cluster.routing.RerouteService; import org.opensearch.cluster.routing.RoutingNodes; import org.opensearch.cluster.routing.RoutingTable; import org.opensearch.cluster.routing.ShardRouting; @@ -30,6 +32,8 @@ import org.opensearch.cluster.routing.UnassignedInfo; import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.cluster.routing.allocation.decider.AllocationDeciders; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.Priority; import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; @@ -37,7 +41,9 @@ import org.opensearch.common.util.set.Sets; import org.opensearch.core.index.shard.ShardId; import org.opensearch.snapshots.SnapshotShardSizeInfo; +import org.opensearch.test.ClusterServiceUtils; import org.opensearch.test.gateway.TestShardBatchGatewayAllocator; +import org.opensearch.threadpool.TestThreadPool; import org.junit.Before; import java.util.ArrayList; @@ -47,13 +53,13 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import static org.opensearch.gateway.ShardsBatchGatewayAllocator.PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING; import static org.opensearch.gateway.ShardsBatchGatewayAllocator.PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING_KEY; import static org.opensearch.gateway.ShardsBatchGatewayAllocator.REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING; import static org.opensearch.gateway.ShardsBatchGatewayAllocator.REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING_KEY; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; public class GatewayAllocatorTests extends OpenSearchAllocationTestCase { @@ -426,22 +432,62 @@ public void testReplicaAllocatorTimeout() { assertEquals(-1, REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING.get(build).getMillis()); } - public void testCollectTimedOutShards() throws InterruptedException { + public void testCollectTimedOutShardsAndScheduleReroute_Success() throws InterruptedException { createIndexAndUpdateClusterState(2, 5, 2); - CountDownLatch latch = new CountDownLatch(10); - testShardsBatchGatewayAllocator = new TestShardBatchGatewayAllocator(latch); + TestThreadPool threadPool = new TestThreadPool(getTestName()); + ClusterService clusterService = ClusterServiceUtils.createClusterService(clusterState, threadPool); + final CountDownLatch rerouteLatch = new CountDownLatch(2); + final RerouteService rerouteService = (reason, priority, listener) -> { + listener.onResponse(clusterService.state()); + assertThat(rerouteLatch.getCount(), greaterThanOrEqualTo(0L)); + assertEquals("reroute after existing shards allocator timed out", reason); + assertEquals(Priority.HIGH, priority); + rerouteLatch.countDown(); + }; + CountDownLatch timedOutShardsLatch = new CountDownLatch(20); + testShardsBatchGatewayAllocator = new TestShardBatchGatewayAllocator(timedOutShardsLatch, 1000, rerouteService); testShardsBatchGatewayAllocator.setPrimaryBatchAllocatorTimeout(TimeValue.ZERO); testShardsBatchGatewayAllocator.setReplicaBatchAllocatorTimeout(TimeValue.ZERO); BatchRunnableExecutor executor = testShardsBatchGatewayAllocator.allocateAllUnassignedShards(testAllocation, true); executor.run(); - assertTrue(latch.await(1, TimeUnit.MINUTES)); - latch = new CountDownLatch(10); - testShardsBatchGatewayAllocator = new TestShardBatchGatewayAllocator(latch); + assertEquals(timedOutShardsLatch.getCount(), 10); + assertEquals(1, rerouteLatch.getCount()); + executor = testShardsBatchGatewayAllocator.allocateAllUnassignedShards(testAllocation, false); + executor.run(); + assertEquals(timedOutShardsLatch.getCount(), 0); + assertEquals(0, rerouteLatch.getCount()); // even with failure it doesn't leak any listeners + final boolean terminated = terminate(threadPool); + assert terminated; + clusterService.close(); + } + + public void testCollectTimedOutShardsAndScheduleReroute_Failure() throws InterruptedException { + createIndexAndUpdateClusterState(2, 5, 2); + TestThreadPool threadPool = new TestThreadPool(getTestName()); + ClusterService clusterService = ClusterServiceUtils.createClusterService(clusterState, threadPool); + final CountDownLatch rerouteLatch = new CountDownLatch(2); + final RerouteService rerouteService = (reason, priority, listener) -> { + listener.onFailure(new OpenSearchException("simulated")); + assertThat(rerouteLatch.getCount(), greaterThanOrEqualTo(0L)); + assertEquals("reroute after existing shards allocator timed out", reason); + assertEquals(Priority.HIGH, priority); + rerouteLatch.countDown(); + }; + CountDownLatch timedOutShardsLatch = new CountDownLatch(20); + testShardsBatchGatewayAllocator = new TestShardBatchGatewayAllocator(timedOutShardsLatch, 1000, rerouteService); testShardsBatchGatewayAllocator.setPrimaryBatchAllocatorTimeout(TimeValue.ZERO); testShardsBatchGatewayAllocator.setReplicaBatchAllocatorTimeout(TimeValue.ZERO); + BatchRunnableExecutor executor = testShardsBatchGatewayAllocator.allocateAllUnassignedShards(testAllocation, true); + executor.run(); + assertEquals(timedOutShardsLatch.getCount(), 10); + assertEquals(1, rerouteLatch.getCount()); executor = testShardsBatchGatewayAllocator.allocateAllUnassignedShards(testAllocation, false); executor.run(); - assertTrue(latch.await(1, TimeUnit.MINUTES)); + assertEquals(timedOutShardsLatch.getCount(), 0); + assertEquals(0, rerouteLatch.getCount()); // even with failure it doesn't leak any listeners + final boolean terminated = terminate(threadPool); + assert terminated; + clusterService.close(); } private void createIndexAndUpdateClusterState(int count, int numberOfShards, int numberOfReplicas) { diff --git a/test/framework/src/main/java/org/opensearch/test/gateway/TestShardBatchGatewayAllocator.java b/test/framework/src/main/java/org/opensearch/test/gateway/TestShardBatchGatewayAllocator.java index 156b1d7c620e6..c2ff228a6bf3a 100644 --- a/test/framework/src/main/java/org/opensearch/test/gateway/TestShardBatchGatewayAllocator.java +++ b/test/framework/src/main/java/org/opensearch/test/gateway/TestShardBatchGatewayAllocator.java @@ -10,6 +10,7 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.RerouteService; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.opensearch.cluster.routing.allocation.RoutingAllocation; @@ -39,12 +40,13 @@ public TestShardBatchGatewayAllocator() { } - public TestShardBatchGatewayAllocator(CountDownLatch latch) { + public TestShardBatchGatewayAllocator(CountDownLatch latch, long maxBatchSize, RerouteService rerouteService) { + super(maxBatchSize, rerouteService); this.latch = latch; } public TestShardBatchGatewayAllocator(long maxBatchSize) { - super(maxBatchSize); + super(maxBatchSize, null); } Map> knownAllocations = new HashMap<>();