From 6d2a9d3554483ace2aa6239afc4ff1f67daadf2a Mon Sep 17 00:00:00 2001 From: Bukhtawar Khan Date: Sun, 21 Jul 2024 05:56:47 +0530 Subject: [PATCH 01/35] Simplify batch allocators with timeouts Signed-off-by: Bukhtawar Khan --- .../AllocateUnassignedDecision.java | 2 +- .../routing/allocation/AllocationService.java | 19 ++++++++++-- .../allocation/ExistingShardsAllocator.java | 8 +++-- .../gateway/PrimaryShardBatchAllocator.java | 14 +++++++++ .../gateway/ReplicaShardBatchAllocator.java | 17 +++++++++++ .../gateway/ShardsBatchGatewayAllocator.java | 30 +++++++++++++++---- .../TestShardBatchGatewayAllocator.java | 5 ++-- 7 files changed, 82 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocateUnassignedDecision.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocateUnassignedDecision.java index 4e77ab772f390..43c6c19315cd0 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocateUnassignedDecision.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocateUnassignedDecision.java @@ -105,7 +105,7 @@ public class AllocateUnassignedDecision extends AbstractAllocationDecision { private final long remainingDelayInMillis; private final long configuredDelayInMillis; - private AllocateUnassignedDecision( + public AllocateUnassignedDecision( AllocationStatus allocationStatus, DiscoveryNode assignedNode, String allocationId, diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java index 5ad3a2fd47ce3..9eeb75d4f05aa 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java @@ -73,7 +73,9 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Consumer; import java.util.function.Function; +import java.util.function.Supplier; import java.util.stream.Collectors; import static java.util.Collections.emptyList; @@ -100,6 +102,7 @@ public class AllocationService { private final ClusterInfoService clusterInfoService; private SnapshotsInfoService snapshotsInfoService; private final ClusterManagerMetrics clusterManagerMetrics; + private final long maxRunTimeoutInMillis = 5; // only for tests that use the GatewayAllocator as the unique ExistingShardsAllocator public AllocationService( @@ -617,10 +620,22 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { private void allocateAllUnassignedShards(RoutingAllocation allocation) { ExistingShardsAllocator allocator = existingShardsAllocators.get(ShardsBatchGatewayAllocator.ALLOCATOR_NAME); - allocator.allocateAllUnassignedShards(allocation, true); + executeTimedRunnables(allocator.allocateAllUnassignedShards(allocation, true), () -> maxRunTimeoutInMillis); allocator.afterPrimariesBeforeReplicas(allocation); // Replicas Assignment - allocator.allocateAllUnassignedShards(allocation, false); + executeTimedRunnables(allocator.allocateAllUnassignedShards(allocation, false), () -> maxRunTimeoutInMillis); + } + + private void executeTimedRunnables(List> runnables, Supplier maxRunTimeSupplier) { + Collections.shuffle(runnables); + long startTime = System.nanoTime(); + for (Consumer workQueue : runnables) { + if (System.nanoTime() - startTime < TimeValue.timeValueMillis(maxRunTimeSupplier.get()).nanos()) { + workQueue.accept(false); + } else { + workQueue.accept(true); + } + } } private void disassociateDeadNodes(RoutingAllocation allocation) { diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java index fb2a37237f8b6..097ee95d186fa 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java @@ -41,7 +41,9 @@ import org.opensearch.gateway.GatewayAllocator; import org.opensearch.gateway.ShardsBatchGatewayAllocator; +import java.util.ArrayList; import java.util.List; +import java.util.function.Consumer; /** * Searches for, and allocates, shards for which there is an existing on-disk copy somewhere in the cluster. The default implementation is @@ -108,14 +110,16 @@ void allocateUnassigned( * * Allocation service will currently run the default implementation of it implemented by {@link ShardsBatchGatewayAllocator} */ - default void allocateAllUnassignedShards(RoutingAllocation allocation, boolean primary) { + default List> allocateAllUnassignedShards(RoutingAllocation allocation, boolean primary) { RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); + List> runnables = new ArrayList<>(); while (iterator.hasNext()) { ShardRouting shardRouting = iterator.next(); if (shardRouting.primary() == primary) { - allocateUnassigned(shardRouting, allocation, iterator); + runnables.add((t) -> allocateUnassigned(shardRouting, allocation, iterator)); } } + return runnables; } /** diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java index c493bf717c97f..617902b7506a9 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java @@ -12,6 +12,7 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.routing.RoutingNodes; import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.UnassignedInfo; import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.core.index.shard.ShardId; @@ -121,6 +122,19 @@ public void allocateUnassignedBatch(List shardRoutings, RoutingAll logger.trace("Finished shard allocation execution for unassigned primary shards: {}", shardRoutings.size()); } + public void allocateUnassignedBatchOnTimeout(List shardRoutings, RoutingAllocation allocation) { + Set batchShardRoutingSet = new HashSet<>(shardRoutings); + RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); + while (iterator.hasNext()) { + ShardRouting unassignedShard = iterator.next(); + AllocateUnassignedDecision allocationDecision; + if (unassignedShard.primary() && batchShardRoutingSet.contains(unassignedShard)) { + allocationDecision = new AllocateUnassignedDecision(UnassignedInfo.AllocationStatus.DECIDERS_THROTTLED, null, null, null, false, 0L, 0L); + executeDecision(unassignedShard, allocationDecision, allocation, iterator); + } + } + } + /** * Transforms {@link FetchResult} of {@link NodeGatewayStartedShardsBatch} to {@link List} of {@link TransportNodesListGatewayStartedShards.NodeGatewayStartedShards}. *

diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java index 7c75f2a5d1a8f..99171b17a99e9 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java @@ -168,6 +168,23 @@ public void allocateUnassignedBatch(List shardRoutings, RoutingAll logger.trace("Finished shard allocation execution for unassigned replica shards: {}", shardRoutings.size()); } + public void allocateUnassignedBatchOnTimeout(List shardRoutings, RoutingAllocation allocation) { + Set shardIdsFromBatch = new HashSet<>(); + for (ShardRouting shardRouting : shardRoutings) { + ShardId shardId = shardRouting.shardId(); + shardIdsFromBatch.add(shardId); + } + RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); + while (iterator.hasNext()) { + ShardRouting unassignedShard = iterator.next(); + AllocateUnassignedDecision allocationDecision; + if (!unassignedShard.primary() && shardIdsFromBatch.contains(unassignedShard.shardId())) { + allocationDecision = new AllocateUnassignedDecision(UnassignedInfo.AllocationStatus.DECIDERS_THROTTLED, null, null, null, false, 0L, 0L); + executeDecision(unassignedShard, allocationDecision, allocation, iterator); + } + } + } + private AllocateUnassignedDecision getUnassignedShardAllocationDecision( ShardRouting shardRouting, RoutingAllocation allocation, diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 3c0797cd450d2..6247f4a554f9f 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -41,6 +41,7 @@ import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -51,6 +52,7 @@ import java.util.Set; import java.util.Spliterators; import java.util.concurrent.ConcurrentMap; +import java.util.function.Consumer; import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -187,14 +189,14 @@ public void allocateUnassigned( } @Override - public void allocateAllUnassignedShards(final RoutingAllocation allocation, boolean primary) { + public List> allocateAllUnassignedShards(final RoutingAllocation allocation, boolean primary) { assert primaryShardBatchAllocator != null; assert replicaShardBatchAllocator != null; - innerAllocateUnassignedBatch(allocation, primaryShardBatchAllocator, replicaShardBatchAllocator, primary); + return innerAllocateUnassignedBatch(allocation, primaryShardBatchAllocator, replicaShardBatchAllocator, primary); } - protected void innerAllocateUnassignedBatch( + protected List> innerAllocateUnassignedBatch( RoutingAllocation allocation, PrimaryShardBatchAllocator primaryBatchShardAllocator, ReplicaShardBatchAllocator replicaBatchShardAllocator, @@ -203,21 +205,37 @@ protected void innerAllocateUnassignedBatch( // create batches for unassigned shards Set batchesToAssign = createAndUpdateBatches(allocation, primary); if (batchesToAssign.isEmpty()) { - return; + return Collections.emptyList(); } + List> runnables = new ArrayList<>(); if (primary) { batchIdToStartedShardBatch.values() .stream() .filter(batch -> batchesToAssign.contains(batch.batchId)) .forEach( - shardsBatch -> primaryBatchShardAllocator.allocateUnassignedBatch(shardsBatch.getBatchedShardRoutings(), allocation) + shardsBatch -> runnables.add((timedOut) -> { + if(timedOut) { + primaryBatchShardAllocator.allocateUnassignedBatchOnTimeout(shardsBatch.getBatchedShardRoutings(), allocation); + } else { + primaryBatchShardAllocator.allocateUnassignedBatch(shardsBatch.getBatchedShardRoutings(), allocation); + } + }) ); } else { batchIdToStoreShardBatch.values() .stream() .filter(batch -> batchesToAssign.contains(batch.batchId)) - .forEach(batch -> replicaBatchShardAllocator.allocateUnassignedBatch(batch.getBatchedShardRoutings(), allocation)); + .forEach( + batch -> runnables.add((timedOut) -> { + if(timedOut) { + replicaBatchShardAllocator.allocateUnassignedBatchOnTimeout(batch.getBatchedShardRoutings(), allocation); + } else { + replicaBatchShardAllocator.allocateUnassignedBatch(batch.getBatchedShardRoutings(), allocation); + } + }) + ); } + return runnables; } // visible for testing 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 fbb39c284f0ff..9b11ebc8cd429 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 @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Consumer; public class TestShardBatchGatewayAllocator extends ShardsBatchGatewayAllocator { @@ -102,9 +103,9 @@ protected boolean hasInitiatedFetching(ShardRouting shard) { }; @Override - public void allocateAllUnassignedShards(RoutingAllocation allocation, boolean primary) { + public List> allocateAllUnassignedShards(RoutingAllocation allocation, boolean primary) { currentNodes = allocation.nodes(); - innerAllocateUnassignedBatch(allocation, primaryBatchShardAllocator, replicaBatchShardAllocator, primary); + return innerAllocateUnassignedBatch(allocation, primaryBatchShardAllocator, replicaBatchShardAllocator, primary); } @Override From ef8232a89366d78688753bb4cd71df3e2d3441a3 Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Mon, 22 Jul 2024 11:59:14 +0530 Subject: [PATCH 02/35] Add settings for allocator timeout and add additional logs Signed-off-by: Rishab Nahata --- .../routing/allocation/AllocationService.java | 18 +++-- .../allocation/ExistingShardsAllocator.java | 9 +++ .../allocator/BalancedShardsAllocator.java | 26 +++++-- .../allocator/LocalShardsBalancer.java | 19 ++++- .../allocator/RemoteShardsBalancer.java | 2 +- .../allocation/allocator/ShardsBalancer.java | 2 +- .../common/settings/ClusterSettings.java | 3 + .../gateway/ShardsBatchGatewayAllocator.java | 69 ++++++++++++++++++- 8 files changed, 134 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java index 9eeb75d4f05aa..b888a6854c91c 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java @@ -73,6 +73,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.TimeUnit; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; @@ -102,7 +103,6 @@ public class AllocationService { private final ClusterInfoService clusterInfoService; private SnapshotsInfoService snapshotsInfoService; private final ClusterManagerMetrics clusterManagerMetrics; - private final long maxRunTimeoutInMillis = 5; // only for tests that use the GatewayAllocator as the unique ExistingShardsAllocator public AllocationService( @@ -568,8 +568,14 @@ private void reroute(RoutingAllocation allocation) { long rerouteStartTimeNS = System.nanoTime(); removeDelayMarkers(allocation); + long startTime = System.nanoTime(); allocateExistingUnassignedShards(allocation); // try to allocate existing shard copies first + logger.info("Completing allocate unassigned, elapsed time: [{}]", + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); + startTime = System.nanoTime(); shardsAllocator.allocate(allocation); + logger.info("Shard allocator to allocate all shards, elapsed time: [{}]", + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); clusterManagerMetrics.recordLatency( clusterManagerMetrics.rerouteHistogram, (double) Math.max(0, TimeValue.nsecToMSec(System.nanoTime() - rerouteStartTimeNS)) @@ -620,22 +626,26 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { private void allocateAllUnassignedShards(RoutingAllocation allocation) { ExistingShardsAllocator allocator = existingShardsAllocators.get(ShardsBatchGatewayAllocator.ALLOCATOR_NAME); - executeTimedRunnables(allocator.allocateAllUnassignedShards(allocation, true), () -> maxRunTimeoutInMillis); + executeTimedRunnables(allocator.allocateAllUnassignedShards(allocation, true), () -> allocator.getPrimaryBatchAllocatorTimeout().millis(), true); allocator.afterPrimariesBeforeReplicas(allocation); // Replicas Assignment - executeTimedRunnables(allocator.allocateAllUnassignedShards(allocation, false), () -> maxRunTimeoutInMillis); + executeTimedRunnables(allocator.allocateAllUnassignedShards(allocation, false), () -> allocator.getReplicaBatchAllocatorTimeout().millis(), false); } - private void executeTimedRunnables(List> runnables, Supplier maxRunTimeSupplier) { + private void executeTimedRunnables(List> runnables, Supplier maxRunTimeSupplier, boolean primary) { + logger.info("Executing timed runnables for primary [{}] of size [{}]", primary, runnables.size()); Collections.shuffle(runnables); long startTime = System.nanoTime(); for (Consumer workQueue : runnables) { if (System.nanoTime() - startTime < TimeValue.timeValueMillis(maxRunTimeSupplier.get()).nanos()) { + logger.info("Starting primary [{}] batch to allocate", primary); workQueue.accept(false); } else { + logger.info("Timing out primary [{}] batch to allocate", primary); workQueue.accept(true); } } + logger.info("Time taken to execute timed runnables in this cycle:[{}ms]", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); } private void disassociateDeadNodes(RoutingAllocation allocation) { diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java index 097ee95d186fa..a70fc6e4bc926 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java @@ -38,6 +38,7 @@ import org.opensearch.cluster.routing.UnassignedInfo; import org.opensearch.common.Nullable; import org.opensearch.common.settings.Setting; +import org.opensearch.common.unit.TimeValue; import org.opensearch.gateway.GatewayAllocator; import org.opensearch.gateway.ShardsBatchGatewayAllocator; @@ -122,6 +123,14 @@ default List> allocateAllUnassignedShards(RoutingAllocation al return runnables; } + default TimeValue getPrimaryBatchAllocatorTimeout() { + return TimeValue.MINUS_ONE; + } + + default TimeValue getReplicaBatchAllocatorTimeout() { + return TimeValue.MINUS_ONE; + } + /** * Returns an explanation for a single unassigned shard. */ 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 b2443490dd973..47fe51013b38f 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 @@ -54,6 +54,7 @@ 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 java.util.HashMap; import java.util.HashSet; @@ -162,6 +163,13 @@ public class BalancedShardsAllocator implements ShardsAllocator { Property.NodeScope ); + public static final Setting ALLOCATE_UNASSIGNED_TIMEOUT_SETTING = Setting.timeSetting( + "cluster.routing.allocation.allocate_unassigned_timeout", + TimeValue.MINUS_ONE, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + private volatile boolean movePrimaryFirst; private volatile ShardMovementStrategy shardMovementStrategy; @@ -172,6 +180,7 @@ public class BalancedShardsAllocator implements ShardsAllocator { private volatile float shardBalanceFactor; private volatile WeightFunction weightFunction; private volatile float threshold; + private volatile TimeValue allocateUnassignedTimeout; public BalancedShardsAllocator(Settings settings) { this(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); @@ -187,6 +196,7 @@ public BalancedShardsAllocator(Settings settings, ClusterSettings clusterSetting setPreferPrimaryShardBalance(PREFER_PRIMARY_SHARD_BALANCE.get(settings)); setPreferPrimaryShardRebalance(PREFER_PRIMARY_SHARD_REBALANCE.get(settings)); setShardMovementStrategy(SHARD_MOVEMENT_STRATEGY_SETTING.get(settings)); + setAllocateUnassignedTimeout(ALLOCATE_UNASSIGNED_TIMEOUT_SETTING.get(settings)); clusterSettings.addSettingsUpdateConsumer(PREFER_PRIMARY_SHARD_BALANCE, this::setPreferPrimaryShardBalance); clusterSettings.addSettingsUpdateConsumer(SHARD_MOVE_PRIMARY_FIRST_SETTING, this::setMovePrimaryFirst); clusterSettings.addSettingsUpdateConsumer(SHARD_MOVEMENT_STRATEGY_SETTING, this::setShardMovementStrategy); @@ -195,6 +205,7 @@ public BalancedShardsAllocator(Settings settings, ClusterSettings clusterSetting clusterSettings.addSettingsUpdateConsumer(PRIMARY_SHARD_REBALANCE_BUFFER, this::updatePreferPrimaryShardBalanceBuffer); clusterSettings.addSettingsUpdateConsumer(PREFER_PRIMARY_SHARD_REBALANCE, this::setPreferPrimaryShardRebalance); clusterSettings.addSettingsUpdateConsumer(THRESHOLD_SETTING, this::setThreshold); + clusterSettings.addSettingsUpdateConsumer(ALLOCATE_UNASSIGNED_TIMEOUT_SETTING, this::setAllocateUnassignedTimeout); } /** @@ -269,6 +280,10 @@ private void setThreshold(float threshold) { this.threshold = threshold; } + private void setAllocateUnassignedTimeout(TimeValue allocateUnassignedTimeout) { + this.allocateUnassignedTimeout = allocateUnassignedTimeout; + } + @Override public void allocate(RoutingAllocation allocation) { if (allocation.routingNodes().size() == 0) { @@ -282,7 +297,8 @@ public void allocate(RoutingAllocation allocation) { weightFunction, threshold, preferPrimaryShardBalance, - preferPrimaryShardRebalance + preferPrimaryShardRebalance, + allocateUnassignedTimeout ); localShardsBalancer.allocateUnassigned(); localShardsBalancer.moveShards(); @@ -297,6 +313,7 @@ public void allocate(RoutingAllocation allocation) { @Override public ShardAllocationDecision decideShardAllocation(final ShardRouting shard, final RoutingAllocation allocation) { + long startTime = System.nanoTime(); ShardsBalancer localShardsBalancer = new LocalShardsBalancer( logger, allocation, @@ -304,12 +321,13 @@ public ShardAllocationDecision decideShardAllocation(final ShardRouting shard, f weightFunction, threshold, preferPrimaryShardBalance, - preferPrimaryShardRebalance + preferPrimaryShardRebalance, + allocateUnassignedTimeout ); AllocateUnassignedDecision allocateUnassignedDecision = AllocateUnassignedDecision.NOT_TAKEN; MoveDecision moveDecision = MoveDecision.NOT_TAKEN; if (shard.unassigned()) { - allocateUnassignedDecision = localShardsBalancer.decideAllocateUnassigned(shard); + allocateUnassignedDecision = localShardsBalancer.decideAllocateUnassigned(shard, startTime); } else { moveDecision = localShardsBalancer.decideMove(shard); if (moveDecision.isDecisionTaken() && moveDecision.canRemain()) { @@ -558,7 +576,7 @@ public Balancer( float threshold, boolean preferPrimaryBalance ) { - super(logger, allocation, shardMovementStrategy, weight, threshold, preferPrimaryBalance, false); + super(logger, allocation, shardMovementStrategy, weight, threshold, preferPrimaryBalance, false, TimeValue.MINUS_ONE); } } diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java index 00eb79add9f1d..d2336608a9c23 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java @@ -29,6 +29,7 @@ import org.opensearch.cluster.routing.allocation.decider.Decision; import org.opensearch.cluster.routing.allocation.decider.DiskThresholdDecider; import org.opensearch.common.collect.Tuple; +import org.opensearch.common.unit.TimeValue; import org.opensearch.gateway.PriorityComparator; import java.util.ArrayList; @@ -40,6 +41,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.TimeUnit; import java.util.stream.Stream; import java.util.stream.StreamSupport; @@ -60,6 +62,7 @@ public class LocalShardsBalancer extends ShardsBalancer { private final boolean preferPrimaryBalance; private final boolean preferPrimaryRebalance; + private final TimeValue allocateUnassignedTimeout; private final BalancedShardsAllocator.WeightFunction weight; private final float threshold; @@ -77,7 +80,8 @@ public LocalShardsBalancer( BalancedShardsAllocator.WeightFunction weight, float threshold, boolean preferPrimaryBalance, - boolean preferPrimaryRebalance + boolean preferPrimaryRebalance, + TimeValue allocateUnassignedTimeout ) { this.logger = logger; this.allocation = allocation; @@ -94,6 +98,7 @@ public LocalShardsBalancer( this.preferPrimaryBalance = preferPrimaryBalance; this.preferPrimaryRebalance = preferPrimaryRebalance; this.shardMovementStrategy = shardMovementStrategy; + this.allocateUnassignedTimeout = allocateUnassignedTimeout; } /** @@ -742,6 +747,7 @@ private Map buildModelFromAssigned() */ @Override void allocateUnassigned() { + long startTime = System.nanoTime(); RoutingNodes.UnassignedShards unassigned = routingNodes.unassigned(); assert !nodes.isEmpty(); if (logger.isTraceEnabled()) { @@ -797,7 +803,7 @@ void allocateUnassigned() { do { for (int i = 0; i < primaryLength; i++) { ShardRouting shard = primary[i]; - final AllocateUnassignedDecision allocationDecision = decideAllocateUnassigned(shard); + final AllocateUnassignedDecision allocationDecision = decideAllocateUnassigned(shard, startTime); final String assignedNodeId = allocationDecision.getTargetNode() != null ? allocationDecision.getTargetNode().getId() : null; @@ -870,6 +876,8 @@ void allocateUnassigned() { secondaryLength = 0; } while (primaryLength > 0); // clear everything we have either added it or moved to ignoreUnassigned + logger.debug("Time taken in allocate unassigned [{}]", + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); } /** @@ -879,12 +887,17 @@ void allocateUnassigned() { * is of type {@link Decision.Type#NO}, then the assigned node will be null. */ @Override - AllocateUnassignedDecision decideAllocateUnassigned(final ShardRouting shard) { + AllocateUnassignedDecision decideAllocateUnassigned(final ShardRouting shard, long startTime) { if (shard.assignedToNode()) { // we only make decisions for unassigned shards here return AllocateUnassignedDecision.NOT_TAKEN; } + if (System.nanoTime() - startTime > allocateUnassignedTimeout.nanos()) { + logger.info("Timed out while running Local shard balancer allocate unassigned - outer loop"); + return AllocateUnassignedDecision.throttle(null); + } + final boolean explain = allocation.debugDecision(); Decision shardLevelDecision = allocation.deciders().canAllocate(shard, allocation); if (shardLevelDecision.type() == Decision.Type.NO && explain == false) { 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 a05938c176678..0fc48092c95a2 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 @@ -288,7 +288,7 @@ private Map calculateNodePrimaryShardCount(List re } @Override - AllocateUnassignedDecision decideAllocateUnassigned(ShardRouting shardRouting) { + AllocateUnassignedDecision decideAllocateUnassigned(ShardRouting shardRouting, long startTime) { throw new UnsupportedOperationException("remote shards balancer does not support decision operations"); } diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/ShardsBalancer.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/ShardsBalancer.java index ef2dbd34644a7..f6284797a303d 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/ShardsBalancer.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/ShardsBalancer.java @@ -42,7 +42,7 @@ public abstract class ShardsBalancer { * @param shardRouting the shard for which the decision has to be made * @return the allocation decision */ - abstract AllocateUnassignedDecision decideAllocateUnassigned(ShardRouting shardRouting); + abstract AllocateUnassignedDecision decideAllocateUnassigned(ShardRouting shardRouting, long startTime); /** * Makes a decision on whether to move a started shard to another node. diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index 5dcf23ae52294..d8af832d0d9b2 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -267,6 +267,7 @@ public void apply(Settings value, Settings current, Settings previous) { BalancedShardsAllocator.SHARD_MOVE_PRIMARY_FIRST_SETTING, BalancedShardsAllocator.SHARD_MOVEMENT_STRATEGY_SETTING, BalancedShardsAllocator.THRESHOLD_SETTING, + BalancedShardsAllocator.ALLOCATE_UNASSIGNED_TIMEOUT_SETTING, BreakerSettings.CIRCUIT_BREAKER_LIMIT_SETTING, BreakerSettings.CIRCUIT_BREAKER_OVERHEAD_SETTING, BreakerSettings.CIRCUIT_BREAKER_TYPE, @@ -342,6 +343,8 @@ public void apply(Settings value, Settings current, Settings previous) { GatewayService.RECOVER_AFTER_NODES_SETTING, GatewayService.RECOVER_AFTER_TIME_SETTING, ShardsBatchGatewayAllocator.GATEWAY_ALLOCATOR_BATCH_SIZE, + ShardsBatchGatewayAllocator.PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING, + ShardsBatchGatewayAllocator.REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING, PersistedClusterStateService.SLOW_WRITE_LOGGING_THRESHOLD, NetworkModule.HTTP_DEFAULT_TYPE_SETTING, NetworkModule.TRANSPORT_DEFAULT_TYPE_SETTING, diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 6247f4a554f9f..f47a4bdf73a9f 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -27,8 +27,10 @@ import org.opensearch.common.UUIDs; import org.opensearch.common.inject.Inject; import org.opensearch.common.lease.Releasables; +import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.common.util.set.Sets; import org.opensearch.core.action.ActionListener; @@ -52,6 +54,7 @@ import java.util.Set; import java.util.Spliterators; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; import java.util.function.Consumer; import java.util.function.Predicate; import java.util.stream.Collectors; @@ -70,6 +73,12 @@ public class ShardsBatchGatewayAllocator implements ExistingShardsAllocator { private final long maxBatchSize; private static final short DEFAULT_SHARD_BATCH_SIZE = 2000; + private static final String PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING_KEY = "cluster.routing.allocation.shards_batch_gateway_allocator.primary_allocator_timeout"; + private static final String REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING_KEY = "cluster.routing.allocation.shards_batch_gateway_allocator.replica_allocator_timeout"; + + private TimeValue primaryShardsBatchGatewayAllocatorTimeout; + private TimeValue replicaShardsBatchGatewayAllocatorTimeout; + /** * Number of shards we send in one batch to data nodes for fetching metadata */ @@ -81,6 +90,20 @@ public class ShardsBatchGatewayAllocator implements ExistingShardsAllocator { Setting.Property.NodeScope ); + public static final Setting PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING = Setting.timeSetting( + PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING_KEY, + TimeValue.timeValueSeconds(20), + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + + public static final Setting REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING = Setting.timeSetting( + REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING_KEY, + TimeValue.timeValueSeconds(20), + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + private final RerouteService rerouteService; private final PrimaryShardBatchAllocator primaryShardBatchAllocator; private final ReplicaShardBatchAllocator replicaShardBatchAllocator; @@ -99,7 +122,8 @@ public ShardsBatchGatewayAllocator( RerouteService rerouteService, TransportNodesListGatewayStartedShardsBatch batchStartedAction, TransportNodesListShardStoreMetadataBatch batchStoreAction, - Settings settings + Settings settings, + ClusterSettings clusterSettings ) { this.rerouteService = rerouteService; this.primaryShardBatchAllocator = new InternalPrimaryBatchShardAllocator(); @@ -107,6 +131,16 @@ public ShardsBatchGatewayAllocator( this.batchStartedAction = batchStartedAction; this.batchStoreAction = batchStoreAction; this.maxBatchSize = GATEWAY_ALLOCATOR_BATCH_SIZE.get(settings); + this.primaryShardsBatchGatewayAllocatorTimeout = PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING.get(settings); + clusterSettings.addSettingsUpdateConsumer( + PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING, + this::setPrimaryBatchAllocatorTimeout + ); + this.replicaShardsBatchGatewayAllocatorTimeout = REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING.get(settings); + clusterSettings.addSettingsUpdateConsumer( + REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING, + this::setReplicaBatchAllocatorTimeout + ); } @Override @@ -129,7 +163,28 @@ protected ShardsBatchGatewayAllocator(long batchSize) { this.batchStoreAction = null; this.replicaShardBatchAllocator = null; this.maxBatchSize = batchSize; + this.primaryShardsBatchGatewayAllocatorTimeout = null; + this.replicaShardsBatchGatewayAllocatorTimeout = null; } + + @Override + public TimeValue getPrimaryBatchAllocatorTimeout() { + return this.primaryShardsBatchGatewayAllocatorTimeout; + } + + public void setPrimaryBatchAllocatorTimeout(TimeValue primaryShardsBatchGatewayAllocatorTimeout) { + this.primaryShardsBatchGatewayAllocatorTimeout = primaryShardsBatchGatewayAllocatorTimeout; + } + + @Override + public TimeValue getReplicaBatchAllocatorTimeout() { + return this.primaryShardsBatchGatewayAllocatorTimeout; + } + + public void setReplicaBatchAllocatorTimeout(TimeValue replicaShardsBatchGatewayAllocatorTimeout) { + this.replicaShardsBatchGatewayAllocatorTimeout = replicaShardsBatchGatewayAllocatorTimeout; + } + // for tests @Override @@ -215,9 +270,15 @@ protected List> innerAllocateUnassignedBatch( .forEach( shardsBatch -> runnables.add((timedOut) -> { if(timedOut) { + long startTime = System.nanoTime(); primaryBatchShardAllocator.allocateUnassignedBatchOnTimeout(shardsBatch.getBatchedShardRoutings(), allocation); + logger.info("Time taken to execute allocateUnassignedBatchOnTimeout for unassigned primary batch with id [{}], size : [{}] in this cycle:[{}ms]", + shardsBatch.batchId, shardsBatch.getBatchedShardRoutings().size(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); } else { + long startTime = System.nanoTime(); primaryBatchShardAllocator.allocateUnassignedBatch(shardsBatch.getBatchedShardRoutings(), allocation); + logger.info("Time taken to allocate unassigned primary batch with id [{}], size : [{}] in this cycle:[{}ms]", + shardsBatch.batchId, shardsBatch.getBatchedShardRoutings().size(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); } }) ); @@ -228,9 +289,15 @@ protected List> innerAllocateUnassignedBatch( .forEach( batch -> runnables.add((timedOut) -> { if(timedOut) { + long startTime = System.nanoTime(); replicaBatchShardAllocator.allocateUnassignedBatchOnTimeout(batch.getBatchedShardRoutings(), allocation); + logger.info("Time taken to execute allocateUnassignedBatchOnTimeout for unassigned replica batch with id [{}], size : [{}] in this cycle:[{}ms]", + batch.batchId, batch.getBatchedShardRoutings().size(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); } else { + long startTime = System.nanoTime(); replicaBatchShardAllocator.allocateUnassignedBatch(batch.getBatchedShardRoutings(), allocation); + logger.info("Time taken to allocate unassigned replica batch with id [{}], size : [{}] in this cycle:[{}ms]", + batch.batchId, batch.getBatchedShardRoutings().size(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); } }) ); From bf9cf1623b6cef6986d949866c6b99dd64fb4f4a Mon Sep 17 00:00:00 2001 From: Bukhtawar Khan Date: Mon, 22 Jul 2024 13:51:46 +0530 Subject: [PATCH 03/35] Change to TimeoutAwareRunnable Signed-off-by: Bukhtawar Khan --- .../routing/allocation/AllocationService.java | 9 ++--- .../allocation/ExistingShardsAllocator.java | 17 +++++++-- .../util/concurrent/TimeoutAwareRunnable.java | 17 +++++++++ .../gateway/ShardsBatchGatewayAllocator.java | 36 ++++++++++++------- .../TestShardBatchGatewayAllocator.java | 3 +- 5 files changed, 61 insertions(+), 21 deletions(-) create mode 100644 server/src/main/java/org/opensearch/common/util/concurrent/TimeoutAwareRunnable.java diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java index b888a6854c91c..1a9eaf99a5ad8 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java @@ -58,6 +58,7 @@ import org.opensearch.cluster.routing.allocation.decider.Decision; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.concurrent.TimeoutAwareRunnable; import org.opensearch.gateway.GatewayAllocator; import org.opensearch.gateway.PriorityComparator; import org.opensearch.gateway.ShardsBatchGatewayAllocator; @@ -632,17 +633,17 @@ private void allocateAllUnassignedShards(RoutingAllocation allocation) { executeTimedRunnables(allocator.allocateAllUnassignedShards(allocation, false), () -> allocator.getReplicaBatchAllocatorTimeout().millis(), false); } - private void executeTimedRunnables(List> runnables, Supplier maxRunTimeSupplier, boolean primary) { + private void executeTimedRunnables(List runnables, Supplier maxRunTimeSupplier, boolean primary) { logger.info("Executing timed runnables for primary [{}] of size [{}]", primary, runnables.size()); Collections.shuffle(runnables); long startTime = System.nanoTime(); - for (Consumer workQueue : runnables) { + for (TimeoutAwareRunnable workQueue : runnables) { if (System.nanoTime() - startTime < TimeValue.timeValueMillis(maxRunTimeSupplier.get()).nanos()) { logger.info("Starting primary [{}] batch to allocate", primary); - workQueue.accept(false); + workQueue.run(); } else { logger.info("Timing out primary [{}] batch to allocate", primary); - workQueue.accept(true); + workQueue.onTimeout(); } } logger.info("Time taken to execute timed runnables in this cycle:[{}ms]", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java index a70fc6e4bc926..91d9d08f4c333 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java @@ -39,6 +39,7 @@ import org.opensearch.common.Nullable; import org.opensearch.common.settings.Setting; import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.concurrent.TimeoutAwareRunnable; import org.opensearch.gateway.GatewayAllocator; import org.opensearch.gateway.ShardsBatchGatewayAllocator; @@ -111,13 +112,23 @@ void allocateUnassigned( * * Allocation service will currently run the default implementation of it implemented by {@link ShardsBatchGatewayAllocator} */ - default List> allocateAllUnassignedShards(RoutingAllocation allocation, boolean primary) { + default List allocateAllUnassignedShards(RoutingAllocation allocation, boolean primary) { RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); - List> runnables = new ArrayList<>(); + List runnables = new ArrayList<>(); while (iterator.hasNext()) { ShardRouting shardRouting = iterator.next(); if (shardRouting.primary() == primary) { - runnables.add((t) -> allocateUnassigned(shardRouting, allocation, iterator)); + runnables.add(new TimeoutAwareRunnable() { + @Override + public void onTimeout() { + //do nothing + } + + @Override + public void run() { + allocateUnassigned(shardRouting, allocation, iterator); + } + }); } } return runnables; diff --git a/server/src/main/java/org/opensearch/common/util/concurrent/TimeoutAwareRunnable.java b/server/src/main/java/org/opensearch/common/util/concurrent/TimeoutAwareRunnable.java new file mode 100644 index 0000000000000..b84a43004b54d --- /dev/null +++ b/server/src/main/java/org/opensearch/common/util/concurrent/TimeoutAwareRunnable.java @@ -0,0 +1,17 @@ +/* + * 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.common.util.concurrent; + +/** + * Runnable that is aware of a timeout and can execute another {@link Runnable} when a timeout is reached + */ +public interface TimeoutAwareRunnable extends Runnable { + + void onTimeout(); +} diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index f47a4bdf73a9f..936b25f0dd435 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -32,6 +32,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.ConcurrentCollections; +import org.opensearch.common.util.concurrent.TimeoutAwareRunnable; import org.opensearch.common.util.set.Sets; import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.shard.ShardId; @@ -244,14 +245,14 @@ public void allocateUnassigned( } @Override - public List> allocateAllUnassignedShards(final RoutingAllocation allocation, boolean primary) { + public List allocateAllUnassignedShards(final RoutingAllocation allocation, boolean primary) { assert primaryShardBatchAllocator != null; assert replicaShardBatchAllocator != null; return innerAllocateUnassignedBatch(allocation, primaryShardBatchAllocator, replicaShardBatchAllocator, primary); } - protected List> innerAllocateUnassignedBatch( + protected List innerAllocateUnassignedBatch( RoutingAllocation allocation, PrimaryShardBatchAllocator primaryBatchShardAllocator, ReplicaShardBatchAllocator replicaBatchShardAllocator, @@ -262,45 +263,54 @@ protected List> innerAllocateUnassignedBatch( if (batchesToAssign.isEmpty()) { return Collections.emptyList(); } - List> runnables = new ArrayList<>(); + List runnables = new ArrayList<>(); if (primary) { batchIdToStartedShardBatch.values() .stream() .filter(batch -> batchesToAssign.contains(batch.batchId)) .forEach( - shardsBatch -> runnables.add((timedOut) -> { - if(timedOut) { + shardsBatch -> runnables.add(new TimeoutAwareRunnable() { + @Override + public void onTimeout() { long startTime = System.nanoTime(); primaryBatchShardAllocator.allocateUnassignedBatchOnTimeout(shardsBatch.getBatchedShardRoutings(), allocation); logger.info("Time taken to execute allocateUnassignedBatchOnTimeout for unassigned primary batch with id [{}], size : [{}] in this cycle:[{}ms]", shardsBatch.batchId, shardsBatch.getBatchedShardRoutings().size(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); - } else { + } + + @Override + public void run() { long startTime = System.nanoTime(); primaryBatchShardAllocator.allocateUnassignedBatch(shardsBatch.getBatchedShardRoutings(), allocation); logger.info("Time taken to allocate unassigned primary batch with id [{}], size : [{}] in this cycle:[{}ms]", shardsBatch.batchId, shardsBatch.getBatchedShardRoutings().size(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); + } - }) - ); + })); } else { batchIdToStoreShardBatch.values() .stream() .filter(batch -> batchesToAssign.contains(batch.batchId)) .forEach( - batch -> runnables.add((timedOut) -> { - if(timedOut) { + batch -> runnables.add(new TimeoutAwareRunnable() { + @Override + public void onTimeout() { long startTime = System.nanoTime(); replicaBatchShardAllocator.allocateUnassignedBatchOnTimeout(batch.getBatchedShardRoutings(), allocation); logger.info("Time taken to execute allocateUnassignedBatchOnTimeout for unassigned replica batch with id [{}], size : [{}] in this cycle:[{}ms]", batch.batchId, batch.getBatchedShardRoutings().size(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); - } else { + + } + + @Override + public void run() { long startTime = System.nanoTime(); replicaBatchShardAllocator.allocateUnassignedBatch(batch.getBatchedShardRoutings(), allocation); logger.info("Time taken to allocate unassigned replica batch with id [{}], size : [{}] in this cycle:[{}ms]", batch.batchId, batch.getBatchedShardRoutings().size(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); + } - }) - ); + })); } return runnables; } 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 9b11ebc8cd429..5f76c4f9b8189 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 @@ -13,6 +13,7 @@ import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.opensearch.cluster.routing.allocation.RoutingAllocation; +import org.opensearch.common.util.concurrent.TimeoutAwareRunnable; import org.opensearch.core.index.shard.ShardId; import org.opensearch.gateway.AsyncShardFetch; import org.opensearch.gateway.PrimaryShardBatchAllocator; @@ -103,7 +104,7 @@ protected boolean hasInitiatedFetching(ShardRouting shard) { }; @Override - public List> allocateAllUnassignedShards(RoutingAllocation allocation, boolean primary) { + public List allocateAllUnassignedShards(RoutingAllocation allocation, boolean primary) { currentNodes = allocation.nodes(); return innerAllocateUnassignedBatch(allocation, primaryBatchShardAllocator, replicaBatchShardAllocator, primary); } From de8f6aa2ae11056bb09cae9ba6b0895ce2860e47 Mon Sep 17 00:00:00 2001 From: Bukhtawar Khan Date: Mon, 22 Jul 2024 19:40:48 +0530 Subject: [PATCH 04/35] Introduce BatchRunnableExecutor Signed-off-by: Bukhtawar Khan --- .../routing/allocation/AllocationService.java | 23 +------- .../allocation/ExistingShardsAllocator.java | 14 ++--- .../common/util/BatchRunnableExecutor.java | 54 +++++++++++++++++++ .../util/concurrent/TimeoutAwareRunnable.java | 2 +- .../gateway/ShardsBatchGatewayAllocator.java | 37 +++++-------- .../TestShardBatchGatewayAllocator.java | 5 +- 6 files changed, 76 insertions(+), 59 deletions(-) create mode 100644 server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java index 1a9eaf99a5ad8..48b0e957a826f 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java @@ -58,7 +58,6 @@ import org.opensearch.cluster.routing.allocation.decider.Decision; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; -import org.opensearch.common.util.concurrent.TimeoutAwareRunnable; import org.opensearch.gateway.GatewayAllocator; import org.opensearch.gateway.PriorityComparator; import org.opensearch.gateway.ShardsBatchGatewayAllocator; @@ -75,9 +74,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; -import java.util.function.Consumer; import java.util.function.Function; -import java.util.function.Supplier; import java.util.stream.Collectors; import static java.util.Collections.emptyList; @@ -627,26 +624,10 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { private void allocateAllUnassignedShards(RoutingAllocation allocation) { ExistingShardsAllocator allocator = existingShardsAllocators.get(ShardsBatchGatewayAllocator.ALLOCATOR_NAME); - executeTimedRunnables(allocator.allocateAllUnassignedShards(allocation, true), () -> allocator.getPrimaryBatchAllocatorTimeout().millis(), true); + allocator.allocateAllUnassignedShards(allocation, true).run(); allocator.afterPrimariesBeforeReplicas(allocation); // Replicas Assignment - executeTimedRunnables(allocator.allocateAllUnassignedShards(allocation, false), () -> allocator.getReplicaBatchAllocatorTimeout().millis(), false); - } - - private void executeTimedRunnables(List runnables, Supplier maxRunTimeSupplier, boolean primary) { - logger.info("Executing timed runnables for primary [{}] of size [{}]", primary, runnables.size()); - Collections.shuffle(runnables); - long startTime = System.nanoTime(); - for (TimeoutAwareRunnable workQueue : runnables) { - if (System.nanoTime() - startTime < TimeValue.timeValueMillis(maxRunTimeSupplier.get()).nanos()) { - logger.info("Starting primary [{}] batch to allocate", primary); - workQueue.run(); - } else { - logger.info("Timing out primary [{}] batch to allocate", primary); - workQueue.onTimeout(); - } - } - logger.info("Time taken to execute timed runnables in this cycle:[{}ms]", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); + allocator.allocateAllUnassignedShards(allocation, false).run(); } private void disassociateDeadNodes(RoutingAllocation allocation) { diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java index 91d9d08f4c333..57098dab89b95 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java @@ -39,13 +39,13 @@ import org.opensearch.common.Nullable; import org.opensearch.common.settings.Setting; import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.BatchRunnableExecutor; import org.opensearch.common.util.concurrent.TimeoutAwareRunnable; import org.opensearch.gateway.GatewayAllocator; import org.opensearch.gateway.ShardsBatchGatewayAllocator; import java.util.ArrayList; import java.util.List; -import java.util.function.Consumer; /** * Searches for, and allocates, shards for which there is an existing on-disk copy somewhere in the cluster. The default implementation is @@ -112,7 +112,7 @@ void allocateUnassigned( * * Allocation service will currently run the default implementation of it implemented by {@link ShardsBatchGatewayAllocator} */ - default List allocateAllUnassignedShards(RoutingAllocation allocation, boolean primary) { + default BatchRunnableExecutor allocateAllUnassignedShards(RoutingAllocation allocation, boolean primary) { RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); List runnables = new ArrayList<>(); while (iterator.hasNext()) { @@ -131,15 +131,7 @@ public void run() { }); } } - return runnables; - } - - default TimeValue getPrimaryBatchAllocatorTimeout() { - return TimeValue.MINUS_ONE; - } - - default TimeValue getReplicaBatchAllocatorTimeout() { - return TimeValue.MINUS_ONE; + return new BatchRunnableExecutor(runnables, () -> TimeValue.MINUS_ONE); } /** diff --git a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java new file mode 100644 index 0000000000000..dc610161da22f --- /dev/null +++ b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java @@ -0,0 +1,54 @@ +/* + * 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.common.util; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.concurrent.TimeoutAwareRunnable; + +import java.util.Collections; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; + +/** + * The executor that executes a batch of {@link TimeoutAwareRunnable} and triggers a timeout based on {@link TimeValue} timeout + */ +public class BatchRunnableExecutor implements Runnable { + + private final Supplier timeoutSupplier; + + private final List timeoutAwareRunnables; + + private static final Logger logger = LogManager.getLogger(BatchRunnableExecutor.class); + + public BatchRunnableExecutor(List timeoutAwareRunnables, Supplier timeoutSupplier) { + this.timeoutSupplier = timeoutSupplier; + this.timeoutAwareRunnables = timeoutAwareRunnables; + } + + @Override + public void run() { + logger.debug("Starting execution of runnable of size [{}]", timeoutAwareRunnables.size()); + Collections.shuffle(timeoutAwareRunnables); + long startTime = System.nanoTime(); + for (TimeoutAwareRunnable workQueue : timeoutAwareRunnables) { + if (System.nanoTime() - startTime > timeoutSupplier.get().nanos()) { + workQueue.run(); + } else { + logger.debug("Executing timeout for runnable of size [{}]", timeoutAwareRunnables.size()); + workQueue.onTimeout(); + } + } + logger.debug("Time taken to execute timed runnables in this cycle:[{}ms]", + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); + } + +} diff --git a/server/src/main/java/org/opensearch/common/util/concurrent/TimeoutAwareRunnable.java b/server/src/main/java/org/opensearch/common/util/concurrent/TimeoutAwareRunnable.java index b84a43004b54d..2d890c5e01e45 100644 --- a/server/src/main/java/org/opensearch/common/util/concurrent/TimeoutAwareRunnable.java +++ b/server/src/main/java/org/opensearch/common/util/concurrent/TimeoutAwareRunnable.java @@ -9,7 +9,7 @@ package org.opensearch.common.util.concurrent; /** - * Runnable that is aware of a timeout and can execute another {@link Runnable} when a timeout is reached + * Runnable that is aware of a timeout */ public interface TimeoutAwareRunnable extends Runnable { diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 936b25f0dd435..36a4049bac8fd 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -31,6 +31,7 @@ import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.BatchRunnableExecutor; import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.common.util.concurrent.TimeoutAwareRunnable; import org.opensearch.common.util.set.Sets; @@ -56,7 +57,6 @@ import java.util.Spliterators; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; -import java.util.function.Consumer; import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -168,24 +168,6 @@ protected ShardsBatchGatewayAllocator(long batchSize) { this.replicaShardsBatchGatewayAllocatorTimeout = null; } - @Override - public TimeValue getPrimaryBatchAllocatorTimeout() { - return this.primaryShardsBatchGatewayAllocatorTimeout; - } - - public void setPrimaryBatchAllocatorTimeout(TimeValue primaryShardsBatchGatewayAllocatorTimeout) { - this.primaryShardsBatchGatewayAllocatorTimeout = primaryShardsBatchGatewayAllocatorTimeout; - } - - @Override - public TimeValue getReplicaBatchAllocatorTimeout() { - return this.primaryShardsBatchGatewayAllocatorTimeout; - } - - public void setReplicaBatchAllocatorTimeout(TimeValue replicaShardsBatchGatewayAllocatorTimeout) { - this.replicaShardsBatchGatewayAllocatorTimeout = replicaShardsBatchGatewayAllocatorTimeout; - } - // for tests @Override @@ -245,14 +227,14 @@ public void allocateUnassigned( } @Override - public List allocateAllUnassignedShards(final RoutingAllocation allocation, boolean primary) { + public BatchRunnableExecutor allocateAllUnassignedShards(final RoutingAllocation allocation, boolean primary) { assert primaryShardBatchAllocator != null; assert replicaShardBatchAllocator != null; return innerAllocateUnassignedBatch(allocation, primaryShardBatchAllocator, replicaShardBatchAllocator, primary); } - protected List innerAllocateUnassignedBatch( + protected BatchRunnableExecutor innerAllocateUnassignedBatch( RoutingAllocation allocation, PrimaryShardBatchAllocator primaryBatchShardAllocator, ReplicaShardBatchAllocator replicaBatchShardAllocator, @@ -261,7 +243,7 @@ protected List innerAllocateUnassignedBatch( // create batches for unassigned shards Set batchesToAssign = createAndUpdateBatches(allocation, primary); if (batchesToAssign.isEmpty()) { - return Collections.emptyList(); + return null; } List runnables = new ArrayList<>(); if (primary) { @@ -287,6 +269,7 @@ public void run() { } })); + return new BatchRunnableExecutor(runnables, () -> primaryShardsBatchGatewayAllocatorTimeout); } else { batchIdToStoreShardBatch.values() .stream() @@ -311,8 +294,8 @@ public void run() { } })); + return new BatchRunnableExecutor(runnables, () -> replicaShardsBatchGatewayAllocatorTimeout); } - return runnables; } // visible for testing @@ -816,4 +799,12 @@ public int getNumberOfStartedShardBatches() { public int getNumberOfStoreShardBatches() { return batchIdToStoreShardBatch.size(); } + + private void setPrimaryBatchAllocatorTimeout(TimeValue primaryShardsBatchGatewayAllocatorTimeout) { + this.primaryShardsBatchGatewayAllocatorTimeout = primaryShardsBatchGatewayAllocatorTimeout; + } + + private void setReplicaBatchAllocatorTimeout(TimeValue replicaShardsBatchGatewayAllocatorTimeout) { + this.replicaShardsBatchGatewayAllocatorTimeout = replicaShardsBatchGatewayAllocatorTimeout; + } } 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 5f76c4f9b8189..0eb4bb6935bac 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 @@ -13,7 +13,7 @@ import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.opensearch.cluster.routing.allocation.RoutingAllocation; -import org.opensearch.common.util.concurrent.TimeoutAwareRunnable; +import org.opensearch.common.util.BatchRunnableExecutor; import org.opensearch.core.index.shard.ShardId; import org.opensearch.gateway.AsyncShardFetch; import org.opensearch.gateway.PrimaryShardBatchAllocator; @@ -29,7 +29,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.function.Consumer; public class TestShardBatchGatewayAllocator extends ShardsBatchGatewayAllocator { @@ -104,7 +103,7 @@ protected boolean hasInitiatedFetching(ShardRouting shard) { }; @Override - public List allocateAllUnassignedShards(RoutingAllocation allocation, boolean primary) { + public BatchRunnableExecutor allocateAllUnassignedShards(RoutingAllocation allocation, boolean primary) { currentNodes = allocation.nodes(); return innerAllocateUnassignedBatch(allocation, primaryBatchShardAllocator, replicaBatchShardAllocator, primary); } From d3d78cf53520dce1e925ad24eddcc517ce2fd7ff Mon Sep 17 00:00:00 2001 From: Bukhtawar Khan Date: Mon, 22 Jul 2024 19:44:29 +0530 Subject: [PATCH 05/35] Null checks Signed-off-by: Bukhtawar Khan --- .../org/opensearch/common/util/BatchRunnableExecutor.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java index dc610161da22f..6b32be5490563 100644 --- a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java +++ b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java @@ -12,6 +12,7 @@ import org.apache.logging.log4j.Logger; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.TimeoutAwareRunnable; +import org.opensearch.core.common.util.CollectionUtils; import java.util.Collections; import java.util.List; @@ -39,6 +40,9 @@ public void run() { logger.debug("Starting execution of runnable of size [{}]", timeoutAwareRunnables.size()); Collections.shuffle(timeoutAwareRunnables); long startTime = System.nanoTime(); + if (CollectionUtils.isEmpty(timeoutAwareRunnables)) { + return; + } for (TimeoutAwareRunnable workQueue : timeoutAwareRunnables) { if (System.nanoTime() - startTime > timeoutSupplier.get().nanos()) { workQueue.run(); From 1c5c2117038929f5a0160449d4697c3955beb7c4 Mon Sep 17 00:00:00 2001 From: Bukhtawar Khan Date: Mon, 22 Jul 2024 20:08:20 +0530 Subject: [PATCH 06/35] nitpick Signed-off-by: Bukhtawar Khan --- .../java/org/opensearch/common/util/BatchRunnableExecutor.java | 2 +- .../java/org/opensearch/gateway/PrimaryShardBatchAllocator.java | 2 +- .../java/org/opensearch/gateway/ReplicaShardBatchAllocator.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java index 6b32be5490563..4ae139ada7d2c 100644 --- a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java +++ b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java @@ -44,7 +44,7 @@ public void run() { return; } for (TimeoutAwareRunnable workQueue : timeoutAwareRunnables) { - if (System.nanoTime() - startTime > timeoutSupplier.get().nanos()) { + if (System.nanoTime() - startTime < timeoutSupplier.get().nanos()) { workQueue.run(); } else { logger.debug("Executing timeout for runnable of size [{}]", timeoutAwareRunnables.size()); diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java index 617902b7506a9..f416166069417 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java @@ -122,7 +122,7 @@ public void allocateUnassignedBatch(List shardRoutings, RoutingAll logger.trace("Finished shard allocation execution for unassigned primary shards: {}", shardRoutings.size()); } - public void allocateUnassignedBatchOnTimeout(List shardRoutings, RoutingAllocation allocation) { + protected void allocateUnassignedBatchOnTimeout(List shardRoutings, RoutingAllocation allocation) { Set batchShardRoutingSet = new HashSet<>(shardRoutings); RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); while (iterator.hasNext()) { diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java index 99171b17a99e9..a9da8f3a41161 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java @@ -168,7 +168,7 @@ public void allocateUnassignedBatch(List shardRoutings, RoutingAll logger.trace("Finished shard allocation execution for unassigned replica shards: {}", shardRoutings.size()); } - public void allocateUnassignedBatchOnTimeout(List shardRoutings, RoutingAllocation allocation) { + protected void allocateUnassignedBatchOnTimeout(List shardRoutings, RoutingAllocation allocation) { Set shardIdsFromBatch = new HashSet<>(); for (ShardRouting shardRouting : shardRoutings) { ShardId shardId = shardRouting.shardId(); From 3d8b9e49cb2c051aea67b96686fcfefce3db8a91 Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Mon, 22 Jul 2024 23:10:23 +0530 Subject: [PATCH 07/35] Fix NPE bug Signed-off-by: Rishab Nahata --- .../cluster/routing/allocation/AllocationService.java | 11 +++++++++-- .../common/util/BatchRunnableExecutorTest.java | 2 ++ 2 files changed, 11 insertions(+), 2 deletions(-) create mode 100644 server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java index 48b0e957a826f..d75b5d413ebee 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java @@ -58,6 +58,7 @@ import org.opensearch.cluster.routing.allocation.decider.Decision; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.BatchRunnableExecutor; import org.opensearch.gateway.GatewayAllocator; import org.opensearch.gateway.PriorityComparator; import org.opensearch.gateway.ShardsBatchGatewayAllocator; @@ -624,10 +625,16 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { private void allocateAllUnassignedShards(RoutingAllocation allocation) { ExistingShardsAllocator allocator = existingShardsAllocators.get(ShardsBatchGatewayAllocator.ALLOCATOR_NAME); - allocator.allocateAllUnassignedShards(allocation, true).run(); + BatchRunnableExecutor primaryShardsBatchExecutor = allocator.allocateAllUnassignedShards(allocation, true); + if (primaryShardsBatchExecutor != null) { + primaryShardsBatchExecutor.run(); + } allocator.afterPrimariesBeforeReplicas(allocation); // Replicas Assignment - allocator.allocateAllUnassignedShards(allocation, false).run(); + BatchRunnableExecutor replicaShardsBatchExecutor = allocator.allocateAllUnassignedShards(allocation, false); + if (replicaShardsBatchExecutor != null) { + replicaShardsBatchExecutor.run(); + } } private void disassociateDeadNodes(RoutingAllocation allocation) { diff --git a/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java b/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java new file mode 100644 index 0000000000000..4c9c5def673f6 --- /dev/null +++ b/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java @@ -0,0 +1,2 @@ +package org.opensearch.common.util;public class BatchRunnableExecutorTest { +} From cf9c9ddeec14fb6289e778a19cbc9253c989f800 Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Mon, 22 Jul 2024 23:19:56 +0530 Subject: [PATCH 08/35] Remove timeout from decide allocate unassigned Signed-off-by: Rishab Nahata --- .../allocator/BalancedShardsAllocator.java | 25 +++---------------- .../allocator/LocalShardsBalancer.java | 18 +++---------- .../allocator/RemoteShardsBalancer.java | 2 +- .../allocation/allocator/ShardsBalancer.java | 2 +- .../common/settings/ClusterSettings.java | 1 - 5 files changed, 9 insertions(+), 39 deletions(-) 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 47fe51013b38f..1fe897adeda19 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 @@ -54,7 +54,6 @@ 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 java.util.HashMap; import java.util.HashSet; @@ -163,13 +162,6 @@ public class BalancedShardsAllocator implements ShardsAllocator { Property.NodeScope ); - public static final Setting ALLOCATE_UNASSIGNED_TIMEOUT_SETTING = Setting.timeSetting( - "cluster.routing.allocation.allocate_unassigned_timeout", - TimeValue.MINUS_ONE, - Setting.Property.NodeScope, - Setting.Property.Dynamic - ); - private volatile boolean movePrimaryFirst; private volatile ShardMovementStrategy shardMovementStrategy; @@ -180,7 +172,6 @@ public class BalancedShardsAllocator implements ShardsAllocator { private volatile float shardBalanceFactor; private volatile WeightFunction weightFunction; private volatile float threshold; - private volatile TimeValue allocateUnassignedTimeout; public BalancedShardsAllocator(Settings settings) { this(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); @@ -196,7 +187,6 @@ public BalancedShardsAllocator(Settings settings, ClusterSettings clusterSetting setPreferPrimaryShardBalance(PREFER_PRIMARY_SHARD_BALANCE.get(settings)); setPreferPrimaryShardRebalance(PREFER_PRIMARY_SHARD_REBALANCE.get(settings)); setShardMovementStrategy(SHARD_MOVEMENT_STRATEGY_SETTING.get(settings)); - setAllocateUnassignedTimeout(ALLOCATE_UNASSIGNED_TIMEOUT_SETTING.get(settings)); clusterSettings.addSettingsUpdateConsumer(PREFER_PRIMARY_SHARD_BALANCE, this::setPreferPrimaryShardBalance); clusterSettings.addSettingsUpdateConsumer(SHARD_MOVE_PRIMARY_FIRST_SETTING, this::setMovePrimaryFirst); clusterSettings.addSettingsUpdateConsumer(SHARD_MOVEMENT_STRATEGY_SETTING, this::setShardMovementStrategy); @@ -205,7 +195,6 @@ public BalancedShardsAllocator(Settings settings, ClusterSettings clusterSetting clusterSettings.addSettingsUpdateConsumer(PRIMARY_SHARD_REBALANCE_BUFFER, this::updatePreferPrimaryShardBalanceBuffer); clusterSettings.addSettingsUpdateConsumer(PREFER_PRIMARY_SHARD_REBALANCE, this::setPreferPrimaryShardRebalance); clusterSettings.addSettingsUpdateConsumer(THRESHOLD_SETTING, this::setThreshold); - clusterSettings.addSettingsUpdateConsumer(ALLOCATE_UNASSIGNED_TIMEOUT_SETTING, this::setAllocateUnassignedTimeout); } /** @@ -280,10 +269,6 @@ private void setThreshold(float threshold) { this.threshold = threshold; } - private void setAllocateUnassignedTimeout(TimeValue allocateUnassignedTimeout) { - this.allocateUnassignedTimeout = allocateUnassignedTimeout; - } - @Override public void allocate(RoutingAllocation allocation) { if (allocation.routingNodes().size() == 0) { @@ -297,8 +282,7 @@ public void allocate(RoutingAllocation allocation) { weightFunction, threshold, preferPrimaryShardBalance, - preferPrimaryShardRebalance, - allocateUnassignedTimeout + preferPrimaryShardRebalance ); localShardsBalancer.allocateUnassigned(); localShardsBalancer.moveShards(); @@ -321,13 +305,12 @@ public ShardAllocationDecision decideShardAllocation(final ShardRouting shard, f weightFunction, threshold, preferPrimaryShardBalance, - preferPrimaryShardRebalance, - allocateUnassignedTimeout + preferPrimaryShardRebalance ); AllocateUnassignedDecision allocateUnassignedDecision = AllocateUnassignedDecision.NOT_TAKEN; MoveDecision moveDecision = MoveDecision.NOT_TAKEN; if (shard.unassigned()) { - allocateUnassignedDecision = localShardsBalancer.decideAllocateUnassigned(shard, startTime); + allocateUnassignedDecision = localShardsBalancer.decideAllocateUnassigned(shard); } else { moveDecision = localShardsBalancer.decideMove(shard); if (moveDecision.isDecisionTaken() && moveDecision.canRemain()) { @@ -576,7 +559,7 @@ public Balancer( float threshold, boolean preferPrimaryBalance ) { - super(logger, allocation, shardMovementStrategy, weight, threshold, preferPrimaryBalance, false, TimeValue.MINUS_ONE); + super(logger, allocation, shardMovementStrategy, weight, threshold, preferPrimaryBalance, false); } } diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java index d2336608a9c23..5c097a825fc16 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java @@ -29,7 +29,6 @@ import org.opensearch.cluster.routing.allocation.decider.Decision; import org.opensearch.cluster.routing.allocation.decider.DiskThresholdDecider; import org.opensearch.common.collect.Tuple; -import org.opensearch.common.unit.TimeValue; import org.opensearch.gateway.PriorityComparator; import java.util.ArrayList; @@ -62,7 +61,6 @@ public class LocalShardsBalancer extends ShardsBalancer { private final boolean preferPrimaryBalance; private final boolean preferPrimaryRebalance; - private final TimeValue allocateUnassignedTimeout; private final BalancedShardsAllocator.WeightFunction weight; private final float threshold; @@ -80,8 +78,7 @@ public LocalShardsBalancer( BalancedShardsAllocator.WeightFunction weight, float threshold, boolean preferPrimaryBalance, - boolean preferPrimaryRebalance, - TimeValue allocateUnassignedTimeout + boolean preferPrimaryRebalance ) { this.logger = logger; this.allocation = allocation; @@ -98,7 +95,6 @@ public LocalShardsBalancer( this.preferPrimaryBalance = preferPrimaryBalance; this.preferPrimaryRebalance = preferPrimaryRebalance; this.shardMovementStrategy = shardMovementStrategy; - this.allocateUnassignedTimeout = allocateUnassignedTimeout; } /** @@ -747,7 +743,6 @@ private Map buildModelFromAssigned() */ @Override void allocateUnassigned() { - long startTime = System.nanoTime(); RoutingNodes.UnassignedShards unassigned = routingNodes.unassigned(); assert !nodes.isEmpty(); if (logger.isTraceEnabled()) { @@ -803,7 +798,7 @@ void allocateUnassigned() { do { for (int i = 0; i < primaryLength; i++) { ShardRouting shard = primary[i]; - final AllocateUnassignedDecision allocationDecision = decideAllocateUnassigned(shard, startTime); + final AllocateUnassignedDecision allocationDecision = decideAllocateUnassigned(shard); final String assignedNodeId = allocationDecision.getTargetNode() != null ? allocationDecision.getTargetNode().getId() : null; @@ -876,8 +871,6 @@ void allocateUnassigned() { secondaryLength = 0; } while (primaryLength > 0); // clear everything we have either added it or moved to ignoreUnassigned - logger.debug("Time taken in allocate unassigned [{}]", - TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); } /** @@ -887,17 +880,12 @@ void allocateUnassigned() { * is of type {@link Decision.Type#NO}, then the assigned node will be null. */ @Override - AllocateUnassignedDecision decideAllocateUnassigned(final ShardRouting shard, long startTime) { + AllocateUnassignedDecision decideAllocateUnassigned(final ShardRouting shard) { if (shard.assignedToNode()) { // we only make decisions for unassigned shards here return AllocateUnassignedDecision.NOT_TAKEN; } - if (System.nanoTime() - startTime > allocateUnassignedTimeout.nanos()) { - logger.info("Timed out while running Local shard balancer allocate unassigned - outer loop"); - return AllocateUnassignedDecision.throttle(null); - } - final boolean explain = allocation.debugDecision(); Decision shardLevelDecision = allocation.deciders().canAllocate(shard, allocation); if (shardLevelDecision.type() == Decision.Type.NO && explain == false) { 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 0fc48092c95a2..a05938c176678 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 @@ -288,7 +288,7 @@ private Map calculateNodePrimaryShardCount(List re } @Override - AllocateUnassignedDecision decideAllocateUnassigned(ShardRouting shardRouting, long startTime) { + AllocateUnassignedDecision decideAllocateUnassigned(ShardRouting shardRouting) { throw new UnsupportedOperationException("remote shards balancer does not support decision operations"); } diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/ShardsBalancer.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/ShardsBalancer.java index f6284797a303d..ef2dbd34644a7 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/ShardsBalancer.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/ShardsBalancer.java @@ -42,7 +42,7 @@ public abstract class ShardsBalancer { * @param shardRouting the shard for which the decision has to be made * @return the allocation decision */ - abstract AllocateUnassignedDecision decideAllocateUnassigned(ShardRouting shardRouting, long startTime); + abstract AllocateUnassignedDecision decideAllocateUnassigned(ShardRouting shardRouting); /** * Makes a decision on whether to move a started shard to another node. diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index d8af832d0d9b2..79342fd365025 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -267,7 +267,6 @@ public void apply(Settings value, Settings current, Settings previous) { BalancedShardsAllocator.SHARD_MOVE_PRIMARY_FIRST_SETTING, BalancedShardsAllocator.SHARD_MOVEMENT_STRATEGY_SETTING, BalancedShardsAllocator.THRESHOLD_SETTING, - BalancedShardsAllocator.ALLOCATE_UNASSIGNED_TIMEOUT_SETTING, BreakerSettings.CIRCUIT_BREAKER_LIMIT_SETTING, BreakerSettings.CIRCUIT_BREAKER_OVERHEAD_SETTING, BreakerSettings.CIRCUIT_BREAKER_TYPE, From 25588fa8e5e9a0f8baccf4de48a742f7a0032e73 Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Mon, 22 Jul 2024 23:45:46 +0530 Subject: [PATCH 09/35] Add UT for BatchRunnableExecutor and BatchAllocators Signed-off-by: Rishab Nahata --- .../util/BatchRunnableExecutorTest.java | 99 ++++++++++++++++++- .../PrimaryShardBatchAllocatorTests.java | 50 ++++++++-- .../ReplicaShardBatchAllocatorTests.java | 24 +++++ 3 files changed, 164 insertions(+), 9 deletions(-) diff --git a/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java b/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java index 4c9c5def673f6..779b9d4155ebd 100644 --- a/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java +++ b/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java @@ -1,2 +1,99 @@ -package org.opensearch.common.util;public class BatchRunnableExecutorTest { +/* + * 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.common.util; + +import org.junit.Before; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.concurrent.TimeoutAwareRunnable; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.function.Supplier; + +import static org.mockito.Mockito.*; + +public class BatchRunnableExecutorTest extends OpenSearchTestCase { + private Supplier timeoutSupplier; + private TimeoutAwareRunnable runnable1; + private TimeoutAwareRunnable runnable2; + private TimeoutAwareRunnable runnable3; + private List runnableList; + + public void testRunWithoutTimeout() { + timeoutSupplier = mock(Supplier.class); + runnable1 = mock(TimeoutAwareRunnable.class); + runnable2 = mock(TimeoutAwareRunnable.class); + runnable3 = mock(TimeoutAwareRunnable.class); + runnableList = Arrays.asList(runnable1, runnable2, runnable3); + when(timeoutSupplier.get()).thenReturn(TimeValue.timeValueSeconds(1)); + BatchRunnableExecutor executor = new BatchRunnableExecutor(runnableList, timeoutSupplier); + executor.run(); + verify(runnable1, times(1)).run(); + verify(runnable2, times(1)).run(); + verify(runnable3, times(1)).run(); + verify(runnable1, never()).onTimeout(); + verify(runnable2, never()).onTimeout(); + verify(runnable3, never()).onTimeout(); + } + + public void testRunWithTimeout() { + timeoutSupplier = mock(Supplier.class); + runnable1 = mock(TimeoutAwareRunnable.class); + runnable2 = mock(TimeoutAwareRunnable.class); + runnable3 = mock(TimeoutAwareRunnable.class); + runnableList = Arrays.asList(runnable1, runnable2, runnable3); + when(timeoutSupplier.get()).thenReturn(TimeValue.timeValueNanos(1)); + BatchRunnableExecutor executor = new BatchRunnableExecutor(runnableList, timeoutSupplier); + executor.run(); + verify(runnable1, times(1)).onTimeout(); + verify(runnable2, times(1)).onTimeout(); + verify(runnable3, times(1)).onTimeout(); + verify(runnable1, never()).run(); + verify(runnable2, never()).run(); + verify(runnable3, never()).run(); + } + +// public void testRunWithPartialTimeout() { +// timeoutSupplier = mock(Supplier.class); +// runnable1 = mock(TimeoutAwareRunnable.class); +// runnable2 = mock(TimeoutAwareRunnable.class); +// runnable3 = mock(TimeoutAwareRunnable.class); +// runnableList = Arrays.asList(runnable1, runnable2, runnable3); +// when(timeoutSupplier.get()).thenReturn(TimeValue.timeValueMillis(100)); +// BatchRunnableExecutor executor = new BatchRunnableExecutor(runnableList, timeoutSupplier); +// doAnswer(invocationOnMock -> { +// Thread.sleep(1000); +// return null; +// }).when(runnable1).run(); +// executor.run(); +// verify(runnable1, times(1)).run(); +// verify(runnable2, never()).run(); +// verify(runnable3, never()).run(); +// verify(runnable2, times(1)).onTimeout(); +// verify(runnable3, times(1)).onTimeout(); +// } + + public void testRunWithEmptyRunnableList() { + timeoutSupplier = mock(Supplier.class); + runnable1 = mock(TimeoutAwareRunnable.class); + runnable2 = mock(TimeoutAwareRunnable.class); + runnable3 = mock(TimeoutAwareRunnable.class); + runnableList = Arrays.asList(runnable1, runnable2, runnable3); + BatchRunnableExecutor executor = new BatchRunnableExecutor(Collections.emptyList(), timeoutSupplier); + executor.run(); + verify(runnable1, never()).onTimeout(); + verify(runnable2, never()).onTimeout(); + verify(runnable3, never()).onTimeout(); + verify(runnable1, never()).run(); + verify(runnable2, never()).run(); + verify(runnable3, never()).run(); + } } diff --git a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java index 8ad8bcda95f40..1e5ad732940ce 100644 --- a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java @@ -40,14 +40,7 @@ import org.opensearch.test.IndexSettingsModule; import org.junit.Before; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; +import java.util.*; import java.util.stream.Collectors; import static org.opensearch.cluster.routing.UnassignedInfo.Reason.CLUSTER_RECOVERED; @@ -256,6 +249,47 @@ public void testAllocateUnassignedBatchThrottlingAllocationDeciderIsHonoured() { assertEquals(UnassignedInfo.AllocationStatus.DECIDERS_THROTTLED, ignoredShards.get(0).unassignedInfo().getLastAllocationStatus()); } + public void testAllocateUnassignedBatchOnTimeoutWithMatchingPrimaryShards() { + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + AllocationDeciders allocationDeciders = randomAllocationDeciders(Settings.builder().build(), clusterSettings, random()); + setUpShards(1); + final RoutingAllocation routingAllocation = routingAllocationWithOnePrimary(allocationDeciders, CLUSTER_RECOVERED, "allocId-0"); + ShardRouting shardRouting = routingAllocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).primaryShard(); + + List shardRoutings = Arrays.asList(shardRouting); + batchAllocator.allocateUnassignedBatchOnTimeout(shardRoutings, routingAllocation); + + List ignoredShards = routingAllocation.routingNodes().unassigned().ignored(); + assertEquals(1, ignoredShards.size()); + assertEquals(UnassignedInfo.AllocationStatus.DECIDERS_THROTTLED, ignoredShards.get(0).unassignedInfo().getLastAllocationStatus()); + } + + public void testAllocateUnassignedBatchOnTimeoutWithNoMatchingPrimaryShards() { + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + AllocationDeciders allocationDeciders = randomAllocationDeciders(Settings.builder().build(), clusterSettings, random()); + setUpShards(1); + final RoutingAllocation routingAllocation = routingAllocationWithOnePrimary(allocationDeciders, CLUSTER_RECOVERED, "allocId-0"); + List shardRoutings = new ArrayList<>(); + batchAllocator.allocateUnassignedBatchOnTimeout(shardRoutings, routingAllocation); + + List ignoredShards = routingAllocation.routingNodes().unassigned().ignored(); + assertEquals(0, ignoredShards.size()); + } + + public void testAllocateUnassignedBatchOnTimeoutWithNonPrimaryShards() { + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + AllocationDeciders allocationDeciders = randomAllocationDeciders(Settings.builder().build(), clusterSettings, random()); + setUpShards(1); + final RoutingAllocation routingAllocation = routingAllocationWithOnePrimary(allocationDeciders, CLUSTER_RECOVERED, "allocId-0"); + + ShardRouting shardRouting = routingAllocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).replicaShards().get(0); + List shardRoutings = Arrays.asList(shardRouting); + batchAllocator.allocateUnassignedBatchOnTimeout(shardRoutings, routingAllocation); + + List ignoredShards = routingAllocation.routingNodes().unassigned().ignored(); + assertEquals(0, ignoredShards.size()); + } + private RoutingAllocation routingAllocationWithOnePrimary( AllocationDeciders deciders, UnassignedInfo.Reason reason, diff --git a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java index 2e148c2bc8130..a648ed29b5698 100644 --- a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java @@ -698,6 +698,30 @@ public void testAllocateUnassignedBatchThrottlingAllocationDeciderIsHonoured() t assertEquals(UnassignedInfo.AllocationStatus.DECIDERS_THROTTLED, allocateUnassignedDecision.getAllocationStatus()); } + public void testAllocateUnassignedBatchOnTimeoutWithUnassignedReplicaShard() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); + final RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); + List shards = new ArrayList<>(); + while (iterator.hasNext()) { + shards.add(iterator.next()); + } + testBatchAllocator.allocateUnassignedBatchOnTimeout(shards, allocation); + assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); + assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); + assertEquals(UnassignedInfo.AllocationStatus.NO_ATTEMPT, allocation.routingNodes().unassigned().ignored().get(0).unassignedInfo().getLastAllocationStatus()); + } + + public void testAllocateUnassignedBatchOnTimeoutWithAlreadyRecoveringReplicaShard() { + RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders()); + final RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); + List shards = new ArrayList<>(); + while (iterator.hasNext()) { + shards.add(iterator.next()); + } + testBatchAllocator.allocateUnassignedBatchOnTimeout(shards, allocation); + assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(0)); + } + private RoutingAllocation onePrimaryOnNode1And1Replica(AllocationDeciders deciders) { return onePrimaryOnNode1And1Replica(deciders, Settings.EMPTY, UnassignedInfo.Reason.CLUSTER_RECOVERED); } From d5ede503fe5df509c99ae12e52a6c4633eab492e Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Tue, 23 Jul 2024 04:09:35 +0530 Subject: [PATCH 10/35] Add cluster green IT Signed-off-by: Rishab Nahata --- .../gateway/RecoveryFromGatewayIT.java | 128 +++++++++++++++++- .../common/util/BatchRunnableExecutor.java | 2 +- .../gateway/ShardsBatchGatewayAllocator.java | 4 +- 3 files changed, 130 insertions(+), 4 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 6296608c64d37..4085cc3890f30 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -769,7 +769,7 @@ public void testMessyElectionsStillMakeClusterGoGreen() throws Exception { ensureGreen("test"); } - public void testBatchModeEnabled() throws Exception { + public void testBatchModeEnabledWithoutTimeout() throws Exception { internalCluster().startClusterManagerOnlyNodes( 1, Settings.builder().put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.getKey(), true).build() @@ -810,6 +810,132 @@ public void testBatchModeEnabled() throws Exception { assertEquals(0, gatewayAllocator.getNumberOfInFlightFetches()); } + public void testBatchModeEnabledWithSufficientTimeoutAndClusterGreen() throws Exception { + internalCluster().startClusterManagerOnlyNodes( + 1, + Settings.builder() + .put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.getKey(), true) + .put(ShardsBatchGatewayAllocator.PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING.getKey(), "20s") + .put(ShardsBatchGatewayAllocator.REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING.getKey(), "20s") + .build() + ); + List dataOnlyNodes = internalCluster().startDataOnlyNodes(2); + createIndex( + "test", + Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).build() + ); + ensureGreen("test"); + Settings node0DataPathSettings = internalCluster().dataPathSettings(dataOnlyNodes.get(0)); + Settings node1DataPathSettings = internalCluster().dataPathSettings(dataOnlyNodes.get(1)); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(dataOnlyNodes.get(0))); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(dataOnlyNodes.get(1))); + ensureRed("test"); + ensureStableCluster(1); + + logger.info("--> Now do a protective reroute"); + ClusterRerouteResponse clusterRerouteResponse = client().admin().cluster().prepareReroute().setRetryFailed(true).get(); + assertTrue(clusterRerouteResponse.isAcknowledged()); + + ShardsBatchGatewayAllocator gatewayAllocator = internalCluster().getInstance( + ShardsBatchGatewayAllocator.class, + internalCluster().getClusterManagerName() + ); + assertTrue(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.get(internalCluster().clusterService().getSettings())); + assertEquals(1, gatewayAllocator.getNumberOfStartedShardBatches()); + assertEquals(1, gatewayAllocator.getNumberOfStoreShardBatches()); + + // Now start both data nodes and ensure batch mode is working + logger.info("--> restarting the stopped nodes"); + internalCluster().startDataOnlyNode(Settings.builder().put("node.name", dataOnlyNodes.get(0)).put(node0DataPathSettings).build()); + internalCluster().startDataOnlyNode(Settings.builder().put("node.name", dataOnlyNodes.get(1)).put(node1DataPathSettings).build()); + ensureStableCluster(3); + ensureGreen("test"); + assertEquals(0, gatewayAllocator.getNumberOfStartedShardBatches()); + assertEquals(0, gatewayAllocator.getNumberOfStoreShardBatches()); + assertEquals(0, gatewayAllocator.getNumberOfInFlightFetches()); + } + + public void testBatchModeEnabledWithInSufficientTimeoutButClusterGreen() throws Exception { + + internalCluster().startClusterManagerOnlyNodes( + 1, + Settings.builder().put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.getKey(), true).build() + ); + List dataOnlyNodes = internalCluster().startDataOnlyNodes(2); + createNIndices(50, "test"); // this will create 50p, 50r shards + ensureStableCluster(3); + IndicesStatsResponse indicesStats = dataNodeClient().admin().indices().prepareStats().get(); + assertThat(indicesStats.getSuccessfulShards(), equalTo(100)); + ClusterHealthResponse health = client().admin() + .cluster() + .health(Requests.clusterHealthRequest().waitForGreenStatus().timeout("1m")) + .actionGet(); + assertFalse(health.isTimedOut()); + assertEquals(GREEN, health.getStatus()); + + String clusterManagerName = internalCluster().getClusterManagerName(); + Settings clusterManagerDataPathSettings = internalCluster().dataPathSettings(clusterManagerName); + Settings node0DataPathSettings = internalCluster().dataPathSettings(dataOnlyNodes.get(0)); + Settings node1DataPathSettings = internalCluster().dataPathSettings(dataOnlyNodes.get(1)); + + internalCluster().stopCurrentClusterManagerNode(); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(dataOnlyNodes.get(0))); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(dataOnlyNodes.get(1))); + + // Now start cluster manager node and post that verify batches created + internalCluster().startClusterManagerOnlyNodes( + 1, + Settings.builder() + .put("node.name", clusterManagerName) + .put(clusterManagerDataPathSettings) + .put(ShardsBatchGatewayAllocator.GATEWAY_ALLOCATOR_BATCH_SIZE.getKey(), 5) + .put(ShardsBatchGatewayAllocator.PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING.getKey(), "10ms") + .put(ShardsBatchGatewayAllocator.REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING.getKey(), "10ms") + .put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.getKey(), true) + .build() + ); + ensureStableCluster(1); + + logger.info("--> Now do a protective reroute"); // to avoid any race condition in test + ClusterRerouteResponse clusterRerouteResponse = client().admin().cluster().prepareReroute().setRetryFailed(true).get(); + assertTrue(clusterRerouteResponse.isAcknowledged()); + + ShardsBatchGatewayAllocator gatewayAllocator = internalCluster().getInstance( + ShardsBatchGatewayAllocator.class, + internalCluster().getClusterManagerName() + ); + + assertTrue(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.get(internalCluster().clusterService().getSettings())); + assertEquals(10, gatewayAllocator.getNumberOfStartedShardBatches()); + assertEquals(10, gatewayAllocator.getNumberOfStoreShardBatches()); + health = client(internalCluster().getClusterManagerName()).admin().cluster().health(Requests.clusterHealthRequest()).actionGet(); + assertFalse(health.isTimedOut()); + assertEquals(RED, health.getStatus()); + assertEquals(100, health.getUnassignedShards()); + assertEquals(0, health.getInitializingShards()); + assertEquals(0, health.getActiveShards()); + assertEquals(0, health.getRelocatingShards()); + assertEquals(0, health.getNumberOfDataNodes()); + + // Now start both data nodes and ensure batch mode is working + logger.info("--> restarting the stopped nodes"); + internalCluster().startDataOnlyNode(Settings.builder().put("node.name", dataOnlyNodes.get(0)).put(node0DataPathSettings).build()); + internalCluster().startDataOnlyNode(Settings.builder().put("node.name", dataOnlyNodes.get(1)).put(node1DataPathSettings).build()); + ensureStableCluster(3); + + // wait for cluster to turn green + health = client().admin().cluster().health(Requests.clusterHealthRequest().waitForGreenStatus().timeout("5m")).actionGet(); + assertFalse(health.isTimedOut()); + assertEquals(GREEN, health.getStatus()); + assertEquals(0, health.getUnassignedShards()); + assertEquals(0, health.getInitializingShards()); + assertEquals(100, health.getActiveShards()); + assertEquals(0, health.getRelocatingShards()); + assertEquals(2, health.getNumberOfDataNodes()); + assertEquals(0, gatewayAllocator.getNumberOfStartedShardBatches()); + assertEquals(0, gatewayAllocator.getNumberOfStoreShardBatches()); + } + public void testBatchModeDisabled() throws Exception { internalCluster().startClusterManagerOnlyNodes( 1, diff --git a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java index 4ae139ada7d2c..58edaa4882ea4 100644 --- a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java +++ b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java @@ -44,7 +44,7 @@ public void run() { return; } for (TimeoutAwareRunnable workQueue : timeoutAwareRunnables) { - if (System.nanoTime() - startTime < timeoutSupplier.get().nanos()) { + if (timeoutSupplier.get().nanos() < 0 || System.nanoTime() - startTime < timeoutSupplier.get().nanos()) { workQueue.run(); } else { logger.debug("Executing timeout for runnable of size [{}]", timeoutAwareRunnables.size()); diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 36a4049bac8fd..68c06264e99ed 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -93,14 +93,14 @@ public class ShardsBatchGatewayAllocator implements ExistingShardsAllocator { public static final Setting PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING = Setting.timeSetting( PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING_KEY, - TimeValue.timeValueSeconds(20), + TimeValue.MINUS_ONE, Setting.Property.NodeScope, Setting.Property.Dynamic ); public static final Setting REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING = Setting.timeSetting( REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING_KEY, - TimeValue.timeValueSeconds(20), + TimeValue.MINUS_ONE, Setting.Property.NodeScope, Setting.Property.Dynamic ); From b9a90271c77c3142857aa632b736969ad4153134 Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Tue, 23 Jul 2024 04:19:30 +0530 Subject: [PATCH 11/35] Minor fix and spotless apply Signed-off-by: Rishab Nahata --- .../routing/allocation/AllocationService.java | 9 +- .../allocation/ExistingShardsAllocator.java | 2 +- .../allocator/BalancedShardsAllocator.java | 1 - .../allocator/LocalShardsBalancer.java | 1 - .../common/util/BatchRunnableExecutor.java | 6 +- .../gateway/PrimaryShardBatchAllocator.java | 10 +- .../gateway/ReplicaShardBatchAllocator.java | 10 +- .../gateway/ShardsBatchGatewayAllocator.java | 108 ++++++++++-------- .../util/BatchRunnableExecutorTest.java | 45 ++++---- .../PrimaryShardBatchAllocatorTests.java | 17 ++- .../ReplicaShardBatchAllocatorTests.java | 5 +- 11 files changed, 130 insertions(+), 84 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java index d75b5d413ebee..3ac792491b622 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java @@ -569,12 +569,13 @@ private void reroute(RoutingAllocation allocation) { long startTime = System.nanoTime(); allocateExistingUnassignedShards(allocation); // try to allocate existing shard copies first - logger.info("Completing allocate unassigned, elapsed time: [{}]", - TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); + logger.info("Completing allocate unassigned, elapsed time: [{}]", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); startTime = System.nanoTime(); shardsAllocator.allocate(allocation); - logger.info("Shard allocator to allocate all shards, elapsed time: [{}]", - TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); + logger.info( + "Shard allocator to allocate all shards, elapsed time: [{}]", + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime) + ); clusterManagerMetrics.recordLatency( clusterManagerMetrics.rerouteHistogram, (double) Math.max(0, TimeValue.nsecToMSec(System.nanoTime() - rerouteStartTimeNS)) diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java index 57098dab89b95..115369bd5ab05 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java @@ -121,7 +121,7 @@ default BatchRunnableExecutor allocateAllUnassignedShards(RoutingAllocation allo runnables.add(new TimeoutAwareRunnable() { @Override public void onTimeout() { - //do nothing + // do nothing } @Override 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 1fe897adeda19..b2443490dd973 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 @@ -297,7 +297,6 @@ public void allocate(RoutingAllocation allocation) { @Override public ShardAllocationDecision decideShardAllocation(final ShardRouting shard, final RoutingAllocation allocation) { - long startTime = System.nanoTime(); ShardsBalancer localShardsBalancer = new LocalShardsBalancer( logger, allocation, diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java index 5c097a825fc16..00eb79add9f1d 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java @@ -40,7 +40,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.TimeUnit; import java.util.stream.Stream; import java.util.stream.StreamSupport; diff --git a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java index 58edaa4882ea4..014f13091b6b8 100644 --- a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java +++ b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java @@ -51,8 +51,10 @@ public void run() { workQueue.onTimeout(); } } - logger.debug("Time taken to execute timed runnables in this cycle:[{}ms]", - TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); + logger.debug( + "Time taken to execute timed runnables in this cycle:[{}ms]", + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime) + ); } } diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java index f416166069417..66c81df7f3853 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java @@ -129,7 +129,15 @@ protected void allocateUnassignedBatchOnTimeout(List shardRoutings ShardRouting unassignedShard = iterator.next(); AllocateUnassignedDecision allocationDecision; if (unassignedShard.primary() && batchShardRoutingSet.contains(unassignedShard)) { - allocationDecision = new AllocateUnassignedDecision(UnassignedInfo.AllocationStatus.DECIDERS_THROTTLED, null, null, null, false, 0L, 0L); + allocationDecision = new AllocateUnassignedDecision( + UnassignedInfo.AllocationStatus.DECIDERS_THROTTLED, + null, + null, + null, + false, + 0L, + 0L + ); executeDecision(unassignedShard, allocationDecision, allocation, iterator); } } diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java index a9da8f3a41161..83fe7e3b10ee1 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java @@ -179,7 +179,15 @@ protected void allocateUnassignedBatchOnTimeout(List shardRoutings ShardRouting unassignedShard = iterator.next(); AllocateUnassignedDecision allocationDecision; if (!unassignedShard.primary() && shardIdsFromBatch.contains(unassignedShard.shardId())) { - allocationDecision = new AllocateUnassignedDecision(UnassignedInfo.AllocationStatus.DECIDERS_THROTTLED, null, null, null, false, 0L, 0L); + allocationDecision = new AllocateUnassignedDecision( + UnassignedInfo.AllocationStatus.DECIDERS_THROTTLED, + null, + null, + null, + false, + 0L, + 0L + ); executeDecision(unassignedShard, allocationDecision, allocation, iterator); } } diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 68c06264e99ed..91bd2d3f3e164 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -74,8 +74,10 @@ public class ShardsBatchGatewayAllocator implements ExistingShardsAllocator { private final long maxBatchSize; private static final short DEFAULT_SHARD_BATCH_SIZE = 2000; - private static final String PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING_KEY = "cluster.routing.allocation.shards_batch_gateway_allocator.primary_allocator_timeout"; - private static final String REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING_KEY = "cluster.routing.allocation.shards_batch_gateway_allocator.replica_allocator_timeout"; + private static final String PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING_KEY = + "cluster.routing.allocation.shards_batch_gateway_allocator.primary_allocator_timeout"; + private static final String REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING_KEY = + "cluster.routing.allocation.shards_batch_gateway_allocator.replica_allocator_timeout"; private TimeValue primaryShardsBatchGatewayAllocatorTimeout; private TimeValue replicaShardsBatchGatewayAllocatorTimeout; @@ -133,15 +135,9 @@ public ShardsBatchGatewayAllocator( this.batchStoreAction = batchStoreAction; this.maxBatchSize = GATEWAY_ALLOCATOR_BATCH_SIZE.get(settings); this.primaryShardsBatchGatewayAllocatorTimeout = PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING.get(settings); - clusterSettings.addSettingsUpdateConsumer( - PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING, - this::setPrimaryBatchAllocatorTimeout - ); + clusterSettings.addSettingsUpdateConsumer(PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING, this::setPrimaryBatchAllocatorTimeout); this.replicaShardsBatchGatewayAllocatorTimeout = REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING.get(settings); - clusterSettings.addSettingsUpdateConsumer( - REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING, - this::setReplicaBatchAllocatorTimeout - ); + clusterSettings.addSettingsUpdateConsumer(REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING, this::setReplicaBatchAllocatorTimeout); } @Override @@ -250,50 +246,64 @@ protected BatchRunnableExecutor innerAllocateUnassignedBatch( batchIdToStartedShardBatch.values() .stream() .filter(batch -> batchesToAssign.contains(batch.batchId)) - .forEach( - shardsBatch -> runnables.add(new TimeoutAwareRunnable() { - @Override - public void onTimeout() { - long startTime = System.nanoTime(); - primaryBatchShardAllocator.allocateUnassignedBatchOnTimeout(shardsBatch.getBatchedShardRoutings(), allocation); - logger.info("Time taken to execute allocateUnassignedBatchOnTimeout for unassigned primary batch with id [{}], size : [{}] in this cycle:[{}ms]", - shardsBatch.batchId, shardsBatch.getBatchedShardRoutings().size(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); - } - - @Override - public void run() { - long startTime = System.nanoTime(); - primaryBatchShardAllocator.allocateUnassignedBatch(shardsBatch.getBatchedShardRoutings(), allocation); - logger.info("Time taken to allocate unassigned primary batch with id [{}], size : [{}] in this cycle:[{}ms]", - shardsBatch.batchId, shardsBatch.getBatchedShardRoutings().size(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); - - } - })); + .forEach(shardsBatch -> runnables.add(new TimeoutAwareRunnable() { + @Override + public void onTimeout() { + long startTime = System.nanoTime(); + primaryBatchShardAllocator.allocateUnassignedBatchOnTimeout(shardsBatch.getBatchedShardRoutings(), allocation); + logger.info( + "Time taken to execute allocateUnassignedBatchOnTimeout for unassigned primary batch with id [{}], size : [{}] in this cycle:[{}ms]", + shardsBatch.batchId, + shardsBatch.getBatchedShardRoutings().size(), + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime) + ); + } + + @Override + public void run() { + long startTime = System.nanoTime(); + primaryBatchShardAllocator.allocateUnassignedBatch(shardsBatch.getBatchedShardRoutings(), allocation); + logger.info( + "Time taken to allocate unassigned primary batch with id [{}], size : [{}] in this cycle:[{}ms]", + shardsBatch.batchId, + shardsBatch.getBatchedShardRoutings().size(), + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime) + ); + + } + })); return new BatchRunnableExecutor(runnables, () -> primaryShardsBatchGatewayAllocatorTimeout); } else { batchIdToStoreShardBatch.values() .stream() .filter(batch -> batchesToAssign.contains(batch.batchId)) - .forEach( - batch -> runnables.add(new TimeoutAwareRunnable() { - @Override - public void onTimeout() { - long startTime = System.nanoTime(); - replicaBatchShardAllocator.allocateUnassignedBatchOnTimeout(batch.getBatchedShardRoutings(), allocation); - logger.info("Time taken to execute allocateUnassignedBatchOnTimeout for unassigned replica batch with id [{}], size : [{}] in this cycle:[{}ms]", - batch.batchId, batch.getBatchedShardRoutings().size(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); - - } - - @Override - public void run() { - long startTime = System.nanoTime(); - replicaBatchShardAllocator.allocateUnassignedBatch(batch.getBatchedShardRoutings(), allocation); - logger.info("Time taken to allocate unassigned replica batch with id [{}], size : [{}] in this cycle:[{}ms]", - batch.batchId, batch.getBatchedShardRoutings().size(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); - - } - })); + .forEach(batch -> runnables.add(new TimeoutAwareRunnable() { + @Override + public void onTimeout() { + long startTime = System.nanoTime(); + replicaBatchShardAllocator.allocateUnassignedBatchOnTimeout(batch.getBatchedShardRoutings(), allocation); + logger.info( + "Time taken to execute allocateUnassignedBatchOnTimeout for unassigned replica batch with id [{}], size : [{}] in this cycle:[{}ms]", + batch.batchId, + batch.getBatchedShardRoutings().size(), + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime) + ); + + } + + @Override + public void run() { + long startTime = System.nanoTime(); + replicaBatchShardAllocator.allocateUnassignedBatch(batch.getBatchedShardRoutings(), allocation); + logger.info( + "Time taken to allocate unassigned replica batch with id [{}], size : [{}] in this cycle:[{}ms]", + batch.batchId, + batch.getBatchedShardRoutings().size(), + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime) + ); + + } + })); return new BatchRunnableExecutor(runnables, () -> replicaShardsBatchGatewayAllocatorTimeout); } } diff --git a/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java b/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java index 779b9d4155ebd..350e0b2a4b357 100644 --- a/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java +++ b/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java @@ -8,7 +8,6 @@ package org.opensearch.common.util; -import org.junit.Before; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.TimeoutAwareRunnable; import org.opensearch.test.OpenSearchTestCase; @@ -18,7 +17,11 @@ import java.util.List; import java.util.function.Supplier; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class BatchRunnableExecutorTest extends OpenSearchTestCase { private Supplier timeoutSupplier; @@ -61,25 +64,25 @@ public void testRunWithTimeout() { verify(runnable3, never()).run(); } -// public void testRunWithPartialTimeout() { -// timeoutSupplier = mock(Supplier.class); -// runnable1 = mock(TimeoutAwareRunnable.class); -// runnable2 = mock(TimeoutAwareRunnable.class); -// runnable3 = mock(TimeoutAwareRunnable.class); -// runnableList = Arrays.asList(runnable1, runnable2, runnable3); -// when(timeoutSupplier.get()).thenReturn(TimeValue.timeValueMillis(100)); -// BatchRunnableExecutor executor = new BatchRunnableExecutor(runnableList, timeoutSupplier); -// doAnswer(invocationOnMock -> { -// Thread.sleep(1000); -// return null; -// }).when(runnable1).run(); -// executor.run(); -// verify(runnable1, times(1)).run(); -// verify(runnable2, never()).run(); -// verify(runnable3, never()).run(); -// verify(runnable2, times(1)).onTimeout(); -// verify(runnable3, times(1)).onTimeout(); -// } + // public void testRunWithPartialTimeout() { + // timeoutSupplier = mock(Supplier.class); + // runnable1 = mock(TimeoutAwareRunnable.class); + // runnable2 = mock(TimeoutAwareRunnable.class); + // runnable3 = mock(TimeoutAwareRunnable.class); + // runnableList = Arrays.asList(runnable1, runnable2, runnable3); + // when(timeoutSupplier.get()).thenReturn(TimeValue.timeValueMillis(100)); + // BatchRunnableExecutor executor = new BatchRunnableExecutor(runnableList, timeoutSupplier); + // doAnswer(invocationOnMock -> { + // Thread.sleep(1000); + // return null; + // }).when(runnable1).run(); + // executor.run(); + // verify(runnable1, times(1)).run(); + // verify(runnable2, never()).run(); + // verify(runnable3, never()).run(); + // verify(runnable2, times(1)).onTimeout(); + // verify(runnable3, times(1)).onTimeout(); + // } public void testRunWithEmptyRunnableList() { timeoutSupplier = mock(Supplier.class); diff --git a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java index 1e5ad732940ce..6e163849b2c29 100644 --- a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java @@ -40,7 +40,15 @@ import org.opensearch.test.IndexSettingsModule; import org.junit.Before; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; import static org.opensearch.cluster.routing.UnassignedInfo.Reason.CLUSTER_RECOVERED; @@ -282,7 +290,12 @@ public void testAllocateUnassignedBatchOnTimeoutWithNonPrimaryShards() { setUpShards(1); final RoutingAllocation routingAllocation = routingAllocationWithOnePrimary(allocationDeciders, CLUSTER_RECOVERED, "allocId-0"); - ShardRouting shardRouting = routingAllocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).replicaShards().get(0); + ShardRouting shardRouting = routingAllocation.routingTable() + .getIndicesRouting() + .get("test") + .shard(shardId.id()) + .replicaShards() + .get(0); List shardRoutings = Arrays.asList(shardRouting); batchAllocator.allocateUnassignedBatchOnTimeout(shardRoutings, routingAllocation); diff --git a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java index a648ed29b5698..0d12a2f031a60 100644 --- a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java @@ -708,7 +708,10 @@ public void testAllocateUnassignedBatchOnTimeoutWithUnassignedReplicaShard() { testBatchAllocator.allocateUnassignedBatchOnTimeout(shards, allocation); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); - assertEquals(UnassignedInfo.AllocationStatus.NO_ATTEMPT, allocation.routingNodes().unassigned().ignored().get(0).unassignedInfo().getLastAllocationStatus()); + assertEquals( + UnassignedInfo.AllocationStatus.NO_ATTEMPT, + allocation.routingNodes().unassigned().ignored().get(0).unassignedInfo().getLastAllocationStatus() + ); } public void testAllocateUnassignedBatchOnTimeoutWithAlreadyRecoveringReplicaShard() { From 0cde01c823d46e18f97dc1d0033244f2f99c3c12 Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Tue, 23 Jul 2024 04:21:05 +0530 Subject: [PATCH 12/35] Add changelog Signed-off-by: Rishab Nahata --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index fbe2b7f50d446..dde54f01c43cb 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -45,6 +45,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Allow @InternalApi annotation on classes not meant to be constructed outside of the OpenSearch core ([#14575](https://github.com/opensearch-project/OpenSearch/pull/14575)) - Add @InternalApi annotation to japicmp exclusions ([#14597](https://github.com/opensearch-project/OpenSearch/pull/14597)) - Allow system index warning in OpenSearchRestTestCase.refreshAllIndices ([#14635](https://github.com/opensearch-project/OpenSearch/pull/14635)) +- Make reroute iteration time-bound for large shard allocations ([#14848](https://github.com/opensearch-project/OpenSearch/pull/14848)) ### Deprecated From 3b4baae8b049027ae1c578763f0f760c5d2f5604 Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Tue, 23 Jul 2024 10:17:47 +0530 Subject: [PATCH 13/35] Remove redundant code Signed-off-by: Rishab Nahata --- .../routing/allocation/AllocationService.java | 6 ++--- .../gateway/BaseGatewayShardAllocator.java | 16 ++++++++++++ .../gateway/PrimaryShardBatchAllocator.java | 21 ---------------- .../gateway/ReplicaShardBatchAllocator.java | 25 ------------------- .../gateway/ShardsBatchGatewayAllocator.java | 4 +-- .../PrimaryShardBatchAllocatorTests.java | 6 ++--- .../ReplicaShardBatchAllocatorTests.java | 4 +-- 7 files changed, 26 insertions(+), 56 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java index 3ac792491b622..f2cf4e8e01d6b 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java @@ -569,11 +569,11 @@ private void reroute(RoutingAllocation allocation) { long startTime = System.nanoTime(); allocateExistingUnassignedShards(allocation); // try to allocate existing shard copies first - logger.info("Completing allocate unassigned, elapsed time: [{}]", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); + logger.debug("Completing allocateExistingUnassignedShards, elapsed time: [{}]", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); startTime = System.nanoTime(); shardsAllocator.allocate(allocation); - logger.info( - "Shard allocator to allocate all shards, elapsed time: [{}]", + logger.debug( + "Completing shardsAllocator allocate, elapsed time: [{}]", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime) ); clusterManagerMetrics.recordLatency( diff --git a/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java b/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java index 58982e869794f..c7b769fae72de 100644 --- a/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java @@ -36,6 +36,7 @@ import org.apache.logging.log4j.Logger; import org.opensearch.cluster.routing.RecoverySource; import org.opensearch.cluster.routing.RoutingNode; +import org.opensearch.cluster.routing.RoutingNodes; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.opensearch.cluster.routing.allocation.AllocationDecision; @@ -45,7 +46,9 @@ import org.opensearch.cluster.routing.allocation.decider.Decision; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; +import java.util.Set; /** * An abstract class that implements basic functionality for allocating @@ -78,6 +81,19 @@ public void allocateUnassigned( executeDecision(shardRouting, allocateUnassignedDecision, allocation, unassignedAllocationHandler); } + protected void allocateUnassignedBatchOnTimeout(List shardRoutings, RoutingAllocation allocation, boolean primary) { + Set batchShardRoutingSet = new HashSet<>(shardRoutings); + RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); + while (iterator.hasNext()) { + ShardRouting unassignedShard = iterator.next(); + AllocateUnassignedDecision allocationDecision; + if (unassignedShard.primary() == primary && batchShardRoutingSet.contains(unassignedShard)) { + allocationDecision = AllocateUnassignedDecision.throttle(null); + executeDecision(unassignedShard, allocationDecision, allocation, iterator); + } + } + } + protected void executeDecision( ShardRouting shardRouting, AllocateUnassignedDecision allocateUnassignedDecision, diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java index 66c81df7f3853..c50d8a40fd781 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java @@ -122,27 +122,6 @@ public void allocateUnassignedBatch(List shardRoutings, RoutingAll logger.trace("Finished shard allocation execution for unassigned primary shards: {}", shardRoutings.size()); } - protected void allocateUnassignedBatchOnTimeout(List shardRoutings, RoutingAllocation allocation) { - Set batchShardRoutingSet = new HashSet<>(shardRoutings); - RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); - while (iterator.hasNext()) { - ShardRouting unassignedShard = iterator.next(); - AllocateUnassignedDecision allocationDecision; - if (unassignedShard.primary() && batchShardRoutingSet.contains(unassignedShard)) { - allocationDecision = new AllocateUnassignedDecision( - UnassignedInfo.AllocationStatus.DECIDERS_THROTTLED, - null, - null, - null, - false, - 0L, - 0L - ); - executeDecision(unassignedShard, allocationDecision, allocation, iterator); - } - } - } - /** * Transforms {@link FetchResult} of {@link NodeGatewayStartedShardsBatch} to {@link List} of {@link TransportNodesListGatewayStartedShards.NodeGatewayStartedShards}. *

diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java index 83fe7e3b10ee1..7c75f2a5d1a8f 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java @@ -168,31 +168,6 @@ public void allocateUnassignedBatch(List shardRoutings, RoutingAll logger.trace("Finished shard allocation execution for unassigned replica shards: {}", shardRoutings.size()); } - protected void allocateUnassignedBatchOnTimeout(List shardRoutings, RoutingAllocation allocation) { - Set shardIdsFromBatch = new HashSet<>(); - for (ShardRouting shardRouting : shardRoutings) { - ShardId shardId = shardRouting.shardId(); - shardIdsFromBatch.add(shardId); - } - RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); - while (iterator.hasNext()) { - ShardRouting unassignedShard = iterator.next(); - AllocateUnassignedDecision allocationDecision; - if (!unassignedShard.primary() && shardIdsFromBatch.contains(unassignedShard.shardId())) { - allocationDecision = new AllocateUnassignedDecision( - UnassignedInfo.AllocationStatus.DECIDERS_THROTTLED, - null, - null, - null, - false, - 0L, - 0L - ); - executeDecision(unassignedShard, allocationDecision, allocation, iterator); - } - } - } - private AllocateUnassignedDecision getUnassignedShardAllocationDecision( ShardRouting shardRouting, RoutingAllocation allocation, diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 91bd2d3f3e164..9020d21b6b31e 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -250,7 +250,7 @@ protected BatchRunnableExecutor innerAllocateUnassignedBatch( @Override public void onTimeout() { long startTime = System.nanoTime(); - primaryBatchShardAllocator.allocateUnassignedBatchOnTimeout(shardsBatch.getBatchedShardRoutings(), allocation); + primaryBatchShardAllocator.allocateUnassignedBatchOnTimeout(shardsBatch.getBatchedShardRoutings(), allocation, true); logger.info( "Time taken to execute allocateUnassignedBatchOnTimeout for unassigned primary batch with id [{}], size : [{}] in this cycle:[{}ms]", shardsBatch.batchId, @@ -281,7 +281,7 @@ public void run() { @Override public void onTimeout() { long startTime = System.nanoTime(); - replicaBatchShardAllocator.allocateUnassignedBatchOnTimeout(batch.getBatchedShardRoutings(), allocation); + replicaBatchShardAllocator.allocateUnassignedBatchOnTimeout(batch.getBatchedShardRoutings(), allocation, false); logger.info( "Time taken to execute allocateUnassignedBatchOnTimeout for unassigned replica batch with id [{}], size : [{}] in this cycle:[{}ms]", batch.batchId, diff --git a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java index 6e163849b2c29..cb79b0fef19aa 100644 --- a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java @@ -265,7 +265,7 @@ public void testAllocateUnassignedBatchOnTimeoutWithMatchingPrimaryShards() { ShardRouting shardRouting = routingAllocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).primaryShard(); List shardRoutings = Arrays.asList(shardRouting); - batchAllocator.allocateUnassignedBatchOnTimeout(shardRoutings, routingAllocation); + batchAllocator.allocateUnassignedBatchOnTimeout(shardRoutings, routingAllocation, true); List ignoredShards = routingAllocation.routingNodes().unassigned().ignored(); assertEquals(1, ignoredShards.size()); @@ -278,7 +278,7 @@ public void testAllocateUnassignedBatchOnTimeoutWithNoMatchingPrimaryShards() { setUpShards(1); final RoutingAllocation routingAllocation = routingAllocationWithOnePrimary(allocationDeciders, CLUSTER_RECOVERED, "allocId-0"); List shardRoutings = new ArrayList<>(); - batchAllocator.allocateUnassignedBatchOnTimeout(shardRoutings, routingAllocation); + batchAllocator.allocateUnassignedBatchOnTimeout(shardRoutings, routingAllocation, true); List ignoredShards = routingAllocation.routingNodes().unassigned().ignored(); assertEquals(0, ignoredShards.size()); @@ -297,7 +297,7 @@ public void testAllocateUnassignedBatchOnTimeoutWithNonPrimaryShards() { .replicaShards() .get(0); List shardRoutings = Arrays.asList(shardRouting); - batchAllocator.allocateUnassignedBatchOnTimeout(shardRoutings, routingAllocation); + batchAllocator.allocateUnassignedBatchOnTimeout(shardRoutings, routingAllocation, true); List ignoredShards = routingAllocation.routingNodes().unassigned().ignored(); assertEquals(0, ignoredShards.size()); diff --git a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java index 0d12a2f031a60..0dc09c32d2959 100644 --- a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java @@ -705,7 +705,7 @@ public void testAllocateUnassignedBatchOnTimeoutWithUnassignedReplicaShard() { while (iterator.hasNext()) { shards.add(iterator.next()); } - testBatchAllocator.allocateUnassignedBatchOnTimeout(shards, allocation); + testBatchAllocator.allocateUnassignedBatchOnTimeout(shards, allocation, false); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); assertEquals( @@ -721,7 +721,7 @@ public void testAllocateUnassignedBatchOnTimeoutWithAlreadyRecoveringReplicaShar while (iterator.hasNext()) { shards.add(iterator.next()); } - testBatchAllocator.allocateUnassignedBatchOnTimeout(shards, allocation); + testBatchAllocator.allocateUnassignedBatchOnTimeout(shards, allocation, false); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(0)); } From 109616aacc2585202ce65399d7c72fb71c44cb2f Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Tue, 23 Jul 2024 10:20:21 +0530 Subject: [PATCH 14/35] Add min value for timeout Signed-off-by: Rishab Nahata --- .../org/opensearch/gateway/ShardsBatchGatewayAllocator.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 9020d21b6b31e..e72876f7b6aec 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -96,6 +96,7 @@ public class ShardsBatchGatewayAllocator implements ExistingShardsAllocator { public static final Setting PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING = Setting.timeSetting( PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING_KEY, TimeValue.MINUS_ONE, + TimeValue.timeValueSeconds(20), Setting.Property.NodeScope, Setting.Property.Dynamic ); @@ -103,6 +104,7 @@ public class ShardsBatchGatewayAllocator implements ExistingShardsAllocator { public static final Setting REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING = Setting.timeSetting( REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING_KEY, TimeValue.MINUS_ONE, + TimeValue.timeValueSeconds(20), Setting.Property.NodeScope, Setting.Property.Dynamic ); From 5c40c999cd77a506734b379bf816e68f1d2fe8db Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Tue, 23 Jul 2024 11:16:32 +0530 Subject: [PATCH 15/35] Fix spotless check Signed-off-by: Rishab Nahata --- .../routing/allocation/AllocateUnassignedDecision.java | 2 +- .../cluster/routing/allocation/AllocationService.java | 5 ++++- .../org/opensearch/gateway/PrimaryShardBatchAllocator.java | 1 - .../org/opensearch/gateway/ShardsBatchGatewayAllocator.java | 6 +++++- 4 files changed, 10 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocateUnassignedDecision.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocateUnassignedDecision.java index 43c6c19315cd0..4e77ab772f390 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocateUnassignedDecision.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocateUnassignedDecision.java @@ -105,7 +105,7 @@ public class AllocateUnassignedDecision extends AbstractAllocationDecision { private final long remainingDelayInMillis; private final long configuredDelayInMillis; - public AllocateUnassignedDecision( + private AllocateUnassignedDecision( AllocationStatus allocationStatus, DiscoveryNode assignedNode, String allocationId, diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java index f2cf4e8e01d6b..2d87be3fa005a 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java @@ -569,7 +569,10 @@ private void reroute(RoutingAllocation allocation) { long startTime = System.nanoTime(); allocateExistingUnassignedShards(allocation); // try to allocate existing shard copies first - logger.debug("Completing allocateExistingUnassignedShards, elapsed time: [{}]", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); + logger.debug( + "Completing allocateExistingUnassignedShards, elapsed time: [{}]", + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime) + ); startTime = System.nanoTime(); shardsAllocator.allocate(allocation); logger.debug( diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java index c50d8a40fd781..c493bf717c97f 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java @@ -12,7 +12,6 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.routing.RoutingNodes; import org.opensearch.cluster.routing.ShardRouting; -import org.opensearch.cluster.routing.UnassignedInfo; import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.core.index.shard.ShardId; diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index e72876f7b6aec..d92a23cc8f76e 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -252,7 +252,11 @@ protected BatchRunnableExecutor innerAllocateUnassignedBatch( @Override public void onTimeout() { long startTime = System.nanoTime(); - primaryBatchShardAllocator.allocateUnassignedBatchOnTimeout(shardsBatch.getBatchedShardRoutings(), allocation, true); + primaryBatchShardAllocator.allocateUnassignedBatchOnTimeout( + shardsBatch.getBatchedShardRoutings(), + allocation, + true + ); logger.info( "Time taken to execute allocateUnassignedBatchOnTimeout for unassigned primary batch with id [{}], size : [{}] in this cycle:[{}ms]", shardsBatch.batchId, From f5501fae79c59dd0d6e2d46a5db94aad412971ff Mon Sep 17 00:00:00 2001 From: Bukhtawar Khan Date: Tue, 23 Jul 2024 12:56:11 +0530 Subject: [PATCH 16/35] Switch to shuffle Signed-off-by: Bukhtawar Khan --- .../java/org/opensearch/common/util/BatchRunnableExecutor.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java index 014f13091b6b8..4e9d35cafd67d 100644 --- a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java +++ b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java @@ -10,6 +10,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.opensearch.common.Randomness; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.TimeoutAwareRunnable; import org.opensearch.core.common.util.CollectionUtils; @@ -38,11 +39,11 @@ public BatchRunnableExecutor(List timeoutAwareRunnables, S @Override public void run() { logger.debug("Starting execution of runnable of size [{}]", timeoutAwareRunnables.size()); - Collections.shuffle(timeoutAwareRunnables); long startTime = System.nanoTime(); if (CollectionUtils.isEmpty(timeoutAwareRunnables)) { return; } + Randomness.shuffle(timeoutAwareRunnables); for (TimeoutAwareRunnable workQueue : timeoutAwareRunnables) { if (timeoutSupplier.get().nanos() < 0 || System.nanoTime() - startTime < timeoutSupplier.get().nanos()) { workQueue.run(); From 341fe5fdc0b2cee2510b7232ec09e75320bd2238 Mon Sep 17 00:00:00 2001 From: Bukhtawar Khan Date: Tue, 23 Jul 2024 13:16:23 +0530 Subject: [PATCH 17/35] Switch to shuffle Signed-off-by: Bukhtawar Khan --- .../java/org/opensearch/common/util/BatchRunnableExecutor.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java index 4e9d35cafd67d..79b5a4854daf7 100644 --- a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java +++ b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java @@ -15,7 +15,6 @@ import org.opensearch.common.util.concurrent.TimeoutAwareRunnable; import org.opensearch.core.common.util.CollectionUtils; -import java.util.Collections; import java.util.List; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; From e760f08a5e2a1d4b9d3a3dbe8f8763d67071f9aa Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Tue, 23 Jul 2024 13:29:43 +0530 Subject: [PATCH 18/35] Revert "Add min value for timeout" This reverts commit 109616aacc2585202ce65399d7c72fb71c44cb2f. Signed-off-by: Rishab Nahata --- .../org/opensearch/gateway/ShardsBatchGatewayAllocator.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index d92a23cc8f76e..1a42a06656a1a 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -96,7 +96,6 @@ public class ShardsBatchGatewayAllocator implements ExistingShardsAllocator { public static final Setting PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING = Setting.timeSetting( PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING_KEY, TimeValue.MINUS_ONE, - TimeValue.timeValueSeconds(20), Setting.Property.NodeScope, Setting.Property.Dynamic ); @@ -104,7 +103,6 @@ public class ShardsBatchGatewayAllocator implements ExistingShardsAllocator { public static final Setting REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING = Setting.timeSetting( REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING_KEY, TimeValue.MINUS_ONE, - TimeValue.timeValueSeconds(20), Setting.Property.NodeScope, Setting.Property.Dynamic ); From 23a8fba3625517aac2e95a882a8a06812340c58d Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Tue, 23 Jul 2024 15:45:01 +0530 Subject: [PATCH 19/35] Trigger Build Signed-off-by: Rishab Nahata From f78903721272d62358eeeae84171dd68cb8039b1 Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Tue, 23 Jul 2024 16:57:15 +0530 Subject: [PATCH 20/35] Fix BatchRunnableExecutorTest test Signed-off-by: Rishab Nahata --- .../util/BatchRunnableExecutorTest.java | 62 ++++++++----------- 1 file changed, 27 insertions(+), 35 deletions(-) diff --git a/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java b/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java index 350e0b2a4b357..0a0b28a63afed 100644 --- a/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java +++ b/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java @@ -8,6 +8,7 @@ package org.opensearch.common.util; +import org.mockito.ArgumentCaptor; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.TimeoutAwareRunnable; import org.opensearch.test.OpenSearchTestCase; @@ -17,11 +18,7 @@ import java.util.List; import java.util.function.Supplier; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; +import static org.mockito.Mockito.*; public class BatchRunnableExecutorTest extends OpenSearchTestCase { private Supplier timeoutSupplier; @@ -30,12 +27,16 @@ public class BatchRunnableExecutorTest extends OpenSearchTestCase { private TimeoutAwareRunnable runnable3; private List runnableList; - public void testRunWithoutTimeout() { + public void setupRunnables() { timeoutSupplier = mock(Supplier.class); runnable1 = mock(TimeoutAwareRunnable.class); runnable2 = mock(TimeoutAwareRunnable.class); runnable3 = mock(TimeoutAwareRunnable.class); runnableList = Arrays.asList(runnable1, runnable2, runnable3); + } + + public void testRunWithoutTimeout() { + setupRunnables(); when(timeoutSupplier.get()).thenReturn(TimeValue.timeValueSeconds(1)); BatchRunnableExecutor executor = new BatchRunnableExecutor(runnableList, timeoutSupplier); executor.run(); @@ -48,11 +49,7 @@ public void testRunWithoutTimeout() { } public void testRunWithTimeout() { - timeoutSupplier = mock(Supplier.class); - runnable1 = mock(TimeoutAwareRunnable.class); - runnable2 = mock(TimeoutAwareRunnable.class); - runnable3 = mock(TimeoutAwareRunnable.class); - runnableList = Arrays.asList(runnable1, runnable2, runnable3); + setupRunnables(); when(timeoutSupplier.get()).thenReturn(TimeValue.timeValueNanos(1)); BatchRunnableExecutor executor = new BatchRunnableExecutor(runnableList, timeoutSupplier); executor.run(); @@ -64,32 +61,27 @@ public void testRunWithTimeout() { verify(runnable3, never()).run(); } - // public void testRunWithPartialTimeout() { - // timeoutSupplier = mock(Supplier.class); - // runnable1 = mock(TimeoutAwareRunnable.class); - // runnable2 = mock(TimeoutAwareRunnable.class); - // runnable3 = mock(TimeoutAwareRunnable.class); - // runnableList = Arrays.asList(runnable1, runnable2, runnable3); - // when(timeoutSupplier.get()).thenReturn(TimeValue.timeValueMillis(100)); - // BatchRunnableExecutor executor = new BatchRunnableExecutor(runnableList, timeoutSupplier); - // doAnswer(invocationOnMock -> { - // Thread.sleep(1000); - // return null; - // }).when(runnable1).run(); - // executor.run(); - // verify(runnable1, times(1)).run(); - // verify(runnable2, never()).run(); - // verify(runnable3, never()).run(); - // verify(runnable2, times(1)).onTimeout(); - // verify(runnable3, times(1)).onTimeout(); - // } + public void testRunWithPartialTimeout() { + setupRunnables(); + when(timeoutSupplier.get()).thenReturn(TimeValue.timeValueMillis(50)); + BatchRunnableExecutor executor = new BatchRunnableExecutor(runnableList, timeoutSupplier); + doAnswer(invocation -> { + Thread.sleep(100); + return null; + }).when(runnable1).run(); + executor.run(); + ArgumentCaptor runnableCaptor = ArgumentCaptor.forClass(Runnable.class); + verify(runnable1, atMost(1)).run(); + verify(runnable2, atMost(1)).run(); + verify(runnable3, atMost(1)).run(); + verify(runnable2, atMost(1)).onTimeout(); + verify(runnable3, atMost(1)).onTimeout(); + verify(runnable2, atMost(1)).onTimeout(); + verify(runnable3, atMost(1)).onTimeout(); + } public void testRunWithEmptyRunnableList() { - timeoutSupplier = mock(Supplier.class); - runnable1 = mock(TimeoutAwareRunnable.class); - runnable2 = mock(TimeoutAwareRunnable.class); - runnable3 = mock(TimeoutAwareRunnable.class); - runnableList = Arrays.asList(runnable1, runnable2, runnable3); + setupRunnables(); BatchRunnableExecutor executor = new BatchRunnableExecutor(Collections.emptyList(), timeoutSupplier); executor.run(); verify(runnable1, never()).onTimeout(); From 0d0ac70cfb9d79006d2e8cd56e441c2b3206cae4 Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Tue, 23 Jul 2024 19:19:59 +0530 Subject: [PATCH 21/35] Trigger Build Signed-off-by: Rishab Nahata From 65b315927545a9b1c510a4dac31f6965e648a4d4 Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Tue, 23 Jul 2024 20:24:05 +0530 Subject: [PATCH 22/35] Change log levels and minor comment fix Signed-off-by: Rishab Nahata --- .../routing/allocation/ExistingShardsAllocator.java | 2 +- .../opensearch/gateway/ShardsBatchGatewayAllocator.java | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java index 115369bd5ab05..75bb83e94e889 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java @@ -121,7 +121,7 @@ default BatchRunnableExecutor allocateAllUnassignedShards(RoutingAllocation allo runnables.add(new TimeoutAwareRunnable() { @Override public void onTimeout() { - // do nothing + throw new UnsupportedOperationException("Timeout not supported for non batched allocator"); } @Override diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 1a42a06656a1a..7751ac8a5c312 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -255,7 +255,7 @@ public void onTimeout() { allocation, true ); - logger.info( + logger.debug( "Time taken to execute allocateUnassignedBatchOnTimeout for unassigned primary batch with id [{}], size : [{}] in this cycle:[{}ms]", shardsBatch.batchId, shardsBatch.getBatchedShardRoutings().size(), @@ -267,7 +267,7 @@ public void onTimeout() { public void run() { long startTime = System.nanoTime(); primaryBatchShardAllocator.allocateUnassignedBatch(shardsBatch.getBatchedShardRoutings(), allocation); - logger.info( + logger.debug( "Time taken to allocate unassigned primary batch with id [{}], size : [{}] in this cycle:[{}ms]", shardsBatch.batchId, shardsBatch.getBatchedShardRoutings().size(), @@ -286,7 +286,7 @@ public void run() { public void onTimeout() { long startTime = System.nanoTime(); replicaBatchShardAllocator.allocateUnassignedBatchOnTimeout(batch.getBatchedShardRoutings(), allocation, false); - logger.info( + logger.debug( "Time taken to execute allocateUnassignedBatchOnTimeout for unassigned replica batch with id [{}], size : [{}] in this cycle:[{}ms]", batch.batchId, batch.getBatchedShardRoutings().size(), @@ -299,7 +299,7 @@ public void onTimeout() { public void run() { long startTime = System.nanoTime(); replicaBatchShardAllocator.allocateUnassignedBatch(batch.getBatchedShardRoutings(), allocation); - logger.info( + logger.debug( "Time taken to allocate unassigned replica batch with id [{}], size : [{}] in this cycle:[{}ms]", batch.batchId, batch.getBatchedShardRoutings().size(), From 65ea95064e10c715ea7d9009ccf459ffd8c9f770 Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Tue, 23 Jul 2024 21:22:12 +0530 Subject: [PATCH 23/35] Minor fix Signed-off-by: Rishab Nahata --- .../opensearch/common/util/BatchRunnableExecutor.java | 2 ++ .../common/util/concurrent/TimeoutAwareRunnable.java | 2 ++ .../common/util/BatchRunnableExecutorTest.java | 11 +++++++++-- 3 files changed, 13 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java index 79b5a4854daf7..37b1cc37a24d0 100644 --- a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java +++ b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java @@ -21,6 +21,8 @@ /** * The executor that executes a batch of {@link TimeoutAwareRunnable} and triggers a timeout based on {@link TimeValue} timeout + * + * @opensearch.internal */ public class BatchRunnableExecutor implements Runnable { diff --git a/server/src/main/java/org/opensearch/common/util/concurrent/TimeoutAwareRunnable.java b/server/src/main/java/org/opensearch/common/util/concurrent/TimeoutAwareRunnable.java index 2d890c5e01e45..8d3357ad93095 100644 --- a/server/src/main/java/org/opensearch/common/util/concurrent/TimeoutAwareRunnable.java +++ b/server/src/main/java/org/opensearch/common/util/concurrent/TimeoutAwareRunnable.java @@ -10,6 +10,8 @@ /** * Runnable that is aware of a timeout + * + * @opensearch.internal */ public interface TimeoutAwareRunnable extends Runnable { diff --git a/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java b/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java index 0a0b28a63afed..b8c18059e5c8b 100644 --- a/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java +++ b/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java @@ -8,7 +8,6 @@ package org.opensearch.common.util; -import org.mockito.ArgumentCaptor; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.TimeoutAwareRunnable; import org.opensearch.test.OpenSearchTestCase; @@ -18,7 +17,15 @@ import java.util.List; import java.util.function.Supplier; -import static org.mockito.Mockito.*; +import org.mockito.ArgumentCaptor; + +import static org.mockito.Mockito.atMost; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class BatchRunnableExecutorTest extends OpenSearchTestCase { private Supplier timeoutSupplier; From 7a440afaa5989625e8b441184f5bf867de1d3cef Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Tue, 23 Jul 2024 22:46:54 +0530 Subject: [PATCH 24/35] Fix naming convention Signed-off-by: Rishab Nahata --- ...unnableExecutorTest.java => BatchRunnableExecutorTests.java} | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename server/src/test/java/org/opensearch/common/util/{BatchRunnableExecutorTest.java => BatchRunnableExecutorTests.java} (98%) diff --git a/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java b/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTests.java similarity index 98% rename from server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java rename to server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTests.java index b8c18059e5c8b..7599949e5ace7 100644 --- a/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTest.java +++ b/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTests.java @@ -27,7 +27,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -public class BatchRunnableExecutorTest extends OpenSearchTestCase { +public class BatchRunnableExecutorTests extends OpenSearchTestCase { private Supplier timeoutSupplier; private TimeoutAwareRunnable runnable1; private TimeoutAwareRunnable runnable2; From aa11ddf77913d4ac86cf691caa6926b6a3546b46 Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Tue, 23 Jul 2024 23:27:14 +0530 Subject: [PATCH 25/35] Replace null check with Optional Signed-off-by: Rishab Nahata --- .../cluster/routing/allocation/AllocationService.java | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java index 2d87be3fa005a..c14c93a947dfb 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java @@ -73,6 +73,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.function.Function; @@ -629,16 +630,10 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { private void allocateAllUnassignedShards(RoutingAllocation allocation) { ExistingShardsAllocator allocator = existingShardsAllocators.get(ShardsBatchGatewayAllocator.ALLOCATOR_NAME); - BatchRunnableExecutor primaryShardsBatchExecutor = allocator.allocateAllUnassignedShards(allocation, true); - if (primaryShardsBatchExecutor != null) { - primaryShardsBatchExecutor.run(); - } + Optional.ofNullable(allocator.allocateAllUnassignedShards(allocation, true)).ifPresent(BatchRunnableExecutor::run); allocator.afterPrimariesBeforeReplicas(allocation); // Replicas Assignment - BatchRunnableExecutor replicaShardsBatchExecutor = allocator.allocateAllUnassignedShards(allocation, false); - if (replicaShardsBatchExecutor != null) { - replicaShardsBatchExecutor.run(); - } + Optional.ofNullable(allocator.allocateAllUnassignedShards(allocation, false)).ifPresent(BatchRunnableExecutor::run); } private void disassociateDeadNodes(RoutingAllocation allocation) { From 6cfdfaee80527c71fbed8ba0dd00ec5f45d0f6ca Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Wed, 24 Jul 2024 00:52:26 +0530 Subject: [PATCH 26/35] Remove unnecessary logs Signed-off-by: Rishab Nahata --- .../routing/allocation/AllocationService.java | 10 ------ .../gateway/ShardsBatchGatewayAllocator.java | 32 ------------------- 2 files changed, 42 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java index c14c93a947dfb..9d76cab22607a 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java @@ -568,18 +568,8 @@ private void reroute(RoutingAllocation allocation) { long rerouteStartTimeNS = System.nanoTime(); removeDelayMarkers(allocation); - long startTime = System.nanoTime(); allocateExistingUnassignedShards(allocation); // try to allocate existing shard copies first - logger.debug( - "Completing allocateExistingUnassignedShards, elapsed time: [{}]", - TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime) - ); - startTime = System.nanoTime(); shardsAllocator.allocate(allocation); - logger.debug( - "Completing shardsAllocator allocate, elapsed time: [{}]", - TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime) - ); clusterManagerMetrics.recordLatency( clusterManagerMetrics.rerouteHistogram, (double) Math.max(0, TimeValue.nsecToMSec(System.nanoTime() - rerouteStartTimeNS)) diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 7751ac8a5c312..3190e80590bf9 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -249,31 +249,16 @@ protected BatchRunnableExecutor innerAllocateUnassignedBatch( .forEach(shardsBatch -> runnables.add(new TimeoutAwareRunnable() { @Override public void onTimeout() { - long startTime = System.nanoTime(); primaryBatchShardAllocator.allocateUnassignedBatchOnTimeout( shardsBatch.getBatchedShardRoutings(), allocation, true ); - logger.debug( - "Time taken to execute allocateUnassignedBatchOnTimeout for unassigned primary batch with id [{}], size : [{}] in this cycle:[{}ms]", - shardsBatch.batchId, - shardsBatch.getBatchedShardRoutings().size(), - TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime) - ); } @Override public void run() { - long startTime = System.nanoTime(); primaryBatchShardAllocator.allocateUnassignedBatch(shardsBatch.getBatchedShardRoutings(), allocation); - logger.debug( - "Time taken to allocate unassigned primary batch with id [{}], size : [{}] in this cycle:[{}ms]", - shardsBatch.batchId, - shardsBatch.getBatchedShardRoutings().size(), - TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime) - ); - } })); return new BatchRunnableExecutor(runnables, () -> primaryShardsBatchGatewayAllocatorTimeout); @@ -284,28 +269,11 @@ public void run() { .forEach(batch -> runnables.add(new TimeoutAwareRunnable() { @Override public void onTimeout() { - long startTime = System.nanoTime(); replicaBatchShardAllocator.allocateUnassignedBatchOnTimeout(batch.getBatchedShardRoutings(), allocation, false); - logger.debug( - "Time taken to execute allocateUnassignedBatchOnTimeout for unassigned replica batch with id [{}], size : [{}] in this cycle:[{}ms]", - batch.batchId, - batch.getBatchedShardRoutings().size(), - TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime) - ); - } - @Override public void run() { - long startTime = System.nanoTime(); replicaBatchShardAllocator.allocateUnassignedBatch(batch.getBatchedShardRoutings(), allocation); - logger.debug( - "Time taken to allocate unassigned replica batch with id [{}], size : [{}] in this cycle:[{}ms]", - batch.batchId, - batch.getBatchedShardRoutings().size(), - TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime) - ); - } })); return new BatchRunnableExecutor(runnables, () -> replicaShardsBatchGatewayAllocatorTimeout); From e83db71b3205acc784591e1bf1f14ee02c0a6014 Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Wed, 24 Jul 2024 01:03:32 +0530 Subject: [PATCH 27/35] Fix spotless Signed-off-by: Rishab Nahata --- .../cluster/routing/allocation/AllocationService.java | 1 - .../org/opensearch/gateway/ShardsBatchGatewayAllocator.java | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java index 9d76cab22607a..4d65e7d70b5ec 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java @@ -75,7 +75,6 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.stream.Collectors; diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 3190e80590bf9..55f5388d8f454 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -56,7 +56,6 @@ import java.util.Set; import java.util.Spliterators; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.TimeUnit; import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -271,6 +270,7 @@ public void run() { public void onTimeout() { replicaBatchShardAllocator.allocateUnassignedBatchOnTimeout(batch.getBatchedShardRoutings(), allocation, false); } + @Override public void run() { replicaBatchShardAllocator.allocateUnassignedBatch(batch.getBatchedShardRoutings(), allocation); From ad77ffa909f5093515947e99d804d89890939892 Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Wed, 24 Jul 2024 03:39:24 +0530 Subject: [PATCH 28/35] Update server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java Co-authored-by: Andrew Ross Signed-off-by: Rishab Nahata --- .../java/org/opensearch/common/util/BatchRunnableExecutor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java index 37b1cc37a24d0..e30b779a6e007 100644 --- a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java +++ b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java @@ -20,7 +20,7 @@ import java.util.function.Supplier; /** - * The executor that executes a batch of {@link TimeoutAwareRunnable} and triggers a timeout based on {@link TimeValue} timeout + * A {@link Runnable} that iteratively executes a batch of {@link TimeoutAwareRunnable}s. If the elapsed time exceeds the timeout defined by {@link TimeValue} timeout, then all subsequent {@link TimeoutAwareRunnable}s will have their {@link TimeoutAwareRunnable#onTimeout} method invoked and will not be run. * * @opensearch.internal */ From cc6b5dd2ed103739d6ddb64f6881bc2de07e90a7 Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Wed, 24 Jul 2024 03:29:10 +0530 Subject: [PATCH 29/35] Add null check test Signed-off-by: Rishab Nahata --- .../opensearch/common/util/BatchRunnableExecutor.java | 2 +- .../org/opensearch/gateway/GatewayAllocatorTests.java | 10 ++++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java index e30b779a6e007..1b9b8c7c9e10f 100644 --- a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java +++ b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java @@ -41,7 +41,7 @@ public BatchRunnableExecutor(List timeoutAwareRunnables, S public void run() { logger.debug("Starting execution of runnable of size [{}]", timeoutAwareRunnables.size()); long startTime = System.nanoTime(); - if (CollectionUtils.isEmpty(timeoutAwareRunnables)) { + if (timeoutAwareRunnables.isEmpty()) { return; } Randomness.shuffle(timeoutAwareRunnables); diff --git a/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java index aa31c710c1fbd..9769c67333fac 100644 --- a/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java @@ -32,6 +32,7 @@ import org.opensearch.cluster.routing.allocation.decider.AllocationDeciders; import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.BatchRunnableExecutor; import org.opensearch.common.util.set.Sets; import org.opensearch.core.index.shard.ShardId; import org.opensearch.snapshots.SnapshotShardSizeInfo; @@ -46,6 +47,8 @@ import java.util.Set; import java.util.stream.Collectors; +import static org.opensearch.index.store.Store.verify; + public class GatewayAllocatorTests extends OpenSearchAllocationTestCase { private final Logger logger = LogManager.getLogger(GatewayAllocatorTests.class); @@ -61,6 +64,13 @@ public void setUp() throws Exception { testShardsBatchGatewayAllocator = new TestShardBatchGatewayAllocator(); } + public void testExecutorNotNull() { + createIndexAndUpdateClusterState(1, 3, 1); + createBatchesAndAssert(1); + BatchRunnableExecutor executor = testShardsBatchGatewayAllocator.allocateAllUnassignedShards(testAllocation, true); + assertNotNull(executor); + } + public void testSingleBatchCreation() { createIndexAndUpdateClusterState(1, 3, 1); createBatchesAndAssert(1); From d03f8cf6db05d4bb877cf7436a77a70dd0f95818 Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Wed, 24 Jul 2024 03:52:32 +0530 Subject: [PATCH 30/35] Resolve PR Comments Signed-off-by: Rishab Nahata --- .../routing/allocation/AllocationService.java | 4 ++-- .../allocation/ExistingShardsAllocator.java | 18 ++++-------------- .../common/util/BatchRunnableExecutor.java | 6 +++--- .../util/BatchRunnableExecutorTests.java | 7 +++---- 4 files changed, 12 insertions(+), 23 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java index 4d65e7d70b5ec..40efe37314878 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java @@ -619,10 +619,10 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { private void allocateAllUnassignedShards(RoutingAllocation allocation) { ExistingShardsAllocator allocator = existingShardsAllocators.get(ShardsBatchGatewayAllocator.ALLOCATOR_NAME); - Optional.ofNullable(allocator.allocateAllUnassignedShards(allocation, true)).ifPresent(BatchRunnableExecutor::run); + Optional.ofNullable(allocator.allocateAllUnassignedShards(allocation, true)).ifPresent(Runnable::run); allocator.afterPrimariesBeforeReplicas(allocation); // Replicas Assignment - Optional.ofNullable(allocator.allocateAllUnassignedShards(allocation, false)).ifPresent(BatchRunnableExecutor::run); + Optional.ofNullable(allocator.allocateAllUnassignedShards(allocation, false)).ifPresent(Runnable::run); } private void disassociateDeadNodes(RoutingAllocation allocation) { diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java index 75bb83e94e889..9ed9afebe7008 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java @@ -112,26 +112,16 @@ void allocateUnassigned( * * Allocation service will currently run the default implementation of it implemented by {@link ShardsBatchGatewayAllocator} */ - default BatchRunnableExecutor allocateAllUnassignedShards(RoutingAllocation allocation, boolean primary) { + default Runnable allocateAllUnassignedShards(RoutingAllocation allocation, boolean primary) { RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); - List runnables = new ArrayList<>(); + List runnables = new ArrayList<>(); while (iterator.hasNext()) { ShardRouting shardRouting = iterator.next(); if (shardRouting.primary() == primary) { - runnables.add(new TimeoutAwareRunnable() { - @Override - public void onTimeout() { - throw new UnsupportedOperationException("Timeout not supported for non batched allocator"); - } - - @Override - public void run() { - allocateUnassigned(shardRouting, allocation, iterator); - } - }); + runnables.add(() -> allocateUnassigned(shardRouting, allocation, iterator)); } } - return new BatchRunnableExecutor(runnables, () -> TimeValue.MINUS_ONE); + return () -> runnables.forEach(Runnable::run); } /** diff --git a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java index 1b9b8c7c9e10f..846512f6beaa5 100644 --- a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java +++ b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java @@ -45,12 +45,12 @@ public void run() { return; } Randomness.shuffle(timeoutAwareRunnables); - for (TimeoutAwareRunnable workQueue : timeoutAwareRunnables) { + for (TimeoutAwareRunnable runnable : timeoutAwareRunnables) { if (timeoutSupplier.get().nanos() < 0 || System.nanoTime() - startTime < timeoutSupplier.get().nanos()) { - workQueue.run(); + runnable.run(); } else { logger.debug("Executing timeout for runnable of size [{}]", timeoutAwareRunnables.size()); - workQueue.onTimeout(); + runnable.onTimeout(); } } logger.debug( diff --git a/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTests.java b/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTests.java index 7599949e5ace7..4a546a9f480a7 100644 --- a/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTests.java +++ b/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTests.java @@ -44,7 +44,7 @@ public void setupRunnables() { public void testRunWithoutTimeout() { setupRunnables(); - when(timeoutSupplier.get()).thenReturn(TimeValue.timeValueSeconds(1)); + timeoutSupplier = () -> TimeValue.timeValueSeconds(1); BatchRunnableExecutor executor = new BatchRunnableExecutor(runnableList, timeoutSupplier); executor.run(); verify(runnable1, times(1)).run(); @@ -57,7 +57,7 @@ public void testRunWithoutTimeout() { public void testRunWithTimeout() { setupRunnables(); - when(timeoutSupplier.get()).thenReturn(TimeValue.timeValueNanos(1)); + timeoutSupplier = () -> TimeValue.timeValueNanos(1); BatchRunnableExecutor executor = new BatchRunnableExecutor(runnableList, timeoutSupplier); executor.run(); verify(runnable1, times(1)).onTimeout(); @@ -70,14 +70,13 @@ public void testRunWithTimeout() { public void testRunWithPartialTimeout() { setupRunnables(); - when(timeoutSupplier.get()).thenReturn(TimeValue.timeValueMillis(50)); + timeoutSupplier = () -> TimeValue.timeValueMillis(50); BatchRunnableExecutor executor = new BatchRunnableExecutor(runnableList, timeoutSupplier); doAnswer(invocation -> { Thread.sleep(100); return null; }).when(runnable1).run(); executor.run(); - ArgumentCaptor runnableCaptor = ArgumentCaptor.forClass(Runnable.class); verify(runnable1, atMost(1)).run(); verify(runnable2, atMost(1)).run(); verify(runnable3, atMost(1)).run(); From 21c4c1cc7d1fba614d8e681c268154584961f0b8 Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Wed, 24 Jul 2024 03:54:02 +0530 Subject: [PATCH 31/35] Fix spotless Signed-off-by: Rishab Nahata --- .../cluster/routing/allocation/AllocationService.java | 1 - .../cluster/routing/allocation/ExistingShardsAllocator.java | 3 --- .../java/org/opensearch/common/util/BatchRunnableExecutor.java | 1 - .../org/opensearch/common/util/BatchRunnableExecutorTests.java | 3 --- .../java/org/opensearch/gateway/GatewayAllocatorTests.java | 2 -- 5 files changed, 10 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java index 40efe37314878..e29a81a2c131f 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java @@ -58,7 +58,6 @@ import org.opensearch.cluster.routing.allocation.decider.Decision; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; -import org.opensearch.common.util.BatchRunnableExecutor; import org.opensearch.gateway.GatewayAllocator; import org.opensearch.gateway.PriorityComparator; import org.opensearch.gateway.ShardsBatchGatewayAllocator; diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java index 9ed9afebe7008..eb7a1e7209c37 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java @@ -38,9 +38,6 @@ import org.opensearch.cluster.routing.UnassignedInfo; import org.opensearch.common.Nullable; import org.opensearch.common.settings.Setting; -import org.opensearch.common.unit.TimeValue; -import org.opensearch.common.util.BatchRunnableExecutor; -import org.opensearch.common.util.concurrent.TimeoutAwareRunnable; import org.opensearch.gateway.GatewayAllocator; import org.opensearch.gateway.ShardsBatchGatewayAllocator; diff --git a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java index 846512f6beaa5..ed968e628c838 100644 --- a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java +++ b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java @@ -13,7 +13,6 @@ import org.opensearch.common.Randomness; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.TimeoutAwareRunnable; -import org.opensearch.core.common.util.CollectionUtils; import java.util.List; import java.util.concurrent.TimeUnit; diff --git a/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTests.java b/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTests.java index 4a546a9f480a7..269f89faec54d 100644 --- a/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTests.java +++ b/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTests.java @@ -17,15 +17,12 @@ import java.util.List; import java.util.function.Supplier; -import org.mockito.ArgumentCaptor; - import static org.mockito.Mockito.atMost; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; public class BatchRunnableExecutorTests extends OpenSearchTestCase { private Supplier timeoutSupplier; diff --git a/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java index 9769c67333fac..de15c341d4738 100644 --- a/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java @@ -47,8 +47,6 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.opensearch.index.store.Store.verify; - public class GatewayAllocatorTests extends OpenSearchAllocationTestCase { private final Logger logger = LogManager.getLogger(GatewayAllocatorTests.class); From 25ca9519cca1fe0eb086acd7ec7ab67d39c4114f Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Wed, 24 Jul 2024 12:46:18 +0530 Subject: [PATCH 32/35] Fix Signed-off-by: Rishab Nahata --- .../opensearch/gateway/BaseGatewayShardAllocator.java | 9 +++++++-- .../gateway/PrimaryShardBatchAllocatorTests.java | 4 ++-- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java b/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java index c7b769fae72de..0d6af943d39e0 100644 --- a/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java @@ -44,6 +44,7 @@ import org.opensearch.cluster.routing.allocation.NodeAllocationResult; import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.cluster.routing.allocation.decider.Decision; +import org.opensearch.core.index.shard.ShardId; import java.util.ArrayList; import java.util.HashSet; @@ -82,12 +83,16 @@ public void allocateUnassigned( } protected void allocateUnassignedBatchOnTimeout(List shardRoutings, RoutingAllocation allocation, boolean primary) { - Set batchShardRoutingSet = new HashSet<>(shardRoutings); + Set shardIdsFromBatch = new HashSet<>(); + for (ShardRouting shardRouting : shardRoutings) { + ShardId shardId = shardRouting.shardId(); + shardIdsFromBatch.add(shardId); + } RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); while (iterator.hasNext()) { ShardRouting unassignedShard = iterator.next(); AllocateUnassignedDecision allocationDecision; - if (unassignedShard.primary() == primary && batchShardRoutingSet.contains(unassignedShard)) { + if (unassignedShard.primary() == primary && shardIdsFromBatch.contains(unassignedShard.shardId())) { allocationDecision = AllocateUnassignedDecision.throttle(null); executeDecision(unassignedShard, allocationDecision, allocation, iterator); } diff --git a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java index cb79b0fef19aa..270cf465d0f80 100644 --- a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java @@ -297,10 +297,10 @@ public void testAllocateUnassignedBatchOnTimeoutWithNonPrimaryShards() { .replicaShards() .get(0); List shardRoutings = Arrays.asList(shardRouting); - batchAllocator.allocateUnassignedBatchOnTimeout(shardRoutings, routingAllocation, true); + batchAllocator.allocateUnassignedBatchOnTimeout(shardRoutings, routingAllocation, false); List ignoredShards = routingAllocation.routingNodes().unassigned().ignored(); - assertEquals(0, ignoredShards.size()); + assertEquals(1, ignoredShards.size()); } private RoutingAllocation routingAllocationWithOnePrimary( From b1a612e7d04623bc9d25491ebd796a9750bf4280 Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Wed, 24 Jul 2024 14:34:17 +0530 Subject: [PATCH 33/35] Add ExistingShardsAllocatorTests Signed-off-by: Rishab Nahata --- .../ExistingShardsAllocatorTests.java | 118 ++++++++++++++++++ 1 file changed, 118 insertions(+) create mode 100644 server/src/test/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocatorTests.java diff --git a/server/src/test/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocatorTests.java b/server/src/test/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocatorTests.java new file mode 100644 index 0000000000000..1da8f5ef7f695 --- /dev/null +++ b/server/src/test/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocatorTests.java @@ -0,0 +1,118 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.cluster.routing.allocation; + +import org.opensearch.Version; +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.OpenSearchAllocationTestCase; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.common.settings.Settings; + +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +public class ExistingShardsAllocatorTests extends OpenSearchAllocationTestCase { + + public void testRunnablesExecutedForUnassignedShards() throws InterruptedException { + + Metadata metadata = Metadata.builder() + .put(IndexMetadata.builder("test").settings(settings(Version.CURRENT)).numberOfShards(3).numberOfReplicas(2)) + .build(); + RoutingTable initialRoutingTable = RoutingTable.builder().addAsNew(metadata.index("test")).build(); + + ClusterState clusterState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(initialRoutingTable) + .build(); + clusterState = ClusterState.builder(clusterState) + .nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2")).add(newNode("node3"))) + .build(); + RoutingAllocation allocation = new RoutingAllocation( + yesAllocationDeciders(), + clusterState.getRoutingNodes(), + clusterState, + null, + null, + 0L + ); + CountDownLatch expectedStateLatch = new CountDownLatch(3); + TestAllocator testAllocator = new TestAllocator(expectedStateLatch); + testAllocator.allocateAllUnassignedShards(allocation, true).run(); + // if the below condition is passed, then we are sure runnable executed for all primary shards + assertTrue(expectedStateLatch.await(30, TimeUnit.SECONDS)); + + expectedStateLatch = new CountDownLatch(6); + testAllocator = new TestAllocator(expectedStateLatch); + testAllocator.allocateAllUnassignedShards(allocation, false).run(); + // if the below condition is passed, then we are sure runnable executed for all replica shards + assertTrue(expectedStateLatch.await(30, TimeUnit.SECONDS)); + } + + private static class TestAllocator implements ExistingShardsAllocator { + + final CountDownLatch countDownLatch; + + TestAllocator(CountDownLatch latch) { + this.countDownLatch = latch; + } + + @Override + public void beforeAllocation(RoutingAllocation allocation) { + + } + + @Override + public void afterPrimariesBeforeReplicas(RoutingAllocation allocation) { + + } + + @Override + public void allocateUnassigned( + ShardRouting shardRouting, + RoutingAllocation allocation, + UnassignedAllocationHandler unassignedAllocationHandler + ) { + countDownLatch.countDown(); + } + + @Override + public AllocateUnassignedDecision explainUnassignedShardAllocation( + ShardRouting unassignedShard, + RoutingAllocation routingAllocation + ) { + return null; + } + + @Override + public void cleanCaches() { + + } + + @Override + public void applyStartedShards(List startedShards, RoutingAllocation allocation) { + + } + + @Override + public void applyFailedShards(List failedShards, RoutingAllocation allocation) { + + } + + @Override + public int getNumberOfInFlightFetches() { + return 0; + } + } +} From 07befa284268dddf2c5b5a63025a40b9d0b175ab Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Wed, 24 Jul 2024 15:06:08 +0530 Subject: [PATCH 34/35] Add tests for ShardsBatchGatewayAllocator Signed-off-by: Rishab Nahata --- .../common/util/BatchRunnableExecutor.java | 5 ++++ .../gateway/GatewayAllocatorTests.java | 24 +++++++++++++++++++ 2 files changed, 29 insertions(+) diff --git a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java index ed968e628c838..d3d3304cb909a 100644 --- a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java +++ b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java @@ -36,6 +36,11 @@ public BatchRunnableExecutor(List timeoutAwareRunnables, S this.timeoutAwareRunnables = timeoutAwareRunnables; } + // for tests + public List getTimeoutAwareRunnables() { + return this.timeoutAwareRunnables; + } + @Override public void run() { logger.debug("Starting execution of runnable of size [{}]", timeoutAwareRunnables.size()); diff --git a/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java index de15c341d4738..bd56123f6df1f 100644 --- a/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java @@ -344,6 +344,30 @@ public void testGetBatchIdNonExisting() { allShardRoutings.forEach(shard -> assertNull(testShardsBatchGatewayAllocator.getBatchId(shard, shard.primary()))); } + public void testCreatePrimaryAndReplicaExecutorOfSizeOne() { + createIndexAndUpdateClusterState(1, 3, 2); + BatchRunnableExecutor executor = testShardsBatchGatewayAllocator.allocateAllUnassignedShards(testAllocation, true); + assertEquals(executor.getTimeoutAwareRunnables().size(), 1); + executor = testShardsBatchGatewayAllocator.allocateAllUnassignedShards(testAllocation, false); + assertEquals(executor.getTimeoutAwareRunnables().size(), 1); + } + + public void testCreatePrimaryExecutorOfSizeOneAndReplicaExecutorOfSizeZero() { + createIndexAndUpdateClusterState(1, 3, 0); + BatchRunnableExecutor executor = testShardsBatchGatewayAllocator.allocateAllUnassignedShards(testAllocation, true); + assertEquals(executor.getTimeoutAwareRunnables().size(), 1); + executor = testShardsBatchGatewayAllocator.allocateAllUnassignedShards(testAllocation, false); + assertNull(executor); + } + + public void testCreatePrimaryAndReplicaExecutorOfSizeTwo() { + createIndexAndUpdateClusterState(2, 1001, 1); + BatchRunnableExecutor executor = testShardsBatchGatewayAllocator.allocateAllUnassignedShards(testAllocation, true); + assertEquals(executor.getTimeoutAwareRunnables().size(), 2); + executor = testShardsBatchGatewayAllocator.allocateAllUnassignedShards(testAllocation, false); + assertEquals(executor.getTimeoutAwareRunnables().size(), 2); + } + private void createIndexAndUpdateClusterState(int count, int numberOfShards, int numberOfReplicas) { if (count == 0) return; Metadata.Builder metadata = Metadata.builder(); From 6ddb155a7b04b3bb56121adbb80f7ab317ec8dbb Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Wed, 24 Jul 2024 15:33:15 +0530 Subject: [PATCH 35/35] Trigger Build Signed-off-by: Rishab Nahata