From 7f13975963251e07d54bf5feeeac3f9eeaa048ee Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Fri, 14 Jul 2023 16:12:42 +0530 Subject: [PATCH 01/45] Draft changes for creation of batches Signed-off-by: Gaurav Chandani --- .../allocation/ExistingShardsAllocator.java | 8 +++ .../opensearch/gateway/GatewayAllocator.java | 71 +++++++++++++++++++ 2 files changed, 79 insertions(+) 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 f1889cdf780d4..8c6aabae1e4df 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 @@ -80,6 +80,14 @@ void allocateUnassigned( UnassignedAllocationHandler unassignedAllocationHandler ); + default void allocateBatchUnassigned(RoutingAllocation allocation){ + RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); + while (iterator.hasNext()) { + ShardRouting shardRouting = iterator.next(); + allocateUnassigned(shardRouting, allocation, iterator); + } + } + /** * Returns an explanation for a single unassigned shard. */ diff --git a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java index cdcf813d9ede0..060ea23c138ac 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java @@ -43,12 +43,14 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.cluster.routing.RerouteService; +import org.opensearch.cluster.routing.RoutingNodes; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.opensearch.cluster.routing.allocation.ExistingShardsAllocator; import org.opensearch.cluster.routing.allocation.FailedShard; import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.common.Priority; +import org.opensearch.common.UUIDs; import org.opensearch.common.inject.Inject; import org.opensearch.common.lease.Releasables; import org.opensearch.common.util.concurrent.ConcurrentCollections; @@ -57,7 +59,10 @@ import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentMap; import java.util.stream.Collectors; @@ -73,6 +78,7 @@ public class GatewayAllocator implements ExistingShardsAllocator { public static final String ALLOCATOR_NAME = "gateway_allocator"; private static final Logger logger = LogManager.getLogger(GatewayAllocator.class); + private static final long MAX_BATCH_SIZE = 2000; // will change it to a dynamic setting later private final RerouteService rerouteService; @@ -87,6 +93,8 @@ public class GatewayAllocator implements ExistingShardsAllocator { ConcurrentCollections.newConcurrentMap(); private Set lastSeenEphemeralIds = Collections.emptySet(); + private final ConcurrentMap> asyncBatchFetchStarted = ConcurrentCollections.newConcurrentMap(); + @Inject public GatewayAllocator( RerouteService rerouteService, @@ -168,6 +176,51 @@ public void allocateUnassigned( innerAllocatedUnassigned(allocation, primaryShardAllocator, replicaShardAllocator, shardRouting, unassignedAllocationHandler); } + @Override + public void allocateBatchUnassigned(final RoutingAllocation allocation){ + assert primaryShardAllocator != null; + assert replicaShardAllocator != null; + + // create batches + createBatchesForShardsStarted(allocation); + + // execute those batches + // execution involves two parts: fetching and allocation + + } + + private void createBatchesForShardsStarted(RoutingAllocation allocation) { + RoutingNodes.UnassignedShards unassigned = allocation.routingNodes().unassigned(); + // fetch all current batch shards + Set currentBatchedShards = asyncBatchFetchStarted.keySet().stream().flatMap(shardsBatcher -> shardsBatcher.getBatchedShards().stream()).collect(Collectors.toSet()); + Set shardsToBatch = Sets.newHashSet(); + unassigned.forEach(shardRouting -> { + if (currentBatchedShards.contains(shardRouting.shardId()) == false) { + shardsToBatch.add(shardRouting); + } + }); + Iterator iterator = shardsToBatch.iterator(); + long batchSize = MAX_BATCH_SIZE; + Map addToCurrentBatch = new HashMap<>(); + while (iterator.hasNext()) { + ShardRouting currentShard = iterator.next(); + if (batchSize > 0) { + addToCurrentBatch.put(currentShard.shardId(), IndexMetadata.INDEX_DATA_PATH_SETTING.get(allocation.metadata().index(currentShard.index()).getSettings())); + batchSize--; + iterator.remove(); + } + // add to batch if batch size full or last shard + if (batchSize == 0 || iterator.hasNext() == false) { + String batchUUId = UUIDs.base64UUID(); + ShardsBatcher shardsBatcher = new ShardsBatcher(batchUUId, addToCurrentBatch); + // add to main asyncShardFetchStarted after Async object ready + + addToCurrentBatch.clear(); + batchSize = MAX_BATCH_SIZE; + } + } + } + // allow for testing infra to change shard allocators implementation protected static void innerAllocatedUnassigned( RoutingAllocation allocation, @@ -341,4 +394,22 @@ protected boolean hasInitiatedFetching(ShardRouting shard) { return asyncFetchStore.get(shard.shardId()) != null; } } + + private class ShardsBatcher { + private final String uuid; + private Map shardsToCustomDataPathMap; + private ShardsBatcher(String uuid, Map shardsToCustomDataPathMap) { + this.uuid = uuid; + this.shardsToCustomDataPathMap = shardsToCustomDataPathMap; + } + void removeFromBatch(ShardId shardId) { + shardsToCustomDataPathMap.remove(shardId); + } + + Set getBatchedShards() { + return shardsToCustomDataPathMap.keySet(); + } + } + + } From 637fca07c9030ed228348b43a4801809ace47af9 Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Tue, 18 Jul 2023 12:45:24 +0530 Subject: [PATCH 02/45] WIP changes for GA changes for fetching Signed-off-by: Gaurav Chandani --- .../routing/allocation/AllocationService.java | 29 +++++ .../allocation/ExistingShardsAllocator.java | 2 +- .../gateway/BaseGatewayShardAllocator.java | 6 ++ .../opensearch/gateway/GatewayAllocator.java | 100 ++++++++++++++++-- .../gateway/PrimaryShardAllocator.java | 7 ++ 5 files changed, 132 insertions(+), 12 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 5169e63aeb9a5..6e442d43ac430 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 @@ -547,6 +547,18 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { for (final ExistingShardsAllocator existingShardsAllocator : existingShardsAllocators.values()) { existingShardsAllocator.beforeAllocation(allocation); } + // batch Mode enabled setting to be added + boolean batchModeEnabled = true; + if (batchModeEnabled) { + // since allocators is per index setting, to have batch assignment verify allocators same for all shards + // if not fallback to single assignment + ExistingShardsAllocator allocator = verifySameAllocatorForAllShards(allocation); + if (allocator != null) { + allocator.allocateBatchUnassigned(allocation, true); + + } + } + final RoutingNodes.UnassignedShards.UnassignedIterator primaryIterator = allocation.routingNodes().unassigned().iterator(); while (primaryIterator.hasNext()) { @@ -569,6 +581,23 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { } } + private ExistingShardsAllocator verifySameAllocatorForAllShards(RoutingAllocation allocation) { + RoutingNodes.UnassignedShards unassignedShards = allocation.routingNodes().unassigned(); + RoutingNodes.UnassignedShards.UnassignedIterator iterator = unassignedShards.iterator(); + ExistingShardsAllocator currentAllocatorForShard =null; + if (unassignedShards.size() > 0) { + ShardRouting shard = iterator.next(); + currentAllocatorForShard= getAllocatorForShard(shard, allocation); + while (iterator.hasNext()){ + ExistingShardsAllocator allocatorForShard = getAllocatorForShard(iterator.next(), allocation); + if (currentAllocatorForShard.getClass().getName().equals(allocatorForShard.getClass().getName())==false){ + return null; + } + } + } + return currentAllocatorForShard; + } + private void disassociateDeadNodes(RoutingAllocation allocation) { for (Iterator it = allocation.routingNodes().mutableIterator(); it.hasNext();) { RoutingNode node = it.next(); 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 8c6aabae1e4df..2013877804305 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 @@ -80,7 +80,7 @@ void allocateUnassigned( UnassignedAllocationHandler unassignedAllocationHandler ); - default void allocateBatchUnassigned(RoutingAllocation allocation){ + default void allocateBatchUnassigned(RoutingAllocation allocation, boolean primary){ RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); while (iterator.hasNext()) { ShardRouting shardRouting = iterator.next(); diff --git a/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java b/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java index 59ef894958cbe..e418e19353c2c 100644 --- a/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java @@ -92,6 +92,9 @@ public void allocateUnassigned( } } + public void allocateBatchUnassigned(String batchId, RoutingAllocation allocation){ + } + protected long getExpectedShardSize(ShardRouting shardRouting, RoutingAllocation allocation) { if (shardRouting.primary()) { if (shardRouting.recoverySource().getType() == RecoverySource.Type.SNAPSHOT) { @@ -120,6 +123,9 @@ public abstract AllocateUnassignedDecision makeAllocationDecision( Logger logger ); + /** + * Returns a list of {@link NodeAllocationResult} + /** * Builds decisions for all nodes in the cluster, so that the explain API can provide information on * allocation decisions for each node, while still waiting to allocate the shard (e.g. due to fetching shard data). diff --git a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java index 060ea23c138ac..ae41036c8d997 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java @@ -85,6 +85,10 @@ public class GatewayAllocator implements ExistingShardsAllocator { private final PrimaryShardAllocator primaryShardAllocator; private final ReplicaShardAllocator replicaShardAllocator; + private final PrimaryShardAllocator primaryBatchShardAllocator; + private final TransportNodesBatchListGatewayStartedShards batchStartedAction; + + private final ConcurrentMap< ShardId, AsyncShardFetch> asyncFetchStarted = ConcurrentCollections @@ -93,17 +97,20 @@ public class GatewayAllocator implements ExistingShardsAllocator { ConcurrentCollections.newConcurrentMap(); private Set lastSeenEphemeralIds = Collections.emptySet(); - private final ConcurrentMap> asyncBatchFetchStarted = ConcurrentCollections.newConcurrentMap(); + private final ConcurrentMap> asyncBatchFetchStarted = ConcurrentCollections.newConcurrentMap(); @Inject public GatewayAllocator( RerouteService rerouteService, TransportNodesListGatewayStartedShards startedAction, - TransportNodesListShardStoreMetadata storeAction + TransportNodesListShardStoreMetadata storeAction, + TransportNodesBatchListGatewayStartedShards batchStartedAction ) { this.rerouteService = rerouteService; this.primaryShardAllocator = new InternalPrimaryShardAllocator(startedAction); this.replicaShardAllocator = new InternalReplicaShardAllocator(storeAction); + this.batchStartedAction = batchStartedAction; + this.primaryBatchShardAllocator = new InternalPrimaryBatchShardAllocator(batchStartedAction); } @Override @@ -119,6 +126,8 @@ protected GatewayAllocator() { this.rerouteService = null; this.primaryShardAllocator = null; this.replicaShardAllocator = null; + this.batchStartedAction=null; + this.primaryBatchShardAllocator =null; } @Override @@ -177,25 +186,30 @@ public void allocateUnassigned( } @Override - public void allocateBatchUnassigned(final RoutingAllocation allocation){ + public void allocateBatchUnassigned(final RoutingAllocation allocation, boolean primary){ assert primaryShardAllocator != null; assert replicaShardAllocator != null; - // create batches - createBatchesForShardsStarted(allocation); - - // execute those batches - // execution involves two parts: fetching and allocation + if (primary) { + createBatchesForShardsStarted(allocation); + asyncBatchFetchStarted.keySet().forEach(batch -> primaryBatchShardAllocator.allocateBatchUnassigned(batch.getBatchId(), allocation)); + } + else { + createBatchesForShardsStore(allocation); + asyncBatchFetchStore.keySet().forEach(batch -> replicaShardAllocator.allocateBatchUnassigned(batch.getBatchId(), allocation)); + } } private void createBatchesForShardsStarted(RoutingAllocation allocation) { RoutingNodes.UnassignedShards unassigned = allocation.routingNodes().unassigned(); - // fetch all current batch shards + // fetch all current batched shards Set currentBatchedShards = asyncBatchFetchStarted.keySet().stream().flatMap(shardsBatcher -> shardsBatcher.getBatchedShards().stream()).collect(Collectors.toSet()); Set shardsToBatch = Sets.newHashSet(); + // add all unassigned shards to the batch if they are not already in a batch unassigned.forEach(shardRouting -> { - if (currentBatchedShards.contains(shardRouting.shardId()) == false) { + if (currentBatchedShards.contains(shardRouting.shardId()) == false && shardRouting.primary()) { + assert shardRouting.unassigned(); shardsToBatch.add(shardRouting); } }); @@ -209,12 +223,22 @@ private void createBatchesForShardsStarted(RoutingAllocation allocation) { batchSize--; iterator.remove(); } - // add to batch if batch size full or last shard + // add to batch if batch size full or last shard in unassigned list if (batchSize == 0 || iterator.hasNext() == false) { String batchUUId = UUIDs.base64UUID(); ShardsBatcher shardsBatcher = new ShardsBatcher(batchUUId, addToCurrentBatch); // add to main asyncShardFetchStarted after Async object ready + asyncBatchFetchStarted.computeIfAbsent( + shardsBatcher, + batch -> new InternalBatchAsyncFetch<>( + logger, + "batch_shards_started", + batch.getShardsToCustomDataPathMap(), + this.batchStartedAction, + batch.getBatchId() + ) + ); addToCurrentBatch.clear(); batchSize = MAX_BATCH_SIZE; } @@ -321,6 +345,32 @@ protected void reroute(ShardId shardId, String reason) { } } + class InternalBatchAsyncFetch extends AsyncBatchShardFetch { + + InternalBatchAsyncFetch(Logger logger, + String type, + Map map, + AsyncBatchShardFetch.Lister, T> action, + String batchUUId + ) { + super(logger, type, map, action, batchUUId); + } + + @Override + protected void reroute(String batchUUId, String reason) { + logger.trace("{} scheduling reroute for {}", batchUUId, reason); + assert rerouteService != null; + rerouteService.reroute( + "async_shard_fetch", + Priority.HIGH, + ActionListener.wrap( + r -> logger.trace("{} scheduled reroute completed for {}", batchUUId, reason), + e -> logger.debug(new ParameterizedMessage("{} scheduled reroute failed for {}", batchUUId, reason), e) + ) + ); + } + } + class InternalPrimaryShardAllocator extends PrimaryShardAllocator { private final TransportNodesListGatewayStartedShards startedAction; @@ -356,6 +406,25 @@ protected AsyncShardFetch.FetchResult fetchData(ShardRouting shard, RoutingAllocation allocation) { + return null; + } + + @Override + protected AsyncBatchShardFetch.AdaptedResultsForShard fetchBatchData(String batchId, RoutingAllocation allocation){ + // send adapted data to PSA for processing. + return null; + } + } + class InternalReplicaShardAllocator extends ReplicaShardAllocator { private final TransportNodesListShardStoreMetadata storeAction; @@ -397,6 +466,11 @@ protected boolean hasInitiatedFetching(ShardRouting shard) { private class ShardsBatcher { private final String uuid; + + public Map getShardsToCustomDataPathMap() { + return shardsToCustomDataPathMap; + } + private Map shardsToCustomDataPathMap; private ShardsBatcher(String uuid, Map shardsToCustomDataPathMap) { this.uuid = uuid; @@ -409,6 +483,10 @@ void removeFromBatch(ShardId shardId) { Set getBatchedShards() { return shardsToCustomDataPathMap.keySet(); } + + public String getBatchId() { + return uuid; + } } diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java index 4dc9396751fc9..af12123187430 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java @@ -89,6 +89,7 @@ private static boolean isResponsibleFor(final ShardRouting shard) { || shard.recoverySource().getType() == RecoverySource.Type.SNAPSHOT); } + @Override public AllocateUnassignedDecision makeAllocationDecision( final ShardRouting unassignedShard, @@ -112,6 +113,8 @@ public AllocateUnassignedDecision makeAllocationDecision( } final FetchResult shardState = fetchData(unassignedShard, allocation); +// final FetchResult batchFetchResult = fetchBatchData(allocation); + if (shardState.hasData() == false) { allocation.setHasPendingAsyncFetch(); List nodeDecisions = null; @@ -456,6 +459,10 @@ private static NodesToAllocate buildNodesToAllocate( } protected abstract FetchResult fetchData(ShardRouting shard, RoutingAllocation allocation); + protected AsyncBatchShardFetch.AdaptedResultsForShard fetchBatchData(String batchId, RoutingAllocation allocation){ + return null; + } + private static class NodeShardsResult { final List orderedAllocationCandidates; From 49f7ce16b017410a65667e2e4892b81716af0213 Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Tue, 18 Jul 2023 14:46:56 +0530 Subject: [PATCH 03/45] Batcher changes in GA Signed-off-by: Gaurav Chandani --- .../routing/allocation/AllocationService.java | 31 +---- .../allocation/ExistingShardsAllocator.java | 2 +- .../gateway/BaseGatewayShardAllocator.java | 2 +- .../opensearch/gateway/GatewayAllocator.java | 108 ++++++++++++------ .../gateway/PrimaryShardAllocator.java | 4 - 5 files changed, 75 insertions(+), 72 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 6e442d43ac430..ed05e9ff0c81c 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 @@ -547,18 +547,6 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { for (final ExistingShardsAllocator existingShardsAllocator : existingShardsAllocators.values()) { existingShardsAllocator.beforeAllocation(allocation); } - // batch Mode enabled setting to be added - boolean batchModeEnabled = true; - if (batchModeEnabled) { - // since allocators is per index setting, to have batch assignment verify allocators same for all shards - // if not fallback to single assignment - ExistingShardsAllocator allocator = verifySameAllocatorForAllShards(allocation); - if (allocator != null) { - allocator.allocateBatchUnassigned(allocation, true); - - } - } - final RoutingNodes.UnassignedShards.UnassignedIterator primaryIterator = allocation.routingNodes().unassigned().iterator(); while (primaryIterator.hasNext()) { @@ -580,24 +568,7 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { } } } - - private ExistingShardsAllocator verifySameAllocatorForAllShards(RoutingAllocation allocation) { - RoutingNodes.UnassignedShards unassignedShards = allocation.routingNodes().unassigned(); - RoutingNodes.UnassignedShards.UnassignedIterator iterator = unassignedShards.iterator(); - ExistingShardsAllocator currentAllocatorForShard =null; - if (unassignedShards.size() > 0) { - ShardRouting shard = iterator.next(); - currentAllocatorForShard= getAllocatorForShard(shard, allocation); - while (iterator.hasNext()){ - ExistingShardsAllocator allocatorForShard = getAllocatorForShard(iterator.next(), allocation); - if (currentAllocatorForShard.getClass().getName().equals(allocatorForShard.getClass().getName())==false){ - return null; - } - } - } - return currentAllocatorForShard; - } - + private void disassociateDeadNodes(RoutingAllocation allocation) { for (Iterator it = allocation.routingNodes().mutableIterator(); it.hasNext();) { RoutingNode node = it.next(); 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 2013877804305..b715f602b096e 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 @@ -80,7 +80,7 @@ void allocateUnassigned( UnassignedAllocationHandler unassignedAllocationHandler ); - default void allocateBatchUnassigned(RoutingAllocation allocation, boolean primary){ + default void allocateUnassignedBatch(RoutingAllocation allocation, boolean primary){ RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); while (iterator.hasNext()) { ShardRouting shardRouting = iterator.next(); diff --git a/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java b/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java index e418e19353c2c..fd2feefbe8f1f 100644 --- a/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java @@ -92,7 +92,7 @@ public void allocateUnassigned( } } - public void allocateBatchUnassigned(String batchId, RoutingAllocation allocation){ + public void allocateUnassignedBatch(String batchId, RoutingAllocation allocation){ } protected long getExpectedShardSize(ShardRouting shardRouting, RoutingAllocation allocation) { diff --git a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java index ae41036c8d997..23b49824d1d77 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java @@ -57,6 +57,7 @@ import org.opensearch.common.util.set.Sets; import org.opensearch.index.shard.ShardId; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; import java.util.Collections; import java.util.HashMap; @@ -86,7 +87,9 @@ public class GatewayAllocator implements ExistingShardsAllocator { private final ReplicaShardAllocator replicaShardAllocator; private final PrimaryShardAllocator primaryBatchShardAllocator; - private final TransportNodesBatchListGatewayStartedShards batchStartedAction; + private final ReplicaShardAllocator replicaBatchShardAllocator; + private final TransportNodesListGatewayStartedShardsBatch batchStartedAction; + private final TransportNodesListShardStoreMetadataBatch batchStoreAction; private final ConcurrentMap< @@ -97,20 +100,24 @@ public class GatewayAllocator implements ExistingShardsAllocator { ConcurrentCollections.newConcurrentMap(); private Set lastSeenEphemeralIds = Collections.emptySet(); - private final ConcurrentMap> asyncBatchFetchStarted = ConcurrentCollections.newConcurrentMap(); + private final ConcurrentMap> asyncBatchFetchStarted = ConcurrentCollections.newConcurrentMap(); + private final ConcurrentMap> asyncBatchFetchStore = ConcurrentCollections.newConcurrentMap(); @Inject public GatewayAllocator( RerouteService rerouteService, TransportNodesListGatewayStartedShards startedAction, TransportNodesListShardStoreMetadata storeAction, - TransportNodesBatchListGatewayStartedShards batchStartedAction + TransportNodesListGatewayStartedShardsBatch batchStartedAction, + TransportNodesListShardStoreMetadataBatch batchStoreAction ) { this.rerouteService = rerouteService; this.primaryShardAllocator = new InternalPrimaryShardAllocator(startedAction); this.replicaShardAllocator = new InternalReplicaShardAllocator(storeAction); this.batchStartedAction = batchStartedAction; this.primaryBatchShardAllocator = new InternalPrimaryBatchShardAllocator(batchStartedAction); + this.batchStoreAction = batchStoreAction; + this.replicaBatchShardAllocator = new InternalReplicaBatchShardAllocator(batchStoreAction); } @Override @@ -128,6 +135,8 @@ protected GatewayAllocator() { this.replicaShardAllocator = null; this.batchStartedAction=null; this.primaryBatchShardAllocator =null; + this.batchStoreAction = null; + this.replicaBatchShardAllocator = null; } @Override @@ -186,29 +195,31 @@ public void allocateUnassigned( } @Override - public void allocateBatchUnassigned(final RoutingAllocation allocation, boolean primary){ - assert primaryShardAllocator != null; - assert replicaShardAllocator != null; + public void allocateUnassignedBatch(final RoutingAllocation allocation, boolean primary){ + // create batches for unassigned shards + createBatches(allocation, primary); if (primary) { - createBatchesForShardsStarted(allocation); - asyncBatchFetchStarted.keySet().forEach(batch -> primaryBatchShardAllocator.allocateBatchUnassigned(batch.getBatchId(), allocation)); + asyncBatchFetchStarted.keySet().forEach(batch -> primaryBatchShardAllocator.allocateUnassignedBatch(batch.getBatchId(), allocation)); } else { - createBatchesForShardsStore(allocation); - asyncBatchFetchStore.keySet().forEach(batch -> replicaShardAllocator.allocateBatchUnassigned(batch.getBatchId(), allocation)); + asyncBatchFetchStore.keySet().forEach(batch -> replicaBatchShardAllocator.allocateUnassignedBatch(batch.getBatchId(), allocation)); } - } - private void createBatchesForShardsStarted(RoutingAllocation allocation) { + private void createBatches(RoutingAllocation allocation, boolean primary) { RoutingNodes.UnassignedShards unassigned = allocation.routingNodes().unassigned(); // fetch all current batched shards - Set currentBatchedShards = asyncBatchFetchStarted.keySet().stream().flatMap(shardsBatcher -> shardsBatcher.getBatchedShards().stream()).collect(Collectors.toSet()); + Set currentBatchedShards; + if (primary) { + currentBatchedShards = asyncBatchFetchStarted.keySet().stream().flatMap(shardsBatch -> shardsBatch.getBatchedShards().stream()).collect(Collectors.toSet()); + } else { + currentBatchedShards = asyncBatchFetchStore.keySet().stream().flatMap(shardsBatch -> shardsBatch.getBatchedShards().stream()).collect(Collectors.toSet()); + } Set shardsToBatch = Sets.newHashSet(); // add all unassigned shards to the batch if they are not already in a batch unassigned.forEach(shardRouting -> { - if (currentBatchedShards.contains(shardRouting.shardId()) == false && shardRouting.primary()) { + if ((currentBatchedShards.contains(shardRouting.shardId()) == false) && (shardRouting.primary() == primary)) { assert shardRouting.unassigned(); shardsToBatch.add(shardRouting); } @@ -226,19 +237,29 @@ private void createBatchesForShardsStarted(RoutingAllocation allocation) { // add to batch if batch size full or last shard in unassigned list if (batchSize == 0 || iterator.hasNext() == false) { String batchUUId = UUIDs.base64UUID(); - ShardsBatcher shardsBatcher = new ShardsBatcher(batchUUId, addToCurrentBatch); - // add to main asyncShardFetchStarted after Async object ready - - asyncBatchFetchStarted.computeIfAbsent( - shardsBatcher, - batch -> new InternalBatchAsyncFetch<>( - logger, - "batch_shards_started", - batch.getShardsToCustomDataPathMap(), - this.batchStartedAction, - batch.getBatchId() - ) - ); + ShardsBatch shardsBatch = new ShardsBatch(batchUUId, addToCurrentBatch); + if(primary) { + asyncBatchFetchStarted.computeIfAbsent( + shardsBatch, + batch -> new InternalBatchAsyncFetch<>( + logger, + "batch_shards_started", + batch.getShardsToCustomDataPathMap(), + this.batchStartedAction, + batch.getBatchId() + )); + } + else { + asyncBatchFetchStore.computeIfAbsent( + shardsBatch, + batch -> new InternalBatchAsyncFetch<>( + logger, + "batch_shards_store", + batch.getShardsToCustomDataPathMap(), + this.batchStoreAction, + batch.getBatchId() + )); + } addToCurrentBatch.clear(); batchSize = MAX_BATCH_SIZE; } @@ -407,9 +428,9 @@ protected AsyncShardFetch.FetchResult fetchBatchData(String batchId, RoutingAllocation allocation){ - // send adapted data to PSA for processing. - return null; - } } class InternalReplicaShardAllocator extends ReplicaShardAllocator { @@ -464,7 +480,27 @@ protected boolean hasInitiatedFetching(ShardRouting shard) { } } - private class ShardsBatcher { + class InternalReplicaBatchShardAllocator extends ReplicaShardAllocator { + + private final TransportNodesListShardStoreMetadataBatch storeAction; + + InternalReplicaBatchShardAllocator(TransportNodesListShardStoreMetadataBatch storeAction) { + this.storeAction = storeAction; + } + + @Override + protected AsyncShardFetch.FetchResult fetchData(ShardRouting shard, RoutingAllocation allocation) { + return null; + } + + @Override + protected boolean hasInitiatedFetching(ShardRouting shard) { + return false; + } + } + + + private class ShardsBatch { private final String uuid; public Map getShardsToCustomDataPathMap() { @@ -472,7 +508,7 @@ public Map getShardsToCustomDataPathMap() { } private Map shardsToCustomDataPathMap; - private ShardsBatcher(String uuid, Map shardsToCustomDataPathMap) { + private ShardsBatch(String uuid, Map shardsToCustomDataPathMap) { this.uuid = uuid; this.shardsToCustomDataPathMap = shardsToCustomDataPathMap; } diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java index af12123187430..9fe705ab5eb1c 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java @@ -459,10 +459,6 @@ private static NodesToAllocate buildNodesToAllocate( } protected abstract FetchResult fetchData(ShardRouting shard, RoutingAllocation allocation); - protected AsyncBatchShardFetch.AdaptedResultsForShard fetchBatchData(String batchId, RoutingAllocation allocation){ - return null; - } - private static class NodeShardsResult { final List orderedAllocationCandidates; From 9b51dfb65026f04b89cd1d60432775094b9a3aea Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Tue, 18 Jul 2023 16:51:53 +0530 Subject: [PATCH 04/45] Binding of Trasnport Classes to Injector Signed-off-by: Gaurav Chandani --- .../java/org/opensearch/gateway/BaseGatewayShardAllocator.java | 3 --- .../src/main/java/org/opensearch/gateway/GatewayAllocator.java | 1 - server/src/main/java/org/opensearch/gateway/GatewayModule.java | 2 ++ .../java/org/opensearch/gateway/PrimaryShardAllocator.java | 2 -- server/src/main/java/org/opensearch/indices/IndicesModule.java | 2 ++ 5 files changed, 4 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java b/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java index fd2feefbe8f1f..64008d7618b0e 100644 --- a/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java @@ -123,9 +123,6 @@ public abstract AllocateUnassignedDecision makeAllocationDecision( Logger logger ); - /** - * Returns a list of {@link NodeAllocationResult} - /** * Builds decisions for all nodes in the cluster, so that the explain API can provide information on * allocation decisions for each node, while still waiting to allocate the shard (e.g. due to fetching shard data). diff --git a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java index 23b49824d1d77..5acc3429b9b98 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java @@ -91,7 +91,6 @@ public class GatewayAllocator implements ExistingShardsAllocator { private final TransportNodesListGatewayStartedShardsBatch batchStartedAction; private final TransportNodesListShardStoreMetadataBatch batchStoreAction; - private final ConcurrentMap< ShardId, AsyncShardFetch> asyncFetchStarted = ConcurrentCollections diff --git a/server/src/main/java/org/opensearch/gateway/GatewayModule.java b/server/src/main/java/org/opensearch/gateway/GatewayModule.java index 59ec0243c88c9..72f543cb6742e 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayModule.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayModule.java @@ -33,6 +33,7 @@ package org.opensearch.gateway; import org.opensearch.common.inject.AbstractModule; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; /** * Binds the gateway module @@ -47,6 +48,7 @@ protected void configure() { bind(GatewayService.class).asEagerSingleton(); bind(TransportNodesListGatewayMetaState.class).asEagerSingleton(); bind(TransportNodesListGatewayStartedShards.class).asEagerSingleton(); + bind(TransportNodesListGatewayStartedShardsBatch.class).asEagerSingleton(); bind(LocalAllocateDangledIndices.class).asEagerSingleton(); } } diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java index 9fe705ab5eb1c..00680c3ee51f6 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java @@ -113,8 +113,6 @@ public AllocateUnassignedDecision makeAllocationDecision( } final FetchResult shardState = fetchData(unassignedShard, allocation); -// final FetchResult batchFetchResult = fetchBatchData(allocation); - if (shardState.hasData() == false) { allocation.setHasPendingAsyncFetch(); List nodeDecisions = null; diff --git a/server/src/main/java/org/opensearch/indices/IndicesModule.java b/server/src/main/java/org/opensearch/indices/IndicesModule.java index 5a5c79c37a0ac..804e4cad7767e 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesModule.java +++ b/server/src/main/java/org/opensearch/indices/IndicesModule.java @@ -78,6 +78,7 @@ import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.indices.store.IndicesStore; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; import org.opensearch.plugins.MapperPlugin; import java.util.ArrayList; @@ -277,6 +278,7 @@ protected void configure() { bind(IndicesStore.class).asEagerSingleton(); bind(IndicesClusterStateService.class).asEagerSingleton(); bind(TransportNodesListShardStoreMetadata.class).asEagerSingleton(); + bind(TransportNodesListShardStoreMetadataBatch.class).asEagerSingleton(); bind(GlobalCheckpointSyncAction.class).asEagerSingleton(); bind(TransportResyncReplicationAction.class).asEagerSingleton(); bind(PrimaryReplicaSyncer.class).asEagerSingleton(); From 0075bb271351027351833f145c57ccc40c5a3663 Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Wed, 19 Jul 2023 14:07:07 +0530 Subject: [PATCH 05/45] Updated ShardsBatch to take ShardRouting into account instead of ShardIds Signed-off-by: Gaurav Chandani --- .../opensearch/gateway/GatewayAllocator.java | 32 +++++++++++-------- 1 file changed, 18 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java index 5acc3429b9b98..0d2549f43eeaf 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java @@ -199,17 +199,17 @@ public void allocateUnassignedBatch(final RoutingAllocation allocation, boolean createBatches(allocation, primary); if (primary) { - asyncBatchFetchStarted.keySet().forEach(batch -> primaryBatchShardAllocator.allocateUnassignedBatch(batch.getBatchId(), allocation)); + asyncBatchFetchStarted.keySet().forEach(batch -> primaryBatchShardAllocator.allocateUnassignedBatch(batch.getBatchedShards(), allocation)); } else { - asyncBatchFetchStore.keySet().forEach(batch -> replicaBatchShardAllocator.allocateUnassignedBatch(batch.getBatchId(), allocation)); + asyncBatchFetchStore.keySet().forEach(batch -> replicaBatchShardAllocator.allocateUnassignedBatch(batch.getBatchedShards(), allocation)); } } private void createBatches(RoutingAllocation allocation, boolean primary) { RoutingNodes.UnassignedShards unassigned = allocation.routingNodes().unassigned(); // fetch all current batched shards - Set currentBatchedShards; + Set currentBatchedShards; if (primary) { currentBatchedShards = asyncBatchFetchStarted.keySet().stream().flatMap(shardsBatch -> shardsBatch.getBatchedShards().stream()).collect(Collectors.toSet()); } else { @@ -218,18 +218,18 @@ private void createBatches(RoutingAllocation allocation, boolean primary) { Set shardsToBatch = Sets.newHashSet(); // add all unassigned shards to the batch if they are not already in a batch unassigned.forEach(shardRouting -> { - if ((currentBatchedShards.contains(shardRouting.shardId()) == false) && (shardRouting.primary() == primary)) { + if ((currentBatchedShards.contains(shardRouting) == false) && (shardRouting.primary() == primary)) { assert shardRouting.unassigned(); shardsToBatch.add(shardRouting); } }); Iterator iterator = shardsToBatch.iterator(); long batchSize = MAX_BATCH_SIZE; - Map addToCurrentBatch = new HashMap<>(); + Map addToCurrentBatch = new HashMap<>(); while (iterator.hasNext()) { ShardRouting currentShard = iterator.next(); if (batchSize > 0) { - addToCurrentBatch.put(currentShard.shardId(), IndexMetadata.INDEX_DATA_PATH_SETTING.get(allocation.metadata().index(currentShard.index()).getSettings())); + addToCurrentBatch.put(currentShard, IndexMetadata.INDEX_DATA_PATH_SETTING.get(allocation.metadata().index(currentShard.index()).getSettings())); batchSize--; iterator.remove(); } @@ -237,13 +237,17 @@ private void createBatches(RoutingAllocation allocation, boolean primary) { if (batchSize == 0 || iterator.hasNext() == false) { String batchUUId = UUIDs.base64UUID(); ShardsBatch shardsBatch = new ShardsBatch(batchUUId, addToCurrentBatch); + Map shardIdsMap = addToCurrentBatch.entrySet().stream().collect(Collectors.toMap( + entry -> entry.getKey().shardId(), + Map.Entry::getValue + )); if(primary) { asyncBatchFetchStarted.computeIfAbsent( shardsBatch, batch -> new InternalBatchAsyncFetch<>( logger, "batch_shards_started", - batch.getShardsToCustomDataPathMap(), + shardIdsMap, this.batchStartedAction, batch.getBatchId() )); @@ -254,7 +258,7 @@ private void createBatches(RoutingAllocation allocation, boolean primary) { batch -> new InternalBatchAsyncFetch<>( logger, "batch_shards_store", - batch.getShardsToCustomDataPathMap(), + shardIdsMap, this.batchStoreAction, batch.getBatchId() )); @@ -502,20 +506,20 @@ protected boolean hasInitiatedFetching(ShardRouting shard) { private class ShardsBatch { private final String uuid; - public Map getShardsToCustomDataPathMap() { + public Map getShardsToCustomDataPathMap() { return shardsToCustomDataPathMap; } - private Map shardsToCustomDataPathMap; - private ShardsBatch(String uuid, Map shardsToCustomDataPathMap) { + private Map shardsToCustomDataPathMap; + private ShardsBatch(String uuid, Map shardsToCustomDataPathMap) { this.uuid = uuid; this.shardsToCustomDataPathMap = shardsToCustomDataPathMap; } - void removeFromBatch(ShardId shardId) { - shardsToCustomDataPathMap.remove(shardId); + void removeFromBatch(ShardRouting shard) { + shardsToCustomDataPathMap.remove(shard); } - Set getBatchedShards() { + Set getBatchedShards() { return shardsToCustomDataPathMap.keySet(); } From 38205cf8701fed4c2871ec64fa9be76a1236d8c8 Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Tue, 25 Jul 2023 15:56:40 +0530 Subject: [PATCH 06/45] Cleaned Shards from batches after allocation attempt and moved keys to ShardIds in ShardsBatch 1. Cleaning of shards from batches in shard started and shardFailed 2. ShardRouting Equals method have nodeId and unassignedInfo for equals. The state changes after assignment so removing it from batcher class becomes difficult Therefore added ShardsId as map key in Shardsbatch class and new EntryClass for it to hold the values for it Signed-off-by: Gaurav Chandani --- .../opensearch/gateway/GatewayAllocator.java | 231 +++++++++++++----- 1 file changed, 172 insertions(+), 59 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java index 0d2549f43eeaf..be3ae441e2b06 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java @@ -64,6 +64,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.concurrent.ConcurrentMap; import java.util.stream.Collectors; @@ -98,9 +99,10 @@ public class GatewayAllocator implements ExistingShardsAllocator { private final ConcurrentMap> asyncFetchStore = ConcurrentCollections.newConcurrentMap(); private Set lastSeenEphemeralIds = Collections.emptySet(); - - private final ConcurrentMap> asyncBatchFetchStarted = ConcurrentCollections.newConcurrentMap(); - private final ConcurrentMap> asyncBatchFetchStore = ConcurrentCollections.newConcurrentMap(); + private final ConcurrentMap startedShardBatchLookup = ConcurrentCollections.newConcurrentMap(); + private final ConcurrentMap batchIdToStartedShardBatch = ConcurrentCollections.newConcurrentMap(); + private final ConcurrentMap storeShardBatchLookup = ConcurrentCollections.newConcurrentMap(); + private final ConcurrentMap batchIdToStoreShardBatch = ConcurrentCollections.newConcurrentMap(); @Inject public GatewayAllocator( @@ -125,6 +127,10 @@ public void cleanCaches() { asyncFetchStarted.clear(); Releasables.close(asyncFetchStore.values()); asyncFetchStore.clear(); + batchIdToStartedShardBatch.clear(); + batchIdToStoreShardBatch.clear(); + startedShardBatchLookup.clear(); + storeShardBatchLookup.clear(); } // for tests @@ -132,8 +138,8 @@ protected GatewayAllocator() { this.rerouteService = null; this.primaryShardAllocator = null; this.replicaShardAllocator = null; - this.batchStartedAction=null; - this.primaryBatchShardAllocator =null; + this.batchStartedAction = null; + this.primaryBatchShardAllocator = null; this.batchStoreAction = null; this.replicaBatchShardAllocator = null; } @@ -155,6 +161,7 @@ public void applyStartedShards(final List startedShards, final Rou for (ShardRouting startedShard : startedShards) { Releasables.close(asyncFetchStarted.remove(startedShard.shardId())); Releasables.close(asyncFetchStore.remove(startedShard.shardId())); + safelyRemoveShardFromBatch(startedShard); } } @@ -163,6 +170,7 @@ public void applyFailedShards(final List failedShards, final Routin for (FailedShard failedShard : failedShards) { Releasables.close(asyncFetchStarted.remove(failedShard.getRoutingEntry().shardId())); Releasables.close(asyncFetchStore.remove(failedShard.getRoutingEntry().shardId())); + safelyRemoveShardFromBatch(failedShard.getRoutingEntry()); } } @@ -194,81 +202,97 @@ public void allocateUnassigned( } @Override - public void allocateUnassignedBatch(final RoutingAllocation allocation, boolean primary){ + public void allocateUnassignedBatch(final RoutingAllocation allocation, boolean primary) { // create batches for unassigned shards createBatches(allocation, primary); + assert primaryBatchShardAllocator != null; + assert replicaBatchShardAllocator != null; if (primary) { - asyncBatchFetchStarted.keySet().forEach(batch -> primaryBatchShardAllocator.allocateUnassignedBatch(batch.getBatchedShards(), allocation)); - } - else { - asyncBatchFetchStore.keySet().forEach(batch -> replicaBatchShardAllocator.allocateUnassignedBatch(batch.getBatchedShards(), allocation)); + batchIdToStartedShardBatch.values().forEach(shardsBatch -> primaryBatchShardAllocator.allocateUnassignedBatch(shardsBatch.getBatchedShardRoutings(), allocation)); + } else { + batchIdToStoreShardBatch.values().forEach(batch -> replicaBatchShardAllocator.allocateUnassignedBatch(batch.getBatchedShardRoutings(), allocation)); } } private void createBatches(RoutingAllocation allocation, boolean primary) { RoutingNodes.UnassignedShards unassigned = allocation.routingNodes().unassigned(); // fetch all current batched shards - Set currentBatchedShards; - if (primary) { - currentBatchedShards = asyncBatchFetchStarted.keySet().stream().flatMap(shardsBatch -> shardsBatch.getBatchedShards().stream()).collect(Collectors.toSet()); - } else { - currentBatchedShards = asyncBatchFetchStore.keySet().stream().flatMap(shardsBatch -> shardsBatch.getBatchedShards().stream()).collect(Collectors.toSet()); - } + Set currentBatchedShards = primary? startedShardBatchLookup.keySet() : storeShardBatchLookup.keySet(); Set shardsToBatch = Sets.newHashSet(); // add all unassigned shards to the batch if they are not already in a batch unassigned.forEach(shardRouting -> { - if ((currentBatchedShards.contains(shardRouting) == false) && (shardRouting.primary() == primary)) { + if ((currentBatchedShards.contains(shardRouting.shardId()) == false) && (shardRouting.primary() == primary)) { assert shardRouting.unassigned(); shardsToBatch.add(shardRouting); } }); Iterator iterator = shardsToBatch.iterator(); long batchSize = MAX_BATCH_SIZE; - Map addToCurrentBatch = new HashMap<>(); + Map addToCurrentBatch = new HashMap<>(); while (iterator.hasNext()) { ShardRouting currentShard = iterator.next(); if (batchSize > 0) { - addToCurrentBatch.put(currentShard, IndexMetadata.INDEX_DATA_PATH_SETTING.get(allocation.metadata().index(currentShard.index()).getSettings())); + ShardBatchEntry shardBatchEntry = new ShardBatchEntry(IndexMetadata.INDEX_DATA_PATH_SETTING.get(allocation.metadata().index(currentShard.index()).getSettings()) + , currentShard); + addToCurrentBatch.put(currentShard.shardId(), shardBatchEntry); batchSize--; iterator.remove(); } // add to batch if batch size full or last shard in unassigned list if (batchSize == 0 || iterator.hasNext() == false) { String batchUUId = UUIDs.base64UUID(); - ShardsBatch shardsBatch = new ShardsBatch(batchUUId, addToCurrentBatch); - Map shardIdsMap = addToCurrentBatch.entrySet().stream().collect(Collectors.toMap( - entry -> entry.getKey().shardId(), - Map.Entry::getValue - )); - if(primary) { - asyncBatchFetchStarted.computeIfAbsent( - shardsBatch, - batch -> new InternalBatchAsyncFetch<>( - logger, - "batch_shards_started", - shardIdsMap, - this.batchStartedAction, - batch.getBatchId() - )); - } - else { - asyncBatchFetchStore.computeIfAbsent( - shardsBatch, - batch -> new InternalBatchAsyncFetch<>( - logger, - "batch_shards_store", - shardIdsMap, - this.batchStoreAction, - batch.getBatchId() - )); - } + ShardsBatch shardsBatch = new ShardsBatch(batchUUId, addToCurrentBatch, primary); + // add the batch to list of current batches + addBatch(shardsBatch, primary); + addShardsIdsToLookup(addToCurrentBatch.keySet(), batchUUId, primary); addToCurrentBatch.clear(); batchSize = MAX_BATCH_SIZE; } } } + private void addBatch(ShardsBatch shardsBatch, boolean primary) { + ConcurrentMap batches = primary ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; + if (batches.containsKey(shardsBatch.getBatchId())) { + throw new IllegalStateException("Batch already exists. BatchId = " + shardsBatch.getBatchId()); + } + batches.put(shardsBatch.getBatchId(), shardsBatch); + } + + private void addShardsIdsToLookup(Set shards, String batchId, boolean primary) { + ConcurrentMap lookupMap = primary ? startedShardBatchLookup : storeShardBatchLookup; + shards.forEach(shardId -> { + if(lookupMap.containsKey(shardId)){ + throw new IllegalStateException("Shard is already Batched. ShardId = " + shardId + "Batch Id="+ lookupMap.get(shardId)); + } + lookupMap.put(shardId, batchId); + }); + } + + /** + * Safely remove a shard from the appropriate batch. + * If the shard is not in a batch, this is a no-op. + * Cleans the batch if it is empty after removing the shard. + * This method should be called when removing the shard from the batch instead {@link ShardsBatch#removeFromBatch(ShardRouting)} + * so that we can clean up the batch if it is empty and release the fetching resources + * @param shardRouting + */ + private void safelyRemoveShardFromBatch(ShardRouting shardRouting) { + String batchId = shardRouting.primary() ? startedShardBatchLookup.get(shardRouting.shardId()) : storeShardBatchLookup.get(shardRouting.shardId()); + if (batchId == null) { + return; + } + ConcurrentMap batches = shardRouting.primary() ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; + ShardsBatch batch = batches.get(batchId); + batch.removeFromBatch(shardRouting); + // remove the batch if it is empty + if (batch.getBatchedShards().isEmpty()) { + Releasables.close(batch.getAsyncFetcher()); + batches.remove(batchId); + } + } + // allow for testing infra to change shard allocators implementation protected static void innerAllocatedUnassigned( RoutingAllocation allocation, @@ -502,31 +526,120 @@ protected boolean hasInitiatedFetching(ShardRouting shard) { } } - + /** + * Holds information about a batch of shards to be allocated. + * Async fetcher is used to fetch the data for the batch. + */ private class ShardsBatch { - private final String uuid; + private final String batchId; + boolean primary; + + private final AsyncBatchShardFetch asyncBatch; + + private final Map batchInfo; + + public ShardsBatch(String batchId, Map shardsWithInfo, boolean primary) { + this.batchId = batchId; + this.batchInfo = new HashMap<>(shardsWithInfo); + // create a ShardId -> customDataPath map for async fetch + Map shardIdsMap = batchInfo.entrySet().stream().collect(Collectors.toMap( + Map.Entry::getKey, + entry -> entry.getValue().getCustomDataPath() + )); + this.primary = primary; + if (primary) { + asyncBatch = new InternalBatchAsyncFetch<>( + logger, + "batch_shards_started", + shardIdsMap, + batchStartedAction, + batchId); + } else { + asyncBatch = new InternalBatchAsyncFetch<>( + logger, + "batch_shards_started", + shardIdsMap, + batchStoreAction, + batchId); - public Map getShardsToCustomDataPathMap() { - return shardsToCustomDataPathMap; + } } - private Map shardsToCustomDataPathMap; - private ShardsBatch(String uuid, Map shardsToCustomDataPathMap) { - this.uuid = uuid; - this.shardsToCustomDataPathMap = shardsToCustomDataPathMap; + private void removeFromBatch(ShardRouting shard) { + + batchInfo.remove(shard.shardId()); + asyncBatch.shardsToCustomDataPathMap.remove(shard.shardId()); + assert shard.primary() == primary : "Illegal call to delete shard from batch"; + // remove from lookup + if (this.primary) { + startedShardBatchLookup.remove(shard.shardId()); + } else { + storeShardBatchLookup.remove(shard.shardId()); + } + // assert that fetcher and shards are the same as batched shards + assert batchInfo.size() == asyncBatch.shardsToCustomDataPathMap.size() : "Shards size is not equal to fetcher size"; } - void removeFromBatch(ShardRouting shard) { - shardsToCustomDataPathMap.remove(shard); + + Set getBatchedShardRoutings() { + return batchInfo.values().stream().map(ShardBatchEntry::getShardRouting).collect(Collectors.toSet()); } - Set getBatchedShards() { - return shardsToCustomDataPathMap.keySet(); + Set getBatchedShards() { + return batchInfo.keySet(); } public String getBatchId() { - return uuid; + return batchId; + } + + AsyncBatchShardFetch getAsyncFetcher() { + return asyncBatch; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || o instanceof ShardsBatch == false) { + return false; + } + ShardsBatch shardsBatch = (ShardsBatch) o; + return batchId.equals(shardsBatch.getBatchId()) && batchInfo.keySet().equals(shardsBatch.getBatchedShards()); + } + + @Override + public int hashCode() { + return Objects.hash(batchId); + } + + @Override + public String toString() { + return "batchId: " + batchId; } + } + /** + * Holds information about a shard to be allocated in a batch. + */ + private class ShardBatchEntry { + + private final String customDataPath; + private final ShardRouting shardRouting; + + public ShardBatchEntry(String customDataPath, ShardRouting shardRouting) { + this.customDataPath = customDataPath; + this.shardRouting = shardRouting; + } + + public ShardRouting getShardRouting() { + return shardRouting; + } + + public String getCustomDataPath() { + return customDataPath; + } + } } From 9bbba2ebce3be48e1beeffb18b1a5a8b3ecc62b3 Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Wed, 26 Jul 2023 10:51:12 +0530 Subject: [PATCH 07/45] Allocation service changes for batch assignment Signed-off-by: Gaurav Chandani --- .../routing/allocation/AllocationService.java | 39 ++++++++++++++++++- 1 file changed, 38 insertions(+), 1 deletion(-) 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 ed05e9ff0c81c..68df385248ce2 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 @@ -547,6 +547,22 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { for (final ExistingShardsAllocator existingShardsAllocator : existingShardsAllocators.values()) { existingShardsAllocator.beforeAllocation(allocation); } + // batch Mode enabled setting to be added + boolean batchModeEnabled = true; + if (batchModeEnabled) { + // since allocators is per index setting, to have batch assignment verify allocators same for all shards + // if not fallback to single assignment + ExistingShardsAllocator allocator = verifySameAllocatorForAllShards(allocation); + if (allocator != null) { + allocator.allocateUnassignedBatch(allocation, true); + for (final ExistingShardsAllocator existingShardsAllocator : existingShardsAllocators.values()) { + existingShardsAllocator.afterPrimariesBeforeReplicas(allocation); + } + allocator.allocateUnassignedBatch(allocation, false); + return; + } + } + final RoutingNodes.UnassignedShards.UnassignedIterator primaryIterator = allocation.routingNodes().unassigned().iterator(); while (primaryIterator.hasNext()) { @@ -568,7 +584,28 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { } } } - + + private ExistingShardsAllocator verifySameAllocatorForAllShards(RoutingAllocation allocation) { + // if there is a single Allocator set in Allocation Service then use it for all shards + if (existingShardsAllocators.size() == 1) { + return existingShardsAllocators.values().iterator().next(); + } + RoutingNodes.UnassignedShards unassignedShards = allocation.routingNodes().unassigned(); + RoutingNodes.UnassignedShards.UnassignedIterator iterator = unassignedShards.iterator(); + ExistingShardsAllocator currentAllocatorForShard =null; + if (unassignedShards.size() > 0) { + ShardRouting shard = iterator.next(); + currentAllocatorForShard= getAllocatorForShard(shard, allocation); + while (iterator.hasNext()){ + ExistingShardsAllocator allocatorForShard = getAllocatorForShard(iterator.next(), allocation); + if (currentAllocatorForShard.getClass().getName().equals(allocatorForShard.getClass().getName())==false){ + return null; + } + } + } + return currentAllocatorForShard; + } + private void disassociateDeadNodes(RoutingAllocation allocation) { for (Iterator it = allocation.routingNodes().mutableIterator(); it.hasNext();) { RoutingNode node = it.next(); From 818ef1d3c060345eff6030d52aa31528f3183e57 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Wed, 30 Aug 2023 13:18:04 +0530 Subject: [PATCH 08/45] Added in-flight fetches for batchMode Signed-off-by: Shivansh Arora --- .../opensearch/gateway/GatewayAllocator.java | 23 +++++++++++++++---- 1 file changed, 19 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java index be3ae441e2b06..5be4a1b656919 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java @@ -146,12 +146,23 @@ protected GatewayAllocator() { @Override public int getNumberOfInFlightFetches() { + boolean batchModeEnabled = true; int count = 0; - for (AsyncShardFetch fetch : asyncFetchStarted.values()) { - count += fetch.getNumberOfInFlightFetches(); + if (batchModeEnabled) { + for (ShardsBatch batch : batchIdToStartedShardBatch.values()) { + count += batch.getNumberOfInFlightFetches(); + } + for (ShardsBatch batch : batchIdToStoreShardBatch.values()) { + count += batch.getNumberOfInFlightFetches(); + } } - for (AsyncShardFetch fetch : asyncFetchStore.values()) { - count += fetch.getNumberOfInFlightFetches(); + else { + for (AsyncShardFetch fetch : asyncFetchStarted.values()) { + count += fetch.getNumberOfInFlightFetches(); + } + for (AsyncShardFetch fetch : asyncFetchStore.values()) { + count += fetch.getNumberOfInFlightFetches(); + } } return count; } @@ -596,6 +607,10 @@ AsyncBatchShardFetch getAsyncFetcher() { return asyncBatch; } + public int getNumberOfInFlightFetches() { + return asyncBatch.getNumberOfInFlightFetches(); + } + @Override public boolean equals(Object o) { if (this == o) { From 1eabedf28e4b761ebd9c22c3754d1577a2e399a5 Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Wed, 6 Sep 2023 14:10:40 +0530 Subject: [PATCH 09/45] Fixed bugs 1. updating shard routing in batches in accordance with previous behaviour 2. Closing of AsyncShardFetch in cleanCaches() 3. Updated in getNumberOfInflightFetches for batch maps 4. Renamed variable names Signed-off-by: Gaurav Chandani --- .../routing/allocation/AllocationService.java | 1 - .../allocation/ExistingShardsAllocator.java | 2 +- .../gateway/BaseGatewayShardAllocator.java | 3 +- .../opensearch/gateway/GatewayAllocator.java | 265 +++++++++++------- .../org/opensearch/gateway/GatewayModule.java | 1 - .../gateway/PrimaryShardAllocator.java | 1 - 6 files changed, 167 insertions(+), 106 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 68df385248ce2..9e91648f48109 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 @@ -605,7 +605,6 @@ private ExistingShardsAllocator verifySameAllocatorForAllShards(RoutingAllocatio } return currentAllocatorForShard; } - private void disassociateDeadNodes(RoutingAllocation allocation) { for (Iterator it = allocation.routingNodes().mutableIterator(); it.hasNext();) { RoutingNode node = it.next(); 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 b715f602b096e..34cc462f9fe83 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 @@ -80,7 +80,7 @@ void allocateUnassigned( UnassignedAllocationHandler unassignedAllocationHandler ); - default void allocateUnassignedBatch(RoutingAllocation allocation, boolean primary){ + default void allocateUnassignedBatch(RoutingAllocation allocation, boolean primary) { RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); while (iterator.hasNext()) { ShardRouting shardRouting = iterator.next(); diff --git a/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java b/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java index 64008d7618b0e..cca21400cc173 100644 --- a/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java @@ -92,8 +92,7 @@ public void allocateUnassigned( } } - public void allocateUnassignedBatch(String batchId, RoutingAllocation allocation){ - } + public void allocateUnassignedBatch(String batchId, RoutingAllocation allocation) {} protected long getExpectedShardSize(ShardRouting shardRouting, RoutingAllocation allocation) { if (shardRouting.primary()) { diff --git a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java index 5be4a1b656919..5c0e505c3f7af 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java @@ -53,6 +53,8 @@ import org.opensearch.common.UUIDs; import org.opensearch.common.inject.Inject; import org.opensearch.common.lease.Releasables; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Settings; import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.common.util.set.Sets; import org.opensearch.index.shard.ShardId; @@ -61,6 +63,7 @@ import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -80,7 +83,7 @@ public class GatewayAllocator implements ExistingShardsAllocator { public static final String ALLOCATOR_NAME = "gateway_allocator"; private static final Logger logger = LogManager.getLogger(GatewayAllocator.class); - private static final long MAX_BATCH_SIZE = 2000; // will change it to a dynamic setting later + private long maxBatchSize; private final RerouteService rerouteService; @@ -99,26 +102,35 @@ public class GatewayAllocator implements ExistingShardsAllocator { private final ConcurrentMap> asyncFetchStore = ConcurrentCollections.newConcurrentMap(); private Set lastSeenEphemeralIds = Collections.emptySet(); - private final ConcurrentMap startedShardBatchLookup = ConcurrentCollections.newConcurrentMap(); private final ConcurrentMap batchIdToStartedShardBatch = ConcurrentCollections.newConcurrentMap(); - private final ConcurrentMap storeShardBatchLookup = ConcurrentCollections.newConcurrentMap(); private final ConcurrentMap batchIdToStoreShardBatch = ConcurrentCollections.newConcurrentMap(); + // Number of shards we send in one batch to data nodes for fetching metadata + public static final Setting GATEWAY_ALLOCATOR_BATCH_SIZE = Setting.longSetting( + "cluster.allocator.gateway.batch_size", + 2000, + 1, + 10000, + Setting.Property.NodeScope + ); + @Inject public GatewayAllocator( RerouteService rerouteService, TransportNodesListGatewayStartedShards startedAction, TransportNodesListShardStoreMetadata storeAction, TransportNodesListGatewayStartedShardsBatch batchStartedAction, - TransportNodesListShardStoreMetadataBatch batchStoreAction + TransportNodesListShardStoreMetadataBatch batchStoreAction, + Settings settings ) { this.rerouteService = rerouteService; this.primaryShardAllocator = new InternalPrimaryShardAllocator(startedAction); this.replicaShardAllocator = new InternalReplicaShardAllocator(storeAction); this.batchStartedAction = batchStartedAction; - this.primaryBatchShardAllocator = new InternalPrimaryBatchShardAllocator(batchStartedAction); + this.primaryBatchShardAllocator = new InternalPrimaryBatchShardAllocator(); this.batchStoreAction = batchStoreAction; - this.replicaBatchShardAllocator = new InternalReplicaBatchShardAllocator(batchStoreAction); + this.replicaBatchShardAllocator = new InternalReplicaBatchShardAllocator(); + this.maxBatchSize = GATEWAY_ALLOCATOR_BATCH_SIZE.get(settings); } @Override @@ -127,10 +139,14 @@ public void cleanCaches() { asyncFetchStarted.clear(); Releasables.close(asyncFetchStore.values()); asyncFetchStore.clear(); + Releasables.close( + batchIdToStartedShardBatch.values().stream().map(shardsBatch -> shardsBatch.asyncBatch).collect(Collectors.toList()) + ); batchIdToStartedShardBatch.clear(); + Releasables.close( + batchIdToStoreShardBatch.values().stream().map(shardsBatch -> shardsBatch.asyncBatch).collect(Collectors.toList()) + ); batchIdToStoreShardBatch.clear(); - startedShardBatchLookup.clear(); - storeShardBatchLookup.clear(); } // for tests @@ -146,23 +162,20 @@ protected GatewayAllocator() { @Override public int getNumberOfInFlightFetches() { - boolean batchModeEnabled = true; int count = 0; - if (batchModeEnabled) { - for (ShardsBatch batch : batchIdToStartedShardBatch.values()) { - count += batch.getNumberOfInFlightFetches(); - } - for (ShardsBatch batch : batchIdToStoreShardBatch.values()) { - count += batch.getNumberOfInFlightFetches(); - } + // If fetching is done in non batched-mode then maps to maintain batches will be empty and vice versa for batch-mode + for (ShardsBatch batch : batchIdToStartedShardBatch.values()) { + count += (batch.getNumberOfInFlightFetches() * batch.getBatchedShards().size()); } - else { - for (AsyncShardFetch fetch : asyncFetchStarted.values()) { - count += fetch.getNumberOfInFlightFetches(); - } - for (AsyncShardFetch fetch : asyncFetchStore.values()) { - count += fetch.getNumberOfInFlightFetches(); - } + for (ShardsBatch batch : batchIdToStoreShardBatch.values()) { + count += (batch.getNumberOfInFlightFetches() * batch.getBatchedShards().size()); + } + + for (AsyncShardFetch fetch : asyncFetchStarted.values()) { + count += fetch.getNumberOfInFlightFetches(); + } + for (AsyncShardFetch fetch : asyncFetchStore.values()) { + count += fetch.getNumberOfInFlightFetches(); } return count; } @@ -172,7 +185,7 @@ public void applyStartedShards(final List startedShards, final Rou for (ShardRouting startedShard : startedShards) { Releasables.close(asyncFetchStarted.remove(startedShard.shardId())); Releasables.close(asyncFetchStore.remove(startedShard.shardId())); - safelyRemoveShardFromBatch(startedShard); + safelyRemoveShardFromBothBatch(startedShard); } } @@ -181,7 +194,7 @@ public void applyFailedShards(final List failedShards, final Routin for (FailedShard failedShard : failedShards) { Releasables.close(asyncFetchStarted.remove(failedShard.getRoutingEntry().shardId())); Releasables.close(asyncFetchStore.remove(failedShard.getRoutingEntry().shardId())); - safelyRemoveShardFromBatch(failedShard.getRoutingEntry()); + safelyRemoveShardFromBothBatch(failedShard.getRoutingEntry()); } } @@ -215,21 +228,36 @@ public void allocateUnassigned( @Override public void allocateUnassignedBatch(final RoutingAllocation allocation, boolean primary) { // create batches for unassigned shards - createBatches(allocation, primary); + Set batchesToAssign = createAndUpdateBatches(allocation, primary); assert primaryBatchShardAllocator != null; assert replicaBatchShardAllocator != null; if (primary) { - batchIdToStartedShardBatch.values().forEach(shardsBatch -> primaryBatchShardAllocator.allocateUnassignedBatch(shardsBatch.getBatchedShardRoutings(), allocation)); + batchIdToStartedShardBatch.values() + .stream() + .filter(batch -> batchesToAssign.contains(batch.batchId)) + .forEach( + shardsBatch -> primaryBatchShardAllocator.allocateUnassignedBatch(shardsBatch.getBatchedShardRoutings(), allocation) + ); } else { - batchIdToStoreShardBatch.values().forEach(batch -> replicaBatchShardAllocator.allocateUnassignedBatch(batch.getBatchedShardRoutings(), allocation)); + batchIdToStoreShardBatch.values() + .stream() + .filter(batch -> batchesToAssign.contains(batch.batchId)) + .forEach(batch -> replicaBatchShardAllocator.allocateUnassignedBatch(batch.getBatchedShardRoutings(), allocation)); } } - private void createBatches(RoutingAllocation allocation, boolean primary) { + private Set createAndUpdateBatches(RoutingAllocation allocation, boolean primary) { + Set batchesToBeAssigned = new HashSet<>(); RoutingNodes.UnassignedShards unassigned = allocation.routingNodes().unassigned(); - // fetch all current batched shards - Set currentBatchedShards = primary? startedShardBatchLookup.keySet() : storeShardBatchLookup.keySet(); + ConcurrentMap currentBatches = primary ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; + // get all batched shards + Set currentBatchedShards = currentBatches.values() + .stream() + .map(ShardsBatch::getBatchedShards) + .flatMap(Set::stream) + .collect(Collectors.toSet()); + Set shardsToBatch = Sets.newHashSet(); // add all unassigned shards to the batch if they are not already in a batch unassigned.forEach(shardRouting -> { @@ -237,16 +265,24 @@ private void createBatches(RoutingAllocation allocation, boolean primary) { assert shardRouting.unassigned(); shardsToBatch.add(shardRouting); } + // if shard is already batched, update to latest shardRouting information in the batches + else if (shardRouting.primary() == primary) { + String batchId = getBatchId(shardRouting, shardRouting.primary()); + batchesToBeAssigned.add(batchId); + currentBatches.get(batchId).batchInfo.get(shardRouting.shardId()).setShardRouting(shardRouting); + } }); Iterator iterator = shardsToBatch.iterator(); - long batchSize = MAX_BATCH_SIZE; - Map addToCurrentBatch = new HashMap<>(); + long batchSize = maxBatchSize; + Map addToCurrentBatch = new HashMap<>(); while (iterator.hasNext()) { ShardRouting currentShard = iterator.next(); if (batchSize > 0) { - ShardBatchEntry shardBatchEntry = new ShardBatchEntry(IndexMetadata.INDEX_DATA_PATH_SETTING.get(allocation.metadata().index(currentShard.index()).getSettings()) - , currentShard); - addToCurrentBatch.put(currentShard.shardId(), shardBatchEntry); + SharEntry sharEntry = new SharEntry( + IndexMetadata.INDEX_DATA_PATH_SETTING.get(allocation.metadata().index(currentShard.index()).getSettings()), + currentShard + ); + addToCurrentBatch.put(currentShard.shardId(), sharEntry); batchSize--; iterator.remove(); } @@ -256,11 +292,12 @@ private void createBatches(RoutingAllocation allocation, boolean primary) { ShardsBatch shardsBatch = new ShardsBatch(batchUUId, addToCurrentBatch, primary); // add the batch to list of current batches addBatch(shardsBatch, primary); - addShardsIdsToLookup(addToCurrentBatch.keySet(), batchUUId, primary); + batchesToBeAssigned.add(batchUUId); addToCurrentBatch.clear(); - batchSize = MAX_BATCH_SIZE; + batchSize = maxBatchSize; } } + return batchesToBeAssigned; } private void addBatch(ShardsBatch shardsBatch, boolean primary) { @@ -271,26 +308,17 @@ private void addBatch(ShardsBatch shardsBatch, boolean primary) { batches.put(shardsBatch.getBatchId(), shardsBatch); } - private void addShardsIdsToLookup(Set shards, String batchId, boolean primary) { - ConcurrentMap lookupMap = primary ? startedShardBatchLookup : storeShardBatchLookup; - shards.forEach(shardId -> { - if(lookupMap.containsKey(shardId)){ - throw new IllegalStateException("Shard is already Batched. ShardId = " + shardId + "Batch Id="+ lookupMap.get(shardId)); - } - lookupMap.put(shardId, batchId); - }); - } - /** - * Safely remove a shard from the appropriate batch. + * Safely remove a shard from the appropriate batch depending on if it is primary or replica * If the shard is not in a batch, this is a no-op. * Cleans the batch if it is empty after removing the shard. * This method should be called when removing the shard from the batch instead {@link ShardsBatch#removeFromBatch(ShardRouting)} * so that we can clean up the batch if it is empty and release the fetching resources - * @param shardRouting + * + * @param shardRouting shard to be removed */ private void safelyRemoveShardFromBatch(ShardRouting shardRouting) { - String batchId = shardRouting.primary() ? startedShardBatchLookup.get(shardRouting.shardId()) : storeShardBatchLookup.get(shardRouting.shardId()); + String batchId = shardRouting.primary() ? getBatchId(shardRouting, true) : getBatchId(shardRouting, false); if (batchId == null) { return; } @@ -304,6 +332,49 @@ private void safelyRemoveShardFromBatch(ShardRouting shardRouting) { } } + /** + * Safely remove shard from both the batches irrespective of its primary or replica, + * For the corresponding shardId. The method intends to clean up the batch if it is empty + * after removing the shard + * @param shardRouting shard to remove + */ + private void safelyRemoveShardFromBothBatch(ShardRouting shardRouting) { + String primaryBatchId = getBatchId(shardRouting, true); + String replicaBatchId = getBatchId(shardRouting, false); + if (primaryBatchId == null && replicaBatchId == null) { + return; + } + if (primaryBatchId != null) { + ShardsBatch batch = batchIdToStartedShardBatch.get(primaryBatchId); + batch.removeFromBatch(shardRouting); + // remove the batch if it is empty + if (batch.getBatchedShards().isEmpty()) { + Releasables.close(batch.getAsyncFetcher()); + batchIdToStartedShardBatch.remove(primaryBatchId); + } + } + if (replicaBatchId != null) { + ShardsBatch batch = batchIdToStoreShardBatch.get(replicaBatchId); + batch.removeFromBatch(shardRouting); + // remove the batch if it is empty + if (batch.getBatchedShards().isEmpty()) { + Releasables.close(batch.getAsyncFetcher()); + batchIdToStoreShardBatch.remove(replicaBatchId); + } + } + } + + private String getBatchId(ShardRouting shardRouting, boolean primary) { + ConcurrentMap batches = primary ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; + + return batches.entrySet() + .stream() + .filter(entry -> entry.getValue().getBatchedShards().contains(shardRouting.shardId())) + .findFirst() + .map(Map.Entry::getKey) + .orElse(null); + } + // allow for testing infra to change shard allocators implementation protected static void innerAllocatedUnassigned( RoutingAllocation allocation, @@ -404,27 +475,28 @@ protected void reroute(ShardId shardId, String reason) { } } - class InternalBatchAsyncFetch extends AsyncBatchShardFetch { + class InternalBatchAsyncFetch extends AsyncShardFetch { - InternalBatchAsyncFetch(Logger logger, - String type, - Map map, - AsyncBatchShardFetch.Lister, T> action, - String batchUUId + InternalBatchAsyncFetch( + Logger logger, + String type, + Map map, + AsyncShardFetch.Lister, T> action, + String batchUUId ) { super(logger, type, map, action, batchUUId); } @Override - protected void reroute(String batchUUId, String reason) { - logger.trace("{} scheduling reroute for {}", batchUUId, reason); + protected void reroute(String logKey, String reason) { + logger.trace("{} scheduling reroute for {}", logKey, reason); assert rerouteService != null; rerouteService.reroute( - "async_shard_fetch", + "async_shard_batch_fetch", Priority.HIGH, ActionListener.wrap( - r -> logger.trace("{} scheduled reroute completed for {}", batchUUId, reason), - e -> logger.debug(new ParameterizedMessage("{} scheduled reroute failed for {}", batchUUId, reason), e) + r -> logger.trace("{} scheduled reroute completed for {}", logKey, reason), + e -> logger.debug(new ParameterizedMessage("{} scheduled reroute failed for {}", logKey, reason), e) ) ); } @@ -473,7 +545,10 @@ class InternalPrimaryBatchShardAllocator extends PrimaryShardAllocator { } @Override - protected AsyncShardFetch.FetchResult fetchData(ShardRouting shard, RoutingAllocation allocation) { + protected AsyncShardFetch.FetchResult fetchData( + ShardRouting shard, + RoutingAllocation allocation + ) { return null; } @@ -527,7 +602,10 @@ class InternalReplicaBatchShardAllocator extends ReplicaShardAllocator { } @Override - protected AsyncShardFetch.FetchResult fetchData(ShardRouting shard, RoutingAllocation allocation) { + protected AsyncShardFetch.FetchResult fetchData( + ShardRouting shard, + RoutingAllocation allocation + ) { return null; } @@ -543,56 +621,37 @@ protected boolean hasInitiatedFetching(ShardRouting shard) { */ private class ShardsBatch { private final String batchId; - boolean primary; + private final boolean primary; - private final AsyncBatchShardFetch asyncBatch; + private final AsyncShardFetch asyncBatch; - private final Map batchInfo; + private final Map batchInfo; - public ShardsBatch(String batchId, Map shardsWithInfo, boolean primary) { + public ShardsBatch(String batchId, Map shardsWithInfo, boolean primary) { this.batchId = batchId; this.batchInfo = new HashMap<>(shardsWithInfo); // create a ShardId -> customDataPath map for async fetch - Map shardIdsMap = batchInfo.entrySet().stream().collect(Collectors.toMap( - Map.Entry::getKey, - entry -> entry.getValue().getCustomDataPath() - )); + Map shardIdsMap = batchInfo.entrySet() + .stream() + .collect(Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().getCustomDataPath())); this.primary = primary; if (primary) { - asyncBatch = new InternalBatchAsyncFetch<>( - logger, - "batch_shards_started", - shardIdsMap, - batchStartedAction, - batchId); + asyncBatch = new InternalBatchAsyncFetch<>(logger, "batch_shards_started", shardIdsMap, batchStartedAction, batchId); } else { - asyncBatch = new InternalBatchAsyncFetch<>( - logger, - "batch_shards_started", - shardIdsMap, - batchStoreAction, - batchId); + asyncBatch = new InternalBatchAsyncFetch<>(logger, "batch_shards_started", shardIdsMap, batchStoreAction, batchId); } } - private void removeFromBatch(ShardRouting shard) { - + public void removeFromBatch(ShardRouting shard) { batchInfo.remove(shard.shardId()); - asyncBatch.shardsToCustomDataPathMap.remove(shard.shardId()); - assert shard.primary() == primary : "Illegal call to delete shard from batch"; - // remove from lookup - if (this.primary) { - startedShardBatchLookup.remove(shard.shardId()); - } else { - storeShardBatchLookup.remove(shard.shardId()); - } + asyncBatch.shardToCustomDataPath.remove(shard.shardId()); // assert that fetcher and shards are the same as batched shards - assert batchInfo.size() == asyncBatch.shardsToCustomDataPathMap.size() : "Shards size is not equal to fetcher size"; + assert batchInfo.size() == asyncBatch.shardToCustomDataPath.size() : "Shards size is not equal to fetcher size"; } Set getBatchedShardRoutings() { - return batchInfo.values().stream().map(ShardBatchEntry::getShardRouting).collect(Collectors.toSet()); + return batchInfo.values().stream().map(SharEntry::getShardRouting).collect(Collectors.toSet()); } Set getBatchedShards() { @@ -603,7 +662,7 @@ public String getBatchId() { return batchId; } - AsyncBatchShardFetch getAsyncFetcher() { + AsyncShardFetch getAsyncFetcher() { return asyncBatch; } @@ -638,12 +697,18 @@ public String toString() { /** * Holds information about a shard to be allocated in a batch. */ - private class ShardBatchEntry { + private class SharEntry { private final String customDataPath; - private final ShardRouting shardRouting; - public ShardBatchEntry(String customDataPath, ShardRouting shardRouting) { + public SharEntry setShardRouting(ShardRouting shardRouting) { + this.shardRouting = shardRouting; + return this; + } + + private ShardRouting shardRouting; + + public SharEntry(String customDataPath, ShardRouting shardRouting) { this.customDataPath = customDataPath; this.shardRouting = shardRouting; } diff --git a/server/src/main/java/org/opensearch/gateway/GatewayModule.java b/server/src/main/java/org/opensearch/gateway/GatewayModule.java index 72f543cb6742e..847ba01737332 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayModule.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayModule.java @@ -33,7 +33,6 @@ package org.opensearch.gateway; import org.opensearch.common.inject.AbstractModule; -import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; /** * Binds the gateway module diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java index 00680c3ee51f6..4dc9396751fc9 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java @@ -89,7 +89,6 @@ private static boolean isResponsibleFor(final ShardRouting shard) { || shard.recoverySource().getType() == RecoverySource.Type.SNAPSHOT); } - @Override public AllocateUnassignedDecision makeAllocationDecision( final ShardRouting unassignedShard, From 80afedfde85c94fc5e31f497a1abd0d6463dfc77 Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Thu, 28 Sep 2023 13:19:33 +0530 Subject: [PATCH 10/45] Fixed typos and added tests Signed-off-by: Gaurav Chandani Signed-off-by: Shivansh Arora --- .../gateway/RecoveryFromGatewayIT.java | 174 +++++++++ .../opensearch/gateway/GatewayAllocator.java | 119 +++--- .../gateway/GatewayAllocatorTests.java | 352 ++++++++++++++++++ .../test/gateway/TestGatewayAllocator.java | 84 +++++ 4 files changed, 685 insertions(+), 44 deletions(-) create mode 100644 server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 11af1fb3cbfab..375e3c4962b12 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -38,10 +38,14 @@ import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsRequest; import org.opensearch.action.admin.cluster.configuration.ClearVotingConfigExclusionsAction; import org.opensearch.action.admin.cluster.configuration.ClearVotingConfigExclusionsRequest; +import org.opensearch.action.admin.cluster.health.ClusterHealthResponse; +import org.opensearch.action.admin.cluster.reroute.ClusterRerouteResponse; import org.opensearch.action.admin.indices.recovery.RecoveryResponse; import org.opensearch.action.admin.indices.stats.IndexStats; +import org.opensearch.action.admin.indices.stats.IndicesStatsResponse; import org.opensearch.action.admin.indices.stats.ShardStats; import org.opensearch.action.support.ActionTestUtils; +import org.opensearch.client.Requests; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.coordination.ElectionSchedulerFactory; import org.opensearch.cluster.metadata.IndexMetadata; @@ -85,6 +89,8 @@ import java.util.stream.IntStream; import static org.opensearch.cluster.coordination.ClusterBootstrapService.INITIAL_CLUSTER_MANAGER_NODES_SETTING; +import static org.opensearch.cluster.health.ClusterHealthStatus.GREEN; +import static org.opensearch.cluster.health.ClusterHealthStatus.RED; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS; import static org.opensearch.common.xcontent.XContentFactory.jsonBuilder; @@ -740,4 +746,172 @@ public void testMessyElectionsStillMakeClusterGoGreen() throws Exception { internalCluster().fullRestart(); ensureGreen("test"); } + + public void testBatchModeEnabled() throws Exception { + internalCluster().startClusterManagerOnlyNodes(1); + 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().stopRandomDataNode(); + internalCluster().stopRandomDataNode(); + ensureRed("test"); + ensureStableCluster(1); + + logger.info("--> Now do a protective reroute"); + ClusterRerouteResponse clusterRerouteResponse = client().admin().cluster().prepareReroute().setRetryFailed(true).get(); + assertTrue(clusterRerouteResponse.isAcknowledged()); + + GatewayAllocator gatewayAllocator = internalCluster().getInstance( + GatewayAllocator.class, + internalCluster().getClusterManagerName() + ); + assertEquals(1, gatewayAllocator.getNumberOfStartedShardBatches()); + assertEquals(1, gatewayAllocator.getNumberOfStoreShardBatches()); + assertTrue( + ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE_ENABLED.get(internalCluster().clusterService().getSettings()) + ); + + // 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()); + } + + public void testBatchModeDisabled() throws Exception { + internalCluster().startClusterManagerOnlyNodes( + 1, + Settings.builder().put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE_ENABLED.getKey(), false).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().stopRandomDataNode(); + internalCluster().stopRandomDataNode(); + ensureStableCluster(1); + + logger.info("--> Now do a protective reroute"); + ClusterRerouteResponse clusterRerouteResponse = client().admin().cluster().prepareReroute().setRetryFailed(true).get(); + assertTrue(clusterRerouteResponse.isAcknowledged()); + + GatewayAllocator gatewayAllocator = internalCluster().getInstance( + GatewayAllocator.class, + internalCluster().getClusterManagerName() + ); + ensureRed("test"); + + // assert no batches created + assertEquals(0, gatewayAllocator.getNumberOfStartedShardBatches()); + assertEquals(0, gatewayAllocator.getNumberOfStoreShardBatches()); + + 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"); + } + + public void testNBatchesCreationAndAssignment() throws Exception { + // we will reduce batch size to 5 to make sure we have enough batches to test assignment + // Total number of primary shards = 50 (50 indices*1) + // Total number of replica shards = 50 (50 indices*1) + // Total batches creation for primaries and replicas will be 10 each + + internalCluster().startClusterManagerOnlyNodes(1); + List dataOnlyNodes = internalCluster().startDataOnlyNodes(2); + createNIndices(50, "test"); + 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()); + + // Now we will first stop cluster manager node and then stop data nodes. This will ensure to avoid any scenarios + // of more number of batch creation. + 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().stopRandomDataNode(); + internalCluster().stopRandomDataNode(); + + // Now start cluster manager node and post that verify batches created + internalCluster().startClusterManagerOnlyNodes( + 1, + Settings.builder() + .put("node.name", clusterManagerName) + .put(clusterManagerDataPathSettings) + .put(GatewayAllocator.GATEWAY_ALLOCATOR_BATCH_SIZE.getKey(), 5) + .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()); + + GatewayAllocator gatewayAllocator = internalCluster().getInstance( + GatewayAllocator.class, + internalCluster().getClusterManagerName() + ); + 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()); + } + + private void createNIndices(int n, String prefix) { + + for (int i = 0; i < n; i++) { + createIndex( + prefix + i, + Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).build() + ); + ensureGreen(prefix + i); + } + } } diff --git a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java index 5c0e505c3f7af..482c9d3159639 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java @@ -83,16 +83,18 @@ public class GatewayAllocator implements ExistingShardsAllocator { public static final String ALLOCATOR_NAME = "gateway_allocator"; private static final Logger logger = LogManager.getLogger(GatewayAllocator.class); - private long maxBatchSize; + private final long maxBatchSize; + + private static final short DEFAULT_BATCH_SIZE = 2000; private final RerouteService rerouteService; private final PrimaryShardAllocator primaryShardAllocator; private final ReplicaShardAllocator replicaShardAllocator; - private final PrimaryShardAllocator primaryBatchShardAllocator; - private final ReplicaShardAllocator replicaBatchShardAllocator; - private final TransportNodesListGatewayStartedShardsBatch batchStartedAction; + private final PrimaryShardBatchAllocator primaryBatchShardAllocator; + private final ReplicaShardBatchAllocator replicaBatchShardAllocator; + private final TransportNodesListGatewayStartedBatchShards batchStartedAction; private final TransportNodesListShardStoreMetadataBatch batchStoreAction; private final ConcurrentMap< @@ -102,13 +104,17 @@ public class GatewayAllocator implements ExistingShardsAllocator { private final ConcurrentMap> asyncFetchStore = ConcurrentCollections.newConcurrentMap(); private Set lastSeenEphemeralIds = Collections.emptySet(); - private final ConcurrentMap batchIdToStartedShardBatch = ConcurrentCollections.newConcurrentMap(); - private final ConcurrentMap batchIdToStoreShardBatch = ConcurrentCollections.newConcurrentMap(); + + // visble for testing + protected final ConcurrentMap batchIdToStartedShardBatch = ConcurrentCollections.newConcurrentMap(); + + // visible for testing + protected final ConcurrentMap batchIdToStoreShardBatch = ConcurrentCollections.newConcurrentMap(); // Number of shards we send in one batch to data nodes for fetching metadata public static final Setting GATEWAY_ALLOCATOR_BATCH_SIZE = Setting.longSetting( "cluster.allocator.gateway.batch_size", - 2000, + DEFAULT_BATCH_SIZE, 1, 10000, Setting.Property.NodeScope @@ -119,7 +125,7 @@ public GatewayAllocator( RerouteService rerouteService, TransportNodesListGatewayStartedShards startedAction, TransportNodesListShardStoreMetadata storeAction, - TransportNodesListGatewayStartedShardsBatch batchStartedAction, + TransportNodesListGatewayStartedBatchShards batchStartedAction, TransportNodesListShardStoreMetadataBatch batchStoreAction, Settings settings ) { @@ -158,6 +164,7 @@ protected GatewayAllocator() { this.primaryBatchShardAllocator = null; this.batchStoreAction = null; this.replicaBatchShardAllocator = null; + this.maxBatchSize = DEFAULT_BATCH_SIZE; } @Override @@ -202,6 +209,8 @@ public void applyFailedShards(final List failedShards, final Routin public void beforeAllocation(final RoutingAllocation allocation) { assert primaryShardAllocator != null; assert replicaShardAllocator != null; + assert primaryBatchShardAllocator != null; + assert replicaBatchShardAllocator != null; ensureAsyncFetchStorePrimaryRecency(allocation); } @@ -227,11 +236,23 @@ public void allocateUnassigned( @Override public void allocateUnassignedBatch(final RoutingAllocation allocation, boolean primary) { - // create batches for unassigned shards - Set batchesToAssign = createAndUpdateBatches(allocation, primary); assert primaryBatchShardAllocator != null; assert replicaBatchShardAllocator != null; + innerAllocateUnassignedBatch(allocation, primaryBatchShardAllocator, replicaBatchShardAllocator, primary); + } + + protected void innerAllocateUnassignedBatch( + RoutingAllocation allocation, + PrimaryShardBatchAllocator primaryBatchShardAllocator, + ReplicaShardBatchAllocator replicaBatchShardAllocator, + boolean primary + ) { + // create batches for unassigned shards + Set batchesToAssign = createAndUpdateBatches(allocation, primary); + if (batchesToAssign.isEmpty()) { + return; + } if (primary) { batchIdToStartedShardBatch.values() .stream() @@ -247,7 +268,8 @@ public void allocateUnassignedBatch(final RoutingAllocation allocation, boolean } } - private Set createAndUpdateBatches(RoutingAllocation allocation, boolean primary) { + // visible for testing + protected Set createAndUpdateBatches(RoutingAllocation allocation, boolean primary) { Set batchesToBeAssigned = new HashSet<>(); RoutingNodes.UnassignedShards unassigned = allocation.routingNodes().unassigned(); ConcurrentMap currentBatches = primary ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; @@ -265,7 +287,7 @@ private Set createAndUpdateBatches(RoutingAllocation allocation, boolean assert shardRouting.unassigned(); shardsToBatch.add(shardRouting); } - // if shard is already batched, update to latest shardRouting information in the batches + // if shard is already batched update to latest shardRouting information in the batches else if (shardRouting.primary() == primary) { String batchId = getBatchId(shardRouting, shardRouting.primary()); batchesToBeAssigned.add(batchId); @@ -273,12 +295,14 @@ else if (shardRouting.primary() == primary) { } }); Iterator iterator = shardsToBatch.iterator(); + assert maxBatchSize > 0 : "Shards batch size must be greater than 0"; + long batchSize = maxBatchSize; - Map addToCurrentBatch = new HashMap<>(); + Map addToCurrentBatch = new HashMap<>(); while (iterator.hasNext()) { ShardRouting currentShard = iterator.next(); if (batchSize > 0) { - SharEntry sharEntry = new SharEntry( + ShardEntry sharEntry = new ShardEntry( IndexMetadata.INDEX_DATA_PATH_SETTING.get(allocation.metadata().index(currentShard.index()).getSettings()), currentShard ); @@ -317,7 +341,7 @@ private void addBatch(ShardsBatch shardsBatch, boolean primary) { * * @param shardRouting shard to be removed */ - private void safelyRemoveShardFromBatch(ShardRouting shardRouting) { + protected void safelyRemoveShardFromBatch(ShardRouting shardRouting) { String batchId = shardRouting.primary() ? getBatchId(shardRouting, true) : getBatchId(shardRouting, false); if (batchId == null) { return; @@ -325,11 +349,7 @@ private void safelyRemoveShardFromBatch(ShardRouting shardRouting) { ConcurrentMap batches = shardRouting.primary() ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; ShardsBatch batch = batches.get(batchId); batch.removeFromBatch(shardRouting); - // remove the batch if it is empty - if (batch.getBatchedShards().isEmpty()) { - Releasables.close(batch.getAsyncFetcher()); - batches.remove(batchId); - } + deleteBatchIfEmpty(batches, batchId); } /** @@ -338,7 +358,7 @@ private void safelyRemoveShardFromBatch(ShardRouting shardRouting) { * after removing the shard * @param shardRouting shard to remove */ - private void safelyRemoveShardFromBothBatch(ShardRouting shardRouting) { + protected void safelyRemoveShardFromBothBatch(ShardRouting shardRouting) { String primaryBatchId = getBatchId(shardRouting, true); String replicaBatchId = getBatchId(shardRouting, false); if (primaryBatchId == null && replicaBatchId == null) { @@ -347,24 +367,26 @@ private void safelyRemoveShardFromBothBatch(ShardRouting shardRouting) { if (primaryBatchId != null) { ShardsBatch batch = batchIdToStartedShardBatch.get(primaryBatchId); batch.removeFromBatch(shardRouting); - // remove the batch if it is empty - if (batch.getBatchedShards().isEmpty()) { - Releasables.close(batch.getAsyncFetcher()); - batchIdToStartedShardBatch.remove(primaryBatchId); - } + deleteBatchIfEmpty(batchIdToStartedShardBatch, primaryBatchId); } if (replicaBatchId != null) { ShardsBatch batch = batchIdToStoreShardBatch.get(replicaBatchId); batch.removeFromBatch(shardRouting); - // remove the batch if it is empty + deleteBatchIfEmpty(batchIdToStoreShardBatch, replicaBatchId); + } + } + + private void deleteBatchIfEmpty(ConcurrentMap batches, String batchId) { + if (batches.containsKey(batchId)) { + ShardsBatch batch = batches.get(batchId); if (batch.getBatchedShards().isEmpty()) { Releasables.close(batch.getAsyncFetcher()); - batchIdToStoreShardBatch.remove(replicaBatchId); + batches.remove(batchId); } } } - private String getBatchId(ShardRouting shardRouting, boolean primary) { + protected String getBatchId(ShardRouting shardRouting, boolean primary) { ConcurrentMap batches = primary ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; return batches.entrySet() @@ -461,15 +483,15 @@ class InternalAsyncFetch extends AsyncShardFetch } @Override - protected void reroute(ShardId shardId, String reason) { - logger.trace("{} scheduling reroute for {}", shardId, reason); + protected void reroute(String logKey, String reason) { + logger.trace("{} scheduling reroute for {}", logKey, reason); assert rerouteService != null; rerouteService.reroute( "async_shard_fetch", Priority.HIGH, ActionListener.wrap( - r -> logger.trace("{} scheduled reroute completed for {}", shardId, reason), - e -> logger.debug(new ParameterizedMessage("{} scheduled reroute failed for {}", shardId, reason), e) + r -> logger.trace("{} scheduled reroute completed for {}", logKey, reason), + e -> logger.debug(new ParameterizedMessage("{} scheduled reroute failed for {}", logKey, reason), e) ) ); } @@ -618,16 +640,18 @@ protected boolean hasInitiatedFetching(ShardRouting shard) { /** * Holds information about a batch of shards to be allocated. * Async fetcher is used to fetch the data for the batch. + * + * Visible for testing */ - private class ShardsBatch { + public class ShardsBatch { private final String batchId; private final boolean primary; private final AsyncShardFetch asyncBatch; - private final Map batchInfo; + private final Map batchInfo; - public ShardsBatch(String batchId, Map shardsWithInfo, boolean primary) { + public ShardsBatch(String batchId, Map shardsWithInfo, boolean primary) { this.batchId = batchId; this.batchInfo = new HashMap<>(shardsWithInfo); // create a ShardId -> customDataPath map for async fetch @@ -643,18 +667,18 @@ public ShardsBatch(String batchId, Map shardsWithInfo, boole } } - public void removeFromBatch(ShardRouting shard) { + private void removeFromBatch(ShardRouting shard) { batchInfo.remove(shard.shardId()); asyncBatch.shardToCustomDataPath.remove(shard.shardId()); // assert that fetcher and shards are the same as batched shards assert batchInfo.size() == asyncBatch.shardToCustomDataPath.size() : "Shards size is not equal to fetcher size"; } - Set getBatchedShardRoutings() { - return batchInfo.values().stream().map(SharEntry::getShardRouting).collect(Collectors.toSet()); + public Set getBatchedShardRoutings() { + return batchInfo.values().stream().map(ShardEntry::getShardRouting).collect(Collectors.toSet()); } - Set getBatchedShards() { + public Set getBatchedShards() { return batchInfo.keySet(); } @@ -662,7 +686,7 @@ public String getBatchId() { return batchId; } - AsyncShardFetch getAsyncFetcher() { + public AsyncShardFetch getAsyncFetcher() { return asyncBatch; } @@ -697,18 +721,18 @@ public String toString() { /** * Holds information about a shard to be allocated in a batch. */ - private class SharEntry { + private class ShardEntry { private final String customDataPath; - public SharEntry setShardRouting(ShardRouting shardRouting) { + public ShardEntry setShardRouting(ShardRouting shardRouting) { this.shardRouting = shardRouting; return this; } private ShardRouting shardRouting; - public SharEntry(String customDataPath, ShardRouting shardRouting) { + public ShardEntry(String customDataPath, ShardRouting shardRouting) { this.customDataPath = customDataPath; this.shardRouting = shardRouting; } @@ -722,4 +746,11 @@ public String getCustomDataPath() { } } + public int getNumberOfStartedShardBatches() { + return batchIdToStoreShardBatch.size(); + } + + public int getNumberOfStoreShardBatches() { + return batchIdToStoreShardBatch.size(); + } } diff --git a/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java new file mode 100644 index 0000000000000..ba06cda68facf --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java @@ -0,0 +1,352 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.Version; +import org.opensearch.action.support.nodes.BaseNodeResponse; +import org.opensearch.cluster.ClusterInfo; +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.routing.IndexShardRoutingTable; +import org.opensearch.cluster.routing.RoutingNodes; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.allocation.RoutingAllocation; +import org.opensearch.cluster.routing.allocation.decider.AllocationDeciders; +import org.opensearch.common.collect.Tuple; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.snapshots.SnapshotShardSizeInfo; +import org.opensearch.test.gateway.TestGatewayAllocator; +import org.junit.Before; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +public class GatewayAllocatorTests extends OpenSearchAllocationTestCase { + + private final Logger logger = LogManager.getLogger(GatewayAllocatorTests.class); + TestGatewayAllocator testGatewayAllocator = null; + ClusterState clusterState = null; + RoutingAllocation testAllocation = null; + String indexPrefix = "TEST"; + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + testGatewayAllocator = new TestGatewayAllocator(); + } + + public void testSingleBatchCreation() { + createIndexAndUpdateClusterState(1, 3, 1); + createBatchesAndAssert(1); + } + + public void testTwoBatchCreation() { + createIndexAndUpdateClusterState(2, 1020, 1); + createBatchesAndAssert(2); + + List listOfBatches = new ArrayList<>(testGatewayAllocator.getBatchIdToStartedShardBatch().values()); + assertNotEquals(listOfBatches.get(0), listOfBatches.get(1)); + + // test for replicas + listOfBatches = new ArrayList<>(testGatewayAllocator.getBatchIdToStoreShardBatch().values()); + assertNotEquals(listOfBatches.get(0), listOfBatches.get(1)); + } + + public void testNonDuplicationOfBatch() { + createIndexAndUpdateClusterState(1, 3, 1); + Tuple, Set> batches = createBatchesAndAssert(1); + assertEquals(1, batches.v1().size()); + assertEquals(1, batches.v2().size()); + + // again try to create batch and verify no new batch is created since shard is already batched and no new unassigned shard + assertEquals(batches.v1(), testGatewayAllocator.createAndUpdateBatches(testAllocation, true)); + assertEquals(batches.v2(), testGatewayAllocator.createAndUpdateBatches(testAllocation, false)); + } + + public void testCorrectnessOfBatch() { + createIndexAndUpdateClusterState(2, 1020, 1); + createBatchesAndAssert(2); + Set shardsSet1 = clusterState.routingTable() + .index(indexPrefix + 0) + .getShards() + .values() + .stream() + .map(IndexShardRoutingTable::getShardId) + .collect(Collectors.toSet()); + Set shardsSet2 = clusterState.routingTable() + .index(indexPrefix + 1) + .getShards() + .values() + .stream() + .map(IndexShardRoutingTable::getShardId) + .collect(Collectors.toSet()); + shardsSet1.addAll(shardsSet2); + + Set shardsInAllbatches = testGatewayAllocator.getBatchIdToStartedShardBatch() + .values() + .stream() + .map(GatewayAllocator.ShardsBatch::getBatchedShards) + .flatMap(Set::stream) + .collect(Collectors.toSet()); + assertEquals(shardsInAllbatches, shardsSet1); + shardsInAllbatches = testGatewayAllocator.getBatchIdToStoreShardBatch() + .values() + .stream() + .map(GatewayAllocator.ShardsBatch::getBatchedShards) + .flatMap(Set::stream) + .collect(Collectors.toSet()); + assertEquals(shardsInAllbatches, shardsSet1); + + Set primariesInAllBatches = testGatewayAllocator.getBatchIdToStartedShardBatch() + .values() + .stream() + .map(GatewayAllocator.ShardsBatch::getBatchedShardRoutings) + .flatMap(Set::stream) + .collect(Collectors.toSet()); + primariesInAllBatches.forEach(shardRouting -> assertTrue(shardRouting.unassigned() && shardRouting.primary() == true)); + + Set replicasInAllBatches = testGatewayAllocator.getBatchIdToStoreShardBatch() + .values() + .stream() + .map(GatewayAllocator.ShardsBatch::getBatchedShardRoutings) + .flatMap(Set::stream) + .collect(Collectors.toSet()); + + replicasInAllBatches.forEach(shardRouting -> assertTrue(shardRouting.unassigned() && shardRouting.primary() == false)); + } + + public void testAsyncFetcherCreationInBatch() { + createIndexAndUpdateClusterState(1, 3, 1); + Tuple, Set> batchesTuple = createBatchesAndAssert(1); + Set primaryBatches = batchesTuple.v1(); + Set replicaBatches = batchesTuple.v2(); + + GatewayAllocator.ShardsBatch shardsBatch = testGatewayAllocator.getBatchIdToStartedShardBatch() + .get(primaryBatches.iterator().next()); + AsyncShardFetch asyncFetcher = shardsBatch.getAsyncFetcher(); + // assert asyncFetcher is not null + assertNotNull(asyncFetcher); + shardsBatch = testGatewayAllocator.getBatchIdToStoreShardBatch().get(replicaBatches.iterator().next()); + asyncFetcher = shardsBatch.getAsyncFetcher(); + assertNotNull(asyncFetcher); + } + + public void testSafelyRemoveShardFromBatch() { + createIndexAndUpdateClusterState(2, 1023, 1); + + Tuple, Set> batchesTuple = createBatchesAndAssert(2); + Set primaryBatches = batchesTuple.v1(); + Set replicaBatches = batchesTuple.v2(); + + GatewayAllocator.ShardsBatch primaryShardsBatch = testGatewayAllocator.getBatchIdToStartedShardBatch() + .get(primaryBatches.iterator().next()); + ShardRouting primaryShardRouting = primaryShardsBatch.getBatchedShardRoutings().iterator().next(); + assertEquals(2, replicaBatches.size()); + GatewayAllocator.ShardsBatch replicaShardsBatch = testGatewayAllocator.getBatchIdToStoreShardBatch() + .get(replicaBatches.iterator().next()); + ShardRouting replicaShardRouting = replicaShardsBatch.getBatchedShardRoutings().iterator().next(); + + // delete 1 shard routing from each batch + testGatewayAllocator.safelyRemoveShardFromBatch(primaryShardRouting); + + testGatewayAllocator.safelyRemoveShardFromBatch(replicaShardRouting); + // verify that shard routing is removed from both batches + assertFalse(primaryShardsBatch.getBatchedShards().contains(primaryShardRouting.shardId())); + assertFalse(replicaShardsBatch.getBatchedShards().contains(replicaShardRouting.shardId())); + + // try to remove that shard again to see if its no op and doent result in exception + testGatewayAllocator.safelyRemoveShardFromBatch(primaryShardRouting); + testGatewayAllocator.safelyRemoveShardFromBatch(replicaShardRouting); + + // now remove all shard routings to verify that batch only gets deleted + primaryShardsBatch.getBatchedShardRoutings().forEach(testGatewayAllocator::safelyRemoveShardFromBatch); + replicaShardsBatch.getBatchedShardRoutings().forEach(testGatewayAllocator::safelyRemoveShardFromBatch); + + assertFalse(testGatewayAllocator.getBatchIdToStartedShardBatch().containsKey(primaryShardsBatch.getBatchId())); + assertFalse(testGatewayAllocator.getBatchIdToStoreShardBatch().containsKey(replicaShardsBatch.getBatchId())); + assertEquals(1, testGatewayAllocator.getBatchIdToStartedShardBatch().size()); + assertEquals(1, testGatewayAllocator.getBatchIdToStoreShardBatch().size()); + } + + public void testSafelyRemoveShardFromBothBatch() { + createIndexAndUpdateClusterState(1, 3, 1); + createBatchesAndAssert(1); + GatewayAllocator.ShardsBatch primaryShardsBatch = testGatewayAllocator.getBatchIdToStartedShardBatch().values().iterator().next(); + GatewayAllocator.ShardsBatch replicaShardsBatch = testGatewayAllocator.getBatchIdToStoreShardBatch().values().iterator().next(); + + ShardRouting anyPrimary = primaryShardsBatch.getBatchedShardRoutings().iterator().next(); + // remove first shard routing from both batches + testGatewayAllocator.safelyRemoveShardFromBothBatch(anyPrimary); + + // verify that shard routing is removed from both batches + assertFalse(primaryShardsBatch.getBatchedShards().contains(anyPrimary.shardId())); + assertFalse(replicaShardsBatch.getBatchedShards().contains(anyPrimary.shardId())); + + // try to remove that shard again to see if its no op and doesnt result in exception + testGatewayAllocator.safelyRemoveShardFromBothBatch(anyPrimary); + + // now remove all shard routings to verify that batch gets deleted + primaryShardsBatch.getBatchedShardRoutings().forEach(testGatewayAllocator::safelyRemoveShardFromBothBatch); + replicaShardsBatch.getBatchedShardRoutings().forEach(testGatewayAllocator::safelyRemoveShardFromBothBatch); + + assertFalse(testGatewayAllocator.getBatchIdToStartedShardBatch().containsKey(primaryShardsBatch.getBatchId())); + assertFalse(testGatewayAllocator.getBatchIdToStoreShardBatch().containsKey(replicaShardsBatch.getBatchId())); + assertEquals(0, testGatewayAllocator.getBatchIdToStartedShardBatch().size()); + assertEquals(0, testGatewayAllocator.getBatchIdToStoreShardBatch().size()); + } + + public void testGetBatchIdExisting() { + createIndexAndUpdateClusterState(2, 1020, 1); + // get all shardsRoutings for test index + List allShardRoutings1 = clusterState.routingTable() + .index(indexPrefix + 0) + .getShards() + .values() + .stream() + .map(IndexShardRoutingTable::getShards) + .flatMap(List::stream) + .collect(Collectors.toList()); + List allShardRouting2 = clusterState.routingTable() + .index(indexPrefix + 1) + .getShards() + .values() + .stream() + .map(IndexShardRoutingTable::getShards) + .flatMap(List::stream) + .collect(Collectors.toList()); + + Tuple, Set> batchesTuple = createBatchesAndAssert(2); + Set primaryBatches = batchesTuple.v1(); + Set replicaBatches = batchesTuple.v2(); + + // create a map of shards to batch id for primaries + + Map shardIdToBatchIdForStartedShards = new HashMap<>(); + allShardRoutings1.addAll(allShardRouting2); + assertEquals(4080, allShardRoutings1.size()); + for (ShardRouting shardRouting : allShardRoutings1) { + for (String batchId : primaryBatches) { + if (shardRouting.primary() == true + && testGatewayAllocator.getBatchIdToStartedShardBatch() + .get(batchId) + .getBatchedShards() + .contains(shardRouting.shardId())) { + if (shardIdToBatchIdForStartedShards.containsKey(shardRouting.shardId())) { + fail("found duplicate shard routing for shard. One shard cant be in multiple batches " + shardRouting.shardId()); + } + assertTrue(shardRouting.primary()); + shardIdToBatchIdForStartedShards.put(shardRouting.shardId(), batchId); + } + } + } + Map shardIdToBatchIdForStoreShards = new HashMap<>(); + + for (ShardRouting shardRouting : allShardRoutings1) { + for (String batchId : replicaBatches) { + if (shardRouting.primary() == false + && testGatewayAllocator.getBatchIdToStoreShardBatch() + .get(batchId) + .getBatchedShards() + .contains(shardRouting.shardId())) { + if (shardIdToBatchIdForStoreShards.containsKey(shardRouting.shardId())) { + fail("found duplicate shard routing for shard. One shard cant be in multiple batches " + shardRouting.shardId()); + } + assertFalse(shardRouting.primary()); + shardIdToBatchIdForStoreShards.put(shardRouting.shardId(), batchId); + } + } + } + + assertEquals(4080, shardIdToBatchIdForStartedShards.size() + shardIdToBatchIdForStoreShards.size()); + // now compare the maps with getBatchId() call + for (ShardRouting shardRouting : allShardRoutings1) { + if (shardRouting.primary()) { + assertEquals( + shardIdToBatchIdForStartedShards.get(shardRouting.shardId()), + testGatewayAllocator.getBatchId(shardRouting, true) + ); + } else { + assertEquals( + shardIdToBatchIdForStoreShards.get(shardRouting.shardId()), + testGatewayAllocator.getBatchId(shardRouting, false) + ); + } + } + } + + public void testGetBatchIdNonExisting() { + createIndexAndUpdateClusterState(1, 1, 1); + List allShardRoutings = clusterState.routingTable() + .index(indexPrefix + 0) + .getShards() + .values() + .stream() + .map(IndexShardRoutingTable::getShards) + .flatMap(List::stream) + .collect(Collectors.toList()); + allShardRoutings.forEach(shard -> assertNull(testGatewayAllocator.getBatchId(shard, shard.primary()))); + } + + private void createIndexAndUpdateClusterState(int count, int numberOfShards, int numberOfReplicas) { + if (count == 0) return; + Metadata.Builder metadata = Metadata.builder(); + RoutingTable.Builder routingTableBuilder = RoutingTable.builder(); + for (int i = 0; i < count; i++) { + String indexName = indexPrefix + i; + metadata.put( + IndexMetadata.builder(indexName) + .settings(settings(Version.CURRENT)) + .numberOfShards(numberOfShards) + .numberOfReplicas(numberOfReplicas) + ); + } + for (int i = 0; i < count; i++) { + String indexName = indexPrefix + i; + routingTableBuilder = routingTableBuilder.addAsNew(metadata.build().index(indexName)); + } + clusterState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata.build()) + .routingTable(routingTableBuilder.build()) + .build(); + testAllocation = new RoutingAllocation( + new AllocationDeciders(Collections.emptyList()), + new RoutingNodes(clusterState, false), + clusterState, + ClusterInfo.EMPTY, + SnapshotShardSizeInfo.EMPTY, + System.nanoTime() + ); + } + + // call this after index creation and update cluster state + private Tuple, Set> createBatchesAndAssert(int expectedBatchSize) { + Set primaryBatches = testGatewayAllocator.createAndUpdateBatches(testAllocation, true); + Set replicaBatches = testGatewayAllocator.createAndUpdateBatches(testAllocation, false); + assertEquals(expectedBatchSize, primaryBatches.size()); + assertEquals(expectedBatchSize, replicaBatches.size()); + assertEquals(expectedBatchSize, testGatewayAllocator.getBatchIdToStartedShardBatch().size()); + assertEquals(expectedBatchSize, testGatewayAllocator.getBatchIdToStoreShardBatch().size()); + assertEquals(testGatewayAllocator.getBatchIdToStartedShardBatch().keySet(), primaryBatches); + assertEquals(testGatewayAllocator.getBatchIdToStoreShardBatch().keySet(), replicaBatches); + return new Tuple<>(primaryBatches, replicaBatches); + } +} diff --git a/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java b/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java index a36dc26685eb4..64d1cccf72e57 100644 --- a/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java +++ b/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java @@ -40,11 +40,15 @@ import org.opensearch.gateway.AsyncShardFetch; import org.opensearch.gateway.GatewayAllocator; import org.opensearch.gateway.PrimaryShardAllocator; +import org.opensearch.gateway.PrimaryShardBatchAllocator; import org.opensearch.gateway.ReplicaShardAllocator; +import org.opensearch.gateway.ReplicaShardBatchAllocator; import org.opensearch.gateway.TransportNodesListGatewayStartedShards.NodeGatewayStartedShards; +import org.opensearch.gateway.TransportNodesListGatewayStartedBatchShards; import org.opensearch.index.shard.ShardId; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata.NodeStoreFilesMetadata; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; import java.util.Collections; import java.util.HashMap; @@ -102,6 +106,41 @@ protected AsyncShardFetch.FetchResult fetchData(ShardR } }; + + PrimaryShardBatchAllocator primaryShardBatchAllocator = new PrimaryShardBatchAllocator() { + @Override + protected AsyncShardFetch.FetchResult fetchData(Set shardsEligibleForFetch, + Set inEligibleShards, + RoutingAllocation allocation) { + Map foundShards = new HashMap<>(); + HashMap> shardsToIgnoreNodes = new HashMap<>(); + for (Map.Entry> entry : knownAllocations.entrySet()) { + String nodeId = entry.getKey(); + Map shardsOnNode = entry.getValue(); + HashMap adaptedResponse = new HashMap<>(); + + for (ShardRouting shardRouting : shardsEligibleForFetch) { + ShardId shardId = shardRouting.shardId(); + Set ignoreNodes = allocation.getIgnoreNodes(shardId); + + if (shardsOnNode.containsKey(shardId) && ignoreNodes.contains(nodeId) == false && currentNodes.nodeExists(nodeId)) { + TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShards nodeShard = new TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShards( + shardRouting.allocationId().getId(), + shardRouting.primary(), + getReplicationCheckpoint(shardId, nodeId) + ); + adaptedResponse.put(shardId, nodeShard); + shardsToIgnoreNodes.put(shardId, ignoreNodes); + } + foundShards.put(currentNodes.get(nodeId), + new TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShardsBatch(currentNodes.get(nodeId), adaptedResponse)); + } + } + return new AsyncShardFetch.FetchResult<>(foundShards, shardsToIgnoreNodes); + } + }; + + private ReplicationCheckpoint getReplicationCheckpoint(ShardId shardId, String nodeName) { return shardIdNodeToReplicationCheckPointMap.getOrDefault(getReplicationCheckPointKey(shardId, nodeName), null); } @@ -120,6 +159,19 @@ protected boolean hasInitiatedFetching(ShardRouting shard) { } }; + ReplicaShardBatchAllocator replicaShardBatchAllocator = new ReplicaShardBatchAllocator() { + + @Override + protected AsyncShardFetch.FetchResult fetchData(Set shardsEligibleForFetch, + Set inEligibleShards, + RoutingAllocation allocation) { + return new AsyncShardFetch.FetchResult<>(Collections.emptyMap(), Collections.emptyMap()); + } + @Override + protected boolean hasInitiatedFetching(ShardRouting shard) { + return true; + } + }; @Override public void applyStartedShards(List startedShards, RoutingAllocation allocation) { currentNodes = allocation.nodes(); @@ -157,6 +209,12 @@ public void allocateUnassigned( innerAllocatedUnassigned(allocation, primaryShardAllocator, replicaShardAllocator, shardRouting, unassignedAllocationHandler); } + @Override + public void allocateUnassignedBatch(RoutingAllocation allocation, boolean primary){ + currentNodes = allocation.nodes(); + innerAllocateUnassignedBatch(allocation, primaryShardBatchAllocator, replicaShardBatchAllocator, primary); + } + /** * manually add a specific shard to the allocations the gateway keeps track of */ @@ -171,4 +229,30 @@ public String getReplicationCheckPointKey(ShardId shardId, String nodeName) { public void addReplicationCheckpoint(ShardId shardId, String nodeName, ReplicationCheckpoint replicationCheckpoint) { shardIdNodeToReplicationCheckPointMap.putIfAbsent(getReplicationCheckPointKey(shardId, nodeName), replicationCheckpoint); } + + + public Set createAndUpdateBatches(RoutingAllocation allocation, boolean primary) { + return super.createAndUpdateBatches(allocation, primary); + } + + public void safelyRemoveShardFromBatch(ShardRouting shard){ + super.safelyRemoveShardFromBatch(shard); + } + + + public void safelyRemoveShardFromBothBatch(ShardRouting shardRouting) { + super.safelyRemoveShardFromBothBatch(shardRouting); + } + + public String getBatchId(ShardRouting shard, boolean primary) { + return super.getBatchId(shard, primary); + } + + public Map getBatchIdToStartedShardBatch(){ + return batchIdToStartedShardBatch; + } + + public Map getBatchIdToStoreShardBatch(){ + return batchIdToStoreShardBatch; + } } From 9238ebfd06a4f2d16269a512089cedd12a013e4c Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Thu, 28 Sep 2023 13:27:39 +0530 Subject: [PATCH 11/45] Added batch mode setting Signed-off-by: Gaurav Chandani --- .../routing/allocation/AllocationService.java | 46 +++++++++++++++---- .../allocation/ExistingShardsAllocator.java | 7 +++ 2 files changed, 43 insertions(+), 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 9e91648f48109..b689f84e34bfa 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 @@ -35,6 +35,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.opensearch.Version; import org.opensearch.cluster.ClusterInfoService; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.RestoreInProgress; @@ -55,6 +56,7 @@ import org.opensearch.cluster.routing.allocation.decider.AllocationDeciders; import org.opensearch.cluster.routing.allocation.decider.Decision; import org.opensearch.common.collect.ImmutableOpenMap; +import org.opensearch.common.settings.Settings; import org.opensearch.gateway.GatewayAllocator; import org.opensearch.gateway.PriorityComparator; import org.opensearch.snapshots.SnapshotsInfoService; @@ -73,6 +75,7 @@ import static java.util.Collections.emptyList; import static java.util.Collections.singletonList; import static org.opensearch.cluster.routing.UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING; +import static org.opensearch.cluster.routing.allocation.ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE_ENABLED; /** * This service manages the node allocation of a cluster. For this reason the @@ -87,6 +90,7 @@ public class AllocationService { private static final Logger logger = LogManager.getLogger(AllocationService.class); private final AllocationDeciders allocationDeciders; + private Settings settings; private Map existingShardsAllocators; private final ShardsAllocator shardsAllocator; private final ClusterInfoService clusterInfoService; @@ -114,6 +118,22 @@ public AllocationService( this.shardsAllocator = shardsAllocator; this.clusterInfoService = clusterInfoService; this.snapshotsInfoService = snapshotsInfoService; + this.settings = Settings.EMPTY; + } + + public AllocationService( + AllocationDeciders allocationDeciders, + ShardsAllocator shardsAllocator, + ClusterInfoService clusterInfoService, + SnapshotsInfoService snapshotsInfoService, + Settings settings + + ) { + this.allocationDeciders = allocationDeciders; + this.shardsAllocator = shardsAllocator; + this.clusterInfoService = clusterInfoService; + this.snapshotsInfoService = snapshotsInfoService; + this.settings = settings; } /** @@ -547,12 +567,13 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { for (final ExistingShardsAllocator existingShardsAllocator : existingShardsAllocators.values()) { existingShardsAllocator.beforeAllocation(allocation); } - // batch Mode enabled setting to be added - boolean batchModeEnabled = true; - if (batchModeEnabled) { + + Boolean batchModeEnabled = EXISTING_SHARDS_ALLOCATOR_BATCH_MODE_ENABLED.get(settings); + + if (batchModeEnabled && allocation.nodes().getMinNodeVersion().onOrAfter(Version.CURRENT)) { // since allocators is per index setting, to have batch assignment verify allocators same for all shards // if not fallback to single assignment - ExistingShardsAllocator allocator = verifySameAllocatorForAllShards(allocation); + ExistingShardsAllocator allocator = verifySameAllocatorForAllUnassignedShards(allocation); if (allocator != null) { allocator.allocateUnassignedBatch(allocation, true); for (final ExistingShardsAllocator existingShardsAllocator : existingShardsAllocators.values()) { @@ -563,7 +584,6 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { } } - final RoutingNodes.UnassignedShards.UnassignedIterator primaryIterator = allocation.routingNodes().unassigned().iterator(); while (primaryIterator.hasNext()) { final ShardRouting shardRouting = primaryIterator.next(); @@ -585,20 +605,26 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { } } - private ExistingShardsAllocator verifySameAllocatorForAllShards(RoutingAllocation allocation) { + /** + * Verify if all unassigned shards are allocated by the same allocator, if yes then return the allocator, else + * return null + * @param allocation {@link RoutingAllocation} + * @return {@link ExistingShardsAllocator} or null + */ + private ExistingShardsAllocator verifySameAllocatorForAllUnassignedShards(RoutingAllocation allocation) { // if there is a single Allocator set in Allocation Service then use it for all shards if (existingShardsAllocators.size() == 1) { return existingShardsAllocators.values().iterator().next(); } RoutingNodes.UnassignedShards unassignedShards = allocation.routingNodes().unassigned(); RoutingNodes.UnassignedShards.UnassignedIterator iterator = unassignedShards.iterator(); - ExistingShardsAllocator currentAllocatorForShard =null; + ExistingShardsAllocator currentAllocatorForShard = null; if (unassignedShards.size() > 0) { ShardRouting shard = iterator.next(); - currentAllocatorForShard= getAllocatorForShard(shard, allocation); - while (iterator.hasNext()){ + currentAllocatorForShard = getAllocatorForShard(shard, allocation); + while (iterator.hasNext()) { ExistingShardsAllocator allocatorForShard = getAllocatorForShard(iterator.next(), allocation); - if (currentAllocatorForShard.getClass().getName().equals(allocatorForShard.getClass().getName())==false){ + if (currentAllocatorForShard.getClass().getName().equals(allocatorForShard.getClass().getName()) == false) { return null; } } 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 34cc462f9fe83..dd169f247d1c4 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 @@ -60,6 +60,13 @@ public interface ExistingShardsAllocator { Setting.Property.PrivateIndex ); + public static final Setting EXISTING_SHARDS_ALLOCATOR_BATCH_MODE_ENABLED = Setting.boolSetting( + "cluster.allocator.existing_shards_allocator.batch_enable", + true, + Setting.Property.NodeScope + ); + + /** * Called before starting a round of allocation, allowing the allocator to invalidate some caches if appropriate. */ From 728167cfaaba1bebf1db6fe4b8d5e5ea31e69d32 Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Wed, 29 Nov 2023 17:15:02 +0530 Subject: [PATCH 12/45] Added missed clusterModule changes Signed-off-by: Gaurav Chandani --- server/src/main/java/org/opensearch/cluster/ClusterModule.java | 2 +- .../java/org/opensearch/common/settings/ClusterSettings.java | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/cluster/ClusterModule.java b/server/src/main/java/org/opensearch/cluster/ClusterModule.java index 8a4e17e5c0dc3..cbf514d051662 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterModule.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterModule.java @@ -150,7 +150,7 @@ public ClusterModule( this.shardsAllocator = createShardsAllocator(settings, clusterService.getClusterSettings(), clusterPlugins); this.clusterService = clusterService; this.indexNameExpressionResolver = new IndexNameExpressionResolver(threadContext); - this.allocationService = new AllocationService(allocationDeciders, shardsAllocator, clusterInfoService, snapshotsInfoService); + this.allocationService = new AllocationService(allocationDeciders, shardsAllocator, clusterInfoService, snapshotsInfoService, settings); } public static List getNamedWriteables() { 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 208a358d38395..eadf0abaee92b 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -79,6 +79,7 @@ import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.routing.OperationRouting; import org.opensearch.cluster.routing.allocation.DiskThresholdSettings; +import org.opensearch.cluster.routing.allocation.ExistingShardsAllocator; import org.opensearch.cluster.routing.allocation.allocator.BalancedShardsAllocator; import org.opensearch.cluster.routing.allocation.decider.AwarenessAllocationDecider; import org.opensearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider; @@ -246,6 +247,7 @@ public void apply(Settings value, Settings current, Settings previous) { DanglingIndicesState.AUTO_IMPORT_DANGLING_INDICES_SETTING, EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING, EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING, + ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE_ENABLED, FilterAllocationDecider.CLUSTER_ROUTING_INCLUDE_GROUP_SETTING, FilterAllocationDecider.CLUSTER_ROUTING_EXCLUDE_GROUP_SETTING, FilterAllocationDecider.CLUSTER_ROUTING_REQUIRE_GROUP_SETTING, From 7fe67098c59d4cbeaa330cb63e123d931d5eae19 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Thu, 30 Nov 2023 17:35:03 +0530 Subject: [PATCH 13/45] Modify GA for running RSBA Signed-off-by: Shivansh Arora --- .../opensearch/gateway/GatewayAllocator.java | 20 +++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java index 482c9d3159639..21aa6c45673dd 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java @@ -137,6 +137,7 @@ public GatewayAllocator( this.batchStoreAction = batchStoreAction; this.replicaBatchShardAllocator = new InternalReplicaBatchShardAllocator(); this.maxBatchSize = GATEWAY_ALLOCATOR_BATCH_SIZE.get(settings); + this.batchMode = EXISTING_SHARDS_ALLOCATOR_BATCH_MODE_ENABLED.get(settings); } @Override @@ -216,10 +217,21 @@ public void beforeAllocation(final RoutingAllocation allocation) { @Override public void afterPrimariesBeforeReplicas(RoutingAllocation allocation) { - assert replicaShardAllocator != null; - if (allocation.routingNodes().hasInactiveShards()) { - // cancel existing recoveries if we have a better match - replicaShardAllocator.processExistingRecoveries(allocation); + if (this.batchMode) { + assert replicaBatchShardAllocator != null; + List> storedShardBatches = batchIdToStoreShardBatch.values().stream() + .map(ShardsBatch::getBatchedShardRoutings) + .collect(Collectors.toList()); + if (allocation.routingNodes().hasInactiveShards()) { + // cancel existing recoveries if we have a better match + replicaBatchShardAllocator.processExistingRecoveries(allocation, storedShardBatches); + } + } else { + assert replicaShardAllocator != null; + if (allocation.routingNodes().hasInactiveShards()) { + // cancel existing recoveries if we have a better match + replicaShardAllocator.processExistingRecoveries(allocation); + } } } From 214aaa4fe8186180b8694409ac004615ed753942 Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Tue, 5 Dec 2023 19:03:22 +0530 Subject: [PATCH 14/45] Addressed PR comments 1. Moved setting of batch enable disable in this PR 2. Added java docs Signed-off-by: Gaurav Chandani --- .../org/opensearch/cluster/ClusterModule.java | 8 +++- .../routing/allocation/AllocationService.java | 10 ++--- .../allocation/ExistingShardsAllocator.java | 38 +++++++++++++++++-- 3 files changed, 45 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/ClusterModule.java b/server/src/main/java/org/opensearch/cluster/ClusterModule.java index cbf514d051662..17e24a5b6220e 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterModule.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterModule.java @@ -150,7 +150,13 @@ public ClusterModule( this.shardsAllocator = createShardsAllocator(settings, clusterService.getClusterSettings(), clusterPlugins); this.clusterService = clusterService; this.indexNameExpressionResolver = new IndexNameExpressionResolver(threadContext); - this.allocationService = new AllocationService(allocationDeciders, shardsAllocator, clusterInfoService, snapshotsInfoService, settings); + this.allocationService = new AllocationService( + allocationDeciders, + shardsAllocator, + clusterInfoService, + snapshotsInfoService, + settings + ); } public static List getNamedWriteables() { 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 b689f84e34bfa..d1e27c8c5670e 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,7 @@ import static java.util.Collections.emptyList; import static java.util.Collections.singletonList; import static org.opensearch.cluster.routing.UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING; -import static org.opensearch.cluster.routing.allocation.ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE_ENABLED; +import static org.opensearch.cluster.routing.allocation.ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE; /** * This service manages the node allocation of a cluster. For this reason the @@ -114,11 +114,7 @@ public AllocationService( ClusterInfoService clusterInfoService, SnapshotsInfoService snapshotsInfoService ) { - this.allocationDeciders = allocationDeciders; - this.shardsAllocator = shardsAllocator; - this.clusterInfoService = clusterInfoService; - this.snapshotsInfoService = snapshotsInfoService; - this.settings = Settings.EMPTY; + this(allocationDeciders, shardsAllocator, clusterInfoService, snapshotsInfoService, Settings.EMPTY); } public AllocationService( @@ -568,7 +564,7 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { existingShardsAllocator.beforeAllocation(allocation); } - Boolean batchModeEnabled = EXISTING_SHARDS_ALLOCATOR_BATCH_MODE_ENABLED.get(settings); + Boolean batchModeEnabled = EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.get(settings); if (batchModeEnabled && allocation.nodes().getMinNodeVersion().onOrAfter(Version.CURRENT)) { // since allocators is per index setting, to have batch assignment verify allocators same for all shards 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 dd169f247d1c4..17a3f842b642d 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 @@ -60,13 +60,30 @@ public interface ExistingShardsAllocator { Setting.Property.PrivateIndex ); - public static final Setting EXISTING_SHARDS_ALLOCATOR_BATCH_MODE_ENABLED = Setting.boolSetting( + /** + * Boolean setting to enable/disable batch allocation of unassigned shards already existing on disk. + * This will allow sending all Unassigned Shards to the ExistingShard Allocator to make decision to allocate + * in one or more go. + * + * Enable this setting if your ExistingShardAllocator is implementing the + * {@link ExistingShardsAllocator#allocateUnassignedBatch(RoutingAllocation, boolean)} method. + * The default implementation of this method is not optimized and assigns shards one by one. + * + * If enable to true then it expects all indices of the shard to use same {@link ExistingShardsAllocator}, otherwise + * Allocation Service will fallback to default implementation i.e. {@link ExistingShardsAllocator#allocateUnassigned(ShardRouting, RoutingAllocation, UnassignedAllocationHandler)} + * + * If no plugin overrides {@link ExistingShardsAllocator} then default implementation will be use for it , i.e, + * {@link GatewayAllocator} + * + * TODO: Currently its implementation is WIP for GatewayAllocator so setting enabling wont have any effect + * https://github.com/opensearch-project/OpenSearch/issues/5098 + */ + Setting EXISTING_SHARDS_ALLOCATOR_BATCH_MODE = Setting.boolSetting( "cluster.allocator.existing_shards_allocator.batch_enable", - true, + false, Setting.Property.NodeScope ); - /** * Called before starting a round of allocation, allowing the allocator to invalidate some caches if appropriate. */ @@ -95,6 +112,21 @@ default void allocateUnassignedBatch(RoutingAllocation allocation, boolean prima } } + /** + * Allocate all unassigned shards in the given {@link RoutingAllocation} for which this {@link ExistingShardsAllocator} is responsible. + * Default implementation calls {@link #allocateUnassigned(ShardRouting, RoutingAllocation, UnassignedAllocationHandler)} for each Unassigned shard + * and is kept here for backward compatibility. + */ + default void allocateUnassignedBatch(RoutingAllocation allocation, boolean primary) { + RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); + while (iterator.hasNext()) { + ShardRouting shardRouting = iterator.next(); + if (shardRouting.primary() == primary) { + allocateUnassigned(shardRouting, allocation, iterator); + } + } + } + /** * Returns an explanation for a single unassigned shard. */ From 59922357ac2fad746ceb17c477b4bacebcdd6727 Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Wed, 13 Dec 2023 14:09:34 +0530 Subject: [PATCH 15/45] Addressed PR comments 1. Added a serparate class for batch mode of GA 2. nit comments of renaming variables 3. Added ClupritShardTest Signed-off-by: Shivansh Arora --- .../gateway/RecoveryFromGatewayIT.java | 142 +++- .../org/opensearch/cluster/ClusterModule.java | 8 +- .../common/settings/ClusterSettings.java | 5 +- .../opensearch/gateway/GatewayAllocator.java | 459 +----------- .../gateway/ShardsBatchGatewayAllocator.java | 703 ++++++++++++++++++ .../main/java/org/opensearch/node/Node.java | 10 +- .../cluster/reroute/ClusterRerouteTests.java | 4 +- .../cluster/ClusterModuleTests.java | 13 +- .../gateway/GatewayAllocatorTests.java | 120 +-- .../test/gateway/TestGatewayAllocator.java | 92 +-- .../TestShardBatchGatewayAllocator.java | 142 ++++ 11 files changed, 1098 insertions(+), 600 deletions(-) create mode 100644 server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java create mode 100644 test/framework/src/main/java/org/opensearch/test/gateway/TestShardBatchGatewayAllocator.java diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 375e3c4962b12..73de310cd65fc 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -51,6 +51,7 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.routing.UnassignedInfo; +import org.opensearch.cluster.routing.allocation.ExistingShardsAllocator; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; @@ -748,7 +749,10 @@ public void testMessyElectionsStillMakeClusterGoGreen() throws Exception { } public void testBatchModeEnabled() throws Exception { - internalCluster().startClusterManagerOnlyNodes(1); + internalCluster().startClusterManagerOnlyNodes( + 1, + Settings.builder().put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.getKey(), true).build() + ); List dataOnlyNodes = internalCluster().startDataOnlyNodes(2); createIndex( "test", @@ -766,15 +770,13 @@ public void testBatchModeEnabled() throws Exception { ClusterRerouteResponse clusterRerouteResponse = client().admin().cluster().prepareReroute().setRetryFailed(true).get(); assertTrue(clusterRerouteResponse.isAcknowledged()); - GatewayAllocator gatewayAllocator = internalCluster().getInstance( - GatewayAllocator.class, + 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()); - assertTrue( - ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE_ENABLED.get(internalCluster().clusterService().getSettings()) - ); // Now start both data nodes and ensure batch mode is working logger.info("--> restarting the stopped nodes"); @@ -789,7 +791,7 @@ public void testBatchModeEnabled() throws Exception { public void testBatchModeDisabled() throws Exception { internalCluster().startClusterManagerOnlyNodes( 1, - Settings.builder().put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE_ENABLED.getKey(), false).build() + Settings.builder().put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.getKey(), false).build() ); List dataOnlyNodes = internalCluster().startDataOnlyNodes(2); createIndex( @@ -808,12 +810,14 @@ public void testBatchModeDisabled() throws Exception { ClusterRerouteResponse clusterRerouteResponse = client().admin().cluster().prepareReroute().setRetryFailed(true).get(); assertTrue(clusterRerouteResponse.isAcknowledged()); - GatewayAllocator gatewayAllocator = internalCluster().getInstance( - GatewayAllocator.class, + ShardsBatchGatewayAllocator gatewayAllocator = internalCluster().getInstance( + ShardsBatchGatewayAllocator.class, internalCluster().getClusterManagerName() ); ensureRed("test"); + assertFalse(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.get(internalCluster().clusterService().getSettings())); + // assert no batches created assertEquals(0, gatewayAllocator.getNumberOfStartedShardBatches()); assertEquals(0, gatewayAllocator.getNumberOfStoreShardBatches()); @@ -831,7 +835,10 @@ public void testNBatchesCreationAndAssignment() throws Exception { // Total number of replica shards = 50 (50 indices*1) // Total batches creation for primaries and replicas will be 10 each - internalCluster().startClusterManagerOnlyNodes(1); + internalCluster().startClusterManagerOnlyNodes( + 1, + Settings.builder().put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.getKey(), true).build() + ); List dataOnlyNodes = internalCluster().startDataOnlyNodes(2); createNIndices(50, "test"); ensureStableCluster(3); @@ -844,8 +851,6 @@ public void testNBatchesCreationAndAssignment() throws Exception { assertFalse(health.isTimedOut()); assertEquals(GREEN, health.getStatus()); - // Now we will first stop cluster manager node and then stop data nodes. This will ensure to avoid any scenarios - // of more number of batch creation. String clusterManagerName = internalCluster().getClusterManagerName(); Settings clusterManagerDataPathSettings = internalCluster().dataPathSettings(clusterManagerName); Settings node0DataPathSettings = internalCluster().dataPathSettings(dataOnlyNodes.get(0)); @@ -861,7 +866,8 @@ public void testNBatchesCreationAndAssignment() throws Exception { Settings.builder() .put("node.name", clusterManagerName) .put(clusterManagerDataPathSettings) - .put(GatewayAllocator.GATEWAY_ALLOCATOR_BATCH_SIZE.getKey(), 5) + .put(ShardsBatchGatewayAllocator.GATEWAY_ALLOCATOR_BATCH_SIZE.getKey(), 5) + .put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.getKey(), true) .build() ); ensureStableCluster(1); @@ -870,10 +876,11 @@ public void testNBatchesCreationAndAssignment() throws Exception { ClusterRerouteResponse clusterRerouteResponse = client().admin().cluster().prepareReroute().setRetryFailed(true).get(); assertTrue(clusterRerouteResponse.isAcknowledged()); - GatewayAllocator gatewayAllocator = internalCluster().getInstance( - GatewayAllocator.class, + 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(); @@ -904,6 +911,106 @@ public void testNBatchesCreationAndAssignment() throws Exception { assertEquals(0, gatewayAllocator.getNumberOfStoreShardBatches()); } + public void testCulpritShardInBatch() throws Exception { + internalCluster().startClusterManagerOnlyNodes( + 1, + Settings.builder().put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.getKey(), true).build() + ); + List dataOnlyNodes = internalCluster().startDataOnlyNodes(3); + createNIndices(4, "test"); + ensureStableCluster(4); + ClusterHealthResponse health = client().admin() + .cluster() + .health(Requests.clusterHealthRequest().waitForGreenStatus().timeout("5m")) + .actionGet(); + assertFalse(health.isTimedOut()); + assertEquals(GREEN, health.getStatus()); + assertEquals(8, health.getActiveShards()); + + String culpritShardIndexName = "test0"; + final String customDataPath = IndexMetadata.INDEX_DATA_PATH_SETTING.get( + client().admin().indices().prepareGetSettings(culpritShardIndexName).get().getIndexToSettings().get(culpritShardIndexName) + ); + final Index index = resolveIndex(culpritShardIndexName); + final ShardId shardId = new ShardId(index, 0); + + for (String dataNode : dataOnlyNodes) { + for (Path path : internalCluster().getInstance(NodeEnvironment.class, dataNode).availableShardPaths(shardId)) { + final Path indexPath = path.resolve(ShardPath.INDEX_FOLDER_NAME); + if (Files.exists(indexPath)) { + try (DirectoryStream stream = Files.newDirectoryStream(indexPath)) { + for (Path item : stream) { + if (item.getFileName().toString().startsWith("segments_")) { + logger.debug("--> deleting [{}]", item); + Files.delete(item); + } + } + } + } + } + } + String clusterManagerName = internalCluster().getClusterManagerName(); + Settings clusterManagerDataPathSettings = internalCluster().dataPathSettings(clusterManagerName); + Settings node0DataPathSettings = internalCluster().dataPathSettings(dataOnlyNodes.get(0)); + Settings node1DataPathSettings = internalCluster().dataPathSettings(dataOnlyNodes.get(1)); + Settings node2DataPathSettings = internalCluster().dataPathSettings(dataOnlyNodes.get(2)); + + internalCluster().stopCurrentClusterManagerNode(); + internalCluster().stopRandomDataNode(); + internalCluster().stopRandomDataNode(); + internalCluster().stopRandomDataNode(); + + // Now start cluster manager node and post that verify batches created + internalCluster().startClusterManagerOnlyNodes( + 1, + Settings.builder() + .put("node.name", clusterManagerName) + .put(clusterManagerDataPathSettings) + .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(1, gatewayAllocator.getNumberOfStartedShardBatches()); + assertEquals(1, gatewayAllocator.getNumberOfStoreShardBatches()); + assertTrue(clusterRerouteResponse.isAcknowledged()); + health = client(internalCluster().getClusterManagerName()).admin().cluster().health(Requests.clusterHealthRequest()).actionGet(); + assertFalse(health.isTimedOut()); + assertEquals(RED, health.getStatus()); + assertEquals(8, health.getUnassignedShards()); + assertEquals(0, health.getInitializingShards()); + assertEquals(0, health.getActiveShards()); + assertEquals(0, health.getRelocatingShards()); + assertEquals(0, health.getNumberOfDataNodes()); + + 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()); + internalCluster().startDataOnlyNode(Settings.builder().put("node.name", dataOnlyNodes.get(2)).put(node2DataPathSettings).build()); + ensureStableCluster(4); + + health = client().admin().cluster().health(Requests.clusterHealthRequest().waitForGreenStatus().timeout("1m")).actionGet(); + + assertEquals(RED, health.getStatus()); + assertTrue(health.isTimedOut()); + assertEquals(0, health.getNumberOfPendingTasks()); + assertEquals(0, health.getNumberOfInFlightFetch()); + assertEquals(6, health.getActiveShards()); + assertEquals(2, health.getUnassignedShards()); + assertEquals(0, health.getInitializingShards()); + assertEquals(0, health.getRelocatingShards()); + assertEquals(3, health.getNumberOfDataNodes()); + } + private void createNIndices(int n, String prefix) { for (int i = 0; i < n; i++) { @@ -911,6 +1018,11 @@ private void createNIndices(int n, String prefix) { prefix + i, Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).build() ); + // index doc2 + client().prepareIndex(prefix + i).setId("1").setSource("foo", "bar").get(); + + // index doc 2 + client().prepareIndex(prefix + i).setId("2").setSource("foo2", "bar2").get(); ensureGreen(prefix + i); } } diff --git a/server/src/main/java/org/opensearch/cluster/ClusterModule.java b/server/src/main/java/org/opensearch/cluster/ClusterModule.java index 17e24a5b6220e..cd1ffd898509f 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterModule.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterModule.java @@ -90,6 +90,7 @@ import org.opensearch.common.util.set.Sets; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.gateway.GatewayAllocator; +import org.opensearch.gateway.ShardsBatchGatewayAllocator; import org.opensearch.ingest.IngestMetadata; import org.opensearch.persistent.PersistentTasksCustomMetadata; import org.opensearch.persistent.PersistentTasksNodeService; @@ -421,6 +422,7 @@ public AllocationService getAllocationService() { @Override protected void configure() { bind(GatewayAllocator.class).asEagerSingleton(); + bind(ShardsBatchGatewayAllocator.class).asEagerSingleton(); bind(AllocationService.class).toInstance(allocationService); bind(ClusterService.class).toInstance(clusterService); bind(NodeConnectionsService.class).asEagerSingleton(); @@ -440,10 +442,8 @@ protected void configure() { bind(ShardsAllocator.class).toInstance(shardsAllocator); } - public void setExistingShardsAllocators(GatewayAllocator gatewayAllocator) { - final Map existingShardsAllocators = new HashMap<>(); - existingShardsAllocators.put(GatewayAllocator.ALLOCATOR_NAME, gatewayAllocator); - + public void setExistingShardsAllocators(Map gatewayAllocators) { + final Map existingShardsAllocators = new HashMap<>(gatewayAllocators); for (ClusterPlugin clusterPlugin : clusterPlugins) { for (Map.Entry existingShardsAllocatorEntry : clusterPlugin.getExistingShardsAllocators() .entrySet()) { 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 eadf0abaee92b..404ade615f7bd 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -94,6 +94,7 @@ import org.opensearch.cluster.service.ClusterApplierService; import org.opensearch.cluster.service.ClusterService; import org.opensearch.cluster.service.ClusterManagerTaskThrottler; +import org.opensearch.cluster.service.ClusterService; import org.opensearch.cluster.service.ClusterManagerService; import org.opensearch.common.logging.Loggers; import org.opensearch.common.network.NetworkModule; @@ -112,6 +113,7 @@ import org.opensearch.gateway.DanglingIndicesState; import org.opensearch.gateway.GatewayService; import org.opensearch.gateway.PersistedClusterStateService; +import org.opensearch.gateway.ShardsBatchGatewayAllocator; import org.opensearch.http.HttpTransportSettings; import org.opensearch.indices.IndexingMemoryController; import org.opensearch.indices.IndicesQueryCache; @@ -247,7 +249,7 @@ public void apply(Settings value, Settings current, Settings previous) { DanglingIndicesState.AUTO_IMPORT_DANGLING_INDICES_SETTING, EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING, EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING, - ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE_ENABLED, + ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE, FilterAllocationDecider.CLUSTER_ROUTING_INCLUDE_GROUP_SETTING, FilterAllocationDecider.CLUSTER_ROUTING_EXCLUDE_GROUP_SETTING, FilterAllocationDecider.CLUSTER_ROUTING_REQUIRE_GROUP_SETTING, @@ -304,6 +306,7 @@ public void apply(Settings value, Settings current, Settings previous) { GatewayService.RECOVER_AFTER_MASTER_NODES_SETTING, GatewayService.RECOVER_AFTER_NODES_SETTING, GatewayService.RECOVER_AFTER_TIME_SETTING, + ShardsBatchGatewayAllocator.GATEWAY_ALLOCATOR_BATCH_SIZE, 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/GatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java index 21aa6c45673dd..d6232502d29fa 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java @@ -43,31 +43,22 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.cluster.routing.RerouteService; -import org.opensearch.cluster.routing.RoutingNodes; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.opensearch.cluster.routing.allocation.ExistingShardsAllocator; import org.opensearch.cluster.routing.allocation.FailedShard; import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.common.Priority; -import org.opensearch.common.UUIDs; import org.opensearch.common.inject.Inject; import org.opensearch.common.lease.Releasables; -import org.opensearch.common.settings.Setting; -import org.opensearch.common.settings.Settings; import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.common.util.set.Sets; import org.opensearch.index.shard.ShardId; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; -import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; 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.Objects; import java.util.Set; import java.util.concurrent.ConcurrentMap; import java.util.stream.Collectors; @@ -83,61 +74,29 @@ public class GatewayAllocator implements ExistingShardsAllocator { public static final String ALLOCATOR_NAME = "gateway_allocator"; private static final Logger logger = LogManager.getLogger(GatewayAllocator.class); - private final long maxBatchSize; - - private static final short DEFAULT_BATCH_SIZE = 2000; private final RerouteService rerouteService; private final PrimaryShardAllocator primaryShardAllocator; private final ReplicaShardAllocator replicaShardAllocator; - private final PrimaryShardBatchAllocator primaryBatchShardAllocator; - private final ReplicaShardBatchAllocator replicaBatchShardAllocator; - private final TransportNodesListGatewayStartedBatchShards batchStartedAction; - private final TransportNodesListShardStoreMetadataBatch batchStoreAction; - private final ConcurrentMap< ShardId, AsyncShardFetch> asyncFetchStarted = ConcurrentCollections - .newConcurrentMap(); + .newConcurrentMap(); private final ConcurrentMap> asyncFetchStore = ConcurrentCollections.newConcurrentMap(); private Set lastSeenEphemeralIds = Collections.emptySet(); - // visble for testing - protected final ConcurrentMap batchIdToStartedShardBatch = ConcurrentCollections.newConcurrentMap(); - - // visible for testing - protected final ConcurrentMap batchIdToStoreShardBatch = ConcurrentCollections.newConcurrentMap(); - - // Number of shards we send in one batch to data nodes for fetching metadata - public static final Setting GATEWAY_ALLOCATOR_BATCH_SIZE = Setting.longSetting( - "cluster.allocator.gateway.batch_size", - DEFAULT_BATCH_SIZE, - 1, - 10000, - Setting.Property.NodeScope - ); - @Inject public GatewayAllocator( RerouteService rerouteService, TransportNodesListGatewayStartedShards startedAction, - TransportNodesListShardStoreMetadata storeAction, - TransportNodesListGatewayStartedBatchShards batchStartedAction, - TransportNodesListShardStoreMetadataBatch batchStoreAction, - Settings settings + TransportNodesListShardStoreMetadata storeAction ) { this.rerouteService = rerouteService; this.primaryShardAllocator = new InternalPrimaryShardAllocator(startedAction); this.replicaShardAllocator = new InternalReplicaShardAllocator(storeAction); - this.batchStartedAction = batchStartedAction; - this.primaryBatchShardAllocator = new InternalPrimaryBatchShardAllocator(); - this.batchStoreAction = batchStoreAction; - this.replicaBatchShardAllocator = new InternalReplicaBatchShardAllocator(); - this.maxBatchSize = GATEWAY_ALLOCATOR_BATCH_SIZE.get(settings); - this.batchMode = EXISTING_SHARDS_ALLOCATOR_BATCH_MODE_ENABLED.get(settings); } @Override @@ -146,14 +105,6 @@ public void cleanCaches() { asyncFetchStarted.clear(); Releasables.close(asyncFetchStore.values()); asyncFetchStore.clear(); - Releasables.close( - batchIdToStartedShardBatch.values().stream().map(shardsBatch -> shardsBatch.asyncBatch).collect(Collectors.toList()) - ); - batchIdToStartedShardBatch.clear(); - Releasables.close( - batchIdToStoreShardBatch.values().stream().map(shardsBatch -> shardsBatch.asyncBatch).collect(Collectors.toList()) - ); - batchIdToStoreShardBatch.clear(); } // for tests @@ -161,24 +112,11 @@ protected GatewayAllocator() { this.rerouteService = null; this.primaryShardAllocator = null; this.replicaShardAllocator = null; - this.batchStartedAction = null; - this.primaryBatchShardAllocator = null; - this.batchStoreAction = null; - this.replicaBatchShardAllocator = null; - this.maxBatchSize = DEFAULT_BATCH_SIZE; } @Override public int getNumberOfInFlightFetches() { int count = 0; - // If fetching is done in non batched-mode then maps to maintain batches will be empty and vice versa for batch-mode - for (ShardsBatch batch : batchIdToStartedShardBatch.values()) { - count += (batch.getNumberOfInFlightFetches() * batch.getBatchedShards().size()); - } - for (ShardsBatch batch : batchIdToStoreShardBatch.values()) { - count += (batch.getNumberOfInFlightFetches() * batch.getBatchedShards().size()); - } - for (AsyncShardFetch fetch : asyncFetchStarted.values()) { count += fetch.getNumberOfInFlightFetches(); } @@ -193,7 +131,6 @@ public void applyStartedShards(final List startedShards, final Rou for (ShardRouting startedShard : startedShards) { Releasables.close(asyncFetchStarted.remove(startedShard.shardId())); Releasables.close(asyncFetchStore.remove(startedShard.shardId())); - safelyRemoveShardFromBothBatch(startedShard); } } @@ -202,7 +139,6 @@ public void applyFailedShards(final List failedShards, final Routin for (FailedShard failedShard : failedShards) { Releasables.close(asyncFetchStarted.remove(failedShard.getRoutingEntry().shardId())); Releasables.close(asyncFetchStore.remove(failedShard.getRoutingEntry().shardId())); - safelyRemoveShardFromBothBatch(failedShard.getRoutingEntry()); } } @@ -210,28 +146,15 @@ public void applyFailedShards(final List failedShards, final Routin public void beforeAllocation(final RoutingAllocation allocation) { assert primaryShardAllocator != null; assert replicaShardAllocator != null; - assert primaryBatchShardAllocator != null; - assert replicaBatchShardAllocator != null; ensureAsyncFetchStorePrimaryRecency(allocation); } @Override public void afterPrimariesBeforeReplicas(RoutingAllocation allocation) { - if (this.batchMode) { - assert replicaBatchShardAllocator != null; - List> storedShardBatches = batchIdToStoreShardBatch.values().stream() - .map(ShardsBatch::getBatchedShardRoutings) - .collect(Collectors.toList()); - if (allocation.routingNodes().hasInactiveShards()) { - // cancel existing recoveries if we have a better match - replicaBatchShardAllocator.processExistingRecoveries(allocation, storedShardBatches); - } - } else { - assert replicaShardAllocator != null; - if (allocation.routingNodes().hasInactiveShards()) { - // cancel existing recoveries if we have a better match - replicaShardAllocator.processExistingRecoveries(allocation); - } + assert replicaShardAllocator != null; + if (allocation.routingNodes().hasInactiveShards()) { + // cancel existing recoveries if we have a better match + replicaShardAllocator.processExistingRecoveries(allocation); } } @@ -246,169 +169,6 @@ public void allocateUnassigned( innerAllocatedUnassigned(allocation, primaryShardAllocator, replicaShardAllocator, shardRouting, unassignedAllocationHandler); } - @Override - public void allocateUnassignedBatch(final RoutingAllocation allocation, boolean primary) { - - assert primaryBatchShardAllocator != null; - assert replicaBatchShardAllocator != null; - innerAllocateUnassignedBatch(allocation, primaryBatchShardAllocator, replicaBatchShardAllocator, primary); - } - - protected void innerAllocateUnassignedBatch( - RoutingAllocation allocation, - PrimaryShardBatchAllocator primaryBatchShardAllocator, - ReplicaShardBatchAllocator replicaBatchShardAllocator, - boolean primary - ) { - // create batches for unassigned shards - Set batchesToAssign = createAndUpdateBatches(allocation, primary); - if (batchesToAssign.isEmpty()) { - return; - } - if (primary) { - batchIdToStartedShardBatch.values() - .stream() - .filter(batch -> batchesToAssign.contains(batch.batchId)) - .forEach( - shardsBatch -> primaryBatchShardAllocator.allocateUnassignedBatch(shardsBatch.getBatchedShardRoutings(), allocation) - ); - } else { - batchIdToStoreShardBatch.values() - .stream() - .filter(batch -> batchesToAssign.contains(batch.batchId)) - .forEach(batch -> replicaBatchShardAllocator.allocateUnassignedBatch(batch.getBatchedShardRoutings(), allocation)); - } - } - - // visible for testing - protected Set createAndUpdateBatches(RoutingAllocation allocation, boolean primary) { - Set batchesToBeAssigned = new HashSet<>(); - RoutingNodes.UnassignedShards unassigned = allocation.routingNodes().unassigned(); - ConcurrentMap currentBatches = primary ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; - // get all batched shards - Set currentBatchedShards = currentBatches.values() - .stream() - .map(ShardsBatch::getBatchedShards) - .flatMap(Set::stream) - .collect(Collectors.toSet()); - - Set shardsToBatch = Sets.newHashSet(); - // add all unassigned shards to the batch if they are not already in a batch - unassigned.forEach(shardRouting -> { - if ((currentBatchedShards.contains(shardRouting.shardId()) == false) && (shardRouting.primary() == primary)) { - assert shardRouting.unassigned(); - shardsToBatch.add(shardRouting); - } - // if shard is already batched update to latest shardRouting information in the batches - else if (shardRouting.primary() == primary) { - String batchId = getBatchId(shardRouting, shardRouting.primary()); - batchesToBeAssigned.add(batchId); - currentBatches.get(batchId).batchInfo.get(shardRouting.shardId()).setShardRouting(shardRouting); - } - }); - Iterator iterator = shardsToBatch.iterator(); - assert maxBatchSize > 0 : "Shards batch size must be greater than 0"; - - long batchSize = maxBatchSize; - Map addToCurrentBatch = new HashMap<>(); - while (iterator.hasNext()) { - ShardRouting currentShard = iterator.next(); - if (batchSize > 0) { - ShardEntry sharEntry = new ShardEntry( - IndexMetadata.INDEX_DATA_PATH_SETTING.get(allocation.metadata().index(currentShard.index()).getSettings()), - currentShard - ); - addToCurrentBatch.put(currentShard.shardId(), sharEntry); - batchSize--; - iterator.remove(); - } - // add to batch if batch size full or last shard in unassigned list - if (batchSize == 0 || iterator.hasNext() == false) { - String batchUUId = UUIDs.base64UUID(); - ShardsBatch shardsBatch = new ShardsBatch(batchUUId, addToCurrentBatch, primary); - // add the batch to list of current batches - addBatch(shardsBatch, primary); - batchesToBeAssigned.add(batchUUId); - addToCurrentBatch.clear(); - batchSize = maxBatchSize; - } - } - return batchesToBeAssigned; - } - - private void addBatch(ShardsBatch shardsBatch, boolean primary) { - ConcurrentMap batches = primary ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; - if (batches.containsKey(shardsBatch.getBatchId())) { - throw new IllegalStateException("Batch already exists. BatchId = " + shardsBatch.getBatchId()); - } - batches.put(shardsBatch.getBatchId(), shardsBatch); - } - - /** - * Safely remove a shard from the appropriate batch depending on if it is primary or replica - * If the shard is not in a batch, this is a no-op. - * Cleans the batch if it is empty after removing the shard. - * This method should be called when removing the shard from the batch instead {@link ShardsBatch#removeFromBatch(ShardRouting)} - * so that we can clean up the batch if it is empty and release the fetching resources - * - * @param shardRouting shard to be removed - */ - protected void safelyRemoveShardFromBatch(ShardRouting shardRouting) { - String batchId = shardRouting.primary() ? getBatchId(shardRouting, true) : getBatchId(shardRouting, false); - if (batchId == null) { - return; - } - ConcurrentMap batches = shardRouting.primary() ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; - ShardsBatch batch = batches.get(batchId); - batch.removeFromBatch(shardRouting); - deleteBatchIfEmpty(batches, batchId); - } - - /** - * Safely remove shard from both the batches irrespective of its primary or replica, - * For the corresponding shardId. The method intends to clean up the batch if it is empty - * after removing the shard - * @param shardRouting shard to remove - */ - protected void safelyRemoveShardFromBothBatch(ShardRouting shardRouting) { - String primaryBatchId = getBatchId(shardRouting, true); - String replicaBatchId = getBatchId(shardRouting, false); - if (primaryBatchId == null && replicaBatchId == null) { - return; - } - if (primaryBatchId != null) { - ShardsBatch batch = batchIdToStartedShardBatch.get(primaryBatchId); - batch.removeFromBatch(shardRouting); - deleteBatchIfEmpty(batchIdToStartedShardBatch, primaryBatchId); - } - if (replicaBatchId != null) { - ShardsBatch batch = batchIdToStoreShardBatch.get(replicaBatchId); - batch.removeFromBatch(shardRouting); - deleteBatchIfEmpty(batchIdToStoreShardBatch, replicaBatchId); - } - } - - private void deleteBatchIfEmpty(ConcurrentMap batches, String batchId) { - if (batches.containsKey(batchId)) { - ShardsBatch batch = batches.get(batchId); - if (batch.getBatchedShards().isEmpty()) { - Releasables.close(batch.getAsyncFetcher()); - batches.remove(batchId); - } - } - } - - protected String getBatchId(ShardRouting shardRouting, boolean primary) { - ConcurrentMap batches = primary ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; - - return batches.entrySet() - .stream() - .filter(entry -> entry.getValue().getBatchedShards().contains(shardRouting.shardId())) - .findFirst() - .map(Map.Entry::getKey) - .orElse(null); - } - // allow for testing infra to change shard allocators implementation protected static void innerAllocatedUnassigned( RoutingAllocation allocation, @@ -467,7 +227,9 @@ private static void clearCacheForPrimary( AsyncShardFetch fetch, RoutingAllocation allocation ) { - ShardRouting primary = allocation.routingNodes().activePrimary(fetch.shardId); + assert fetch.shardAttributesMap.size() == 1 : "expected only one shard"; + ShardId shardId = fetch.shardAttributesMap.keySet().iterator().next(); + ShardRouting primary = allocation.routingNodes().activePrimary(shardId); if (primary != null) { fetch.clearCacheForNode(primary.currentNodeId()); } @@ -495,42 +257,15 @@ class InternalAsyncFetch extends AsyncShardFetch } @Override - protected void reroute(String logKey, String reason) { - logger.trace("{} scheduling reroute for {}", logKey, reason); + protected void reroute(String reroutingKey, String reason) { + logger.trace("{} scheduling reroute for {}", reroutingKey, reason); assert rerouteService != null; rerouteService.reroute( "async_shard_fetch", Priority.HIGH, ActionListener.wrap( - r -> logger.trace("{} scheduled reroute completed for {}", logKey, reason), - e -> logger.debug(new ParameterizedMessage("{} scheduled reroute failed for {}", logKey, reason), e) - ) - ); - } - } - - class InternalBatchAsyncFetch extends AsyncShardFetch { - - InternalBatchAsyncFetch( - Logger logger, - String type, - Map map, - AsyncShardFetch.Lister, T> action, - String batchUUId - ) { - super(logger, type, map, action, batchUUId); - } - - @Override - protected void reroute(String logKey, String reason) { - logger.trace("{} scheduling reroute for {}", logKey, reason); - assert rerouteService != null; - rerouteService.reroute( - "async_shard_batch_fetch", - Priority.HIGH, - ActionListener.wrap( - r -> logger.trace("{} scheduled reroute completed for {}", logKey, reason), - e -> logger.debug(new ParameterizedMessage("{} scheduled reroute failed for {}", logKey, reason), e) + r -> logger.trace("{} scheduled reroute completed for {}", reroutingKey, reason), + e -> logger.debug(new ParameterizedMessage("{} scheduled reroute failed for {}", reroutingKey, reason), e) ) ); } @@ -561,7 +296,11 @@ protected AsyncShardFetch.FetchResult shardState = fetch.fetchData( allocation.nodes(), - allocation.getIgnoreNodes(shard.shardId()) + new HashMap<>() { + { + put(shard.shardId(), allocation.getIgnoreNodes(shard.shardId())); + } + } ); if (shardState.hasData()) { @@ -571,23 +310,6 @@ protected AsyncShardFetch.FetchResult fetchData( - ShardRouting shard, - RoutingAllocation allocation - ) { - return null; - } - - } - class InternalReplicaShardAllocator extends ReplicaShardAllocator { private final TransportNodesListShardStoreMetadata storeAction; @@ -613,7 +335,11 @@ protected AsyncShardFetch.FetchResult shardStores = fetch.fetchData( allocation.nodes(), - allocation.getIgnoreNodes(shard.shardId()) + new HashMap<>() { + { + put(shard.shardId(), allocation.getIgnoreNodes(shard.shardId())); + } + } ); if (shardStores.hasData()) { shardStores.processAllocation(allocation); @@ -626,143 +352,4 @@ protected boolean hasInitiatedFetching(ShardRouting shard) { return asyncFetchStore.get(shard.shardId()) != null; } } - - class InternalReplicaBatchShardAllocator extends ReplicaShardAllocator { - - private final TransportNodesListShardStoreMetadataBatch storeAction; - - InternalReplicaBatchShardAllocator(TransportNodesListShardStoreMetadataBatch storeAction) { - this.storeAction = storeAction; - } - - @Override - protected AsyncShardFetch.FetchResult fetchData( - ShardRouting shard, - RoutingAllocation allocation - ) { - return null; - } - - @Override - protected boolean hasInitiatedFetching(ShardRouting shard) { - return false; - } - } - - /** - * Holds information about a batch of shards to be allocated. - * Async fetcher is used to fetch the data for the batch. - * - * Visible for testing - */ - public class ShardsBatch { - private final String batchId; - private final boolean primary; - - private final AsyncShardFetch asyncBatch; - - private final Map batchInfo; - - public ShardsBatch(String batchId, Map shardsWithInfo, boolean primary) { - this.batchId = batchId; - this.batchInfo = new HashMap<>(shardsWithInfo); - // create a ShardId -> customDataPath map for async fetch - Map shardIdsMap = batchInfo.entrySet() - .stream() - .collect(Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().getCustomDataPath())); - this.primary = primary; - if (primary) { - asyncBatch = new InternalBatchAsyncFetch<>(logger, "batch_shards_started", shardIdsMap, batchStartedAction, batchId); - } else { - asyncBatch = new InternalBatchAsyncFetch<>(logger, "batch_shards_started", shardIdsMap, batchStoreAction, batchId); - - } - } - - private void removeFromBatch(ShardRouting shard) { - batchInfo.remove(shard.shardId()); - asyncBatch.shardToCustomDataPath.remove(shard.shardId()); - // assert that fetcher and shards are the same as batched shards - assert batchInfo.size() == asyncBatch.shardToCustomDataPath.size() : "Shards size is not equal to fetcher size"; - } - - public Set getBatchedShardRoutings() { - return batchInfo.values().stream().map(ShardEntry::getShardRouting).collect(Collectors.toSet()); - } - - public Set getBatchedShards() { - return batchInfo.keySet(); - } - - public String getBatchId() { - return batchId; - } - - public AsyncShardFetch getAsyncFetcher() { - return asyncBatch; - } - - public int getNumberOfInFlightFetches() { - return asyncBatch.getNumberOfInFlightFetches(); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || o instanceof ShardsBatch == false) { - return false; - } - ShardsBatch shardsBatch = (ShardsBatch) o; - return batchId.equals(shardsBatch.getBatchId()) && batchInfo.keySet().equals(shardsBatch.getBatchedShards()); - } - - @Override - public int hashCode() { - return Objects.hash(batchId); - } - - @Override - public String toString() { - return "batchId: " + batchId; - } - - } - - /** - * Holds information about a shard to be allocated in a batch. - */ - private class ShardEntry { - - private final String customDataPath; - - public ShardEntry setShardRouting(ShardRouting shardRouting) { - this.shardRouting = shardRouting; - return this; - } - - private ShardRouting shardRouting; - - public ShardEntry(String customDataPath, ShardRouting shardRouting) { - this.customDataPath = customDataPath; - this.shardRouting = shardRouting; - } - - public ShardRouting getShardRouting() { - return shardRouting; - } - - public String getCustomDataPath() { - return customDataPath; - } - } - - public int getNumberOfStartedShardBatches() { - return batchIdToStoreShardBatch.size(); - } - - public int getNumberOfStoreShardBatches() { - return batchIdToStoreShardBatch.size(); - } } diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java new file mode 100644 index 0000000000000..13d2b518d25ef --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -0,0 +1,703 @@ +/* + * 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. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.gateway; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.opensearch.action.support.nodes.BaseNodeResponse; +import org.opensearch.action.support.nodes.BaseNodesResponse; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.RerouteService; +import org.opensearch.cluster.routing.RoutingNodes; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; +import org.opensearch.cluster.routing.allocation.FailedShard; +import org.opensearch.cluster.routing.allocation.RoutingAllocation; +import org.opensearch.common.Priority; +import org.opensearch.common.UUIDs; +import org.opensearch.common.inject.Inject; +import org.opensearch.common.lease.Releasables; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.concurrent.ConcurrentCollections; +import org.opensearch.common.util.set.Sets; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.indices.store.ShardAttributes; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; + +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.Objects; +import java.util.Set; +import java.util.Spliterators; +import java.util.concurrent.ConcurrentMap; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +/** + * Allocator for the Shards batch gateway + * + * @opensearch.internal + */ +public class ShardsBatchGatewayAllocator extends GatewayAllocator { + + public static final String ALLOCATOR_NAME = "shards_batch_gateway_allocator"; + + private static final Logger logger = LogManager.getLogger(ShardsBatchGatewayAllocator.class); + private final long maxBatchSize; + private final boolean batchMode; + private static final short DEFAULT_SHARD_BATCH_SIZE = 2000; + + /** + * Number of shards we send in one batch to data nodes for fetching metadata + */ + public static final Setting GATEWAY_ALLOCATOR_BATCH_SIZE = Setting.longSetting( + "cluster.allocator.gateway.batch_size", + DEFAULT_SHARD_BATCH_SIZE, + 1, + 10000, + Setting.Property.NodeScope + ); + + private final RerouteService rerouteService; + + private PrimaryShardBatchAllocator primaryBatchShardAllocator; + private ReplicaShardBatchAllocator replicaBatchShardAllocator; + + private Set lastSeenEphemeralIds = Collections.emptySet(); + + // visble for testing + protected final ConcurrentMap batchIdToStartedShardBatch = ConcurrentCollections.newConcurrentMap(); + + // visible for testing + protected final ConcurrentMap batchIdToStoreShardBatch = ConcurrentCollections.newConcurrentMap(); + + private final TransportNodesListGatewayStartedBatchShards batchStartedAction; + private final TransportNodesListShardStoreMetadataBatch batchStoreAction; + + @Inject + public ShardsBatchGatewayAllocator( + RerouteService rerouteService, + TransportNodesListGatewayStartedShards startedAction, + TransportNodesListShardStoreMetadata storeAction, + TransportNodesListGatewayStartedBatchShards batchStartedAction, + TransportNodesListShardStoreMetadataBatch batchStoreAction, + Settings settings + ) { + super(rerouteService, startedAction, storeAction); + this.rerouteService = rerouteService; + this.primaryBatchShardAllocator = new InternalPrimaryBatchShardAllocator(); + this.replicaBatchShardAllocator = new InternalReplicaBatchShardAllocator(); + this.batchStartedAction = batchStartedAction; + this.batchStoreAction = batchStoreAction; + this.maxBatchSize = GATEWAY_ALLOCATOR_BATCH_SIZE.get(settings); + this.batchMode = EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.get(settings); + } + + @Override + public void cleanCaches() { + Releasables.close( + batchIdToStartedShardBatch.values().stream().map(shardsBatch -> shardsBatch.asyncBatch).collect(Collectors.toList()) + ); + batchIdToStartedShardBatch.clear(); + Releasables.close( + batchIdToStoreShardBatch.values().stream().map(shardsBatch -> shardsBatch.asyncBatch).collect(Collectors.toList()) + ); + batchIdToStoreShardBatch.clear(); + } + + // for tests + protected ShardsBatchGatewayAllocator() { + this.rerouteService = null; + this.batchStartedAction = null; + this.primaryBatchShardAllocator = null; + this.batchStoreAction = null; + this.replicaBatchShardAllocator = null; + this.maxBatchSize = DEFAULT_SHARD_BATCH_SIZE; + this.batchMode = true; + } + + // for tests + + @Override + public int getNumberOfInFlightFetches() { + int count = 0; + // If fetching is done in non batched-mode then maps to maintain batches will be empty and vice versa for batch-mode + for (ShardsBatch batch : batchIdToStartedShardBatch.values()) { + count += (batch.getNumberOfInFlightFetches() * batch.getBatchedShards().size()); + } + for (ShardsBatch batch : batchIdToStoreShardBatch.values()) { + count += (batch.getNumberOfInFlightFetches() * batch.getBatchedShards().size()); + } + + return count; + } + + @Override + public void applyStartedShards(final List startedShards, final RoutingAllocation allocation) { + for (ShardRouting startedShard : startedShards) { + safelyRemoveShardFromBothBatch(startedShard); + } + } + + @Override + public void applyFailedShards(final List failedShards, final RoutingAllocation allocation) { + for (FailedShard failedShard : failedShards) { + safelyRemoveShardFromBothBatch(failedShard.getRoutingEntry()); + } + } + + @Override + public void beforeAllocation(final RoutingAllocation allocation) { + assert primaryBatchShardAllocator != null; + assert replicaBatchShardAllocator != null; + ensureAsyncFetchStorePrimaryRecency(allocation); + } + + @Override + public void afterPrimariesBeforeReplicas(RoutingAllocation allocation) { + assert replicaBatchShardAllocator != null; + List> storedShardBatches = batchIdToStoreShardBatch.values() + .stream() + .map(ShardsBatch::getBatchedShardRoutings) + .collect(Collectors.toList()); + if (allocation.routingNodes().hasInactiveShards()) { + // cancel existing recoveries if we have a better match + replicaBatchShardAllocator.processExistingRecoveries(allocation, storedShardBatches); + } + } + + @Override + public void allocateUnassignedBatch(final RoutingAllocation allocation, boolean primary) { + + assert primaryBatchShardAllocator != null; + assert replicaBatchShardAllocator != null; + innerAllocateUnassignedBatch(allocation, primaryBatchShardAllocator, replicaBatchShardAllocator, primary); + } + + protected void innerAllocateUnassignedBatch( + RoutingAllocation allocation, + PrimaryShardBatchAllocator primaryBatchShardAllocator, + ReplicaShardBatchAllocator replicaBatchShardAllocator, + boolean primary + ) { + // create batches for unassigned shards + Set batchesToAssign = createAndUpdateBatches(allocation, primary); + if (batchesToAssign.isEmpty()) { + return; + } + if (primary) { + batchIdToStartedShardBatch.values() + .stream() + .filter(batch -> batchesToAssign.contains(batch.batchId)) + .forEach( + shardsBatch -> primaryBatchShardAllocator.allocateUnassignedBatch(shardsBatch.getBatchedShardRoutings(), allocation) + ); + } else { + batchIdToStoreShardBatch.values() + .stream() + .filter(batch -> batchesToAssign.contains(batch.batchId)) + .forEach(batch -> replicaBatchShardAllocator.allocateUnassignedBatch(batch.getBatchedShardRoutings(), allocation)); + } + } + + // visible for testing + protected Set createAndUpdateBatches(RoutingAllocation allocation, boolean primary) { + Set batchesToBeAssigned = new HashSet<>(); + RoutingNodes.UnassignedShards unassigned = allocation.routingNodes().unassigned(); + ConcurrentMap currentBatches = primary ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; + // get all batched shards + Set currentBatchedShards = currentBatches.values() + .stream() + .map(ShardsBatch::getBatchedShards) + .flatMap(Set::stream) + .collect(Collectors.toSet()); + + Set shardsToBatch = Sets.newHashSet(); + // add all unassigned shards to the batch if they are not already in a batch + unassigned.forEach(shardRouting -> { + if ((currentBatchedShards.contains(shardRouting.shardId()) == false) && (shardRouting.primary() == primary)) { + assert shardRouting.unassigned(); + shardsToBatch.add(shardRouting); + } + // if shard is already batched update to latest shardRouting information in the batches + else if (shardRouting.primary() == primary) { + String batchId = getBatchId(shardRouting, shardRouting.primary()); + batchesToBeAssigned.add(batchId); + currentBatches.get(batchId).batchInfo.get(shardRouting.shardId()).setShardRouting(shardRouting); + } + }); + Iterator iterator = shardsToBatch.iterator(); + assert maxBatchSize > 0 : "Shards batch size must be greater than 0"; + + long batchSize = maxBatchSize; + Map shardsToAddToCurrentBatch = new HashMap<>(); + while (iterator.hasNext()) { + ShardRouting currentShard = iterator.next(); + if (batchSize > 0) { + ShardEntry sharEntry = new ShardEntry( + new ShardAttributes( + currentShard.shardId(), + IndexMetadata.INDEX_DATA_PATH_SETTING.get(allocation.metadata().index(currentShard.index()).getSettings()) + ), + currentShard + ); + shardsToAddToCurrentBatch.put(currentShard.shardId(), sharEntry); + batchSize--; + iterator.remove(); + } + // add to batch if batch size full or last shard in unassigned list + if (batchSize == 0 || iterator.hasNext() == false) { + String batchUUId = UUIDs.base64UUID(); + ShardsBatch shardsBatch = new ShardsBatch(batchUUId, shardsToAddToCurrentBatch, primary); + // add the batch to list of current batches + addBatch(shardsBatch, primary); + batchesToBeAssigned.add(batchUUId); + shardsToAddToCurrentBatch.clear(); + batchSize = maxBatchSize; + } + } + return batchesToBeAssigned; + } + + private void addBatch(ShardsBatch shardsBatch, boolean primary) { + ConcurrentMap batches = primary ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; + if (batches.containsKey(shardsBatch.getBatchId())) { + throw new IllegalStateException("Batch already exists. BatchId = " + shardsBatch.getBatchId()); + } + batches.put(shardsBatch.getBatchId(), shardsBatch); + } + + /** + * Safely remove a shard from the appropriate batch depending on if it is primary or replica + * If the shard is not in a batch, this is a no-op. + * Cleans the batch if it is empty after removing the shard. + * This method should be called when removing the shard from the batch instead {@link ShardsBatch#removeFromBatch(ShardRouting)} + * so that we can clean up the batch if it is empty and release the fetching resources + * + * @param shardRouting shard to be removed + */ + protected void safelyRemoveShardFromBatch(ShardRouting shardRouting) { + String batchId = shardRouting.primary() ? getBatchId(shardRouting, true) : getBatchId(shardRouting, false); + if (batchId == null) { + logger.debug("Shard[{}] is not batched", shardRouting); + return; + } + ConcurrentMap batches = shardRouting.primary() ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; + ShardsBatch batch = batches.get(batchId); + batch.removeFromBatch(shardRouting); + deleteBatchIfEmpty(batches, batchId); + } + + /** + * Safely remove shard from both the batches irrespective of its primary or replica, + * For the corresponding shardId. The method intends to clean up the batch if it is empty + * after removing the shard + * @param shardRouting shard to remove + */ + protected void safelyRemoveShardFromBothBatch(ShardRouting shardRouting) { + String primaryBatchId = getBatchId(shardRouting, true); + String replicaBatchId = getBatchId(shardRouting, false); + if (primaryBatchId == null && replicaBatchId == null) { + return; + } + if (primaryBatchId != null) { + ShardsBatch batch = batchIdToStartedShardBatch.get(primaryBatchId); + batch.removeFromBatch(shardRouting); + deleteBatchIfEmpty(batchIdToStartedShardBatch, primaryBatchId); + } + if (replicaBatchId != null) { + ShardsBatch batch = batchIdToStoreShardBatch.get(replicaBatchId); + batch.removeFromBatch(shardRouting); + deleteBatchIfEmpty(batchIdToStoreShardBatch, replicaBatchId); + } + } + + private void deleteBatchIfEmpty(ConcurrentMap batches, String batchId) { + if (batches.containsKey(batchId)) { + ShardsBatch batch = batches.get(batchId); + if (batch.getBatchedShards().isEmpty()) { + Releasables.close(batch.getAsyncFetcher()); + batches.remove(batchId); + } + } + } + + protected String getBatchId(ShardRouting shardRouting, boolean primary) { + ConcurrentMap batches = primary ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; + + return batches.entrySet() + .stream() + .filter(entry -> entry.getValue().getBatchedShards().contains(shardRouting.shardId())) + .findFirst() + .map(Map.Entry::getKey) + .orElse(null); + } + + @Override + public AllocateUnassignedDecision explainUnassignedShardAllocation(ShardRouting unassignedShard, RoutingAllocation routingAllocation) { + assert unassignedShard.unassigned(); + assert routingAllocation.debugDecision(); + if (getBatchId(unassignedShard, unassignedShard.primary()) == null) { + createAndUpdateBatches(routingAllocation, unassignedShard.primary()); + } + assert getBatchId(unassignedShard, unassignedShard.primary()) != null; + if (unassignedShard.primary()) { + assert primaryBatchShardAllocator != null; + return primaryBatchShardAllocator.makeAllocationDecision(unassignedShard, routingAllocation, logger); + } else { + assert replicaBatchShardAllocator != null; + return replicaBatchShardAllocator.makeAllocationDecision(unassignedShard, routingAllocation, logger); + } + } + + /** + * Clear the fetched data for the primary to ensure we do not cancel recoveries based on excessively stale data. + */ + private void ensureAsyncFetchStorePrimaryRecency(RoutingAllocation allocation) { + DiscoveryNodes nodes = allocation.nodes(); + if (hasNewNodes(nodes)) { + final Set newEphemeralIds = StreamSupport.stream(Spliterators.spliterator(nodes.getDataNodes().entrySet(), 0), false) + .map(node -> node.getValue().getEphemeralId()) + .collect(Collectors.toSet()); + // Invalidate the cache if a data node has been added to the cluster. This ensures that we do not cancel a recovery if a node + // drops out, we fetch the shard data, then some indexing happens and then the node rejoins the cluster again. There are other + // ways we could decide to cancel a recovery based on stale data (e.g. changing allocation filters or a primary failure) but + // making the wrong decision here is not catastrophic so we only need to cover the common case. + logger.trace( + () -> new ParameterizedMessage( + "new nodes {} found, clearing primary async-fetch-store cache", + Sets.difference(newEphemeralIds, lastSeenEphemeralIds) + ) + ); + // ToDo : Validate that we don't need below call for batch allocation + // storeShardBatchLookup.values().forEach(batch -> + // clearCacheForBatchPrimary(batchIdToStoreShardBatch.get(batch), allocation) + // ); + batchIdToStoreShardBatch.values().forEach(batch -> clearCacheForBatchPrimary(batch, allocation)); + + // recalc to also (lazily) clear out old nodes. + this.lastSeenEphemeralIds = newEphemeralIds; + } + } + + private static void clearCacheForBatchPrimary(ShardsBatch batch, RoutingAllocation allocation) { + // We're not running below code because for removing a node from cache we need all replica's primaries + // to be assigned on same node. This was easy in single shard case and we're saving a call for a node + // if primary was already assigned for a replica. But here we don't keep track of per shard data in cache + // so it's not feasible to do any removal of node entry just based on single shard. + // ONLY run if single shard is present in the batch, to maintain backward compatibility + if (batch.getBatchedShards().size() == 1) { + List primaries = batch.getBatchedShards() + .stream() + .map(allocation.routingNodes()::activePrimary) + .filter(Objects::nonNull) + .collect(Collectors.toList()); + AsyncShardFetch fetch = batch.getAsyncFetcher(); + primaries.forEach(node -> fetch.clearCacheForNode(node.currentNodeId())); + } + } + + private boolean hasNewNodes(DiscoveryNodes nodes) { + for (final DiscoveryNode node : nodes.getDataNodes().values()) { + if (lastSeenEphemeralIds.contains(node.getEphemeralId()) == false) { + return true; + } + } + return false; + } + + class InternalBatchAsyncFetch extends AsyncShardFetch { + InternalBatchAsyncFetch( + Logger logger, + String type, + Map map, + AsyncShardFetch.Lister, T> action, + String batchUUId + ) { + super(logger, type, map, action, batchUUId); + } + + @Override + protected void reroute(String reroutingKey, String reason) { + logger.trace("{} scheduling reroute for {}", reroutingKey, reason); + assert rerouteService != null; + rerouteService.reroute( + "async_shard_batch_fetch", + Priority.HIGH, + ActionListener.wrap( + r -> logger.trace("{} scheduled reroute completed for {}", reroutingKey, reason), + e -> logger.debug(new ParameterizedMessage("{} scheduled reroute failed for {}", reroutingKey, reason), e) + ) + ); + } + } + + class InternalPrimaryBatchShardAllocator extends PrimaryShardBatchAllocator { + + @Override + @SuppressWarnings("unchecked") + protected AsyncShardFetch.FetchResult fetchData( + Set shardsEligibleForFetch, + Set inEligibleShards, + RoutingAllocation allocation + ) { + ShardRouting shardRouting = shardsEligibleForFetch.iterator().hasNext() ? shardsEligibleForFetch.iterator().next() : null; + shardRouting = shardRouting == null && inEligibleShards.iterator().hasNext() + ? inEligibleShards.iterator().next() + : shardRouting; + if (shardRouting == null) { + return new AsyncShardFetch.FetchResult<>(null, Collections.emptyMap()); + } + String batchId = getBatchId(shardRouting, shardRouting.primary()); + if (batchId == null) { + logger.debug("Shard {} has no batch id", shardRouting); + throw new IllegalStateException("Shard " + shardRouting + " has no batch id. Shard should batched before fetching"); + } + + if (batchIdToStartedShardBatch.containsKey(batchId) == false) { + logger.debug("Batch {} has no started shard batch", batchId); + throw new IllegalStateException("Batch " + batchId + " has no started shard batch"); + } + + ShardsBatch shardsBatch = batchIdToStartedShardBatch.get(batchId); + // remove in eligible shards which allocator is not responsible for + inEligibleShards.forEach(ShardsBatchGatewayAllocator.this::safelyRemoveShardFromBatch); + + if (shardsBatch.getBatchedShards().isEmpty() && shardsEligibleForFetch.isEmpty()) { + logger.debug("Batch {} is empty", batchId); + return new AsyncShardFetch.FetchResult<>(null, Collections.emptyMap()); + } + + Map> shardToIgnoreNodes = new HashMap<>(); + + for (ShardId shardId : shardsBatch.asyncBatch.shardAttributesMap.keySet()) { + shardToIgnoreNodes.put(shardId, allocation.getIgnoreNodes(shardId)); + } + AsyncShardFetch asyncFetcher = shardsBatch.getAsyncFetcher(); + AsyncShardFetch.FetchResult shardBatchState = asyncFetcher.fetchData( + allocation.nodes(), + shardToIgnoreNodes + ); + + if (shardBatchState.hasData()) { + shardBatchState.processAllocation(allocation); + } + return (AsyncShardFetch.FetchResult) shardBatchState; + } + + } + + class InternalReplicaBatchShardAllocator extends ReplicaShardBatchAllocator { + @Override + @SuppressWarnings("unchecked") + protected AsyncShardFetch.FetchResult fetchData( + Set shardsEligibleForFetch, + Set inEligibleShards, + RoutingAllocation allocation + ) { + // get batch id for anyone given shard. We are assuming all shards will have same batch Id + ShardRouting shardRouting = shardsEligibleForFetch.iterator().hasNext() ? shardsEligibleForFetch.iterator().next() : null; + shardRouting = shardRouting == null && inEligibleShards.iterator().hasNext() + ? inEligibleShards.iterator().next() + : shardRouting; + if (shardRouting == null) { + return new AsyncShardFetch.FetchResult<>(null, Collections.emptyMap()); + } + String batchId = getBatchId(shardRouting, shardRouting.primary()); + if (batchId == null) { + logger.debug("Shard {} has no batch id", shardRouting); + throw new IllegalStateException("Shard " + shardRouting + " has no batch id. Shard should batched before fetching"); + } + + if (batchIdToStoreShardBatch.containsKey(batchId) == false) { + logger.debug("Batch {} has no store shard batch", batchId); + throw new IllegalStateException("Batch " + batchId + " has no shard store batch"); + } + + ShardsBatch shardsBatch = batchIdToStoreShardBatch.get(batchId); + // remove in eligible shards which allocator is not responsible for + inEligibleShards.forEach(ShardsBatchGatewayAllocator.this::safelyRemoveShardFromBatch); + + if (shardsBatch.getBatchedShards().isEmpty() && shardsEligibleForFetch.isEmpty()) { + logger.debug("Batch {} is empty", batchId); + return new AsyncShardFetch.FetchResult<>(null, Collections.emptyMap()); + } + Map> shardToIgnoreNodes = new HashMap<>(); + for (ShardId shardId : shardsBatch.asyncBatch.shardAttributesMap.keySet()) { + shardToIgnoreNodes.put(shardId, allocation.getIgnoreNodes(shardId)); + } + AsyncShardFetch asyncFetcher = shardsBatch.getAsyncFetcher(); + AsyncShardFetch.FetchResult shardBatchStores = asyncFetcher.fetchData( + allocation.nodes(), + shardToIgnoreNodes + ); + if (shardBatchStores.hasData()) { + shardBatchStores.processAllocation(allocation); + } + return (AsyncShardFetch.FetchResult) shardBatchStores; + } + + @Override + protected boolean hasInitiatedFetching(ShardRouting shard) { + String batchId = getBatchId(shard, shard.primary()); + return batchId != null; + } + } + + /** + * Holds information about a batch of shards to be allocated. + * Async fetcher is used to fetch the data for the batch. + * + * Visible for testing + */ + public class ShardsBatch { + private final String batchId; + private final boolean primary; + + private final AsyncShardFetch asyncBatch; + + private final Map batchInfo; + + public ShardsBatch(String batchId, Map shardsWithInfo, boolean primary) { + this.batchId = batchId; + this.batchInfo = new HashMap<>(shardsWithInfo); + // create a ShardId -> customDataPath map for async fetch + Map shardIdsMap = batchInfo.entrySet() + .stream() + .collect(Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().getShardAttributes())); + this.primary = primary; + if (primary) { + asyncBatch = new InternalBatchAsyncFetch<>(logger, "batch_shards_started", shardIdsMap, batchStartedAction, batchId); + } else { + asyncBatch = new InternalBatchAsyncFetch<>(logger, "batch_shards_started", shardIdsMap, batchStoreAction, batchId); + + } + } + + private void removeFromBatch(ShardRouting shard) { + batchInfo.remove(shard.shardId()); + asyncBatch.shardAttributesMap.remove(shard.shardId()); + // assert that fetcher and shards are the same as batched shards + assert batchInfo.size() == asyncBatch.shardAttributesMap.size() : "Shards size is not equal to fetcher size"; + } + + public Set getBatchedShardRoutings() { + return batchInfo.values().stream().map(ShardEntry::getShardRouting).collect(Collectors.toSet()); + } + + public Set getBatchedShards() { + return batchInfo.keySet(); + } + + public String getBatchId() { + return batchId; + } + + public AsyncShardFetch getAsyncFetcher() { + return asyncBatch; + } + + public int getNumberOfInFlightFetches() { + return asyncBatch.getNumberOfInFlightFetches(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || o instanceof ShardsBatch == false) { + return false; + } + ShardsBatch shardsBatch = (ShardsBatch) o; + return batchId.equals(shardsBatch.getBatchId()) && batchInfo.keySet().equals(shardsBatch.getBatchedShards()); + } + + @Override + public int hashCode() { + return Objects.hash(batchId); + } + + @Override + public String toString() { + return "batchId: " + batchId; + } + + } + + /** + * Holds information about a shard to be allocated in a batch. + */ + private class ShardEntry { + + private final ShardAttributes shardAttributes; + + public ShardEntry setShardRouting(ShardRouting shardRouting) { + this.shardRouting = shardRouting; + return this; + } + + private ShardRouting shardRouting; + + public ShardEntry(ShardAttributes shardAttributes, ShardRouting shardRouting) { + this.shardAttributes = shardAttributes; + this.shardRouting = shardRouting; + } + + public ShardRouting getShardRouting() { + return shardRouting; + } + + public ShardAttributes getShardAttributes() { + return shardAttributes; + } + } + + public int getNumberOfStartedShardBatches() { + return batchIdToStoreShardBatch.size(); + } + + public int getNumberOfStoreShardBatches() { + return batchIdToStoreShardBatch.size(); + } +} diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index dc4a11cf7102a..41a99d62671ba 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -139,6 +139,7 @@ import org.opensearch.env.NodeEnvironment; import org.opensearch.env.NodeMetadata; import org.opensearch.gateway.GatewayAllocator; +import org.opensearch.gateway.ShardsBatchGatewayAllocator; import org.opensearch.gateway.GatewayMetaState; import org.opensearch.gateway.GatewayModule; import org.opensearch.gateway.GatewayService; @@ -1132,7 +1133,14 @@ protected Node( // completes we trigger another reroute to try the allocation again. This means there is a circular dependency: the allocation // service needs access to the existing shards allocators (e.g. the GatewayAllocator) which need to be able to trigger a // reroute, which needs to call into the allocation service. We close the loop here: - clusterModule.setExistingShardsAllocators(injector.getInstance(GatewayAllocator.class)); + // create Hashmap for existing Allocators + Map gatewayAllocatorMap = new HashMap<>() { + { + put(GatewayAllocator.ALLOCATOR_NAME, injector.getInstance(GatewayAllocator.class)); + put(ShardsBatchGatewayAllocator.ALLOCATOR_NAME, injector.getInstance(ShardsBatchGatewayAllocator.class)); + } + }; + clusterModule.setExistingShardsAllocators(gatewayAllocatorMap); List pluginLifecycleComponents = pluginComponents.stream() .filter(p -> p instanceof LifecycleComponent) diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/reroute/ClusterRerouteTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/reroute/ClusterRerouteTests.java index 859d8ce3bb734..4d93aa94e8b9a 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/reroute/ClusterRerouteTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/reroute/ClusterRerouteTests.java @@ -55,7 +55,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.snapshots.EmptySnapshotsInfoService; -import org.opensearch.test.gateway.TestGatewayAllocator; +import org.opensearch.test.gateway.TestShardBatchGatewayAllocator; import java.io.IOException; import java.util.Collections; @@ -94,7 +94,7 @@ public void testSerializeRequest() throws IOException { public void testClusterStateUpdateTask() { AllocationService allocationService = new AllocationService( new AllocationDeciders(Collections.singleton(new MaxRetryAllocationDecider())), - new TestGatewayAllocator(), + new TestShardBatchGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE, EmptySnapshotsInfoService.INSTANCE diff --git a/server/src/test/java/org/opensearch/cluster/ClusterModuleTests.java b/server/src/test/java/org/opensearch/cluster/ClusterModuleTests.java index 535444cd866b8..f64e6b146e001 100644 --- a/server/src/test/java/org/opensearch/cluster/ClusterModuleTests.java +++ b/server/src/test/java/org/opensearch/cluster/ClusterModuleTests.java @@ -75,6 +75,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -291,7 +292,11 @@ public void testRejectsReservedExistingShardsAllocatorName() { null, threadContext ); - expectThrows(IllegalArgumentException.class, () -> clusterModule.setExistingShardsAllocators(new TestGatewayAllocator())); + expectThrows(IllegalArgumentException.class, () -> clusterModule.setExistingShardsAllocators(new HashMap<>() { + { + put(GatewayAllocator.ALLOCATOR_NAME, new TestGatewayAllocator()); + } + })); } public void testRejectsDuplicateExistingShardsAllocatorName() { @@ -303,7 +308,11 @@ public void testRejectsDuplicateExistingShardsAllocatorName() { null, threadContext ); - expectThrows(IllegalArgumentException.class, () -> clusterModule.setExistingShardsAllocators(new TestGatewayAllocator())); + expectThrows(IllegalArgumentException.class, () -> clusterModule.setExistingShardsAllocators(new HashMap<>() { + { + put(GatewayAllocator.ALLOCATOR_NAME, new TestGatewayAllocator()); + } + })); } private static ClusterPlugin existingShardsAllocatorPlugin(final String allocatorName) { diff --git a/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java index ba06cda68facf..ee269e0264b8b 100644 --- a/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java @@ -28,7 +28,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.core.index.shard.ShardId; import org.opensearch.snapshots.SnapshotShardSizeInfo; -import org.opensearch.test.gateway.TestGatewayAllocator; +import org.opensearch.test.gateway.TestShardBatchGatewayAllocator; import org.junit.Before; import java.util.ArrayList; @@ -42,7 +42,7 @@ public class GatewayAllocatorTests extends OpenSearchAllocationTestCase { private final Logger logger = LogManager.getLogger(GatewayAllocatorTests.class); - TestGatewayAllocator testGatewayAllocator = null; + TestShardBatchGatewayAllocator testShardsBatchGatewayAllocator = null; ClusterState clusterState = null; RoutingAllocation testAllocation = null; String indexPrefix = "TEST"; @@ -51,7 +51,7 @@ public class GatewayAllocatorTests extends OpenSearchAllocationTestCase { @Before public void setUp() throws Exception { super.setUp(); - testGatewayAllocator = new TestGatewayAllocator(); + testShardsBatchGatewayAllocator = new TestShardBatchGatewayAllocator(); } public void testSingleBatchCreation() { @@ -63,11 +63,13 @@ public void testTwoBatchCreation() { createIndexAndUpdateClusterState(2, 1020, 1); createBatchesAndAssert(2); - List listOfBatches = new ArrayList<>(testGatewayAllocator.getBatchIdToStartedShardBatch().values()); + List listOfBatches = new ArrayList<>( + testShardsBatchGatewayAllocator.getBatchIdToStartedShardBatch().values() + ); assertNotEquals(listOfBatches.get(0), listOfBatches.get(1)); // test for replicas - listOfBatches = new ArrayList<>(testGatewayAllocator.getBatchIdToStoreShardBatch().values()); + listOfBatches = new ArrayList<>(testShardsBatchGatewayAllocator.getBatchIdToStoreShardBatch().values()); assertNotEquals(listOfBatches.get(0), listOfBatches.get(1)); } @@ -78,8 +80,8 @@ public void testNonDuplicationOfBatch() { assertEquals(1, batches.v2().size()); // again try to create batch and verify no new batch is created since shard is already batched and no new unassigned shard - assertEquals(batches.v1(), testGatewayAllocator.createAndUpdateBatches(testAllocation, true)); - assertEquals(batches.v2(), testGatewayAllocator.createAndUpdateBatches(testAllocation, false)); + assertEquals(batches.v1(), testShardsBatchGatewayAllocator.createAndUpdateBatches(testAllocation, true)); + assertEquals(batches.v2(), testShardsBatchGatewayAllocator.createAndUpdateBatches(testAllocation, false)); } public void testCorrectnessOfBatch() { @@ -101,33 +103,33 @@ public void testCorrectnessOfBatch() { .collect(Collectors.toSet()); shardsSet1.addAll(shardsSet2); - Set shardsInAllbatches = testGatewayAllocator.getBatchIdToStartedShardBatch() + Set shardsInAllbatches = testShardsBatchGatewayAllocator.getBatchIdToStartedShardBatch() .values() .stream() - .map(GatewayAllocator.ShardsBatch::getBatchedShards) + .map(ShardsBatchGatewayAllocator.ShardsBatch::getBatchedShards) .flatMap(Set::stream) .collect(Collectors.toSet()); assertEquals(shardsInAllbatches, shardsSet1); - shardsInAllbatches = testGatewayAllocator.getBatchIdToStoreShardBatch() + shardsInAllbatches = testShardsBatchGatewayAllocator.getBatchIdToStoreShardBatch() .values() .stream() - .map(GatewayAllocator.ShardsBatch::getBatchedShards) + .map(ShardsBatchGatewayAllocator.ShardsBatch::getBatchedShards) .flatMap(Set::stream) .collect(Collectors.toSet()); assertEquals(shardsInAllbatches, shardsSet1); - Set primariesInAllBatches = testGatewayAllocator.getBatchIdToStartedShardBatch() + Set primariesInAllBatches = testShardsBatchGatewayAllocator.getBatchIdToStartedShardBatch() .values() .stream() - .map(GatewayAllocator.ShardsBatch::getBatchedShardRoutings) + .map(ShardsBatchGatewayAllocator.ShardsBatch::getBatchedShardRoutings) .flatMap(Set::stream) .collect(Collectors.toSet()); primariesInAllBatches.forEach(shardRouting -> assertTrue(shardRouting.unassigned() && shardRouting.primary() == true)); - Set replicasInAllBatches = testGatewayAllocator.getBatchIdToStoreShardBatch() + Set replicasInAllBatches = testShardsBatchGatewayAllocator.getBatchIdToStoreShardBatch() .values() .stream() - .map(GatewayAllocator.ShardsBatch::getBatchedShardRoutings) + .map(ShardsBatchGatewayAllocator.ShardsBatch::getBatchedShardRoutings) .flatMap(Set::stream) .collect(Collectors.toSet()); @@ -140,12 +142,12 @@ public void testAsyncFetcherCreationInBatch() { Set primaryBatches = batchesTuple.v1(); Set replicaBatches = batchesTuple.v2(); - GatewayAllocator.ShardsBatch shardsBatch = testGatewayAllocator.getBatchIdToStartedShardBatch() + ShardsBatchGatewayAllocator.ShardsBatch shardsBatch = testShardsBatchGatewayAllocator.getBatchIdToStartedShardBatch() .get(primaryBatches.iterator().next()); AsyncShardFetch asyncFetcher = shardsBatch.getAsyncFetcher(); // assert asyncFetcher is not null assertNotNull(asyncFetcher); - shardsBatch = testGatewayAllocator.getBatchIdToStoreShardBatch().get(replicaBatches.iterator().next()); + shardsBatch = testShardsBatchGatewayAllocator.getBatchIdToStoreShardBatch().get(replicaBatches.iterator().next()); asyncFetcher = shardsBatch.getAsyncFetcher(); assertNotNull(asyncFetcher); } @@ -157,61 +159,67 @@ public void testSafelyRemoveShardFromBatch() { Set primaryBatches = batchesTuple.v1(); Set replicaBatches = batchesTuple.v2(); - GatewayAllocator.ShardsBatch primaryShardsBatch = testGatewayAllocator.getBatchIdToStartedShardBatch() + ShardsBatchGatewayAllocator.ShardsBatch primaryShardsBatch = testShardsBatchGatewayAllocator.getBatchIdToStartedShardBatch() .get(primaryBatches.iterator().next()); ShardRouting primaryShardRouting = primaryShardsBatch.getBatchedShardRoutings().iterator().next(); assertEquals(2, replicaBatches.size()); - GatewayAllocator.ShardsBatch replicaShardsBatch = testGatewayAllocator.getBatchIdToStoreShardBatch() + ShardsBatchGatewayAllocator.ShardsBatch replicaShardsBatch = testShardsBatchGatewayAllocator.getBatchIdToStoreShardBatch() .get(replicaBatches.iterator().next()); ShardRouting replicaShardRouting = replicaShardsBatch.getBatchedShardRoutings().iterator().next(); // delete 1 shard routing from each batch - testGatewayAllocator.safelyRemoveShardFromBatch(primaryShardRouting); + testShardsBatchGatewayAllocator.safelyRemoveShardFromBatch(primaryShardRouting); - testGatewayAllocator.safelyRemoveShardFromBatch(replicaShardRouting); + testShardsBatchGatewayAllocator.safelyRemoveShardFromBatch(replicaShardRouting); // verify that shard routing is removed from both batches assertFalse(primaryShardsBatch.getBatchedShards().contains(primaryShardRouting.shardId())); assertFalse(replicaShardsBatch.getBatchedShards().contains(replicaShardRouting.shardId())); // try to remove that shard again to see if its no op and doent result in exception - testGatewayAllocator.safelyRemoveShardFromBatch(primaryShardRouting); - testGatewayAllocator.safelyRemoveShardFromBatch(replicaShardRouting); + testShardsBatchGatewayAllocator.safelyRemoveShardFromBatch(primaryShardRouting); + testShardsBatchGatewayAllocator.safelyRemoveShardFromBatch(replicaShardRouting); // now remove all shard routings to verify that batch only gets deleted - primaryShardsBatch.getBatchedShardRoutings().forEach(testGatewayAllocator::safelyRemoveShardFromBatch); - replicaShardsBatch.getBatchedShardRoutings().forEach(testGatewayAllocator::safelyRemoveShardFromBatch); + primaryShardsBatch.getBatchedShardRoutings().forEach(testShardsBatchGatewayAllocator::safelyRemoveShardFromBatch); + replicaShardsBatch.getBatchedShardRoutings().forEach(testShardsBatchGatewayAllocator::safelyRemoveShardFromBatch); - assertFalse(testGatewayAllocator.getBatchIdToStartedShardBatch().containsKey(primaryShardsBatch.getBatchId())); - assertFalse(testGatewayAllocator.getBatchIdToStoreShardBatch().containsKey(replicaShardsBatch.getBatchId())); - assertEquals(1, testGatewayAllocator.getBatchIdToStartedShardBatch().size()); - assertEquals(1, testGatewayAllocator.getBatchIdToStoreShardBatch().size()); + assertFalse(testShardsBatchGatewayAllocator.getBatchIdToStartedShardBatch().containsKey(primaryShardsBatch.getBatchId())); + assertFalse(testShardsBatchGatewayAllocator.getBatchIdToStoreShardBatch().containsKey(replicaShardsBatch.getBatchId())); + assertEquals(1, testShardsBatchGatewayAllocator.getBatchIdToStartedShardBatch().size()); + assertEquals(1, testShardsBatchGatewayAllocator.getBatchIdToStoreShardBatch().size()); } public void testSafelyRemoveShardFromBothBatch() { createIndexAndUpdateClusterState(1, 3, 1); createBatchesAndAssert(1); - GatewayAllocator.ShardsBatch primaryShardsBatch = testGatewayAllocator.getBatchIdToStartedShardBatch().values().iterator().next(); - GatewayAllocator.ShardsBatch replicaShardsBatch = testGatewayAllocator.getBatchIdToStoreShardBatch().values().iterator().next(); + ShardsBatchGatewayAllocator.ShardsBatch primaryShardsBatch = testShardsBatchGatewayAllocator.getBatchIdToStartedShardBatch() + .values() + .iterator() + .next(); + ShardsBatchGatewayAllocator.ShardsBatch replicaShardsBatch = testShardsBatchGatewayAllocator.getBatchIdToStoreShardBatch() + .values() + .iterator() + .next(); ShardRouting anyPrimary = primaryShardsBatch.getBatchedShardRoutings().iterator().next(); // remove first shard routing from both batches - testGatewayAllocator.safelyRemoveShardFromBothBatch(anyPrimary); + testShardsBatchGatewayAllocator.safelyRemoveShardFromBothBatch(anyPrimary); // verify that shard routing is removed from both batches assertFalse(primaryShardsBatch.getBatchedShards().contains(anyPrimary.shardId())); assertFalse(replicaShardsBatch.getBatchedShards().contains(anyPrimary.shardId())); // try to remove that shard again to see if its no op and doesnt result in exception - testGatewayAllocator.safelyRemoveShardFromBothBatch(anyPrimary); + testShardsBatchGatewayAllocator.safelyRemoveShardFromBothBatch(anyPrimary); // now remove all shard routings to verify that batch gets deleted - primaryShardsBatch.getBatchedShardRoutings().forEach(testGatewayAllocator::safelyRemoveShardFromBothBatch); - replicaShardsBatch.getBatchedShardRoutings().forEach(testGatewayAllocator::safelyRemoveShardFromBothBatch); + primaryShardsBatch.getBatchedShardRoutings().forEach(testShardsBatchGatewayAllocator::safelyRemoveShardFromBothBatch); + replicaShardsBatch.getBatchedShardRoutings().forEach(testShardsBatchGatewayAllocator::safelyRemoveShardFromBothBatch); - assertFalse(testGatewayAllocator.getBatchIdToStartedShardBatch().containsKey(primaryShardsBatch.getBatchId())); - assertFalse(testGatewayAllocator.getBatchIdToStoreShardBatch().containsKey(replicaShardsBatch.getBatchId())); - assertEquals(0, testGatewayAllocator.getBatchIdToStartedShardBatch().size()); - assertEquals(0, testGatewayAllocator.getBatchIdToStoreShardBatch().size()); + assertFalse(testShardsBatchGatewayAllocator.getBatchIdToStartedShardBatch().containsKey(primaryShardsBatch.getBatchId())); + assertFalse(testShardsBatchGatewayAllocator.getBatchIdToStoreShardBatch().containsKey(replicaShardsBatch.getBatchId())); + assertEquals(0, testShardsBatchGatewayAllocator.getBatchIdToStartedShardBatch().size()); + assertEquals(0, testShardsBatchGatewayAllocator.getBatchIdToStoreShardBatch().size()); } public void testGetBatchIdExisting() { @@ -246,10 +254,10 @@ public void testGetBatchIdExisting() { for (ShardRouting shardRouting : allShardRoutings1) { for (String batchId : primaryBatches) { if (shardRouting.primary() == true - && testGatewayAllocator.getBatchIdToStartedShardBatch() - .get(batchId) - .getBatchedShards() - .contains(shardRouting.shardId())) { + && testShardsBatchGatewayAllocator.getBatchIdToStartedShardBatch() + .get(batchId) + .getBatchedShards() + .contains(shardRouting.shardId())) { if (shardIdToBatchIdForStartedShards.containsKey(shardRouting.shardId())) { fail("found duplicate shard routing for shard. One shard cant be in multiple batches " + shardRouting.shardId()); } @@ -263,10 +271,10 @@ public void testGetBatchIdExisting() { for (ShardRouting shardRouting : allShardRoutings1) { for (String batchId : replicaBatches) { if (shardRouting.primary() == false - && testGatewayAllocator.getBatchIdToStoreShardBatch() - .get(batchId) - .getBatchedShards() - .contains(shardRouting.shardId())) { + && testShardsBatchGatewayAllocator.getBatchIdToStoreShardBatch() + .get(batchId) + .getBatchedShards() + .contains(shardRouting.shardId())) { if (shardIdToBatchIdForStoreShards.containsKey(shardRouting.shardId())) { fail("found duplicate shard routing for shard. One shard cant be in multiple batches " + shardRouting.shardId()); } @@ -282,12 +290,12 @@ public void testGetBatchIdExisting() { if (shardRouting.primary()) { assertEquals( shardIdToBatchIdForStartedShards.get(shardRouting.shardId()), - testGatewayAllocator.getBatchId(shardRouting, true) + testShardsBatchGatewayAllocator.getBatchId(shardRouting, true) ); } else { assertEquals( shardIdToBatchIdForStoreShards.get(shardRouting.shardId()), - testGatewayAllocator.getBatchId(shardRouting, false) + testShardsBatchGatewayAllocator.getBatchId(shardRouting, false) ); } } @@ -303,7 +311,7 @@ public void testGetBatchIdNonExisting() { .map(IndexShardRoutingTable::getShards) .flatMap(List::stream) .collect(Collectors.toList()); - allShardRoutings.forEach(shard -> assertNull(testGatewayAllocator.getBatchId(shard, shard.primary()))); + allShardRoutings.forEach(shard -> assertNull(testShardsBatchGatewayAllocator.getBatchId(shard, shard.primary()))); } private void createIndexAndUpdateClusterState(int count, int numberOfShards, int numberOfReplicas) { @@ -339,14 +347,14 @@ private void createIndexAndUpdateClusterState(int count, int numberOfShards, int // call this after index creation and update cluster state private Tuple, Set> createBatchesAndAssert(int expectedBatchSize) { - Set primaryBatches = testGatewayAllocator.createAndUpdateBatches(testAllocation, true); - Set replicaBatches = testGatewayAllocator.createAndUpdateBatches(testAllocation, false); + Set primaryBatches = testShardsBatchGatewayAllocator.createAndUpdateBatches(testAllocation, true); + Set replicaBatches = testShardsBatchGatewayAllocator.createAndUpdateBatches(testAllocation, false); assertEquals(expectedBatchSize, primaryBatches.size()); assertEquals(expectedBatchSize, replicaBatches.size()); - assertEquals(expectedBatchSize, testGatewayAllocator.getBatchIdToStartedShardBatch().size()); - assertEquals(expectedBatchSize, testGatewayAllocator.getBatchIdToStoreShardBatch().size()); - assertEquals(testGatewayAllocator.getBatchIdToStartedShardBatch().keySet(), primaryBatches); - assertEquals(testGatewayAllocator.getBatchIdToStoreShardBatch().keySet(), replicaBatches); + assertEquals(expectedBatchSize, testShardsBatchGatewayAllocator.getBatchIdToStartedShardBatch().size()); + assertEquals(expectedBatchSize, testShardsBatchGatewayAllocator.getBatchIdToStoreShardBatch().size()); + assertEquals(testShardsBatchGatewayAllocator.getBatchIdToStartedShardBatch().keySet(), primaryBatches); + assertEquals(testShardsBatchGatewayAllocator.getBatchIdToStoreShardBatch().keySet(), replicaBatches); return new Tuple<>(primaryBatches, replicaBatches); } } diff --git a/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java b/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java index 64d1cccf72e57..54855fcf12920 100644 --- a/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java +++ b/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java @@ -40,15 +40,12 @@ import org.opensearch.gateway.AsyncShardFetch; import org.opensearch.gateway.GatewayAllocator; import org.opensearch.gateway.PrimaryShardAllocator; -import org.opensearch.gateway.PrimaryShardBatchAllocator; import org.opensearch.gateway.ReplicaShardAllocator; -import org.opensearch.gateway.ReplicaShardBatchAllocator; import org.opensearch.gateway.TransportNodesListGatewayStartedShards.NodeGatewayStartedShards; import org.opensearch.gateway.TransportNodesListGatewayStartedBatchShards; import org.opensearch.index.shard.ShardId; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata.NodeStoreFilesMetadata; -import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; import java.util.Collections; import java.util.HashMap; @@ -102,45 +99,14 @@ protected AsyncShardFetch.FetchResult fetchData(ShardR ) ); - return new AsyncShardFetch.FetchResult<>(shardId, foundShards, ignoreNodes); - } - }; - - - PrimaryShardBatchAllocator primaryShardBatchAllocator = new PrimaryShardBatchAllocator() { - @Override - protected AsyncShardFetch.FetchResult fetchData(Set shardsEligibleForFetch, - Set inEligibleShards, - RoutingAllocation allocation) { - Map foundShards = new HashMap<>(); - HashMap> shardsToIgnoreNodes = new HashMap<>(); - for (Map.Entry> entry : knownAllocations.entrySet()) { - String nodeId = entry.getKey(); - Map shardsOnNode = entry.getValue(); - HashMap adaptedResponse = new HashMap<>(); - - for (ShardRouting shardRouting : shardsEligibleForFetch) { - ShardId shardId = shardRouting.shardId(); - Set ignoreNodes = allocation.getIgnoreNodes(shardId); - - if (shardsOnNode.containsKey(shardId) && ignoreNodes.contains(nodeId) == false && currentNodes.nodeExists(nodeId)) { - TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShards nodeShard = new TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShards( - shardRouting.allocationId().getId(), - shardRouting.primary(), - getReplicationCheckpoint(shardId, nodeId) - ); - adaptedResponse.put(shardId, nodeShard); - shardsToIgnoreNodes.put(shardId, ignoreNodes); - } - foundShards.put(currentNodes.get(nodeId), - new TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShardsBatch(currentNodes.get(nodeId), adaptedResponse)); + return new AsyncShardFetch.FetchResult<>(foundShards, new HashMap<>() { + { + put(shardId, ignoreNodes); } - } - return new AsyncShardFetch.FetchResult<>(foundShards, shardsToIgnoreNodes); + }); } }; - private ReplicationCheckpoint getReplicationCheckpoint(ShardId shardId, String nodeName) { return shardIdNodeToReplicationCheckPointMap.getOrDefault(getReplicationCheckPointKey(shardId, nodeName), null); } @@ -150,7 +116,11 @@ private ReplicationCheckpoint getReplicationCheckpoint(ShardId shardId, String n protected AsyncShardFetch.FetchResult fetchData(ShardRouting shard, RoutingAllocation allocation) { // for now, just pretend no node has data final ShardId shardId = shard.shardId(); - return new AsyncShardFetch.FetchResult<>(shardId, Collections.emptyMap(), allocation.getIgnoreNodes(shardId)); + return new AsyncShardFetch.FetchResult<>(Collections.emptyMap(), new HashMap<>() { + { + put(shardId, allocation.getIgnoreNodes(shardId)); + } + }); } @Override @@ -159,19 +129,6 @@ protected boolean hasInitiatedFetching(ShardRouting shard) { } }; - ReplicaShardBatchAllocator replicaShardBatchAllocator = new ReplicaShardBatchAllocator() { - - @Override - protected AsyncShardFetch.FetchResult fetchData(Set shardsEligibleForFetch, - Set inEligibleShards, - RoutingAllocation allocation) { - return new AsyncShardFetch.FetchResult<>(Collections.emptyMap(), Collections.emptyMap()); - } - @Override - protected boolean hasInitiatedFetching(ShardRouting shard) { - return true; - } - }; @Override public void applyStartedShards(List startedShards, RoutingAllocation allocation) { currentNodes = allocation.nodes(); @@ -209,12 +166,6 @@ public void allocateUnassigned( innerAllocatedUnassigned(allocation, primaryShardAllocator, replicaShardAllocator, shardRouting, unassignedAllocationHandler); } - @Override - public void allocateUnassignedBatch(RoutingAllocation allocation, boolean primary){ - currentNodes = allocation.nodes(); - innerAllocateUnassignedBatch(allocation, primaryShardBatchAllocator, replicaShardBatchAllocator, primary); - } - /** * manually add a specific shard to the allocations the gateway keeps track of */ @@ -230,29 +181,4 @@ public void addReplicationCheckpoint(ShardId shardId, String nodeName, Replicati shardIdNodeToReplicationCheckPointMap.putIfAbsent(getReplicationCheckPointKey(shardId, nodeName), replicationCheckpoint); } - - public Set createAndUpdateBatches(RoutingAllocation allocation, boolean primary) { - return super.createAndUpdateBatches(allocation, primary); - } - - public void safelyRemoveShardFromBatch(ShardRouting shard){ - super.safelyRemoveShardFromBatch(shard); - } - - - public void safelyRemoveShardFromBothBatch(ShardRouting shardRouting) { - super.safelyRemoveShardFromBothBatch(shardRouting); - } - - public String getBatchId(ShardRouting shard, boolean primary) { - return super.getBatchId(shard, primary); - } - - public Map getBatchIdToStartedShardBatch(){ - return batchIdToStartedShardBatch; - } - - public Map getBatchIdToStoreShardBatch(){ - return batchIdToStoreShardBatch; - } } 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 new file mode 100644 index 0000000000000..1cd9f49341132 --- /dev/null +++ b/test/framework/src/main/java/org/opensearch/test/gateway/TestShardBatchGatewayAllocator.java @@ -0,0 +1,142 @@ +/* + * 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.test.gateway; + +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; +import org.opensearch.cluster.routing.allocation.RoutingAllocation; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.gateway.AsyncShardFetch; +import org.opensearch.gateway.PrimaryShardBatchAllocator; +import org.opensearch.gateway.ReplicaShardBatchAllocator; +import org.opensearch.gateway.ShardsBatchGatewayAllocator; +import org.opensearch.gateway.TransportNodesListGatewayStartedBatchShards; +import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +public class TestShardBatchGatewayAllocator extends ShardsBatchGatewayAllocator { + + Map> knownAllocations = new HashMap<>(); + DiscoveryNodes currentNodes = DiscoveryNodes.EMPTY_NODES; + Map shardIdNodeToReplicationCheckPointMap = new HashMap<>(); + + PrimaryShardBatchAllocator primaryBatchShardAllocator = new PrimaryShardBatchAllocator() { + @Override + protected AsyncShardFetch.FetchResult fetchData( + Set shardsEligibleForFetch, + Set inEligibleShards, + RoutingAllocation allocation + ) { + Map foundShards = new HashMap<>(); + HashMap> shardsToIgnoreNodes = new HashMap<>(); + for (Map.Entry> entry : knownAllocations.entrySet()) { + String nodeId = entry.getKey(); + Map shardsOnNode = entry.getValue(); + HashMap adaptedResponse = new HashMap<>(); + + for (ShardRouting shardRouting : shardsEligibleForFetch) { + ShardId shardId = shardRouting.shardId(); + Set ignoreNodes = allocation.getIgnoreNodes(shardId); + + if (shardsOnNode.containsKey(shardId) && ignoreNodes.contains(nodeId) == false && currentNodes.nodeExists(nodeId)) { + TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShards nodeShard = + new TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShards( + shardRouting.allocationId().getId(), + shardRouting.primary(), + getReplicationCheckpoint(shardId, nodeId) + ); + adaptedResponse.put(shardId, nodeShard); + shardsToIgnoreNodes.put(shardId, ignoreNodes); + } + foundShards.put( + currentNodes.get(nodeId), + new TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShardsBatch( + currentNodes.get(nodeId), + adaptedResponse + ) + ); + } + } + return new AsyncShardFetch.FetchResult<>(foundShards, shardsToIgnoreNodes); + } + }; + + ReplicaShardBatchAllocator replicaBatchShardAllocator = new ReplicaShardBatchAllocator() { + + @Override + protected AsyncShardFetch.FetchResult fetchData( + Set shardsEligibleForFetch, + Set inEligibleShards, + RoutingAllocation allocation + ) { + return new AsyncShardFetch.FetchResult<>(Collections.emptyMap(), Collections.emptyMap()); + } + + @Override + protected boolean hasInitiatedFetching(ShardRouting shard) { + return true; + } + }; + + @Override + public void allocateUnassignedBatch(RoutingAllocation allocation, boolean primary) { + currentNodes = allocation.nodes(); + innerAllocateUnassignedBatch(allocation, primaryBatchShardAllocator, replicaBatchShardAllocator, primary); + } + + @Override + public void beforeAllocation(RoutingAllocation allocation) {} + + @Override + public void afterPrimariesBeforeReplicas(RoutingAllocation allocation) {} + + public Set createAndUpdateBatches(RoutingAllocation allocation, boolean primary) { + return super.createAndUpdateBatches(allocation, primary); + } + + public void safelyRemoveShardFromBatch(ShardRouting shard) { + super.safelyRemoveShardFromBatch(shard); + } + + public void safelyRemoveShardFromBothBatch(ShardRouting shardRouting) { + super.safelyRemoveShardFromBothBatch(shardRouting); + } + + public String getBatchId(ShardRouting shard, boolean primary) { + return super.getBatchId(shard, primary); + } + + public Map getBatchIdToStartedShardBatch() { + return batchIdToStartedShardBatch; + } + + public Map getBatchIdToStoreShardBatch() { + return batchIdToStoreShardBatch; + } + + @Override + public AllocateUnassignedDecision explainUnassignedShardAllocation(ShardRouting unassignedShard, RoutingAllocation routingAllocation) { + return super.explainUnassignedShardAllocation(unassignedShard, routingAllocation); + } + + protected ReplicationCheckpoint getReplicationCheckpoint(ShardId shardId, String nodeName) { + return shardIdNodeToReplicationCheckPointMap.getOrDefault(getReplicationCheckPointKey(shardId, nodeName), null); + } + + public String getReplicationCheckPointKey(ShardId shardId, String nodeName) { + return shardId.toString() + "_" + nodeName; + } +} From 7eb12b93375ec07164bf3eb69cab6644fd5ebea8 Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Thu, 14 Dec 2023 16:45:03 +0530 Subject: [PATCH 16/45] Added Changes for ShardBatchGatewayAllocator Signed-off-by: Gaurav Chandani --- .../cluster/routing/allocation/AllocationService.java | 11 +++++++++++ 1 file changed, 11 insertions(+) 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 d1e27c8c5670e..7a251824a7f0b 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 @@ -59,11 +59,13 @@ import org.opensearch.common.settings.Settings; import org.opensearch.gateway.GatewayAllocator; import org.opensearch.gateway.PriorityComparator; +import org.opensearch.gateway.ShardsBatchGatewayAllocator; import org.opensearch.snapshots.SnapshotsInfoService; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -571,12 +573,21 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { // if not fallback to single assignment ExistingShardsAllocator allocator = verifySameAllocatorForAllUnassignedShards(allocation); if (allocator != null) { + // use batch mode implementation of GatewayAllocator + if (allocator.getClass() == GatewayAllocator.class) { + allocator = existingShardsAllocators.get(ShardsBatchGatewayAllocator.ALLOCATOR_NAME); + } + allocator.allocateUnassignedBatch(allocation, true); for (final ExistingShardsAllocator existingShardsAllocator : existingShardsAllocators.values()) { existingShardsAllocator.afterPrimariesBeforeReplicas(allocation); } allocator.allocateUnassignedBatch(allocation, false); return; + } else { + // it means though batch mode is enabled but some indices have custom allocator set and we cant do Batch recover in that + // case fallback to single assignment and + logger.debug("Batch mode is enabled but some indices have custom allocator set. Falling back to single assignment"); } } From d741081bf5d2d988bc1efc08f72ec3e1efed7b36 Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Thu, 14 Dec 2023 20:19:05 +0530 Subject: [PATCH 17/45] Fixed culprit shard test Signed-off-by: Gaurav Chandani Signed-off-by: Shivansh Arora --- .../gateway/RecoveryFromGatewayIT.java | 32 +++++++------------ 1 file changed, 12 insertions(+), 20 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 73de310cd65fc..a002369bacef8 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -34,6 +34,7 @@ import com.carrotsearch.hppc.cursors.ObjectCursor; +import org.apache.lucene.index.CorruptIndexException; import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsAction; import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsRequest; import org.opensearch.action.admin.cluster.configuration.ClearVotingConfigExclusionsAction; @@ -63,6 +64,7 @@ import org.opensearch.index.MergePolicyConfig; import org.opensearch.index.engine.Engine; import org.opensearch.index.query.QueryBuilders; +import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.ShardId; import org.opensearch.index.shard.ShardPath; import org.opensearch.indices.IndicesService; @@ -928,27 +930,17 @@ public void testCulpritShardInBatch() throws Exception { assertEquals(8, health.getActiveShards()); String culpritShardIndexName = "test0"; - final String customDataPath = IndexMetadata.INDEX_DATA_PATH_SETTING.get( - client().admin().indices().prepareGetSettings(culpritShardIndexName).get().getIndexToSettings().get(culpritShardIndexName) - ); - final Index index = resolveIndex(culpritShardIndexName); - final ShardId shardId = new ShardId(index, 0); - - for (String dataNode : dataOnlyNodes) { - for (Path path : internalCluster().getInstance(NodeEnvironment.class, dataNode).availableShardPaths(shardId)) { - final Path indexPath = path.resolve(ShardPath.INDEX_FOLDER_NAME); - if (Files.exists(indexPath)) { - try (DirectoryStream stream = Files.newDirectoryStream(indexPath)) { - for (Path item : stream) { - if (item.getFileName().toString().startsWith("segments_")) { - logger.debug("--> deleting [{}]", item); - Files.delete(item); - } - } - } - } - } + Index idx = resolveIndex(culpritShardIndexName); + for (String node : internalCluster().nodesInclude(culpritShardIndexName)) { + IndicesService indexServices = internalCluster().getInstance(IndicesService.class, node); + IndexService indexShards = indexServices.indexServiceSafe(idx); + Integer shardId = 0; + IndexShard shard = indexShards.getShard(0); + logger.debug("--> failing shard [{}] on node [{}]", shardId, node); + shard.failShard("test", new CorruptIndexException("test corrupted", "")); + logger.debug("--> failed shard [{}] on node [{}]", shardId, node); } + String clusterManagerName = internalCluster().getClusterManagerName(); Settings clusterManagerDataPathSettings = internalCluster().dataPathSettings(clusterManagerName); Settings node0DataPathSettings = internalCluster().dataPathSettings(dataOnlyNodes.get(0)); From 809c99fe13a6f15b3ccb5863483e549a5dd77879 Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Wed, 20 Dec 2023 11:46:15 +0530 Subject: [PATCH 18/45] Cosmetic changes in AllocationService Signed-off-by: Gaurav Chandani --- .../cluster/routing/allocation/AllocationService.java | 7 +++---- 1 file changed, 3 insertions(+), 4 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 7a251824a7f0b..69dbd95024626 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 @@ -571,7 +571,7 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { if (batchModeEnabled && allocation.nodes().getMinNodeVersion().onOrAfter(Version.CURRENT)) { // since allocators is per index setting, to have batch assignment verify allocators same for all shards // if not fallback to single assignment - ExistingShardsAllocator allocator = verifySameAllocatorForAllUnassignedShards(allocation); + ExistingShardsAllocator allocator = getAndVerifySameAllocatorForAllUnassignedShards(allocation); if (allocator != null) { // use batch mode implementation of GatewayAllocator if (allocator.getClass() == GatewayAllocator.class) { @@ -618,7 +618,7 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { * @param allocation {@link RoutingAllocation} * @return {@link ExistingShardsAllocator} or null */ - private ExistingShardsAllocator verifySameAllocatorForAllUnassignedShards(RoutingAllocation allocation) { + private ExistingShardsAllocator getAndVerifySameAllocatorForAllUnassignedShards(RoutingAllocation allocation) { // if there is a single Allocator set in Allocation Service then use it for all shards if (existingShardsAllocators.size() == 1) { return existingShardsAllocators.values().iterator().next(); @@ -627,8 +627,7 @@ private ExistingShardsAllocator verifySameAllocatorForAllUnassignedShards(Routin RoutingNodes.UnassignedShards.UnassignedIterator iterator = unassignedShards.iterator(); ExistingShardsAllocator currentAllocatorForShard = null; if (unassignedShards.size() > 0) { - ShardRouting shard = iterator.next(); - currentAllocatorForShard = getAllocatorForShard(shard, allocation); + currentAllocatorForShard = getAllocatorForShard(iterator.next(), allocation); while (iterator.hasNext()) { ExistingShardsAllocator allocatorForShard = getAllocatorForShard(iterator.next(), allocation); if (currentAllocatorForShard.getClass().getName().equals(allocatorForShard.getClass().getName()) == false) { From 340abe41750fa3ac89a95dfe6cbe14a602f98538 Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Fri, 12 Jan 2024 13:34:31 +0530 Subject: [PATCH 19/45] PR comments 1. Made changes so that Allocation Service run only default implementation of batch mode 2. Renamed methods 3. Added and modified documenatation Signed-off-by: Gaurav Chandani Signed-off-by: Shivansh Arora --- .../routing/allocation/AllocationService.java | 52 ++++++++----------- .../allocation/ExistingShardsAllocator.java | 14 +++-- 2 files changed, 31 insertions(+), 35 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 69dbd95024626..8819e53f0a6f3 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 @@ -55,7 +55,6 @@ import org.opensearch.cluster.routing.allocation.command.AllocationCommands; import org.opensearch.cluster.routing.allocation.decider.AllocationDeciders; import org.opensearch.cluster.routing.allocation.decider.Decision; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.settings.Settings; import org.opensearch.gateway.GatewayAllocator; import org.opensearch.gateway.PriorityComparator; @@ -201,9 +200,9 @@ private ClusterState buildResult(ClusterState oldState, RoutingAllocation alloca if (restoreInProgress != null) { RestoreInProgress updatedRestoreInProgress = allocation.updateRestoreInfoWithRoutingChanges(restoreInProgress); if (updatedRestoreInProgress != restoreInProgress) { - ImmutableOpenMap.Builder customsBuilder = ImmutableOpenMap.builder(allocation.getCustoms()); + final Map customsBuilder = new HashMap<>(allocation.getCustoms()); customsBuilder.put(RestoreInProgress.TYPE, updatedRestoreInProgress); - newStateBuilder.customs(customsBuilder.build()); + newStateBuilder.customs(customsBuilder); } } return newStateBuilder.build(); @@ -566,30 +565,22 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { existingShardsAllocator.beforeAllocation(allocation); } + /* + Use batch mode if enabled and there is no custom allocator set for Allocation service + */ Boolean batchModeEnabled = EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.get(settings); - - if (batchModeEnabled && allocation.nodes().getMinNodeVersion().onOrAfter(Version.CURRENT)) { - // since allocators is per index setting, to have batch assignment verify allocators same for all shards - // if not fallback to single assignment - ExistingShardsAllocator allocator = getAndVerifySameAllocatorForAllUnassignedShards(allocation); - if (allocator != null) { - // use batch mode implementation of GatewayAllocator - if (allocator.getClass() == GatewayAllocator.class) { - allocator = existingShardsAllocators.get(ShardsBatchGatewayAllocator.ALLOCATOR_NAME); - } - - allocator.allocateUnassignedBatch(allocation, true); - for (final ExistingShardsAllocator existingShardsAllocator : existingShardsAllocators.values()) { - existingShardsAllocator.afterPrimariesBeforeReplicas(allocation); - } - allocator.allocateUnassignedBatch(allocation, false); - return; - } else { - // it means though batch mode is enabled but some indices have custom allocator set and we cant do Batch recover in that - // case fallback to single assignment and - logger.debug("Batch mode is enabled but some indices have custom allocator set. Falling back to single assignment"); - } - } + if (batchModeEnabled && allocation.nodes().getMinNodeVersion().onOrAfter(Version.CURRENT) && existingShardsAllocators.size() == 2) { + /* + If we do not have any custom allocator set then we will be using ShardsBatchGatewayAllocator + Currently AllocationService will not run any custom Allocator that implements allocateAllUnassignedShards + */ + ExistingShardsAllocator allocator = existingShardsAllocators.get(ShardsBatchGatewayAllocator.ALLOCATOR_NAME); + allocator.allocateAllUnassignedShards(allocation, true); + allocator.afterPrimariesBeforeReplicas(allocation); + allocator.allocateAllUnassignedShards(allocation, false); + return; + } + logger.warn("Falling back to single shard assignment since batch mode disable or multiple custom allocators set"); final RoutingNodes.UnassignedShards.UnassignedIterator primaryIterator = allocation.routingNodes().unassigned().iterator(); while (primaryIterator.hasNext()) { @@ -637,6 +628,7 @@ private ExistingShardsAllocator getAndVerifySameAllocatorForAllUnassignedShards( } return currentAllocatorForShard; } + private void disassociateDeadNodes(RoutingAllocation allocation) { for (Iterator it = allocation.routingNodes().mutableIterator(); it.hasNext();) { RoutingNode node = it.next(); @@ -676,9 +668,9 @@ private void applyStartedShards(RoutingAllocation routingAllocation, List EXISTING_SHARDS_ALLOCATOR_BATCH_MODE = Setting.boolSetting( - "cluster.allocator.existing_shards_allocator.batch_enable", + "cluster.allocator.existing_shards_allocator.batch_enabled", false, Setting.Property.NodeScope ); @@ -116,8 +118,10 @@ default void allocateUnassignedBatch(RoutingAllocation allocation, boolean prima * Allocate all unassigned shards in the given {@link RoutingAllocation} for which this {@link ExistingShardsAllocator} is responsible. * Default implementation calls {@link #allocateUnassigned(ShardRouting, RoutingAllocation, UnassignedAllocationHandler)} for each Unassigned shard * and is kept here for backward compatibility. + * + * Allocation service will currently run the default implementation of it implemented by {@link ShardsBatchGatewayAllocator} */ - default void allocateUnassignedBatch(RoutingAllocation allocation, boolean primary) { + default void allocateAllUnassignedShards(RoutingAllocation allocation, boolean primary) { RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); while (iterator.hasNext()) { ShardRouting shardRouting = iterator.next(); From 4ee204033d3df4004dbd8d551bcad9df598d87a1 Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Mon, 15 Jan 2024 10:47:38 +0530 Subject: [PATCH 20/45] Changes to implement interface for ShardBatchGatewayAllocator Signed-off-by: Gaurav Chandani --- .../org/opensearch/cluster/ClusterModule.java | 6 ++- .../gateway/ShardsBatchGatewayAllocator.java | 44 ++++++------------- .../main/java/org/opensearch/node/Node.java | 14 ++---- .../cluster/reroute/ClusterRerouteTests.java | 4 +- .../cluster/ClusterModuleTests.java | 15 +++---- 5 files changed, 28 insertions(+), 55 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/ClusterModule.java b/server/src/main/java/org/opensearch/cluster/ClusterModule.java index cd1ffd898509f..a2ab51f221e80 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterModule.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterModule.java @@ -442,8 +442,10 @@ protected void configure() { bind(ShardsAllocator.class).toInstance(shardsAllocator); } - public void setExistingShardsAllocators(Map gatewayAllocators) { - final Map existingShardsAllocators = new HashMap<>(gatewayAllocators); + public void setExistingShardsAllocators(GatewayAllocator gatewayAllocator, ShardsBatchGatewayAllocator shardsBatchGatewayAllocator) { + final Map existingShardsAllocators = new HashMap<>(); + existingShardsAllocators.put(GatewayAllocator.ALLOCATOR_NAME, gatewayAllocator); + existingShardsAllocators.put(ShardsBatchGatewayAllocator.ALLOCATOR_NAME, shardsBatchGatewayAllocator); for (ClusterPlugin clusterPlugin : clusterPlugins) { for (Map.Entry existingShardsAllocatorEntry : clusterPlugin.getExistingShardsAllocators() .entrySet()) { diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 13d2b518d25ef..ecae15b31752e 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -6,30 +6,6 @@ * compatible open source license. */ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -/* - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - package org.opensearch.gateway; import org.apache.logging.log4j.LogManager; @@ -44,6 +20,7 @@ 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.ExistingShardsAllocator; import org.opensearch.cluster.routing.allocation.FailedShard; import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.common.Priority; @@ -57,7 +34,6 @@ import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.shard.ShardId; import org.opensearch.indices.store.ShardAttributes; -import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; import java.util.Collections; @@ -74,11 +50,11 @@ import java.util.stream.StreamSupport; /** - * Allocator for the Shards batch gateway + * Allocator for the gateway * * @opensearch.internal */ -public class ShardsBatchGatewayAllocator extends GatewayAllocator { +public class ShardsBatchGatewayAllocator implements ExistingShardsAllocator { public static final String ALLOCATOR_NAME = "shards_batch_gateway_allocator"; @@ -117,13 +93,10 @@ public class ShardsBatchGatewayAllocator extends GatewayAllocator { @Inject public ShardsBatchGatewayAllocator( RerouteService rerouteService, - TransportNodesListGatewayStartedShards startedAction, - TransportNodesListShardStoreMetadata storeAction, TransportNodesListGatewayStartedBatchShards batchStartedAction, TransportNodesListShardStoreMetadataBatch batchStoreAction, Settings settings ) { - super(rerouteService, startedAction, storeAction); this.rerouteService = rerouteService; this.primaryBatchShardAllocator = new InternalPrimaryBatchShardAllocator(); this.replicaBatchShardAllocator = new InternalReplicaBatchShardAllocator(); @@ -207,7 +180,16 @@ public void afterPrimariesBeforeReplicas(RoutingAllocation allocation) { } @Override - public void allocateUnassignedBatch(final RoutingAllocation allocation, boolean primary) { + public void allocateUnassigned( + ShardRouting shardRouting, + RoutingAllocation allocation, + UnassignedAllocationHandler unassignedAllocationHandler + ) { + throw new UnsupportedOperationException("ShardsBatchGatewayAllocator does not support allocating unassigned shards"); + } + + @Override + public void allocateAllUnassignedShards(final RoutingAllocation allocation, boolean primary) { assert primaryBatchShardAllocator != null; assert replicaBatchShardAllocator != null; diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 41a99d62671ba..02decffadf6ef 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -1131,16 +1131,10 @@ protected Node( // We allocate copies of existing shards by looking for a viable copy of the shard in the cluster and assigning the shard there. // The search for viable copies is triggered by an allocation attempt (i.e. a reroute) and is performed asynchronously. When it // completes we trigger another reroute to try the allocation again. This means there is a circular dependency: the allocation - // service needs access to the existing shards allocators (e.g. the GatewayAllocator) which need to be able to trigger a - // reroute, which needs to call into the allocation service. We close the loop here: - // create Hashmap for existing Allocators - Map gatewayAllocatorMap = new HashMap<>() { - { - put(GatewayAllocator.ALLOCATOR_NAME, injector.getInstance(GatewayAllocator.class)); - put(ShardsBatchGatewayAllocator.ALLOCATOR_NAME, injector.getInstance(ShardsBatchGatewayAllocator.class)); - } - }; - clusterModule.setExistingShardsAllocators(gatewayAllocatorMap); + // service needs access to the existing shards allocators (e.g. the GatewayAllocator, ShardsBatchGatewayAllocator) which + // need to be able to trigger a reroute, which needs to call into the allocation service. We close the loop here: + clusterModule.setExistingShardsAllocators(injector.getInstance(GatewayAllocator.class), + injector.getInstance(ShardsBatchGatewayAllocator.class)); List pluginLifecycleComponents = pluginComponents.stream() .filter(p -> p instanceof LifecycleComponent) diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/reroute/ClusterRerouteTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/reroute/ClusterRerouteTests.java index 4d93aa94e8b9a..859d8ce3bb734 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/reroute/ClusterRerouteTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/reroute/ClusterRerouteTests.java @@ -55,7 +55,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.snapshots.EmptySnapshotsInfoService; -import org.opensearch.test.gateway.TestShardBatchGatewayAllocator; +import org.opensearch.test.gateway.TestGatewayAllocator; import java.io.IOException; import java.util.Collections; @@ -94,7 +94,7 @@ public void testSerializeRequest() throws IOException { public void testClusterStateUpdateTask() { AllocationService allocationService = new AllocationService( new AllocationDeciders(Collections.singleton(new MaxRetryAllocationDecider())), - new TestShardBatchGatewayAllocator(), + new TestGatewayAllocator(), new BalancedShardsAllocator(Settings.EMPTY), EmptyClusterInfoService.INSTANCE, EmptySnapshotsInfoService.INSTANCE diff --git a/server/src/test/java/org/opensearch/cluster/ClusterModuleTests.java b/server/src/test/java/org/opensearch/cluster/ClusterModuleTests.java index f64e6b146e001..50a4f87f78d09 100644 --- a/server/src/test/java/org/opensearch/cluster/ClusterModuleTests.java +++ b/server/src/test/java/org/opensearch/cluster/ClusterModuleTests.java @@ -71,6 +71,7 @@ import org.opensearch.gateway.GatewayAllocator; import org.opensearch.plugins.ClusterPlugin; import org.opensearch.test.gateway.TestGatewayAllocator; +import org.opensearch.test.gateway.TestShardBatchGatewayAllocator; import java.util.Arrays; import java.util.Collection; @@ -292,11 +293,8 @@ public void testRejectsReservedExistingShardsAllocatorName() { null, threadContext ); - expectThrows(IllegalArgumentException.class, () -> clusterModule.setExistingShardsAllocators(new HashMap<>() { - { - put(GatewayAllocator.ALLOCATOR_NAME, new TestGatewayAllocator()); - } - })); + expectThrows(IllegalArgumentException.class, () -> clusterModule.setExistingShardsAllocators(new TestGatewayAllocator(), + new TestShardBatchGatewayAllocator())); } public void testRejectsDuplicateExistingShardsAllocatorName() { @@ -308,11 +306,8 @@ public void testRejectsDuplicateExistingShardsAllocatorName() { null, threadContext ); - expectThrows(IllegalArgumentException.class, () -> clusterModule.setExistingShardsAllocators(new HashMap<>() { - { - put(GatewayAllocator.ALLOCATOR_NAME, new TestGatewayAllocator()); - } - })); + expectThrows(IllegalArgumentException.class, () -> clusterModule.setExistingShardsAllocators(new TestGatewayAllocator(), + new TestShardBatchGatewayAllocator())); } private static ClusterPlugin existingShardsAllocatorPlugin(final String allocatorName) { From 9f6284c41588ef92faf4362bda26f5ba550090a9 Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Mon, 12 Feb 2024 15:14:27 +0530 Subject: [PATCH 21/45] Fixed PR comments around documentation Signed-off-by: Gaurav Chandani --- .../cluster/routing/allocation/AllocationService.java | 8 +++++--- .../routing/allocation/ExistingShardsAllocator.java | 9 ++------- 2 files changed, 7 insertions(+), 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 8819e53f0a6f3..7f2967bca2d69 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 @@ -55,6 +55,7 @@ import org.opensearch.cluster.routing.allocation.command.AllocationCommands; import org.opensearch.cluster.routing.allocation.decider.AllocationDeciders; import org.opensearch.cluster.routing.allocation.decider.Decision; +import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.settings.Settings; import org.opensearch.gateway.GatewayAllocator; import org.opensearch.gateway.PriorityComparator; @@ -200,9 +201,9 @@ private ClusterState buildResult(ClusterState oldState, RoutingAllocation alloca if (restoreInProgress != null) { RestoreInProgress updatedRestoreInProgress = allocation.updateRestoreInfoWithRoutingChanges(restoreInProgress); if (updatedRestoreInProgress != restoreInProgress) { - final Map customsBuilder = new HashMap<>(allocation.getCustoms()); + ImmutableOpenMap.Builder customsBuilder = ImmutableOpenMap.builder(allocation.getCustoms()); customsBuilder.put(RestoreInProgress.TYPE, updatedRestoreInProgress); - newStateBuilder.customs(customsBuilder); + newStateBuilder.customs(customsBuilder.build()); } } return newStateBuilder.build(); @@ -577,6 +578,7 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { ExistingShardsAllocator allocator = existingShardsAllocators.get(ShardsBatchGatewayAllocator.ALLOCATOR_NAME); allocator.allocateAllUnassignedShards(allocation, true); allocator.afterPrimariesBeforeReplicas(allocation); + // Replicas Assignment allocator.allocateAllUnassignedShards(allocation, false); return; } @@ -730,7 +732,7 @@ private ExistingShardsAllocator getAllocatorForShard(ShardRouting shardRouting, final String allocatorName = ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_SETTING.get( routingAllocation.metadata().getIndexSafe(shardRouting.index()).getSettings() ); - ExistingShardsAllocator existingShardsAllocator = existingShardsAllocators.get(allocatorName); + final ExistingShardsAllocator existingShardsAllocator = existingShardsAllocators.get(allocatorName); return existingShardsAllocator != null ? existingShardsAllocator : new NotFoundAllocator(allocatorName); } 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 b8ce08ef4aeb8..6f03199a0f4c3 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 @@ -70,15 +70,10 @@ public interface ExistingShardsAllocator { * {@link ExistingShardsAllocator#allocateAllUnassignedShards(RoutingAllocation, boolean)} method. * The default implementation of this method is not optimized and assigns shards one by one. * - * If enable to true then it expects all indices of the shard to use same {@link ExistingShardsAllocator}, otherwise - * Allocation Service will fallback to default implementation i.e. {@link ExistingShardsAllocator#allocateUnassigned(ShardRouting, RoutingAllocation, UnassignedAllocationHandler)} - * * If no plugin overrides {@link ExistingShardsAllocator} then default implementation will be use for it , i.e, - * {@link ShardsBatchGatewayAllocator}. Right now even if plugin implements it, AllocationService will run the - * default implementation to enable Batch mode of assignment + * {@link ShardsBatchGatewayAllocator}. * - * TODO: Currently its implementation is WIP for GatewayAllocator so setting enabling wont have any effect - * https://github.com/opensearch-project/OpenSearch/issues/5098 + * This setting is experimental at this point. */ Setting EXISTING_SHARDS_ALLOCATOR_BATCH_MODE = Setting.boolSetting( "cluster.allocator.existing_shards_allocator.batch_enabled", From 38692a026fafb8d0b43ddf2ba4054e8917f01974 Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Fri, 12 Jan 2024 12:33:33 +0530 Subject: [PATCH 22/45] Renamed allocateUnassignedBatch to allocateAllUnassignedShards Signed-off-by: Gaurav Chandani --- .../opensearch/test/gateway/TestShardBatchGatewayAllocator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 1cd9f49341132..a548dee26ebf5 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 @@ -92,7 +92,7 @@ protected boolean hasInitiatedFetching(ShardRouting shard) { }; @Override - public void allocateUnassignedBatch(RoutingAllocation allocation, boolean primary) { + public void allocateAllUnassignedShards(RoutingAllocation allocation, boolean primary) { currentNodes = allocation.nodes(); innerAllocateUnassignedBatch(allocation, primaryBatchShardAllocator, replicaBatchShardAllocator, primary); } From 20312bda294e26ea3460cf44c08549be1116aafc Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Tue, 6 Feb 2024 10:35:53 +0530 Subject: [PATCH 23/45] Removed batchMode variable from ShardsBatchGatewayAllocator Signed-off-by: Gaurav Chandani --- .../org/opensearch/gateway/ShardsBatchGatewayAllocator.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index ecae15b31752e..77e1edba04935 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -60,7 +60,6 @@ public class ShardsBatchGatewayAllocator implements ExistingShardsAllocator { private static final Logger logger = LogManager.getLogger(ShardsBatchGatewayAllocator.class); private final long maxBatchSize; - private final boolean batchMode; private static final short DEFAULT_SHARD_BATCH_SIZE = 2000; /** @@ -103,7 +102,6 @@ public ShardsBatchGatewayAllocator( this.batchStartedAction = batchStartedAction; this.batchStoreAction = batchStoreAction; this.maxBatchSize = GATEWAY_ALLOCATOR_BATCH_SIZE.get(settings); - this.batchMode = EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.get(settings); } @Override @@ -126,7 +124,6 @@ protected ShardsBatchGatewayAllocator() { this.batchStoreAction = null; this.replicaBatchShardAllocator = null; this.maxBatchSize = DEFAULT_SHARD_BATCH_SIZE; - this.batchMode = true; } // for tests From 8880e806e29b1e52800fb068b743fb6ecd7ce1c9 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Fri, 16 Feb 2024 22:08:08 +0530 Subject: [PATCH 24/45] Fix TestShardBatchGatewayAllocator Signed-off-by: Shivansh Arora --- .../TestShardBatchGatewayAllocator.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) 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 a548dee26ebf5..e34c222c94205 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 @@ -18,7 +18,7 @@ import org.opensearch.gateway.PrimaryShardBatchAllocator; import org.opensearch.gateway.ReplicaShardBatchAllocator; import org.opensearch.gateway.ShardsBatchGatewayAllocator; -import org.opensearch.gateway.TransportNodesListGatewayStartedBatchShards; +import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; @@ -35,27 +35,27 @@ public class TestShardBatchGatewayAllocator extends ShardsBatchGatewayAllocator PrimaryShardBatchAllocator primaryBatchShardAllocator = new PrimaryShardBatchAllocator() { @Override - protected AsyncShardFetch.FetchResult fetchData( + protected AsyncShardFetch.FetchResult fetchData( Set shardsEligibleForFetch, Set inEligibleShards, RoutingAllocation allocation ) { - Map foundShards = new HashMap<>(); + Map foundShards = new HashMap<>(); HashMap> shardsToIgnoreNodes = new HashMap<>(); for (Map.Entry> entry : knownAllocations.entrySet()) { String nodeId = entry.getKey(); Map shardsOnNode = entry.getValue(); - HashMap adaptedResponse = new HashMap<>(); + HashMap adaptedResponse = new HashMap<>(); for (ShardRouting shardRouting : shardsEligibleForFetch) { ShardId shardId = shardRouting.shardId(); Set ignoreNodes = allocation.getIgnoreNodes(shardId); if (shardsOnNode.containsKey(shardId) && ignoreNodes.contains(nodeId) == false && currentNodes.nodeExists(nodeId)) { - TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShards nodeShard = - new TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShards( - shardRouting.allocationId().getId(), - shardRouting.primary(), + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeShard = + new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard( + shardsOnNode.get(shardId).allocationId().getId(), + shardsOnNode.get(shardId).primary(), getReplicationCheckpoint(shardId, nodeId) ); adaptedResponse.put(shardId, nodeShard); @@ -63,7 +63,7 @@ protected AsyncShardFetch.FetchResult Date: Fri, 23 Feb 2024 18:19:48 +0530 Subject: [PATCH 25/45] Renamed variables and added documentations Signed-off-by: Gaurav Chandani --- .../allocation/ExistingShardsAllocator.java | 8 ---- .../common/settings/ClusterSettings.java | 3 +- .../gateway/ShardsBatchGatewayAllocator.java | 41 +++++++++---------- 3 files changed, 22 insertions(+), 30 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 6f03199a0f4c3..fb2a37237f8b6 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 @@ -101,14 +101,6 @@ void allocateUnassigned( UnassignedAllocationHandler unassignedAllocationHandler ); - default void allocateUnassignedBatch(RoutingAllocation allocation, boolean primary) { - RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); - while (iterator.hasNext()) { - ShardRouting shardRouting = iterator.next(); - allocateUnassigned(shardRouting, allocation, iterator); - } - } - /** * Allocate all unassigned shards in the given {@link RoutingAllocation} for which this {@link ExistingShardsAllocator} is responsible. * Default implementation calls {@link #allocateUnassigned(ShardRouting, RoutingAllocation, UnassignedAllocationHandler)} for each Unassigned shard 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 404ade615f7bd..42911c328c9dd 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -165,8 +165,9 @@ /** * Encapsulates all valid cluster level settings. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class ClusterSettings extends AbstractScopedSettings { public ClusterSettings(final Settings nodeSettings, final Set> settingsSet) { diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 77e1edba04935..655154bb880eb 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -50,7 +50,7 @@ import java.util.stream.StreamSupport; /** - * Allocator for the gateway + * Allocator for the gateway to assign batch of shards. * * @opensearch.internal */ @@ -75,8 +75,8 @@ public class ShardsBatchGatewayAllocator implements ExistingShardsAllocator { private final RerouteService rerouteService; - private PrimaryShardBatchAllocator primaryBatchShardAllocator; - private ReplicaShardBatchAllocator replicaBatchShardAllocator; + private PrimaryShardBatchAllocator primaryShardBatchAllocator; + private ReplicaShardBatchAllocator replicaShardBatchAllocator; private Set lastSeenEphemeralIds = Collections.emptySet(); @@ -97,8 +97,8 @@ public ShardsBatchGatewayAllocator( Settings settings ) { this.rerouteService = rerouteService; - this.primaryBatchShardAllocator = new InternalPrimaryBatchShardAllocator(); - this.replicaBatchShardAllocator = new InternalReplicaBatchShardAllocator(); + this.primaryShardBatchAllocator = new InternalPrimaryBatchShardAllocator(); + this.replicaShardBatchAllocator = new InternalReplicaBatchShardAllocator(); this.batchStartedAction = batchStartedAction; this.batchStoreAction = batchStoreAction; this.maxBatchSize = GATEWAY_ALLOCATOR_BATCH_SIZE.get(settings); @@ -120,9 +120,9 @@ public void cleanCaches() { protected ShardsBatchGatewayAllocator() { this.rerouteService = null; this.batchStartedAction = null; - this.primaryBatchShardAllocator = null; + this.primaryShardBatchAllocator = null; this.batchStoreAction = null; - this.replicaBatchShardAllocator = null; + this.replicaShardBatchAllocator = null; this.maxBatchSize = DEFAULT_SHARD_BATCH_SIZE; } @@ -131,7 +131,6 @@ protected ShardsBatchGatewayAllocator() { @Override public int getNumberOfInFlightFetches() { int count = 0; - // If fetching is done in non batched-mode then maps to maintain batches will be empty and vice versa for batch-mode for (ShardsBatch batch : batchIdToStartedShardBatch.values()) { count += (batch.getNumberOfInFlightFetches() * batch.getBatchedShards().size()); } @@ -158,21 +157,21 @@ public void applyFailedShards(final List failedShards, final Routin @Override public void beforeAllocation(final RoutingAllocation allocation) { - assert primaryBatchShardAllocator != null; - assert replicaBatchShardAllocator != null; + assert primaryShardBatchAllocator != null; + assert replicaShardBatchAllocator != null; ensureAsyncFetchStorePrimaryRecency(allocation); } @Override public void afterPrimariesBeforeReplicas(RoutingAllocation allocation) { - assert replicaBatchShardAllocator != null; + assert replicaShardBatchAllocator != null; List> storedShardBatches = batchIdToStoreShardBatch.values() .stream() .map(ShardsBatch::getBatchedShardRoutings) .collect(Collectors.toList()); if (allocation.routingNodes().hasInactiveShards()) { // cancel existing recoveries if we have a better match - replicaBatchShardAllocator.processExistingRecoveries(allocation, storedShardBatches); + replicaShardBatchAllocator.processExistingRecoveries(allocation, storedShardBatches); } } @@ -188,9 +187,9 @@ public void allocateUnassigned( @Override public void allocateAllUnassignedShards(final RoutingAllocation allocation, boolean primary) { - assert primaryBatchShardAllocator != null; - assert replicaBatchShardAllocator != null; - innerAllocateUnassignedBatch(allocation, primaryBatchShardAllocator, replicaBatchShardAllocator, primary); + assert primaryShardBatchAllocator != null; + assert replicaShardBatchAllocator != null; + innerAllocateUnassignedBatch(allocation, primaryShardBatchAllocator, replicaShardBatchAllocator, primary); } protected void innerAllocateUnassignedBatch( @@ -253,14 +252,14 @@ else if (shardRouting.primary() == primary) { while (iterator.hasNext()) { ShardRouting currentShard = iterator.next(); if (batchSize > 0) { - ShardEntry sharEntry = new ShardEntry( + ShardEntry shardEntry = new ShardEntry( new ShardAttributes( currentShard.shardId(), IndexMetadata.INDEX_DATA_PATH_SETTING.get(allocation.metadata().index(currentShard.index()).getSettings()) ), currentShard ); - shardsToAddToCurrentBatch.put(currentShard.shardId(), sharEntry); + shardsToAddToCurrentBatch.put(currentShard.shardId(), shardEntry); batchSize--; iterator.remove(); } @@ -361,11 +360,11 @@ public AllocateUnassignedDecision explainUnassignedShardAllocation(ShardRouting } assert getBatchId(unassignedShard, unassignedShard.primary()) != null; if (unassignedShard.primary()) { - assert primaryBatchShardAllocator != null; - return primaryBatchShardAllocator.makeAllocationDecision(unassignedShard, routingAllocation, logger); + assert primaryShardBatchAllocator != null; + return primaryShardBatchAllocator.makeAllocationDecision(unassignedShard, routingAllocation, logger); } else { - assert replicaBatchShardAllocator != null; - return replicaBatchShardAllocator.makeAllocationDecision(unassignedShard, routingAllocation, logger); + assert replicaShardBatchAllocator != null; + return replicaShardBatchAllocator.makeAllocationDecision(unassignedShard, routingAllocation, logger); } } From c2da82c5a0b13d1d606bbba1d7b60bc29d567793 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Wed, 13 Mar 2024 12:24:18 +0530 Subject: [PATCH 26/45] Use new AsyncShardFetchBatch class for creating cache for batch transport actions Signed-off-by: Aman Khare --- .../gateway/ShardsBatchGatewayAllocator.java | 143 ++++++++++++------ .../TestShardBatchGatewayAllocator.java | 11 +- 2 files changed, 104 insertions(+), 50 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 655154bb880eb..72d134602e88f 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -33,8 +33,10 @@ import org.opensearch.common.util.set.Sets; import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.store.Store; import org.opensearch.indices.store.ShardAttributes; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper; import java.util.Collections; import java.util.HashMap; @@ -46,6 +48,10 @@ import java.util.Set; import java.util.Spliterators; import java.util.concurrent.ConcurrentMap; +import java.util.function.BiFunction; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -57,7 +63,6 @@ public class ShardsBatchGatewayAllocator implements ExistingShardsAllocator { public static final String ALLOCATOR_NAME = "shards_batch_gateway_allocator"; - private static final Logger logger = LogManager.getLogger(ShardsBatchGatewayAllocator.class); private final long maxBatchSize; private static final short DEFAULT_SHARD_BATCH_SIZE = 2000; @@ -74,25 +79,22 @@ public class ShardsBatchGatewayAllocator implements ExistingShardsAllocator { ); private final RerouteService rerouteService; - - private PrimaryShardBatchAllocator primaryShardBatchAllocator; - private ReplicaShardBatchAllocator replicaShardBatchAllocator; - + private final PrimaryShardBatchAllocator primaryShardBatchAllocator; + private final ReplicaShardBatchAllocator replicaShardBatchAllocator; private Set lastSeenEphemeralIds = Collections.emptySet(); - // visble for testing + // visible for testing protected final ConcurrentMap batchIdToStartedShardBatch = ConcurrentCollections.newConcurrentMap(); // visible for testing protected final ConcurrentMap batchIdToStoreShardBatch = ConcurrentCollections.newConcurrentMap(); - - private final TransportNodesListGatewayStartedBatchShards batchStartedAction; + private final TransportNodesListGatewayStartedShardsBatch batchStartedAction; private final TransportNodesListShardStoreMetadataBatch batchStoreAction; @Inject public ShardsBatchGatewayAllocator( RerouteService rerouteService, - TransportNodesListGatewayStartedBatchShards batchStartedAction, + TransportNodesListGatewayStartedShardsBatch batchStartedAction, TransportNodesListShardStoreMetadataBatch batchStoreAction, Settings settings ) { @@ -165,7 +167,7 @@ public void beforeAllocation(final RoutingAllocation allocation) { @Override public void afterPrimariesBeforeReplicas(RoutingAllocation allocation) { assert replicaShardBatchAllocator != null; - List> storedShardBatches = batchIdToStoreShardBatch.values() + List> storedShardBatches = batchIdToStoreShardBatch.values() .stream() .map(ShardsBatch::getBatchedShardRoutings) .collect(Collectors.toList()); @@ -254,7 +256,6 @@ else if (shardRouting.primary() == primary) { if (batchSize > 0) { ShardEntry shardEntry = new ShardEntry( new ShardAttributes( - currentShard.shardId(), IndexMetadata.INDEX_DATA_PATH_SETTING.get(allocation.metadata().index(currentShard.index()).getSettings()) ), currentShard @@ -424,15 +425,31 @@ private boolean hasNewNodes(DiscoveryNodes nodes) { return false; } - class InternalBatchAsyncFetch extends AsyncShardFetch { + class InternalBatchAsyncFetch extends AsyncShardBatchFetch { InternalBatchAsyncFetch( Logger logger, String type, Map map, AsyncShardFetch.Lister, T> action, - String batchUUId + String batchUUId, + Class clazz, + BiFunction, T> responseBuilder, + Function> shardsBatchDataGetter, + Supplier emptyResponseBuilder, + Consumer handleFailedShard ) { - super(logger, type, map, action, batchUUId); + super( + logger, + type, + map, + action, + batchUUId, + clazz, + responseBuilder, + shardsBatchDataGetter, + emptyResponseBuilder, + handleFailedShard + ); } @Override @@ -454,12 +471,12 @@ class InternalPrimaryBatchShardAllocator extends PrimaryShardBatchAllocator { @Override @SuppressWarnings("unchecked") - protected AsyncShardFetch.FetchResult fetchData( - Set shardsEligibleForFetch, - Set inEligibleShards, + protected AsyncShardFetch.FetchResult fetchData( + List eligibleShards, + List inEligibleShards, RoutingAllocation allocation ) { - ShardRouting shardRouting = shardsEligibleForFetch.iterator().hasNext() ? shardsEligibleForFetch.iterator().next() : null; + ShardRouting shardRouting = eligibleShards.iterator().hasNext() ? eligibleShards.iterator().next() : null; shardRouting = shardRouting == null && inEligibleShards.iterator().hasNext() ? inEligibleShards.iterator().next() : shardRouting; @@ -481,7 +498,7 @@ protected AsyncShardFetch.FetchResult(null, Collections.emptyMap()); } @@ -491,7 +508,7 @@ protected AsyncShardFetch.FetchResult asyncFetcher = shardsBatch.getAsyncFetcher(); + AsyncShardBatchFetch asyncFetcher = shardsBatch.getAsyncFetcher(); AsyncShardFetch.FetchResult shardBatchState = asyncFetcher.fetchData( allocation.nodes(), shardToIgnoreNodes @@ -500,7 +517,7 @@ protected AsyncShardFetch.FetchResult) shardBatchState; + return (AsyncShardFetch.FetchResult) shardBatchState; } } @@ -509,12 +526,12 @@ class InternalReplicaBatchShardAllocator extends ReplicaShardBatchAllocator { @Override @SuppressWarnings("unchecked") protected AsyncShardFetch.FetchResult fetchData( - Set shardsEligibleForFetch, - Set inEligibleShards, + List eligibleShards, + List inEligibleShards, RoutingAllocation allocation ) { - // get batch id for anyone given shard. We are assuming all shards will have same batch Id - ShardRouting shardRouting = shardsEligibleForFetch.iterator().hasNext() ? shardsEligibleForFetch.iterator().next() : null; + // get batch id for anyone given shard. We are assuming all shards will have same batchId + ShardRouting shardRouting = eligibleShards.iterator().hasNext() ? eligibleShards.iterator().next() : null; shardRouting = shardRouting == null && inEligibleShards.iterator().hasNext() ? inEligibleShards.iterator().next() : shardRouting; @@ -536,7 +553,7 @@ protected AsyncShardFetch.FetchResult(null, Collections.emptyMap()); } @@ -544,7 +561,7 @@ protected AsyncShardFetch.FetchResult asyncFetcher = shardsBatch.getAsyncFetcher(); + AsyncShardBatchFetch asyncFetcher = shardsBatch.getAsyncFetcher(); AsyncShardFetch.FetchResult shardBatchStores = asyncFetcher.fetchData( allocation.nodes(), shardToIgnoreNodes @@ -565,14 +582,14 @@ protected boolean hasInitiatedFetching(ShardRouting shard) { /** * Holds information about a batch of shards to be allocated. * Async fetcher is used to fetch the data for the batch. - * + *

* Visible for testing */ public class ShardsBatch { private final String batchId; private final boolean primary; - private final AsyncShardFetch asyncBatch; + private final InternalBatchAsyncFetch asyncBatch; private final Map batchInfo; @@ -584,23 +601,59 @@ public ShardsBatch(String batchId, Map shardsWithInfo, bool .stream() .collect(Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().getShardAttributes())); this.primary = primary; - if (primary) { - asyncBatch = new InternalBatchAsyncFetch<>(logger, "batch_shards_started", shardIdsMap, batchStartedAction, batchId); + if (this.primary) { + asyncBatch = new InternalBatchAsyncFetch<>( + logger, + "batch_shards_started", + shardIdsMap, + batchStartedAction, + batchId, + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard.class, + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch::new, + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch::getNodeGatewayStartedShardsBatch, + () -> new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard(null, false, null, null), + this::removeShard + ); } else { - asyncBatch = new InternalBatchAsyncFetch<>(logger, "batch_shards_started", shardIdsMap, batchStoreAction, batchId); - + asyncBatch = new InternalBatchAsyncFetch<>( + logger, + "batch_shards_store", + shardIdsMap, + batchStoreAction, + batchId, + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata.class, + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch::new, + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch::getNodeStoreFilesMetadataBatch, + this::buildEmptyReplicaShardResponse, + this::removeShard + ); } } + protected void removeShard(ShardId shardId) { + this.batchInfo.remove(shardId); + } + + private TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata buildEmptyReplicaShardResponse() { + return new TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata( + new TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata( + null, + Store.MetadataSnapshot.EMPTY, + Collections.emptyList() + ), + null + ); + } + private void removeFromBatch(ShardRouting shard) { - batchInfo.remove(shard.shardId()); - asyncBatch.shardAttributesMap.remove(shard.shardId()); + removeShard(shard.shardId()); + asyncBatch.clearShard(shard.shardId()); // assert that fetcher and shards are the same as batched shards assert batchInfo.size() == asyncBatch.shardAttributesMap.size() : "Shards size is not equal to fetcher size"; } - public Set getBatchedShardRoutings() { - return batchInfo.values().stream().map(ShardEntry::getShardRouting).collect(Collectors.toSet()); + public List getBatchedShardRoutings() { + return batchInfo.values().stream().map(ShardEntry::getShardRouting).collect(Collectors.toList()); } public Set getBatchedShards() { @@ -611,7 +664,7 @@ public String getBatchId() { return batchId; } - public AsyncShardFetch getAsyncFetcher() { + public AsyncShardBatchFetch getAsyncFetcher() { return asyncBatch; } @@ -624,7 +677,7 @@ public boolean equals(Object o) { if (this == o) { return true; } - if (o == null || o instanceof ShardsBatch == false) { + if (o instanceof ShardsBatch == false) { return false; } ShardsBatch shardsBatch = (ShardsBatch) o; @@ -646,15 +699,10 @@ public String toString() { /** * Holds information about a shard to be allocated in a batch. */ - private class ShardEntry { + static class ShardEntry { private final ShardAttributes shardAttributes; - public ShardEntry setShardRouting(ShardRouting shardRouting) { - this.shardRouting = shardRouting; - return this; - } - private ShardRouting shardRouting; public ShardEntry(ShardAttributes shardAttributes, ShardRouting shardRouting) { @@ -669,6 +717,11 @@ public ShardRouting getShardRouting() { public ShardAttributes getShardAttributes() { return shardAttributes; } + + public ShardEntry setShardRouting(ShardRouting shardRouting) { + this.shardRouting = shardRouting; + return this; + } } public int getNumberOfStartedShardBatches() { 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 e34c222c94205..c7fbfe40d82a7 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 @@ -24,6 +24,7 @@ import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Set; @@ -36,8 +37,8 @@ public class TestShardBatchGatewayAllocator extends ShardsBatchGatewayAllocator PrimaryShardBatchAllocator primaryBatchShardAllocator = new PrimaryShardBatchAllocator() { @Override protected AsyncShardFetch.FetchResult fetchData( - Set shardsEligibleForFetch, - Set inEligibleShards, + List eligibleShards, + List inEligibleShards, RoutingAllocation allocation ) { Map foundShards = new HashMap<>(); @@ -47,7 +48,7 @@ protected AsyncShardFetch.FetchResult shardsOnNode = entry.getValue(); HashMap adaptedResponse = new HashMap<>(); - for (ShardRouting shardRouting : shardsEligibleForFetch) { + for (ShardRouting shardRouting : eligibleShards) { ShardId shardId = shardRouting.shardId(); Set ignoreNodes = allocation.getIgnoreNodes(shardId); @@ -78,8 +79,8 @@ protected AsyncShardFetch.FetchResult fetchData( - Set shardsEligibleForFetch, - Set inEligibleShards, + List eligibleShards, + List inEligibleShards, RoutingAllocation allocation ) { return new AsyncShardFetch.FetchResult<>(Collections.emptyMap(), Collections.emptyMap()); From d2e5522735d38e6a739b5c3d1d85a1426f3470a6 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Wed, 13 Mar 2024 12:52:18 +0530 Subject: [PATCH 27/45] Correct version check with 3.0.0 Signed-off-by: Aman Khare --- .../cluster/routing/allocation/AllocationService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 7f2967bca2d69..a934ad506c435 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 @@ -570,7 +570,7 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { Use batch mode if enabled and there is no custom allocator set for Allocation service */ Boolean batchModeEnabled = EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.get(settings); - if (batchModeEnabled && allocation.nodes().getMinNodeVersion().onOrAfter(Version.CURRENT) && existingShardsAllocators.size() == 2) { + if (batchModeEnabled && allocation.nodes().getMinNodeVersion().onOrAfter(Version.V_3_0_0) && existingShardsAllocators.size() == 2) { /* If we do not have any custom allocator set then we will be using ShardsBatchGatewayAllocator Currently AllocationService will not run any custom Allocator that implements allocateAllUnassignedShards From 4e4fa06aafb9a4d0534d0248d13393baec84ac72 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Mon, 18 Mar 2024 16:52:20 +0530 Subject: [PATCH 28/45] Local lookup map for shardId to batchId, more code comments Signed-off-by: Aman Khare --- .../gateway/ShardsBatchGatewayAllocator.java | 81 +++++++++---------- .../TestShardBatchGatewayAllocator.java | 2 +- 2 files changed, 38 insertions(+), 45 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 72d134602e88f..9d327e351338a 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -226,22 +226,29 @@ protected Set createAndUpdateBatches(RoutingAllocation allocation, boole RoutingNodes.UnassignedShards unassigned = allocation.routingNodes().unassigned(); ConcurrentMap currentBatches = primary ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; // get all batched shards - Set currentBatchedShards = currentBatches.values() - .stream() - .map(ShardsBatch::getBatchedShards) - .flatMap(Set::stream) - .collect(Collectors.toSet()); + Map currentBatchShards = new HashMap<>(); + for (Map.Entry batchEntry : currentBatches.entrySet()) { + batchEntry.getValue().getBatchedShards() + .forEach(shardId -> currentBatchShards.put(shardId, + batchEntry.getKey())); + } Set shardsToBatch = Sets.newHashSet(); // add all unassigned shards to the batch if they are not already in a batch unassigned.forEach(shardRouting -> { - if ((currentBatchedShards.contains(shardRouting.shardId()) == false) && (shardRouting.primary() == primary)) { + if ((currentBatchShards.containsKey(shardRouting.shardId()) == false) && (shardRouting.primary() == primary)) { assert shardRouting.unassigned(); shardsToBatch.add(shardRouting); } // if shard is already batched update to latest shardRouting information in the batches + // Replica shard assignment can be cancelled if we get a better match. These ShardRouting objects also + // store other information like relocating node, targetRelocatingShard etc. And it can be updated after + // batches are created. If we don't update the ShardRouting object, stale data would be passed from the + // batch. This stale data can end up creating a same decision which has already been taken, and we'll see + // failure in executeDecision of BaseGatewayShardAllocator. Previous non-batch mode flow also used to + // pass ShardRouting object directly from unassignedIterator, so we're following the same behaviour. else if (shardRouting.primary() == primary) { - String batchId = getBatchId(shardRouting, shardRouting.primary()); + String batchId = currentBatchShards.get(shardRouting.shardId()); batchesToBeAssigned.add(batchId); currentBatches.get(batchId).batchInfo.get(shardRouting.shardId()).setShardRouting(shardRouting); } @@ -253,17 +260,15 @@ else if (shardRouting.primary() == primary) { Map shardsToAddToCurrentBatch = new HashMap<>(); while (iterator.hasNext()) { ShardRouting currentShard = iterator.next(); - if (batchSize > 0) { - ShardEntry shardEntry = new ShardEntry( - new ShardAttributes( - IndexMetadata.INDEX_DATA_PATH_SETTING.get(allocation.metadata().index(currentShard.index()).getSettings()) - ), - currentShard - ); - shardsToAddToCurrentBatch.put(currentShard.shardId(), shardEntry); - batchSize--; - iterator.remove(); - } + ShardEntry shardEntry = new ShardEntry( + new ShardAttributes( + IndexMetadata.INDEX_DATA_PATH_SETTING.get(allocation.metadata().index(currentShard.index()).getSettings()) + ), + currentShard + ); + shardsToAddToCurrentBatch.put(currentShard.shardId(), shardEntry); + batchSize--; + iterator.remove(); // add to batch if batch size full or last shard in unassigned list if (batchSize == 0 || iterator.hasNext() == false) { String batchUUId = UUIDs.base64UUID(); @@ -294,14 +299,15 @@ private void addBatch(ShardsBatch shardsBatch, boolean primary) { * so that we can clean up the batch if it is empty and release the fetching resources * * @param shardRouting shard to be removed + * @param primary from which batch shard needs to be removed */ - protected void safelyRemoveShardFromBatch(ShardRouting shardRouting) { - String batchId = shardRouting.primary() ? getBatchId(shardRouting, true) : getBatchId(shardRouting, false); + protected void safelyRemoveShardFromBatch(ShardRouting shardRouting, boolean primary) { + String batchId = primary ? getBatchId(shardRouting, true) : getBatchId(shardRouting, false); if (batchId == null) { logger.debug("Shard[{}] is not batched", shardRouting); return; } - ConcurrentMap batches = shardRouting.primary() ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; + ConcurrentMap batches = primary ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; ShardsBatch batch = batches.get(batchId); batch.removeFromBatch(shardRouting); deleteBatchIfEmpty(batches, batchId); @@ -314,21 +320,8 @@ protected void safelyRemoveShardFromBatch(ShardRouting shardRouting) { * @param shardRouting shard to remove */ protected void safelyRemoveShardFromBothBatch(ShardRouting shardRouting) { - String primaryBatchId = getBatchId(shardRouting, true); - String replicaBatchId = getBatchId(shardRouting, false); - if (primaryBatchId == null && replicaBatchId == null) { - return; - } - if (primaryBatchId != null) { - ShardsBatch batch = batchIdToStartedShardBatch.get(primaryBatchId); - batch.removeFromBatch(shardRouting); - deleteBatchIfEmpty(batchIdToStartedShardBatch, primaryBatchId); - } - if (replicaBatchId != null) { - ShardsBatch batch = batchIdToStoreShardBatch.get(replicaBatchId); - batch.removeFromBatch(shardRouting); - deleteBatchIfEmpty(batchIdToStoreShardBatch, replicaBatchId); - } + safelyRemoveShardFromBatch(shardRouting, true); + safelyRemoveShardFromBatch(shardRouting, false); } private void deleteBatchIfEmpty(ConcurrentMap batches, String batchId) { @@ -425,7 +418,7 @@ private boolean hasNewNodes(DiscoveryNodes nodes) { return false; } - class InternalBatchAsyncFetch extends AsyncShardBatchFetch { + class InternalBatchAsyncFetch extends AsyncShardBatchFetch { InternalBatchAsyncFetch( Logger logger, String type, @@ -496,7 +489,7 @@ protected AsyncShardFetch.FetchResult safelyRemoveShardFromBatch(sr, sr.primary())); if (shardsBatch.getBatchedShards().isEmpty() && eligibleShards.isEmpty()) { logger.debug("Batch {} is empty", batchId); @@ -508,7 +501,7 @@ protected AsyncShardFetch.FetchResult asyncFetcher = shardsBatch.getAsyncFetcher(); + AsyncShardBatchFetch asyncFetcher = shardsBatch.getAsyncFetcher(); AsyncShardFetch.FetchResult shardBatchState = asyncFetcher.fetchData( allocation.nodes(), shardToIgnoreNodes @@ -551,7 +544,7 @@ protected AsyncShardFetch.FetchResult safelyRemoveShardFromBatch(sr, sr.primary())); if (shardsBatch.getBatchedShards().isEmpty() && eligibleShards.isEmpty()) { logger.debug("Batch {} is empty", batchId); @@ -561,7 +554,7 @@ protected AsyncShardFetch.FetchResult asyncFetcher = shardsBatch.getAsyncFetcher(); + AsyncShardBatchFetch asyncFetcher = shardsBatch.getAsyncFetcher(); AsyncShardFetch.FetchResult shardBatchStores = asyncFetcher.fetchData( allocation.nodes(), shardToIgnoreNodes @@ -589,7 +582,7 @@ public class ShardsBatch { private final String batchId; private final boolean primary; - private final InternalBatchAsyncFetch asyncBatch; + private final InternalBatchAsyncFetch asyncBatch; private final Map batchInfo; @@ -664,7 +657,7 @@ public String getBatchId() { return batchId; } - public AsyncShardBatchFetch getAsyncFetcher() { + public AsyncShardBatchFetch getAsyncFetcher() { return asyncBatch; } @@ -725,7 +718,7 @@ public ShardEntry setShardRouting(ShardRouting shardRouting) { } public int getNumberOfStartedShardBatches() { - return batchIdToStoreShardBatch.size(); + return batchIdToStartedShardBatch.size(); } public int getNumberOfStoreShardBatches() { 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 c7fbfe40d82a7..8721f40b2546b 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 @@ -109,7 +109,7 @@ public Set createAndUpdateBatches(RoutingAllocation allocation, boolean } public void safelyRemoveShardFromBatch(ShardRouting shard) { - super.safelyRemoveShardFromBatch(shard); + super.safelyRemoveShardFromBatch(shard, shard.primary()); } public void safelyRemoveShardFromBothBatch(ShardRouting shardRouting) { From 3b851b92333434423aeda01f18fee56852e74e52 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Thu, 4 Apr 2024 12:45:56 +0530 Subject: [PATCH 29/45] Remove shards from batches if they are not present in unassigned list from allocation object Signed-off-by: Aman Khare Signed-off-by: Shivansh Arora --- .../gateway/RecoveryFromGatewayIT.java | 297 +++++++++++++++--- .../gateway/ShardsBatchGatewayAllocator.java | 28 +- 2 files changed, 287 insertions(+), 38 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index a002369bacef8..b7823e0f4fb85 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -32,50 +32,57 @@ package org.opensearch.gateway; -import com.carrotsearch.hppc.cursors.ObjectCursor; - import org.apache.lucene.index.CorruptIndexException; +import org.opensearch.Version; import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsAction; import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsRequest; import org.opensearch.action.admin.cluster.configuration.ClearVotingConfigExclusionsAction; import org.opensearch.action.admin.cluster.configuration.ClearVotingConfigExclusionsRequest; import org.opensearch.action.admin.cluster.health.ClusterHealthResponse; import org.opensearch.action.admin.cluster.reroute.ClusterRerouteResponse; +import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsGroup; +import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsResponse; +import org.opensearch.action.admin.indices.exists.indices.IndicesExistsResponse; import org.opensearch.action.admin.indices.recovery.RecoveryResponse; import org.opensearch.action.admin.indices.stats.IndexStats; import org.opensearch.action.admin.indices.stats.IndicesStatsResponse; import org.opensearch.action.admin.indices.stats.ShardStats; import org.opensearch.action.support.ActionTestUtils; +import org.opensearch.action.support.master.AcknowledgedResponse; import org.opensearch.client.Requests; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.coordination.ElectionSchedulerFactory; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.UnassignedInfo; import org.opensearch.cluster.routing.allocation.ExistingShardsAllocator; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.core.index.Index; +import org.opensearch.core.index.shard.ShardId; import org.opensearch.env.NodeEnvironment; -import org.opensearch.index.Index; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayStartedShard; import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; -import org.opensearch.index.MergePolicyConfig; +import org.opensearch.index.MergePolicyProvider; import org.opensearch.index.engine.Engine; import org.opensearch.index.query.QueryBuilders; import org.opensearch.index.shard.IndexShard; -import org.opensearch.index.shard.ShardId; import org.opensearch.index.shard.ShardPath; import org.opensearch.indices.IndicesService; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.replication.common.ReplicationLuceneIndex; +import org.opensearch.indices.store.ShardAttributes; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper; import org.opensearch.plugins.Plugin; +import org.opensearch.test.InternalSettingsPlugin; +import org.opensearch.test.InternalTestCluster.RestartCallback; import org.opensearch.test.OpenSearchIntegTestCase; import org.opensearch.test.OpenSearchIntegTestCase.ClusterScope; import org.opensearch.test.OpenSearchIntegTestCase.Scope; -import org.opensearch.test.InternalSettingsPlugin; -import org.opensearch.test.InternalTestCluster.RestartCallback; import org.opensearch.test.store.MockFSIndexStore; import java.nio.file.DirectoryStream; @@ -89,14 +96,20 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ExecutionException; import java.util.stream.IntStream; +import static java.util.Collections.emptyMap; +import static java.util.Collections.emptySet; import static org.opensearch.cluster.coordination.ClusterBootstrapService.INITIAL_CLUSTER_MANAGER_NODES_SETTING; import static org.opensearch.cluster.health.ClusterHealthStatus.GREEN; import static org.opensearch.cluster.health.ClusterHealthStatus.RED; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS; import static org.opensearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.opensearch.gateway.GatewayRecoveryTestUtils.corruptShard; +import static org.opensearch.gateway.GatewayRecoveryTestUtils.getDiscoveryNodes; +import static org.opensearch.gateway.GatewayRecoveryTestUtils.prepareRequestMap; import static org.opensearch.gateway.GatewayService.RECOVER_AFTER_NODES_SETTING; import static org.opensearch.index.query.QueryBuilders.matchAllQuery; import static org.opensearch.index.query.QueryBuilders.termQuery; @@ -121,16 +134,15 @@ public void testOneNodeRecoverFromGateway() throws Exception { internalCluster().startNode(); - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("appAccountIds") - .field("type", "text") - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("appAccountIds") + .field("type", "text") + .endObject() + .endObject() + .endObject() + .toString(); assertAcked(prepareCreate("test").setMapping(mapping)); client().prepareIndex("test") @@ -190,8 +202,7 @@ private Map assertAndCapturePrimaryTerms(Map pre } final Map result = new HashMap<>(); final ClusterState state = client().admin().cluster().prepareState().get().getState(); - for (ObjectCursor cursor : state.metadata().indices().values()) { - final IndexMetadata indexMetadata = cursor.value; + for (final IndexMetadata indexMetadata : state.metadata().indices().values()) { final String index = indexMetadata.getIndex().getName(); final long[] previous = previousTerms.get(index); final long[] current = IntStream.range(0, indexMetadata.getNumberOfShards()).mapToLong(indexMetadata::primaryTerm).toArray(); @@ -216,19 +227,18 @@ private Map assertAndCapturePrimaryTerms(Map pre public void testSingleNodeNoFlush() throws Exception { internalCluster().startNode(); - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("field") - .field("type", "text") - .endObject() - .startObject("num") - .field("type", "integer") - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("field") + .field("type", "text") + .endObject() + .startObject("num") + .field("type", "integer") + .endObject() + .endObject() + .endObject() + .toString(); // note: default replica settings are tied to #data nodes-1 which is 0 here. We can do with 1 in this test. int numberOfShards = numberOfShards(); assertAcked( @@ -534,7 +544,7 @@ public void testReuseInFileBasedPeerRecovery() throws Exception { .put("number_of_replicas", 1) // disable merges to keep segments the same - .put(MergePolicyConfig.INDEX_MERGE_ENABLED, false) + .put(MergePolicyProvider.INDEX_MERGE_ENABLED, false) // expire retention leases quickly .put(IndexService.RETENTION_LEASE_SYNC_INTERVAL_SETTING.getKey(), "100ms") @@ -721,11 +731,11 @@ public Settings onNodeStopped(String nodeName) throws Exception { ); assertThat(response.getNodes(), hasSize(1)); - assertThat(response.getNodes().get(0).allocationId(), notNullValue()); + assertThat(response.getNodes().get(0).getGatewayShardStarted().allocationId(), notNullValue()); if (corrupt) { - assertThat(response.getNodes().get(0).storeException(), notNullValue()); + assertThat(response.getNodes().get(0).getGatewayShardStarted().storeException(), notNullValue()); } else { - assertThat(response.getNodes().get(0).storeException(), nullValue()); + assertThat(response.getNodes().get(0).getGatewayShardStarted().storeException(), nullValue()); } // start another node so cluster consistency checks won't time out due to the lack of state @@ -1018,4 +1028,217 @@ private void createNIndices(int n, String prefix) { ensureGreen(prefix + i); } } + + public void testSingleShardFetchUsingBatchAction() { + String indexName = "test"; + int numOfShards = 1; + prepareIndex(indexName, numOfShards); + Map shardIdShardAttributesMap = prepareRequestMap(new String[] { indexName }, numOfShards); + + ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); + + TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch response; + response = ActionTestUtils.executeBlocking( + internalCluster().getInstance(TransportNodesListGatewayStartedShardsBatch.class), + new TransportNodesListGatewayStartedShardsBatch.Request(searchShardsResponse.getNodes(), shardIdShardAttributesMap) + ); + final Index index = resolveIndex(indexName); + final ShardId shardId = new ShardId(index, 0); + GatewayStartedShard gatewayStartedShard = response.getNodesMap() + .get(searchShardsResponse.getNodes()[0].getId()) + .getNodeGatewayStartedShardsBatch() + .get(shardId); + assertNodeGatewayStartedShardsHappyCase(gatewayStartedShard); + } + + public void testShardFetchMultiNodeMultiIndexesUsingBatchAction() { + // start node + internalCluster().startNode(); + String indexName1 = "test1"; + String indexName2 = "test2"; + int numShards = internalCluster().numDataNodes(); + // assign one primary shard each to the data nodes + prepareIndex(indexName1, numShards); + prepareIndex(indexName2, numShards); + Map shardIdShardAttributesMap = prepareRequestMap(new String[] { indexName1, indexName2 }, numShards); + ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName1, indexName2).get(); + assertEquals(internalCluster().numDataNodes(), searchShardsResponse.getNodes().length); + TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch response; + response = ActionTestUtils.executeBlocking( + internalCluster().getInstance(TransportNodesListGatewayStartedShardsBatch.class), + new TransportNodesListGatewayStartedShardsBatch.Request(searchShardsResponse.getNodes(), shardIdShardAttributesMap) + ); + for (ClusterSearchShardsGroup clusterSearchShardsGroup : searchShardsResponse.getGroups()) { + ShardId shardId = clusterSearchShardsGroup.getShardId(); + assertEquals(1, clusterSearchShardsGroup.getShards().length); + String nodeId = clusterSearchShardsGroup.getShards()[0].currentNodeId(); + GatewayStartedShard gatewayStartedShard = response.getNodesMap().get(nodeId).getNodeGatewayStartedShardsBatch().get(shardId); + assertNodeGatewayStartedShardsHappyCase(gatewayStartedShard); + } + } + + public void testShardFetchCorruptedShardsUsingBatchAction() throws Exception { + String indexName = "test"; + int numOfShards = 1; + prepareIndex(indexName, numOfShards); + Map shardIdShardAttributesMap = prepareRequestMap(new String[] { indexName }, numOfShards); + ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); + final Index index = resolveIndex(indexName); + final ShardId shardId = new ShardId(index, 0); + corruptShard(searchShardsResponse.getNodes()[0].getName(), shardId); + TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch response; + internalCluster().restartNode(searchShardsResponse.getNodes()[0].getName()); + response = ActionTestUtils.executeBlocking( + internalCluster().getInstance(TransportNodesListGatewayStartedShardsBatch.class), + new TransportNodesListGatewayStartedShardsBatch.Request(getDiscoveryNodes(), shardIdShardAttributesMap) + ); + DiscoveryNode[] discoveryNodes = getDiscoveryNodes(); + GatewayStartedShard gatewayStartedShard = response.getNodesMap() + .get(discoveryNodes[0].getId()) + .getNodeGatewayStartedShardsBatch() + .get(shardId); + assertNotNull(gatewayStartedShard.storeException()); + assertNotNull(gatewayStartedShard.allocationId()); + assertTrue(gatewayStartedShard.primary()); + } + + public void testSingleShardStoreFetchUsingBatchAction() throws ExecutionException, InterruptedException { + String indexName = "test"; + DiscoveryNode[] nodes = getDiscoveryNodes(); + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response = prepareAndSendRequest( + new String[] { indexName }, + nodes + ); + Index index = resolveIndex(indexName); + ShardId shardId = new ShardId(index, 0); + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata = response.getNodesMap() + .get(nodes[0].getId()) + .getNodeStoreFilesMetadataBatch() + .get(shardId); + assertNodeStoreFilesMetadataSuccessCase(nodeStoreFilesMetadata, shardId); + } + + public void testShardStoreFetchMultiNodeMultiIndexesUsingBatchAction() throws Exception { + internalCluster().startNodes(2); + String indexName1 = "test1"; + String indexName2 = "test2"; + DiscoveryNode[] nodes = getDiscoveryNodes(); + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response = prepareAndSendRequest( + new String[] { indexName1, indexName2 }, + nodes + ); + ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName1, indexName2).get(); + for (ClusterSearchShardsGroup clusterSearchShardsGroup : searchShardsResponse.getGroups()) { + ShardId shardId = clusterSearchShardsGroup.getShardId(); + ShardRouting[] shardRoutings = clusterSearchShardsGroup.getShards(); + assertEquals(2, shardRoutings.length); + for (ShardRouting shardRouting : shardRoutings) { + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata = response.getNodesMap() + .get(shardRouting.currentNodeId()) + .getNodeStoreFilesMetadataBatch() + .get(shardId); + assertNodeStoreFilesMetadataSuccessCase(nodeStoreFilesMetadata, shardId); + } + } + } + + public void testShardStoreFetchNodeNotConnectedUsingBatchAction() { + DiscoveryNode nonExistingNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); + String indexName = "test"; + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response = prepareAndSendRequest( + new String[] { indexName }, + new DiscoveryNode[] { nonExistingNode } + ); + assertTrue(response.hasFailures()); + assertEquals(1, response.failures().size()); + assertEquals(nonExistingNode.getId(), response.failures().get(0).nodeId()); + } + + public void testShardStoreFetchCorruptedIndexUsingBatchAction() throws Exception { + internalCluster().startNodes(2); + String index1Name = "test1"; + String index2Name = "test2"; + prepareIndices(new String[] { index1Name, index2Name }, 1, 1); + Map shardAttributesMap = prepareRequestMap(new String[] { index1Name, index2Name }, 1); + Index index1 = resolveIndex(index1Name); + ShardId shardId1 = new ShardId(index1, 0); + ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(index1Name).get(); + assertEquals(2, searchShardsResponse.getNodes().length); + + // corrupt test1 index shards + corruptShard(searchShardsResponse.getNodes()[0].getName(), shardId1); + corruptShard(searchShardsResponse.getNodes()[1].getName(), shardId1); + ClusterRerouteResponse clusterRerouteResponse = client().admin().cluster().prepareReroute().setRetryFailed(false).get(); + DiscoveryNode[] discoveryNodes = getDiscoveryNodes(); + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response; + response = ActionTestUtils.executeBlocking( + internalCluster().getInstance(TransportNodesListShardStoreMetadataBatch.class), + new TransportNodesListShardStoreMetadataBatch.Request(shardAttributesMap, discoveryNodes) + ); + Map nodeStoreFilesMetadata = response.getNodesMap() + .get(discoveryNodes[0].getId()) + .getNodeStoreFilesMetadataBatch(); + // We don't store exception in case of corrupt index, rather just return an empty response + assertNull(nodeStoreFilesMetadata.get(shardId1).getStoreFileFetchException()); + assertEquals(shardId1, nodeStoreFilesMetadata.get(shardId1).storeFilesMetadata().shardId()); + assertTrue(nodeStoreFilesMetadata.get(shardId1).storeFilesMetadata().isEmpty()); + + Index index2 = resolveIndex(index2Name); + ShardId shardId2 = new ShardId(index2, 0); + assertNodeStoreFilesMetadataSuccessCase(nodeStoreFilesMetadata.get(shardId2), shardId2); + } + + private void prepareIndices(String[] indices, int numberOfPrimaryShards, int numberOfReplicaShards) { + for (String index : indices) { + createIndex( + index, + Settings.builder() + .put(SETTING_NUMBER_OF_SHARDS, numberOfPrimaryShards) + .put(SETTING_NUMBER_OF_REPLICAS, numberOfReplicaShards) + .build() + ); + index(index, "type", "1", Collections.emptyMap()); + flush(index); + } + } + + private TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch prepareAndSendRequest( + String[] indices, + DiscoveryNode[] nodes + ) { + Map shardAttributesMap = null; + prepareIndices(indices, 1, 1); + shardAttributesMap = prepareRequestMap(indices, 1); + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response; + return ActionTestUtils.executeBlocking( + internalCluster().getInstance(TransportNodesListShardStoreMetadataBatch.class), + new TransportNodesListShardStoreMetadataBatch.Request(shardAttributesMap, nodes) + ); + } + + private void assertNodeStoreFilesMetadataSuccessCase( + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata, + ShardId shardId + ) { + assertNull(nodeStoreFilesMetadata.getStoreFileFetchException()); + TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata storeFileMetadata = nodeStoreFilesMetadata.storeFilesMetadata(); + assertFalse(storeFileMetadata.isEmpty()); + assertEquals(shardId, storeFileMetadata.shardId()); + assertNotNull(storeFileMetadata.peerRecoveryRetentionLeases()); + } + + private void assertNodeGatewayStartedShardsHappyCase(GatewayStartedShard gatewayStartedShard) { + assertNull(gatewayStartedShard.storeException()); + assertNotNull(gatewayStartedShard.allocationId()); + assertTrue(gatewayStartedShard.primary()); + } + + private void prepareIndex(String indexName, int numberOfPrimaryShards) { + createIndex( + indexName, + Settings.builder().put(SETTING_NUMBER_OF_SHARDS, numberOfPrimaryShards).put(SETTING_NUMBER_OF_REPLICAS, 0).build() + ); + index(indexName, "type", "1", Collections.emptyMap()); + flush(indexName); + } } diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 9d327e351338a..5c66b3c91476f 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -234,6 +234,7 @@ protected Set createAndUpdateBatches(RoutingAllocation allocation, boole } Set shardsToBatch = Sets.newHashSet(); + Set batchedShardsToAssign = Sets.newHashSet(); // add all unassigned shards to the batch if they are not already in a batch unassigned.forEach(shardRouting -> { if ((currentBatchShards.containsKey(shardRouting.shardId()) == false) && (shardRouting.primary() == primary)) { @@ -251,8 +252,12 @@ else if (shardRouting.primary() == primary) { String batchId = currentBatchShards.get(shardRouting.shardId()); batchesToBeAssigned.add(batchId); currentBatches.get(batchId).batchInfo.get(shardRouting.shardId()).setShardRouting(shardRouting); + batchedShardsToAssign.add(shardRouting.shardId()); } }); + + refreshShardBatches(currentBatches, batchedShardsToAssign); + Iterator iterator = shardsToBatch.iterator(); assert maxBatchSize > 0 : "Shards batch size must be greater than 0"; @@ -283,6 +288,23 @@ else if (shardRouting.primary() == primary) { return batchesToBeAssigned; } + private void refreshShardBatches(ConcurrentMap currentBatches, Set batchedShardsToAssign) { + // cleanup shard from batches if they are not present in unassigned list from allocation object. This is + // needed as AllocationService.reroute can also be called directly by API flows for example DeleteIndices. + // So, as part of calling reroute, those shards will be removed from allocation object. It'll handle the + // scenarios where shards can be removed from unassigned list without "start" or "failed" event. + for (Map.Entry batchEntry : currentBatches.entrySet()) { + Iterator shardIdIterator = batchEntry.getValue().getBatchedShards().iterator(); + while (shardIdIterator.hasNext()) { + ShardId shardId = shardIdIterator.next(); + if (batchedShardsToAssign.contains(shardId) == false) { + shardIdIterator.remove(); + batchEntry.getValue().clearShardFromCache(shardId); + } + } + } + } + private void addBatch(ShardsBatch shardsBatch, boolean primary) { ConcurrentMap batches = primary ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; if (batches.containsKey(shardsBatch.getBatchId())) { @@ -640,11 +662,15 @@ private TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata buildEm private void removeFromBatch(ShardRouting shard) { removeShard(shard.shardId()); - asyncBatch.clearShard(shard.shardId()); + clearShardFromCache(shard.shardId()); // assert that fetcher and shards are the same as batched shards assert batchInfo.size() == asyncBatch.shardAttributesMap.size() : "Shards size is not equal to fetcher size"; } + private void clearShardFromCache(ShardId shardId) { + asyncBatch.clearShard(shardId); + } + public List getBatchedShardRoutings() { return batchInfo.values().stream().map(ShardEntry::getShardRouting).collect(Collectors.toList()); } From edc66ddb9bad8894268745ad5d165ef77537b04f Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Wed, 10 Apr 2024 12:28:43 +0530 Subject: [PATCH 30/45] Resolved PR comments 1.Stop data nodes in test cases deterministically instead of random 2.Minor rename of variables Signed-off-by: Gaurav Chandani --- .../gateway/RecoveryFromGatewayIT.java | 20 +++++----- .../gateway/ShardsBatchGatewayAllocator.java | 37 +++++++++---------- 2 files changed, 29 insertions(+), 28 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index b7823e0f4fb85..df17ed316f69f 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -79,6 +79,7 @@ import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper; import org.opensearch.plugins.Plugin; import org.opensearch.test.InternalSettingsPlugin; +import org.opensearch.test.InternalTestCluster; import org.opensearch.test.InternalTestCluster.RestartCallback; import org.opensearch.test.OpenSearchIntegTestCase; import org.opensearch.test.OpenSearchIntegTestCase.ClusterScope; @@ -773,8 +774,8 @@ public void testBatchModeEnabled() throws Exception { ensureGreen("test"); Settings node0DataPathSettings = internalCluster().dataPathSettings(dataOnlyNodes.get(0)); Settings node1DataPathSettings = internalCluster().dataPathSettings(dataOnlyNodes.get(1)); - internalCluster().stopRandomDataNode(); - internalCluster().stopRandomDataNode(); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(dataOnlyNodes.get(0))); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(dataOnlyNodes.get(1))); ensureRed("test"); ensureStableCluster(1); @@ -798,6 +799,7 @@ public void testBatchModeEnabled() throws Exception { ensureGreen("test"); assertEquals(0, gatewayAllocator.getNumberOfStartedShardBatches()); assertEquals(0, gatewayAllocator.getNumberOfStoreShardBatches()); + assertEquals(0,gatewayAllocator.getNumberOfInFlightFetches()); } public void testBatchModeDisabled() throws Exception { @@ -814,8 +816,8 @@ public void testBatchModeDisabled() throws Exception { ensureGreen("test"); Settings node0DataPathSettings = internalCluster().dataPathSettings(dataOnlyNodes.get(0)); Settings node1DataPathSettings = internalCluster().dataPathSettings(dataOnlyNodes.get(1)); - internalCluster().stopRandomDataNode(); - internalCluster().stopRandomDataNode(); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(dataOnlyNodes.get(0))); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(dataOnlyNodes.get(1))); ensureStableCluster(1); logger.info("--> Now do a protective reroute"); @@ -869,8 +871,8 @@ public void testNBatchesCreationAndAssignment() throws Exception { Settings node1DataPathSettings = internalCluster().dataPathSettings(dataOnlyNodes.get(1)); internalCluster().stopCurrentClusterManagerNode(); - internalCluster().stopRandomDataNode(); - internalCluster().stopRandomDataNode(); + 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( @@ -958,9 +960,9 @@ public void testCulpritShardInBatch() throws Exception { Settings node2DataPathSettings = internalCluster().dataPathSettings(dataOnlyNodes.get(2)); internalCluster().stopCurrentClusterManagerNode(); - internalCluster().stopRandomDataNode(); - internalCluster().stopRandomDataNode(); - internalCluster().stopRandomDataNode(); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(dataOnlyNodes.get(0))); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(dataOnlyNodes.get(1))); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(dataOnlyNodes.get(2))); // Now start cluster manager node and post that verify batches created internalCluster().startClusterManagerOnlyNodes( diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 5c66b3c91476f..69878d83dfa73 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -53,6 +53,7 @@ import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; +import java.util.stream.Stream; import java.util.stream.StreamSupport; /** @@ -108,14 +109,12 @@ public ShardsBatchGatewayAllocator( @Override public void cleanCaches() { - Releasables.close( - batchIdToStartedShardBatch.values().stream().map(shardsBatch -> shardsBatch.asyncBatch).collect(Collectors.toList()) - ); - batchIdToStartedShardBatch.clear(); - Releasables.close( - batchIdToStoreShardBatch.values().stream().map(shardsBatch -> shardsBatch.asyncBatch).collect(Collectors.toList()) - ); - batchIdToStoreShardBatch.clear(); + Stream.of(batchIdToStartedShardBatch, batchIdToStoreShardBatch).forEach(b -> { + Releasables.close( + b.values().stream().map(shardsBatch -> shardsBatch.asyncBatch).collect(Collectors.toList()) + ); + b.clear(); + }); } // for tests @@ -226,20 +225,20 @@ protected Set createAndUpdateBatches(RoutingAllocation allocation, boole RoutingNodes.UnassignedShards unassigned = allocation.routingNodes().unassigned(); ConcurrentMap currentBatches = primary ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; // get all batched shards - Map currentBatchShards = new HashMap<>(); + Map currentBatchedShards = new HashMap<>(); for (Map.Entry batchEntry : currentBatches.entrySet()) { batchEntry.getValue().getBatchedShards() - .forEach(shardId -> currentBatchShards.put(shardId, + .forEach(shardId -> currentBatchedShards.put(shardId, batchEntry.getKey())); } - Set shardsToBatch = Sets.newHashSet(); + Set newShardsToBatch = Sets.newHashSet(); Set batchedShardsToAssign = Sets.newHashSet(); // add all unassigned shards to the batch if they are not already in a batch unassigned.forEach(shardRouting -> { - if ((currentBatchShards.containsKey(shardRouting.shardId()) == false) && (shardRouting.primary() == primary)) { + if ((currentBatchedShards.containsKey(shardRouting.shardId()) == false) && (shardRouting.primary() == primary)) { assert shardRouting.unassigned(); - shardsToBatch.add(shardRouting); + newShardsToBatch.add(shardRouting); } // if shard is already batched update to latest shardRouting information in the batches // Replica shard assignment can be cancelled if we get a better match. These ShardRouting objects also @@ -249,7 +248,7 @@ protected Set createAndUpdateBatches(RoutingAllocation allocation, boole // failure in executeDecision of BaseGatewayShardAllocator. Previous non-batch mode flow also used to // pass ShardRouting object directly from unassignedIterator, so we're following the same behaviour. else if (shardRouting.primary() == primary) { - String batchId = currentBatchShards.get(shardRouting.shardId()); + String batchId = currentBatchedShards.get(shardRouting.shardId()); batchesToBeAssigned.add(batchId); currentBatches.get(batchId).batchInfo.get(shardRouting.shardId()).setShardRouting(shardRouting); batchedShardsToAssign.add(shardRouting.shardId()); @@ -258,11 +257,11 @@ else if (shardRouting.primary() == primary) { refreshShardBatches(currentBatches, batchedShardsToAssign); - Iterator iterator = shardsToBatch.iterator(); + Iterator iterator = newShardsToBatch.iterator(); assert maxBatchSize > 0 : "Shards batch size must be greater than 0"; long batchSize = maxBatchSize; - Map shardsToAddToCurrentBatch = new HashMap<>(); + Map perBatchShards = new HashMap<>(); while (iterator.hasNext()) { ShardRouting currentShard = iterator.next(); ShardEntry shardEntry = new ShardEntry( @@ -271,17 +270,17 @@ else if (shardRouting.primary() == primary) { ), currentShard ); - shardsToAddToCurrentBatch.put(currentShard.shardId(), shardEntry); + perBatchShards.put(currentShard.shardId(), shardEntry); batchSize--; iterator.remove(); // add to batch if batch size full or last shard in unassigned list if (batchSize == 0 || iterator.hasNext() == false) { String batchUUId = UUIDs.base64UUID(); - ShardsBatch shardsBatch = new ShardsBatch(batchUUId, shardsToAddToCurrentBatch, primary); + ShardsBatch shardsBatch = new ShardsBatch(batchUUId, perBatchShards, primary); // add the batch to list of current batches addBatch(shardsBatch, primary); batchesToBeAssigned.add(batchUUId); - shardsToAddToCurrentBatch.clear(); + perBatchShards.clear(); batchSize = maxBatchSize; } } From 26a90ce2a7c4ad049cbb3c15223655efa3d9fca9 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Fri, 12 Apr 2024 11:53:41 +0530 Subject: [PATCH 31/45] Incorporate AsyncShardBatchFetch class changes Signed-off-by: Aman Khare Signed-off-by: Shivansh Arora --- .../gateway/RecoveryFromGatewayIT.java | 2 +- .../org/opensearch/cluster/ClusterModule.java | 8 +- .../opensearch/gateway/GatewayAllocator.java | 2 +- .../gateway/ShardsBatchGatewayAllocator.java | 54 +- ...sportNodesListShardStoreMetadataBatch.java | 351 ++++++++++++ .../main/java/org/opensearch/node/Node.java | 513 +++++++++++++----- .../cluster/ClusterModuleTests.java | 13 +- .../gateway/GatewayAllocatorTests.java | 12 +- .../gateway/ShardBatchCacheTests.java | 230 ++++++++ .../TestShardBatchGatewayAllocator.java | 7 +- 10 files changed, 990 insertions(+), 202 deletions(-) create mode 100644 server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java create mode 100644 server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index df17ed316f69f..dce96a6579586 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -799,7 +799,7 @@ public void testBatchModeEnabled() throws Exception { ensureGreen("test"); assertEquals(0, gatewayAllocator.getNumberOfStartedShardBatches()); assertEquals(0, gatewayAllocator.getNumberOfStoreShardBatches()); - assertEquals(0,gatewayAllocator.getNumberOfInFlightFetches()); + assertEquals(0, gatewayAllocator.getNumberOfInFlightFetches()); } public void testBatchModeDisabled() throws Exception { diff --git a/server/src/main/java/org/opensearch/cluster/ClusterModule.java b/server/src/main/java/org/opensearch/cluster/ClusterModule.java index a2ab51f221e80..3d56fa6d61255 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterModule.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterModule.java @@ -151,13 +151,7 @@ public ClusterModule( this.shardsAllocator = createShardsAllocator(settings, clusterService.getClusterSettings(), clusterPlugins); this.clusterService = clusterService; this.indexNameExpressionResolver = new IndexNameExpressionResolver(threadContext); - this.allocationService = new AllocationService( - allocationDeciders, - shardsAllocator, - clusterInfoService, - snapshotsInfoService, - settings - ); + this.allocationService = new AllocationService(allocationDeciders, shardsAllocator, clusterInfoService, snapshotsInfoService); } public static List getNamedWriteables() { diff --git a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java index d6232502d29fa..8ada56211327d 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java @@ -83,7 +83,7 @@ public class GatewayAllocator implements ExistingShardsAllocator { private final ConcurrentMap< ShardId, AsyncShardFetch> asyncFetchStarted = ConcurrentCollections - .newConcurrentMap(); + .newConcurrentMap(); private final ConcurrentMap> asyncFetchStore = ConcurrentCollections.newConcurrentMap(); private Set lastSeenEphemeralIds = Collections.emptySet(); diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 69878d83dfa73..0972f7b1398eb 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -33,10 +33,13 @@ import org.opensearch.common.util.set.Sets; import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayStartedShard; import org.opensearch.index.store.Store; import org.opensearch.indices.store.ShardAttributes; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; import java.util.Collections; import java.util.HashMap; @@ -48,10 +51,7 @@ import java.util.Set; import java.util.Spliterators; import java.util.concurrent.ConcurrentMap; -import java.util.function.BiFunction; -import java.util.function.Consumer; -import java.util.function.Function; -import java.util.function.Supplier; +import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; import java.util.stream.StreamSupport; @@ -110,9 +110,7 @@ public ShardsBatchGatewayAllocator( @Override public void cleanCaches() { Stream.of(batchIdToStartedShardBatch, batchIdToStoreShardBatch).forEach(b -> { - Releasables.close( - b.values().stream().map(shardsBatch -> shardsBatch.asyncBatch).collect(Collectors.toList()) - ); + Releasables.close(b.values().stream().map(shardsBatch -> shardsBatch.asyncBatch).collect(Collectors.toList())); b.clear(); }); } @@ -227,9 +225,7 @@ protected Set createAndUpdateBatches(RoutingAllocation allocation, boole // get all batched shards Map currentBatchedShards = new HashMap<>(); for (Map.Entry batchEntry : currentBatches.entrySet()) { - batchEntry.getValue().getBatchedShards() - .forEach(shardId -> currentBatchedShards.put(shardId, - batchEntry.getKey())); + batchEntry.getValue().getBatchedShards().forEach(shardId -> currentBatchedShards.put(shardId, batchEntry.getKey())); } Set newShardsToBatch = Sets.newHashSet(); @@ -447,23 +443,11 @@ class InternalBatchAsyncFetch extends AsyncShardB AsyncShardFetch.Lister, T> action, String batchUUId, Class clazz, - BiFunction, T> responseBuilder, - Function> shardsBatchDataGetter, - Supplier emptyResponseBuilder, - Consumer handleFailedShard + V emptyShardResponse, + Predicate emptyShardResponsePredicate, + ShardBatchResponseFactory responseFactory ) { - super( - logger, - type, - map, - action, - batchUUId, - clazz, - responseBuilder, - shardsBatchDataGetter, - emptyResponseBuilder, - handleFailedShard - ); + super(logger, type, map, action, batchUUId, clazz, emptyShardResponse, emptyShardResponsePredicate, responseFactory); } @Override @@ -622,11 +606,10 @@ public ShardsBatch(String batchId, Map shardsWithInfo, bool shardIdsMap, batchStartedAction, batchId, - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard.class, - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch::new, - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch::getNodeGatewayStartedShardsBatch, - () -> new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard(null, false, null, null), - this::removeShard + GatewayStartedShard.class, + new GatewayStartedShard(null, false, null, null), + GatewayStartedShard::isEmpty, + new ShardBatchResponseFactory<>(true) ); } else { asyncBatch = new InternalBatchAsyncFetch<>( @@ -635,11 +618,10 @@ public ShardsBatch(String batchId, Map shardsWithInfo, bool shardIdsMap, batchStoreAction, batchId, - TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata.class, - TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch::new, - TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch::getNodeStoreFilesMetadataBatch, - this::buildEmptyReplicaShardResponse, - this::removeShard + NodeStoreFilesMetadata.class, + new NodeStoreFilesMetadata(new StoreFilesMetadata(null, Store.MetadataSnapshot.EMPTY, Collections.emptyList()), null), + NodeStoreFilesMetadata::isEmpty, + new ShardBatchResponseFactory<>(false) ); } } diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java new file mode 100644 index 0000000000000..68dffa62f7b2c --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java @@ -0,0 +1,351 @@ +/* + * 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.indices.store; + +import org.opensearch.OpenSearchException; +import org.opensearch.action.ActionType; +import org.opensearch.action.FailedNodeException; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.nodes.BaseNodeResponse; +import org.opensearch.action.support.nodes.BaseNodesRequest; +import org.opensearch.action.support.nodes.BaseNodesResponse; +import org.opensearch.action.support.nodes.TransportNodesAction; +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.inject.Inject; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.env.NodeEnvironment; +import org.opensearch.gateway.AsyncShardFetch; +import org.opensearch.index.store.Store; +import org.opensearch.indices.IndicesService; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportRequest; +import org.opensearch.transport.TransportService; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.INDEX_NOT_FOUND; + +/** + * Transport action for fetching the batch of shard stores Metadata from a list of transport nodes + * + * @opensearch.internal + */ +public class TransportNodesListShardStoreMetadataBatch extends TransportNodesAction< + TransportNodesListShardStoreMetadataBatch.Request, + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch, + TransportNodesListShardStoreMetadataBatch.NodeRequest, + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch> + implements + AsyncShardFetch.Lister< + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch, + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch> { + + public static final String ACTION_NAME = "internal:cluster/nodes/indices/shard/store/batch"; + public static final ActionType TYPE = new ActionType<>( + ACTION_NAME, + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch::new + ); + + private final Settings settings; + private final IndicesService indicesService; + private final NodeEnvironment nodeEnv; + + @Inject + public TransportNodesListShardStoreMetadataBatch( + Settings settings, + ThreadPool threadPool, + ClusterService clusterService, + TransportService transportService, + IndicesService indicesService, + NodeEnvironment nodeEnv, + ActionFilters actionFilters + ) { + super( + ACTION_NAME, + threadPool, + clusterService, + transportService, + actionFilters, + Request::new, + NodeRequest::new, + ThreadPool.Names.FETCH_SHARD_STORE, + NodeStoreFilesMetadataBatch.class + ); + this.settings = settings; + this.indicesService = indicesService; + this.nodeEnv = nodeEnv; + } + + @Override + public void list( + Map shardAttributes, + DiscoveryNode[] nodes, + ActionListener listener + ) { + execute(new TransportNodesListShardStoreMetadataBatch.Request(shardAttributes, nodes), listener); + } + + @Override + protected NodeRequest newNodeRequest(Request request) { + return new NodeRequest(request); + } + + @Override + protected NodeStoreFilesMetadataBatch newNodeResponse(StreamInput in) throws IOException { + return new NodeStoreFilesMetadataBatch(in); + } + + @Override + protected NodesStoreFilesMetadataBatch newResponse( + Request request, + List responses, + List failures + ) { + return new NodesStoreFilesMetadataBatch(clusterService.getClusterName(), responses, failures); + } + + @Override + protected NodeStoreFilesMetadataBatch nodeOperation(NodeRequest request) { + try { + return new NodeStoreFilesMetadataBatch(clusterService.localNode(), listStoreMetadata(request)); + } catch (IOException e) { + throw new OpenSearchException( + "Failed to list store metadata for shards [" + request.getShardAttributes().keySet().stream().map(ShardId::toString) + "]", + e + ); + } + } + + /** + * This method is similar to listStoreMetadata method of {@link TransportNodesListShardStoreMetadata} + * In this case we fetch the shard store files for batch of shards instead of one shard. + */ + private Map listStoreMetadata(NodeRequest request) throws IOException { + Map shardStoreMetadataMap = new HashMap(); + for (Map.Entry shardAttributes : request.getShardAttributes().entrySet()) { + final ShardId shardId = shardAttributes.getKey(); + try { + StoreFilesMetadata storeFilesMetadata = TransportNodesListShardStoreMetadataHelper.listShardMetadataInternal( + logger, + shardId, + nodeEnv, + indicesService, + shardAttributes.getValue().getCustomDataPath(), + settings, + clusterService + ); + shardStoreMetadataMap.put(shardId, new NodeStoreFilesMetadata(storeFilesMetadata, null)); + } catch (Exception e) { + // should return null in case of known exceptions being returned from listShardMetadataInternal method. + if (e.getMessage().contains(INDEX_NOT_FOUND) || e instanceof IOException) { + shardStoreMetadataMap.put(shardId, null); + } else { + // return actual exception as it is for unknown exceptions + shardStoreMetadataMap.put( + shardId, + new NodeStoreFilesMetadata( + new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()), + e + ) + ); + } + } + } + return shardStoreMetadataMap; + } + + /** + * Request is used in constructing the request for making the transport request to set of other node. + * Refer {@link TransportNodesAction} class start method. + * + * @opensearch.internal + */ + public static class Request extends BaseNodesRequest { + + private final Map shardAttributes; + + public Request(StreamInput in) throws IOException { + super(in); + shardAttributes = in.readMap(ShardId::new, ShardAttributes::new); + } + + public Request(Map shardAttributes, DiscoveryNode[] nodes) { + super(nodes); + this.shardAttributes = Objects.requireNonNull(shardAttributes); + } + + public Map getShardAttributes() { + return shardAttributes; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeMap(shardAttributes, (o, k) -> k.writeTo(o), (o, v) -> v.writeTo(o)); + } + } + + /** + * Metadata for the nodes store files + * + * @opensearch.internal + */ + public static class NodesStoreFilesMetadataBatch extends BaseNodesResponse { + + public NodesStoreFilesMetadataBatch(StreamInput in) throws IOException { + super(in); + } + + public NodesStoreFilesMetadataBatch( + ClusterName clusterName, + List nodes, + List failures + ) { + super(clusterName, nodes, failures); + } + + @Override + protected List readNodesFrom(StreamInput in) throws IOException { + return in.readList(NodeStoreFilesMetadataBatch::new); + } + + @Override + protected void writeNodesTo(StreamOutput out, List nodes) throws IOException { + out.writeList(nodes); + } + } + + /** + * The metadata for the node store files + * + * @opensearch.internal + */ + public static class NodeStoreFilesMetadata { + + private StoreFilesMetadata storeFilesMetadata; + private Exception storeFileFetchException; + + public NodeStoreFilesMetadata(StoreFilesMetadata storeFilesMetadata) { + this.storeFilesMetadata = storeFilesMetadata; + this.storeFileFetchException = null; + } + + public NodeStoreFilesMetadata(StreamInput in) throws IOException { + storeFilesMetadata = new StoreFilesMetadata(in); + if (in.readBoolean()) { + this.storeFileFetchException = in.readException(); + } else { + this.storeFileFetchException = null; + } + } + + public NodeStoreFilesMetadata(StoreFilesMetadata storeFilesMetadata, Exception storeFileFetchException) { + this.storeFilesMetadata = storeFilesMetadata; + this.storeFileFetchException = storeFileFetchException; + } + + public StoreFilesMetadata storeFilesMetadata() { + return storeFilesMetadata; + } + + public void writeTo(StreamOutput out) throws IOException { + storeFilesMetadata.writeTo(out); + if (storeFileFetchException != null) { + out.writeBoolean(true); + out.writeException(storeFileFetchException); + } else { + out.writeBoolean(false); + } + } + + public static boolean isEmpty(NodeStoreFilesMetadata response) { + return response.storeFilesMetadata() == null + || response.storeFilesMetadata().isEmpty() && response.getStoreFileFetchException() == null; + } + + public Exception getStoreFileFetchException() { + return storeFileFetchException; + } + + @Override + public String toString() { + return "[[" + storeFilesMetadata + "]]"; + } + } + + /** + * NodeRequest class is for deserializing the request received by this node from other node for this transport action. + * This is used in {@link TransportNodesAction} + * @opensearch.internal + */ + public static class NodeRequest extends TransportRequest { + + private final Map shardAttributes; + + public NodeRequest(StreamInput in) throws IOException { + super(in); + shardAttributes = in.readMap(ShardId::new, ShardAttributes::new); + } + + public NodeRequest(Request request) { + this.shardAttributes = Objects.requireNonNull(request.getShardAttributes()); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeMap(shardAttributes, (o, k) -> k.writeTo(o), (o, v) -> v.writeTo(o)); + } + + public Map getShardAttributes() { + return shardAttributes; + } + } + + /** + * NodeStoreFilesMetadataBatch Response received by the node from other node for this transport action. + * Refer {@link TransportNodesAction} + */ + public static class NodeStoreFilesMetadataBatch extends BaseNodeResponse { + private final Map nodeStoreFilesMetadataBatch; + + protected NodeStoreFilesMetadataBatch(StreamInput in) throws IOException { + super(in); + this.nodeStoreFilesMetadataBatch = in.readMap(ShardId::new, NodeStoreFilesMetadata::new); + } + + public NodeStoreFilesMetadataBatch(DiscoveryNode node, Map nodeStoreFilesMetadataBatch) { + super(node); + this.nodeStoreFilesMetadataBatch = nodeStoreFilesMetadataBatch; + } + + public Map getNodeStoreFilesMetadataBatch() { + return this.nodeStoreFilesMetadataBatch; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeMap(nodeStoreFilesMetadataBatch, (o, k) -> k.writeTo(o), (o, v) -> v.writeTo(o)); + } + } + +} diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 02decffadf6ef..f500994289d30 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -35,36 +35,8 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.util.Constants; -import org.opensearch.ExceptionsHelper; -import org.opensearch.common.SetOnce; -import org.opensearch.common.settings.SettingsException; -import org.opensearch.common.unit.ByteSizeUnit; -import org.opensearch.common.unit.ByteSizeValue; -import org.opensearch.common.util.FeatureFlags; -import org.opensearch.cluster.routing.allocation.AwarenessReplicaBalance; -import org.opensearch.index.IndexModule; -import org.opensearch.index.IndexingPressureService; -import org.opensearch.index.store.remote.filecache.FileCache; -import org.opensearch.index.store.remote.filecache.FileCacheCleaner; -import org.opensearch.index.store.remote.filecache.FileCacheFactory; -import org.opensearch.indices.replication.SegmentReplicationSourceFactory; -import org.opensearch.indices.replication.SegmentReplicationTargetService; -import org.opensearch.indices.replication.SegmentReplicationSourceService; -import org.opensearch.extensions.ExtensionsManager; -import org.opensearch.extensions.NoopExtensionsManager; -import org.opensearch.monitor.fs.FsInfo; -import org.opensearch.monitor.fs.FsProbe; -import org.opensearch.plugins.SearchPipelinePlugin; -import org.opensearch.search.backpressure.SearchBackpressureService; -import org.opensearch.search.backpressure.settings.SearchBackpressureSettings; -import org.opensearch.search.pipeline.SearchPipelineService; -import org.opensearch.tasks.TaskResourceTrackingService; -import org.opensearch.tasks.consumer.TopNSearchTasksLogger; -import org.opensearch.threadpool.RunnableTaskExecutionListener; -import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; -import org.opensearch.watcher.ResourceWatcherService; -import org.opensearch.core.Assertions; import org.opensearch.Build; +import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.OpenSearchTimeoutException; import org.opensearch.Version; @@ -72,8 +44,13 @@ import org.opensearch.action.ActionModule.DynamicActionRegistry; import org.opensearch.action.ActionType; import org.opensearch.action.admin.cluster.snapshots.status.TransportNodesSnapshotsStatus; +import org.opensearch.action.admin.indices.view.ViewService; import org.opensearch.action.search.SearchExecutionStatsCollector; import org.opensearch.action.search.SearchPhaseController; +import org.opensearch.action.search.SearchRequestOperationsCompositeListenerFactory; +import org.opensearch.action.search.SearchRequestOperationsListener; +import org.opensearch.action.search.SearchRequestSlowLog; +import org.opensearch.action.search.SearchRequestStats; import org.opensearch.action.search.SearchTransportService; import org.opensearch.action.support.TransportAction; import org.opensearch.action.update.UpdateHelper; @@ -89,6 +66,7 @@ import org.opensearch.cluster.InternalClusterInfoService; import org.opensearch.cluster.NodeConnectionsService; import org.opensearch.cluster.action.index.MappingUpdatedAction; +import org.opensearch.cluster.coordination.PersistedStateRegistry; import org.opensearch.cluster.metadata.AliasValidator; import org.opensearch.cluster.metadata.IndexTemplateMetadata; import org.opensearch.cluster.metadata.Metadata; @@ -101,18 +79,20 @@ import org.opensearch.cluster.node.DiscoveryNodeRole; import org.opensearch.cluster.routing.BatchedRerouteService; import org.opensearch.cluster.routing.RerouteService; +import org.opensearch.cluster.routing.allocation.AwarenessReplicaBalance; import org.opensearch.cluster.routing.allocation.DiskThresholdMonitor; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.SetOnce; import org.opensearch.common.StopWatch; -import org.opensearch.common.breaker.CircuitBreaker; -import org.opensearch.common.component.Lifecycle; -import org.opensearch.common.component.LifecycleComponent; +import org.opensearch.common.cache.module.CacheModule; +import org.opensearch.common.cache.service.CacheService; import org.opensearch.common.inject.Injector; import org.opensearch.common.inject.Key; import org.opensearch.common.inject.Module; import org.opensearch.common.inject.ModulesBuilder; -import org.opensearch.common.io.stream.NamedWriteableRegistry; import org.opensearch.common.lease.Releasables; +import org.opensearch.common.lifecycle.Lifecycle; +import org.opensearch.common.lifecycle.LifecycleComponent; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.logging.HeaderWarning; import org.opensearch.common.logging.NodeAndClusterIdStateListener; @@ -125,59 +105,93 @@ import org.opensearch.common.settings.Setting.Property; import org.opensearch.common.settings.SettingUpgrader; import org.opensearch.common.settings.Settings; +import org.opensearch.common.settings.SettingsException; import org.opensearch.common.settings.SettingsModule; -import org.opensearch.common.transport.BoundTransportAddress; -import org.opensearch.common.transport.TransportAddress; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.BigArrays; +import org.opensearch.common.util.FeatureFlags; import org.opensearch.common.util.PageCacheRecycler; -import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.Assertions; +import org.opensearch.core.common.breaker.CircuitBreaker; +import org.opensearch.core.common.io.stream.NamedWriteableRegistry; +import org.opensearch.core.common.transport.BoundTransportAddress; +import org.opensearch.core.common.transport.TransportAddress; +import org.opensearch.core.common.unit.ByteSizeUnit; +import org.opensearch.core.common.unit.ByteSizeValue; +import org.opensearch.core.indices.breaker.CircuitBreakerService; +import org.opensearch.core.indices.breaker.NoneCircuitBreakerService; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.crypto.CryptoHandlerRegistry; import org.opensearch.discovery.Discovery; import org.opensearch.discovery.DiscoveryModule; import org.opensearch.env.Environment; import org.opensearch.env.NodeEnvironment; import org.opensearch.env.NodeMetadata; +import org.opensearch.extensions.ExtensionsManager; +import org.opensearch.extensions.NoopExtensionsManager; import org.opensearch.gateway.GatewayAllocator; -import org.opensearch.gateway.ShardsBatchGatewayAllocator; import org.opensearch.gateway.GatewayMetaState; import org.opensearch.gateway.GatewayModule; import org.opensearch.gateway.GatewayService; import org.opensearch.gateway.MetaStateService; import org.opensearch.gateway.PersistedClusterStateService; +import org.opensearch.gateway.ShardsBatchGatewayAllocator; +import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.http.HttpServerTransport; +import org.opensearch.identity.IdentityService; +import org.opensearch.index.IndexModule; import org.opensearch.index.IndexSettings; +import org.opensearch.index.IndexingPressureService; +import org.opensearch.index.SegmentReplicationStatsTracker; import org.opensearch.index.analysis.AnalysisRegistry; import org.opensearch.index.engine.EngineFactory; +import org.opensearch.index.recovery.RemoteStoreRestoreService; +import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; +import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; +import org.opensearch.index.store.remote.filecache.FileCache; +import org.opensearch.index.store.remote.filecache.FileCacheCleaner; +import org.opensearch.index.store.remote.filecache.FileCacheFactory; import org.opensearch.indices.IndicesModule; import org.opensearch.indices.IndicesService; +import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.indices.ShardLimitValidator; import org.opensearch.indices.SystemIndexDescriptor; import org.opensearch.indices.SystemIndices; import org.opensearch.indices.analysis.AnalysisModule; import org.opensearch.indices.breaker.BreakerSettings; -import org.opensearch.indices.breaker.CircuitBreakerService; import org.opensearch.indices.breaker.HierarchyCircuitBreakerService; -import org.opensearch.indices.breaker.NoneCircuitBreakerService; import org.opensearch.indices.cluster.IndicesClusterStateService; import org.opensearch.indices.recovery.PeerRecoverySourceService; import org.opensearch.indices.recovery.PeerRecoveryTargetService; import org.opensearch.indices.recovery.RecoverySettings; +import org.opensearch.indices.replication.SegmentReplicationSourceFactory; +import org.opensearch.indices.replication.SegmentReplicationSourceService; +import org.opensearch.indices.replication.SegmentReplicationTargetService; import org.opensearch.indices.store.IndicesStore; import org.opensearch.ingest.IngestService; import org.opensearch.monitor.MonitorService; import org.opensearch.monitor.fs.FsHealthService; +import org.opensearch.monitor.fs.FsInfo; +import org.opensearch.monitor.fs.FsProbe; import org.opensearch.monitor.jvm.JvmInfo; +import org.opensearch.node.remotestore.RemoteStoreNodeService; +import org.opensearch.node.resource.tracker.NodeResourceUsageTracker; import org.opensearch.persistent.PersistentTasksClusterService; import org.opensearch.persistent.PersistentTasksExecutor; import org.opensearch.persistent.PersistentTasksExecutorRegistry; import org.opensearch.persistent.PersistentTasksService; import org.opensearch.plugins.ActionPlugin; import org.opensearch.plugins.AnalysisPlugin; +import org.opensearch.plugins.CachePlugin; import org.opensearch.plugins.CircuitBreakerPlugin; import org.opensearch.plugins.ClusterPlugin; +import org.opensearch.plugins.CryptoKeyProviderPlugin; +import org.opensearch.plugins.CryptoPlugin; import org.opensearch.plugins.DiscoveryPlugin; import org.opensearch.plugins.EnginePlugin; +import org.opensearch.plugins.ExtensionAwarePlugin; +import org.opensearch.plugins.IdentityPlugin; import org.opensearch.plugins.IndexStorePlugin; import org.opensearch.plugins.IngestPlugin; import org.opensearch.plugins.MapperPlugin; @@ -188,8 +202,13 @@ import org.opensearch.plugins.PluginsService; import org.opensearch.plugins.RepositoryPlugin; import org.opensearch.plugins.ScriptPlugin; +import org.opensearch.plugins.SearchPipelinePlugin; import org.opensearch.plugins.SearchPlugin; +import org.opensearch.plugins.SecureSettingsFactory; import org.opensearch.plugins.SystemIndexPlugin; +import org.opensearch.plugins.TelemetryPlugin; +import org.opensearch.ratelimitting.admissioncontrol.AdmissionControlService; +import org.opensearch.ratelimitting.admissioncontrol.transport.AdmissionControlTransportInterceptor; import org.opensearch.repositories.RepositoriesModule; import org.opensearch.repositories.RepositoriesService; import org.opensearch.rest.RestController; @@ -200,7 +219,10 @@ import org.opensearch.search.SearchModule; import org.opensearch.search.SearchService; import org.opensearch.search.aggregations.support.AggregationUsageService; +import org.opensearch.search.backpressure.SearchBackpressureService; +import org.opensearch.search.backpressure.settings.SearchBackpressureSettings; import org.opensearch.search.fetch.FetchPhase; +import org.opensearch.search.pipeline.SearchPipelineService; import org.opensearch.search.query.QueryPhase; import org.opensearch.snapshots.InternalSnapshotsInfoService; import org.opensearch.snapshots.RestoreService; @@ -208,17 +230,32 @@ import org.opensearch.snapshots.SnapshotsInfoService; import org.opensearch.snapshots.SnapshotsService; import org.opensearch.tasks.Task; +import org.opensearch.tasks.TaskCancellationMonitoringService; +import org.opensearch.tasks.TaskCancellationMonitoringSettings; import org.opensearch.tasks.TaskCancellationService; +import org.opensearch.tasks.TaskResourceTrackingService; import org.opensearch.tasks.TaskResultsService; +import org.opensearch.tasks.consumer.TopNSearchTasksLogger; +import org.opensearch.telemetry.TelemetryModule; +import org.opensearch.telemetry.TelemetrySettings; +import org.opensearch.telemetry.metrics.MetricsRegistry; +import org.opensearch.telemetry.metrics.MetricsRegistryFactory; +import org.opensearch.telemetry.metrics.NoopMetricsRegistryFactory; +import org.opensearch.telemetry.tracing.NoopTracerFactory; +import org.opensearch.telemetry.tracing.Tracer; +import org.opensearch.telemetry.tracing.TracerFactory; import org.opensearch.threadpool.ExecutorBuilder; +import org.opensearch.threadpool.RunnableTaskExecutionListener; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.RemoteClusterService; import org.opensearch.transport.Transport; import org.opensearch.transport.TransportInterceptor; import org.opensearch.transport.TransportService; import org.opensearch.usage.UsageService; +import org.opensearch.watcher.ResourceWatcherService; import javax.net.ssl.SNIHostName; + import java.io.BufferedWriter; import java.io.Closeable; import java.io.IOException; @@ -233,6 +270,7 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; @@ -248,9 +286,11 @@ import java.util.stream.Stream; import static java.util.stream.Collectors.toList; -import static org.opensearch.common.util.FeatureFlags.SEARCH_PIPELINE; +import static org.opensearch.common.util.FeatureFlags.TELEMETRY; import static org.opensearch.env.NodeEnvironment.collectFileCacheDataPath; import static org.opensearch.index.ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED_ATTRIBUTE_KEY; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteStoreAttributePresent; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteStoreClusterStateEnabled; /** * A node represent a node within a cluster ({@code cluster.name}). The {@link #client()} can be used @@ -368,9 +408,13 @@ public static class DiscoverySettings { private final Collection pluginLifecycleComponents; private final LocalNodeFactory localNodeFactory; private final NodeService nodeService; + private final Tracer tracer; + + private final MetricsRegistry metricsRegistry; final NamedWriteableRegistry namedWriteableRegistry; private final AtomicReference runnableTaskListener; private FileCache fileCache; + private final RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory; public Node(Environment environment) { this(environment, Collections.emptyList(), true); @@ -416,7 +460,7 @@ protected Node( Constants.JVM_VERSION ); if (jvmInfo.getBundledJdk()) { - logger.info("JVM home [{}], using bundled JDK [{}]", System.getProperty("java.home"), jvmInfo.getUsingBundledJdk()); + logger.info("JVM home [{}], using bundled JDK/JRE [{}]", System.getProperty("java.home"), jvmInfo.getUsingBundledJdk()); } else { logger.info("JVM home [{}]", System.getProperty("java.home")); deprecationLogger.deprecate( @@ -455,8 +499,22 @@ protected Node( // Ensure to initialize Feature Flags via the settings from opensearch.yml FeatureFlags.initializeFeatureFlags(settings); + final List identityPlugins = new ArrayList<>(); + if (FeatureFlags.isEnabled(FeatureFlags.IDENTITY)) { + // If identity is enabled load plugins implementing the extension point + logger.info("Identity on so found plugins implementing: " + pluginsService.filterPlugins(IdentityPlugin.class).toString()); + identityPlugins.addAll(pluginsService.filterPlugins(IdentityPlugin.class)); + } + + final IdentityService identityService = new IdentityService(settings, identityPlugins); + if (FeatureFlags.isEnabled(FeatureFlags.EXTENSIONS)) { - this.extensionsManager = new ExtensionsManager(tmpSettings, initialEnvironment.extensionDir()); + final List extensionAwarePlugins = pluginsService.filterPlugins(ExtensionAwarePlugin.class); + Set> additionalSettings = new HashSet<>(); + for (ExtensionAwarePlugin extAwarePlugin : extensionAwarePlugins) { + additionalSettings.addAll(extAwarePlugin.getExtensionSettings()); + } + this.extensionsManager = new ExtensionsManager(additionalSettings, identityService); } else { this.extensionsManager = new NoopExtensionsManager(); } @@ -476,7 +534,11 @@ protected Node( */ this.environment = new Environment(settings, initialEnvironment.configDir(), Node.NODE_LOCAL_STORAGE_SETTING.get(settings)); Environment.assertEquivalent(initialEnvironment, this.environment); - nodeEnvironment = new NodeEnvironment(tmpSettings, environment); + if (DiscoveryNode.isSearchNode(settings) == false) { + nodeEnvironment = new NodeEnvironment(tmpSettings, environment); + } else { + nodeEnvironment = new NodeEnvironment(settings, environment, new FileCacheCleaner(this::fileCache)); + } logger.info( "node name [{}], node ID [{}], cluster name [{}], roles {}", NODE_NAME_SETTING.get(tmpSettings), @@ -488,12 +550,15 @@ protected Node( .collect(Collectors.toCollection(LinkedHashSet::new)) ); resourcesToClose.add(nodeEnvironment); - localNodeFactory = new LocalNodeFactory(settings, nodeEnvironment.nodeId()); final List> executorBuilders = pluginsService.getExecutorBuilders(settings); runnableTaskListener = new AtomicReference<>(); final ThreadPool threadPool = new ThreadPool(settings, runnableTaskListener, executorBuilders.toArray(new ExecutorBuilder[0])); + + final SetOnce repositoriesServiceReference = new SetOnce<>(); + final RemoteStoreNodeService remoteStoreNodeService = new RemoteStoreNodeService(repositoriesServiceReference::get, threadPool); + localNodeFactory = new LocalNodeFactory(settings, nodeEnvironment.nodeId(), remoteStoreNodeService); resourcesToClose.add(() -> ThreadPool.terminate(threadPool, 10, TimeUnit.SECONDS)); final ResourceWatcherService resourceWatcherService = new ResourceWatcherService(settings, threadPool); resourcesToClose.add(resourceWatcherService); @@ -547,17 +612,38 @@ protected Node( new ConsistentSettingsService(settings, clusterService, consistentSettings).newHashPublisher() ); } - final IngestService ingestService = new IngestService( - clusterService, - threadPool, - this.environment, - scriptService, - analysisModule.getAnalysisRegistry(), - pluginsService.filterPlugins(IngestPlugin.class), - client - ); - final SetOnce repositoriesServiceReference = new SetOnce<>(); + TracerFactory tracerFactory; + MetricsRegistryFactory metricsRegistryFactory; + if (FeatureFlags.isEnabled(TELEMETRY)) { + final TelemetrySettings telemetrySettings = new TelemetrySettings(settings, clusterService.getClusterSettings()); + if (telemetrySettings.isTracingFeatureEnabled() || telemetrySettings.isMetricsFeatureEnabled()) { + List telemetryPlugins = pluginsService.filterPlugins(TelemetryPlugin.class); + TelemetryModule telemetryModule = new TelemetryModule(telemetryPlugins, telemetrySettings); + if (telemetrySettings.isTracingFeatureEnabled()) { + tracerFactory = new TracerFactory(telemetrySettings, telemetryModule.getTelemetry(), threadPool.getThreadContext()); + } else { + tracerFactory = new NoopTracerFactory(); + } + if (telemetrySettings.isMetricsFeatureEnabled()) { + metricsRegistryFactory = new MetricsRegistryFactory(telemetrySettings, telemetryModule.getTelemetry()); + } else { + metricsRegistryFactory = new NoopMetricsRegistryFactory(); + } + } else { + tracerFactory = new NoopTracerFactory(); + metricsRegistryFactory = new NoopMetricsRegistryFactory(); + } + } else { + tracerFactory = new NoopTracerFactory(); + metricsRegistryFactory = new NoopMetricsRegistryFactory(); + } + + tracer = tracerFactory.getTracer(); + metricsRegistry = metricsRegistryFactory.getMetricsRegistry(); + resourcesToClose.add(tracer::close); + resourcesToClose.add(metricsRegistry::close); + final ClusterInfoService clusterInfoService = newClusterInfoService(settings, clusterService, threadPool, client); final UsageService usageService = new UsageService(); @@ -603,7 +689,6 @@ protected Node( ); // File cache will be initialized by the node once circuit breakers are in place. initializeFileCache(settings, circuitBreakerService.getBreaker(CircuitBreaker.REQUEST)); - final FileCacheCleaner fileCacheCleaner = new FileCacheCleaner(nodeEnvironment, fileCache); final MonitorService monitorService = new MonitorService(settings, nodeEnvironment, threadPool, fileCache); pluginsService.filterPlugins(CircuitBreakerPlugin.class).forEach(plugin -> { @@ -641,6 +726,19 @@ protected Node( clusterService.getClusterSettings(), threadPool::relativeTimeInMillis ); + final RemoteClusterStateService remoteClusterStateService; + if (isRemoteStoreClusterStateEnabled(settings)) { + remoteClusterStateService = new RemoteClusterStateService( + nodeEnvironment.nodeId(), + repositoriesServiceReference::get, + settings, + clusterService.getClusterSettings(), + threadPool::preciseRelativeTimeInNanos, + threadPool + ); + } else { + remoteClusterStateService = null; + } // collect engine factory providers from plugins final Collection enginePlugins = pluginsService.filterPlugins(EnginePlugin.class); @@ -690,65 +788,61 @@ protected Node( rerouteServiceReference.set(rerouteService); clusterService.setRerouteService(rerouteService); + final RecoverySettings recoverySettings = new RecoverySettings(settings, settingsModule.getClusterSettings()); + + final RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(settings, settingsModule.getClusterSettings()); + final IndexStorePlugin.DirectoryFactory remoteDirectoryFactory = new RemoteSegmentStoreDirectoryFactory( - repositoriesServiceReference::get + repositoriesServiceReference::get, + threadPool ); - final IndicesService indicesService; - if (FeatureFlags.isEnabled(FeatureFlags.EXTENSIONS)) { - indicesService = new IndicesService( - settings, - pluginsService, - extensionsManager, - nodeEnvironment, - xContentRegistry, - analysisModule.getAnalysisRegistry(), - clusterModule.getIndexNameExpressionResolver(), - indicesModule.getMapperRegistry(), - namedWriteableRegistry, - threadPool, - settingsModule.getIndexScopedSettings(), - circuitBreakerService, - bigArrays, - scriptService, - clusterService, - client, - metaStateService, - engineFactoryProviders, - Map.copyOf(directoryFactories), - searchModule.getValuesSourceRegistry(), - recoveryStateFactories, - remoteDirectoryFactory, - repositoriesServiceReference::get, - fileCacheCleaner - ); - } else { - indicesService = new IndicesService( - settings, - pluginsService, - nodeEnvironment, - xContentRegistry, - analysisModule.getAnalysisRegistry(), - clusterModule.getIndexNameExpressionResolver(), - indicesModule.getMapperRegistry(), - namedWriteableRegistry, - threadPool, - settingsModule.getIndexScopedSettings(), - circuitBreakerService, - bigArrays, - scriptService, - clusterService, - client, - metaStateService, - engineFactoryProviders, - Map.copyOf(directoryFactories), - searchModule.getValuesSourceRegistry(), - recoveryStateFactories, - remoteDirectoryFactory, - repositoriesServiceReference::get, - fileCacheCleaner - ); - } + final SearchRequestStats searchRequestStats = new SearchRequestStats(clusterService.getClusterSettings()); + final SearchRequestSlowLog searchRequestSlowLog = new SearchRequestSlowLog(clusterService); + + remoteStoreStatsTrackerFactory = new RemoteStoreStatsTrackerFactory(clusterService, settings); + CacheModule cacheModule = new CacheModule(pluginsService.filterPlugins(CachePlugin.class), settings); + CacheService cacheService = cacheModule.getCacheService(); + final IndicesService indicesService = new IndicesService( + settings, + pluginsService, + nodeEnvironment, + xContentRegistry, + analysisModule.getAnalysisRegistry(), + clusterModule.getIndexNameExpressionResolver(), + indicesModule.getMapperRegistry(), + namedWriteableRegistry, + threadPool, + settingsModule.getIndexScopedSettings(), + circuitBreakerService, + bigArrays, + scriptService, + clusterService, + client, + metaStateService, + engineFactoryProviders, + Map.copyOf(directoryFactories), + searchModule.getValuesSourceRegistry(), + recoveryStateFactories, + remoteDirectoryFactory, + repositoriesServiceReference::get, + searchRequestStats, + remoteStoreStatsTrackerFactory, + recoverySettings, + cacheService, + remoteStoreSettings + ); + + final IngestService ingestService = new IngestService( + clusterService, + threadPool, + this.environment, + scriptService, + analysisModule.getAnalysisRegistry(), + pluginsService.filterPlugins(IngestPlugin.class), + client, + indicesService + ); final AliasValidator aliasValidator = new AliasValidator(); @@ -783,6 +877,8 @@ protected Node( metadataCreateIndexService ); + final ViewService viewService = new ViewService(clusterService, client, null); + Collection pluginComponents = pluginsService.filterPlugins(Plugin.class) .stream() .flatMap( @@ -802,6 +898,17 @@ protected Node( ) .collect(Collectors.toList()); + // register all standard SearchRequestOperationsCompositeListenerFactory to the SearchRequestOperationsCompositeListenerFactory + final SearchRequestOperationsCompositeListenerFactory searchRequestOperationsCompositeListenerFactory = + new SearchRequestOperationsCompositeListenerFactory( + Stream.concat( + Stream.of(searchRequestStats, searchRequestSlowLog), + pluginComponents.stream() + .filter(p -> p instanceof SearchRequestOperationsListener) + .map(p -> (SearchRequestOperationsListener) p) + ).toArray(SearchRequestOperationsListener[]::new) + ); + ActionModule actionModule = new ActionModule( settings, clusterModule.getIndexNameExpressionResolver(), @@ -813,12 +920,45 @@ protected Node( client, circuitBreakerService, usageService, - systemIndices + systemIndices, + identityService, + extensionsManager ); modules.add(actionModule); final RestController restController = actionModule.getRestController(); + final NodeResourceUsageTracker nodeResourceUsageTracker = new NodeResourceUsageTracker( + monitorService.fsService(), + threadPool, + settings, + clusterService.getClusterSettings() + ); + final ResourceUsageCollectorService resourceUsageCollectorService = new ResourceUsageCollectorService( + nodeResourceUsageTracker, + clusterService, + threadPool + ); + + final AdmissionControlService admissionControlService = new AdmissionControlService( + settings, + clusterService, + threadPool, + resourceUsageCollectorService + ); + + AdmissionControlTransportInterceptor admissionControlTransportInterceptor = new AdmissionControlTransportInterceptor( + admissionControlService + ); + + final Collection secureSettingsFactories = pluginsService.filterPlugins(Plugin.class) + .stream() + .map(p -> p.getSecureSettingFactory(settings)) + .filter(Optional::isPresent) + .map(Optional::get) + .collect(Collectors.toList()); + + List transportInterceptors = List.of(admissionControlTransportInterceptor); final NetworkModule networkModule = new NetworkModule( settings, pluginsService.filterPlugins(NetworkPlugin.class), @@ -830,8 +970,12 @@ protected Node( xContentRegistry, networkService, restController, - clusterService.getClusterSettings() + clusterService.getClusterSettings(), + tracer, + transportInterceptors, + secureSettingsFactories ); + Collection>> indexTemplateMetadataUpgraders = pluginsService.filterPlugins( Plugin.class ).stream().map(Plugin::getIndexTemplateMetadataUpgrader).collect(Collectors.toList()); @@ -860,20 +1004,21 @@ protected Node( networkModule.getTransportInterceptor(), localNodeFactory, settingsModule.getClusterSettings(), - taskHeaders + taskHeaders, + tracer ); TopNSearchTasksLogger taskConsumer = new TopNSearchTasksLogger(settings, settingsModule.getClusterSettings()); transportService.getTaskManager().registerTaskResourceConsumer(taskConsumer); - if (FeatureFlags.isEnabled(FeatureFlags.EXTENSIONS)) { - this.extensionsManager.initializeServicesAndRestHandler( - actionModule, - settingsModule, - transportService, - clusterService, - environment.settings(), - client - ); - } + this.extensionsManager.initializeServicesAndRestHandler( + actionModule, + settingsModule, + transportService, + clusterService, + environment.settings(), + client, + identityService + ); + final PersistedStateRegistry persistedStateRegistry = new PersistedStateRegistry(); final GatewayMetaState gatewayMetaState = new GatewayMetaState(); final ResponseCollectorService responseCollectorService = new ResponseCollectorService(clusterService); final SearchTransportService searchTransportService = new SearchTransportService( @@ -904,7 +1049,7 @@ protected Node( transportService.getTaskManager() ); - final RecoverySettings recoverySettings = new RecoverySettings(settings, settingsModule.getClusterSettings()); + final SegmentReplicationStatsTracker segmentReplicationStatsTracker = new SegmentReplicationStatsTracker(indicesService); RepositoriesModule repositoriesModule = new RepositoriesModule( this.environment, pluginsService.filterPlugins(RepositoryPlugin.class), @@ -914,6 +1059,11 @@ protected Node( xContentRegistry, recoverySettings ); + CryptoHandlerRegistry.initRegistry( + pluginsService.filterPlugins(CryptoPlugin.class), + pluginsService.filterPlugins(CryptoKeyProviderPlugin.class), + settings + ); RepositoriesService repositoryService = repositoriesModule.getRepositoryService(); repositoriesServiceReference.set(repositoryService); SnapshotsService snapshotsService = new SnapshotsService( @@ -944,8 +1094,18 @@ protected Node( clusterModule.getAllocationService(), metadataCreateIndexService, metadataIndexUpgradeService, - clusterService.getClusterSettings(), - shardLimitValidator + shardLimitValidator, + indicesService, + clusterInfoService::getClusterInfo + ); + + RemoteStoreRestoreService remoteStoreRestoreService = new RemoteStoreRestoreService( + clusterService, + clusterModule.getAllocationService(), + metadataCreateIndexService, + metadataIndexUpgradeService, + shardLimitValidator, + remoteClusterStateService ); final DiskThresholdMonitor diskThresholdMonitor = new DiskThresholdMonitor( @@ -972,7 +1132,9 @@ protected Node( environment.configDir(), gatewayMetaState, rerouteService, - fsHealthService + fsHealthService, + persistedStateRegistry, + remoteStoreNodeService ); final SearchPipelineService searchPipelineService = new SearchPipelineService( clusterService, @@ -983,8 +1145,16 @@ protected Node( xContentRegistry, namedWriteableRegistry, pluginsService.filterPlugins(SearchPipelinePlugin.class), - client, - FeatureFlags.isEnabled(SEARCH_PIPELINE) + client + ); + final TaskCancellationMonitoringSettings taskCancellationMonitoringSettings = new TaskCancellationMonitoringSettings( + settings, + clusterService.getClusterSettings() + ); + final TaskCancellationMonitoringService taskCancellationMonitoringService = new TaskCancellationMonitoringService( + threadPool, + transportService.getTaskManager(), + taskCancellationMonitoringSettings ); this.nodeService = new NodeService( settings, @@ -1006,7 +1176,12 @@ protected Node( searchModule.getValuesSourceRegistry().getUsageService(), searchBackpressureService, searchPipelineService, - fileCache + fileCache, + taskCancellationMonitoringService, + resourceUsageCollectorService, + segmentReplicationStatsTracker, + repositoryService, + admissionControlService ); final SearchService searchService = newSearchService( @@ -1068,6 +1243,7 @@ protected Node( b.bind(IndexingPressureService.class).toInstance(indexingPressureService); b.bind(TaskResourceTrackingService.class).toInstance(taskResourceTrackingService); b.bind(SearchBackpressureService.class).toInstance(searchBackpressureService); + b.bind(AdmissionControlService.class).toInstance(admissionControlService); b.bind(UsageService.class).toInstance(usageService); b.bind(AggregationUsageService.class).toInstance(searchModule.getValuesSourceRegistry().getUsageService()); b.bind(NamedWriteableRegistry.class).toInstance(namedWriteableRegistry); @@ -1075,10 +1251,12 @@ protected Node( b.bind(MetaStateService.class).toInstance(metaStateService); b.bind(PersistedClusterStateService.class).toInstance(lucenePersistedStateFactory); b.bind(IndicesService.class).toInstance(indicesService); + b.bind(RemoteStoreStatsTrackerFactory.class).toInstance(remoteStoreStatsTrackerFactory); b.bind(AliasValidator.class).toInstance(aliasValidator); b.bind(MetadataCreateIndexService.class).toInstance(metadataCreateIndexService); b.bind(AwarenessReplicaBalance.class).toInstance(awarenessReplicaBalance); b.bind(MetadataCreateDataStreamService.class).toInstance(metadataCreateDataStreamService); + b.bind(ViewService.class).toInstance(viewService); b.bind(SearchService.class).toInstance(searchService); b.bind(SearchTransportService.class).toInstance(searchTransportService); b.bind(SearchPhaseController.class) @@ -1105,7 +1283,8 @@ protected Node( recoverySettings, transportService, new SegmentReplicationSourceFactory(transportService, recoverySettings, clusterService), - indicesService + indicesService, + clusterService ) ); b.bind(SegmentReplicationSourceService.class) @@ -1121,10 +1300,22 @@ protected Node( b.bind(SnapshotShardsService.class).toInstance(snapshotShardsService); b.bind(TransportNodesSnapshotsStatus.class).toInstance(nodesSnapshotsStatus); b.bind(RestoreService.class).toInstance(restoreService); + b.bind(RemoteStoreRestoreService.class).toInstance(remoteStoreRestoreService); b.bind(RerouteService.class).toInstance(rerouteService); b.bind(ShardLimitValidator.class).toInstance(shardLimitValidator); b.bind(FsHealthService.class).toInstance(fsHealthService); + b.bind(NodeResourceUsageTracker.class).toInstance(nodeResourceUsageTracker); + b.bind(ResourceUsageCollectorService.class).toInstance(resourceUsageCollectorService); b.bind(SystemIndices.class).toInstance(systemIndices); + b.bind(IdentityService.class).toInstance(identityService); + b.bind(Tracer.class).toInstance(tracer); + b.bind(SearchRequestStats.class).toInstance(searchRequestStats); + b.bind(SearchRequestSlowLog.class).toInstance(searchRequestSlowLog); + b.bind(MetricsRegistry.class).toInstance(metricsRegistry); + b.bind(RemoteClusterStateService.class).toProvider(() -> remoteClusterStateService); + b.bind(PersistedStateRegistry.class).toInstance(persistedStateRegistry); + b.bind(SegmentReplicationStatsTracker.class).toInstance(segmentReplicationStatsTracker); + b.bind(SearchRequestOperationsCompositeListenerFactory.class).toInstance(searchRequestOperationsCompositeListenerFactory); }); injector = modules.createInjector(); @@ -1133,8 +1324,10 @@ protected Node( // completes we trigger another reroute to try the allocation again. This means there is a circular dependency: the allocation // service needs access to the existing shards allocators (e.g. the GatewayAllocator, ShardsBatchGatewayAllocator) which // need to be able to trigger a reroute, which needs to call into the allocation service. We close the loop here: - clusterModule.setExistingShardsAllocators(injector.getInstance(GatewayAllocator.class), - injector.getInstance(ShardsBatchGatewayAllocator.class)); + clusterModule.setExistingShardsAllocators( + injector.getInstance(GatewayAllocator.class), + injector.getInstance(ShardsBatchGatewayAllocator.class) + ); List pluginLifecycleComponents = pluginComponents.stream() .filter(p -> p instanceof LifecycleComponent) @@ -1178,9 +1371,10 @@ protected TransportService newTransportService( TransportInterceptor interceptor, Function localNodeFactory, ClusterSettings clusterSettings, - Set taskHeaders + Set taskHeaders, + Tracer tracer ) { - return new TransportService(settings, transport, threadPool, interceptor, localNodeFactory, clusterSettings, taskHeaders); + return new TransportService(settings, transport, threadPool, interceptor, localNodeFactory, clusterSettings, taskHeaders, tracer); } protected void processRecoverySettings(ClusterSettings clusterSettings, RecoverySettings recoverySettings) { @@ -1234,8 +1428,11 @@ public Node start() throws NodeValidationException { injector.getInstance(RepositoriesService.class).start(); injector.getInstance(SearchService.class).start(); injector.getInstance(FsHealthService.class).start(); + injector.getInstance(NodeResourceUsageTracker.class).start(); + injector.getInstance(ResourceUsageCollectorService.class).start(); nodeService.getMonitorService().start(); nodeService.getSearchBackpressureService().start(); + nodeService.getTaskCancellationMonitoringService().start(); final ClusterService clusterService = injector.getInstance(ClusterService.class); @@ -1261,8 +1458,13 @@ public Node start() throws NodeValidationException { assert transportService.getLocalNode().equals(localNodeFactory.getNode()) : "transportService has a different local node than the factory provided"; injector.getInstance(PeerRecoverySourceService.class).start(); + injector.getInstance(SegmentReplicationTargetService.class).start(); injector.getInstance(SegmentReplicationSourceService.class).start(); + final RemoteClusterStateService remoteClusterStateService = injector.getInstance(RemoteClusterStateService.class); + if (remoteClusterStateService != null) { + remoteClusterStateService.start(); + } // Load (and maybe upgrade) the metadata stored on disk final GatewayMetaState gatewayMetaState = injector.getInstance(GatewayMetaState.class); gatewayMetaState.start( @@ -1272,7 +1474,10 @@ public Node start() throws NodeValidationException { injector.getInstance(MetaStateService.class), injector.getInstance(MetadataIndexUpgradeService.class), injector.getInstance(MetadataUpgrader.class), - injector.getInstance(PersistedClusterStateService.class) + injector.getInstance(PersistedClusterStateService.class), + injector.getInstance(RemoteClusterStateService.class), + injector.getInstance(PersistedStateRegistry.class), + injector.getInstance(RemoteStoreRestoreService.class) ); if (Assertions.ENABLED) { try { @@ -1306,9 +1511,6 @@ public Node start() throws NodeValidationException { assert clusterService.localNode().equals(localNodeFactory.getNode()) : "clusterService has a different local node than the factory provided"; transportService.acceptIncomingRequests(); - if (FeatureFlags.isEnabled(FeatureFlags.EXTENSIONS)) { - extensionsManager.initialize(); - } discovery.startInitialJoin(); final TimeValue initialStateTimeout = DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.get(settings()); configureNodeAndClusterIdStateListener(clusterService); @@ -1358,7 +1560,7 @@ public void onTimeout(TimeValue timeout) { logger.info("started"); - pluginsService.filterPlugins(ClusterPlugin.class).forEach(ClusterPlugin::onNodeStarted); + pluginsService.filterPlugins(ClusterPlugin.class).forEach(plugin -> plugin.onNodeStarted(clusterService.localNode())); return this; } @@ -1391,11 +1593,14 @@ private Node stop() { injector.getInstance(ClusterService.class).stop(); injector.getInstance(NodeConnectionsService.class).stop(); injector.getInstance(FsHealthService.class).stop(); + injector.getInstance(NodeResourceUsageTracker.class).stop(); + injector.getInstance(ResourceUsageCollectorService.class).stop(); nodeService.getMonitorService().stop(); nodeService.getSearchBackpressureService().stop(); injector.getInstance(GatewayService.class).stop(); injector.getInstance(SearchService.class).stop(); injector.getInstance(TransportService.class).stop(); + nodeService.getTaskCancellationMonitoringService().stop(); pluginLifecycleComponents.forEach(LifecycleComponent::stop); // we should stop this last since it waits for resources to get released @@ -1442,6 +1647,7 @@ public synchronized void close() throws IOException { toClose.add(injector.getInstance(IndicesStore.class)); toClose.add(injector.getInstance(PeerRecoverySourceService.class)); toClose.add(injector.getInstance(SegmentReplicationSourceService.class)); + toClose.add(injector.getInstance(SegmentReplicationTargetService.class)); toClose.add(() -> stopWatch.stop().start("cluster")); toClose.add(injector.getInstance(ClusterService.class)); toClose.add(() -> stopWatch.stop().start("node_connections_service")); @@ -1453,12 +1659,17 @@ public synchronized void close() throws IOException { toClose.add(nodeService.getSearchBackpressureService()); toClose.add(() -> stopWatch.stop().start("fsHealth")); toClose.add(injector.getInstance(FsHealthService.class)); + toClose.add(() -> stopWatch.stop().start("resource_usage_tracker")); + toClose.add(injector.getInstance(NodeResourceUsageTracker.class)); + toClose.add(() -> stopWatch.stop().start("resource_usage_collector")); + toClose.add(injector.getInstance(ResourceUsageCollectorService.class)); toClose.add(() -> stopWatch.stop().start("gateway")); toClose.add(injector.getInstance(GatewayService.class)); toClose.add(() -> stopWatch.stop().start("search")); toClose.add(injector.getInstance(SearchService.class)); toClose.add(() -> stopWatch.stop().start("transport")); toClose.add(injector.getInstance(TransportService.class)); + toClose.add(nodeService.getTaskCancellationMonitoringService()); for (LifecycleComponent plugin : pluginLifecycleComponents) { toClose.add(() -> stopWatch.stop().start("plugin(" + plugin.getClass().getName() + ")")); @@ -1481,6 +1692,10 @@ public synchronized void close() throws IOException { toClose.add(() -> stopWatch.stop().start("node_environment")); toClose.add(injector.getInstance(NodeEnvironment.class)); toClose.add(stopWatch::stop); + if (FeatureFlags.isEnabled(TELEMETRY)) { + toClose.add(injector.getInstance(Tracer.class)); + toClose.add(injector.getInstance(MetricsRegistry.class)); + } if (logger.isTraceEnabled()) { toClose.add(() -> logger.trace("Close times for each service:\n{}", stopWatch.prettyPrint())); @@ -1679,15 +1894,27 @@ private static class LocalNodeFactory implements Function localNode = new SetOnce<>(); private final String persistentNodeId; private final Settings settings; + private final RemoteStoreNodeService remoteStoreNodeService; - private LocalNodeFactory(Settings settings, String persistentNodeId) { + private LocalNodeFactory(Settings settings, String persistentNodeId, RemoteStoreNodeService remoteStoreNodeService) { this.persistentNodeId = persistentNodeId; this.settings = settings; + this.remoteStoreNodeService = remoteStoreNodeService; } @Override public DiscoveryNode apply(BoundTransportAddress boundTransportAddress) { - localNode.set(DiscoveryNode.createLocal(settings, boundTransportAddress.publishAddress(), persistentNodeId)); + final DiscoveryNode discoveryNode = DiscoveryNode.createLocal( + settings, + boundTransportAddress.publishAddress(), + persistentNodeId + ); + + if (isRemoteStoreAttributePresent(settings)) { + remoteStoreNodeService.createAndVerifyRepositories(discoveryNode); + } + + localNode.set(discoveryNode); return localNode.get(); } diff --git a/server/src/test/java/org/opensearch/cluster/ClusterModuleTests.java b/server/src/test/java/org/opensearch/cluster/ClusterModuleTests.java index 50a4f87f78d09..dd1971b7bf50a 100644 --- a/server/src/test/java/org/opensearch/cluster/ClusterModuleTests.java +++ b/server/src/test/java/org/opensearch/cluster/ClusterModuleTests.java @@ -76,7 +76,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -293,8 +292,10 @@ public void testRejectsReservedExistingShardsAllocatorName() { null, threadContext ); - expectThrows(IllegalArgumentException.class, () -> clusterModule.setExistingShardsAllocators(new TestGatewayAllocator(), - new TestShardBatchGatewayAllocator())); + expectThrows( + IllegalArgumentException.class, + () -> clusterModule.setExistingShardsAllocators(new TestGatewayAllocator(), new TestShardBatchGatewayAllocator()) + ); } public void testRejectsDuplicateExistingShardsAllocatorName() { @@ -306,8 +307,10 @@ public void testRejectsDuplicateExistingShardsAllocatorName() { null, threadContext ); - expectThrows(IllegalArgumentException.class, () -> clusterModule.setExistingShardsAllocators(new TestGatewayAllocator(), - new TestShardBatchGatewayAllocator())); + expectThrows( + IllegalArgumentException.class, + () -> clusterModule.setExistingShardsAllocators(new TestGatewayAllocator(), new TestShardBatchGatewayAllocator()) + ); } private static ClusterPlugin existingShardsAllocatorPlugin(final String allocatorName) { diff --git a/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java index ee269e0264b8b..604a5e62192c2 100644 --- a/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java @@ -255,9 +255,9 @@ public void testGetBatchIdExisting() { for (String batchId : primaryBatches) { if (shardRouting.primary() == true && testShardsBatchGatewayAllocator.getBatchIdToStartedShardBatch() - .get(batchId) - .getBatchedShards() - .contains(shardRouting.shardId())) { + .get(batchId) + .getBatchedShards() + .contains(shardRouting.shardId())) { if (shardIdToBatchIdForStartedShards.containsKey(shardRouting.shardId())) { fail("found duplicate shard routing for shard. One shard cant be in multiple batches " + shardRouting.shardId()); } @@ -272,9 +272,9 @@ public void testGetBatchIdExisting() { for (String batchId : replicaBatches) { if (shardRouting.primary() == false && testShardsBatchGatewayAllocator.getBatchIdToStoreShardBatch() - .get(batchId) - .getBatchedShards() - .contains(shardRouting.shardId())) { + .get(batchId) + .getBatchedShards() + .contains(shardRouting.shardId())) { if (shardIdToBatchIdForStoreShards.containsKey(shardRouting.shardId())) { fail("found duplicate shard routing for shard. One shard cant be in multiple batches " + shardRouting.shardId()); } diff --git a/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java b/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java new file mode 100644 index 0000000000000..12030ad41d508 --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java @@ -0,0 +1,230 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway; + +import org.opensearch.cluster.OpenSearchAllocationTestCase; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.ShardRoutingState; +import org.opensearch.cluster.routing.TestShardRouting; +import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayStartedShard; +import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch; +import org.opensearch.indices.store.ShardAttributes; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class ShardBatchCacheTests extends OpenSearchAllocationTestCase { + private static final String BATCH_ID = "b1"; + private final DiscoveryNode node1 = newNode("node1"); + private final DiscoveryNode node2 = newNode("node2"); + private final Map batchInfo = new HashMap<>(); + private AsyncShardBatchFetch.ShardBatchCache shardCache; + private List shardsInBatch = new ArrayList<>(); + private static final int NUMBER_OF_SHARDS_DEFAULT = 10; + + private enum ResponseType { + NULL, + EMPTY, + FAILURE, + VALID + } + + public void setupShardBatchCache(String batchId, int numberOfShards) { + Map shardAttributesMap = new HashMap<>(); + fillShards(shardAttributesMap, numberOfShards); + this.shardCache = new AsyncShardBatchFetch.ShardBatchCache<>( + logger, + "batch_shards_started", + shardAttributesMap, + "BatchID=[" + batchId + "]", + GatewayStartedShard.class, + new GatewayStartedShard(null, false, null, null), + GatewayStartedShard::isEmpty, + new ShardBatchResponseFactory<>(true) + ); + } + + public void testClearShardCache() { + setupShardBatchCache(BATCH_ID, NUMBER_OF_SHARDS_DEFAULT); + ShardId shard = shardsInBatch.iterator().next(); + this.shardCache.initData(node1); + this.shardCache.markAsFetching(List.of(node1.getId()), 1); + this.shardCache.putData(node1, new NodeGatewayStartedShardsBatch(node1, getPrimaryResponse(shardsInBatch, ResponseType.EMPTY))); + assertTrue( + this.shardCache.getCacheData(DiscoveryNodes.builder().add(node1).build(), null) + .get(node1) + .getNodeGatewayStartedShardsBatch() + .containsKey(shard) + ); + this.shardCache.deleteShard(shard); + assertFalse( + this.shardCache.getCacheData(DiscoveryNodes.builder().add(node1).build(), null) + .get(node1) + .getNodeGatewayStartedShardsBatch() + .containsKey(shard) + ); + } + + public void testGetCacheData() { + setupShardBatchCache(BATCH_ID, NUMBER_OF_SHARDS_DEFAULT); + ShardId shard = shardsInBatch.iterator().next(); + this.shardCache.initData(node1); + this.shardCache.initData(node2); + this.shardCache.markAsFetching(List.of(node1.getId(), node2.getId()), 1); + this.shardCache.putData(node1, new NodeGatewayStartedShardsBatch(node1, getPrimaryResponse(shardsInBatch, ResponseType.EMPTY))); + assertTrue( + this.shardCache.getCacheData(DiscoveryNodes.builder().add(node1).build(), null) + .get(node1) + .getNodeGatewayStartedShardsBatch() + .containsKey(shard) + ); + assertTrue( + this.shardCache.getCacheData(DiscoveryNodes.builder().add(node2).build(), null) + .get(node2) + .getNodeGatewayStartedShardsBatch() + .isEmpty() + ); + } + + public void testInitCacheData() { + setupShardBatchCache(BATCH_ID, NUMBER_OF_SHARDS_DEFAULT); + this.shardCache.initData(node1); + this.shardCache.initData(node2); + assertEquals(2, shardCache.getCache().size()); + + // test getData without fetch + assertTrue(shardCache.getData(node1).getNodeGatewayStartedShardsBatch().isEmpty()); + } + + public void testPutData() { + // test empty and non-empty responses + setupShardBatchCache(BATCH_ID, NUMBER_OF_SHARDS_DEFAULT); + ShardId shard = shardsInBatch.iterator().next(); + this.shardCache.initData(node1); + this.shardCache.initData(node2); + this.shardCache.markAsFetching(List.of(node1.getId(), node2.getId()), 1); + this.shardCache.putData(node1, new NodeGatewayStartedShardsBatch(node1, getPrimaryResponse(shardsInBatch, ResponseType.VALID))); + this.shardCache.putData(node2, new NodeGatewayStartedShardsBatch(node1, getPrimaryResponse(shardsInBatch, ResponseType.EMPTY))); + + // assert that fetching is done as both node's responses are stored in cache + assertFalse(this.shardCache.getCache().get(node1.getId()).isFetching()); + assertFalse(this.shardCache.getCache().get(node2.getId()).isFetching()); + + Map fetchData = shardCache.getCacheData( + DiscoveryNodes.builder().add(node1).add(node2).build(), + null + ); + assertEquals(2, fetchData.size()); + assertEquals(10, fetchData.get(node1).getNodeGatewayStartedShardsBatch().size()); + assertEquals("alloc-1", fetchData.get(node1).getNodeGatewayStartedShardsBatch().get(shard).allocationId()); + + assertEquals(10, fetchData.get(node2).getNodeGatewayStartedShardsBatch().size()); + assertTrue(GatewayStartedShard.isEmpty(fetchData.get(node2).getNodeGatewayStartedShardsBatch().get(shard))); + + // test GetData after fetch + assertEquals(10, shardCache.getData(node1).getNodeGatewayStartedShardsBatch().size()); + } + + public void testNullResponses() { + setupShardBatchCache(BATCH_ID, NUMBER_OF_SHARDS_DEFAULT); + this.shardCache.initData(node1); + this.shardCache.markAsFetching(List.of(node1.getId()), 1); + this.shardCache.putData(node1, new NodeGatewayStartedShardsBatch(node1, getPrimaryResponse(shardsInBatch, ResponseType.NULL))); + + Map fetchData = shardCache.getCacheData( + DiscoveryNodes.builder().add(node1).build(), + null + ); + assertTrue(fetchData.get(node1).getNodeGatewayStartedShardsBatch().isEmpty()); + } + + public void testShardsDataWithException() { + setupShardBatchCache(BATCH_ID, NUMBER_OF_SHARDS_DEFAULT); + this.shardCache.initData(node1); + this.shardCache.initData(node2); + this.shardCache.markAsFetching(List.of(node1.getId(), node2.getId()), 1); + this.shardCache.putData(node1, new NodeGatewayStartedShardsBatch(node1, getFailedPrimaryResponse(shardsInBatch, 5))); + Map fetchData = shardCache.getCacheData( + DiscoveryNodes.builder().add(node1).add(node2).build(), + null + ); + + assertEquals(10, batchInfo.size()); + assertEquals(2, fetchData.size()); + assertEquals(10, fetchData.get(node1).getNodeGatewayStartedShardsBatch().size()); + assertTrue(fetchData.get(node2).getNodeGatewayStartedShardsBatch().isEmpty()); + } + + private Map getPrimaryResponse(List shards, ResponseType responseType) { + int allocationId = 1; + Map shardData = new HashMap<>(); + for (ShardId shard : shards) { + switch (responseType) { + case NULL: + shardData.put(shard, null); + break; + case EMPTY: + shardData.put(shard, new GatewayStartedShard(null, false, null, null)); + break; + case VALID: + shardData.put(shard, new GatewayStartedShard("alloc-" + allocationId++, false, null, null)); + break; + default: + throw new AssertionError("unknown response type"); + } + } + return shardData; + } + + private Map getFailedPrimaryResponse(List shards, int failedShardsCount) { + int allocationId = 1; + Map shardData = new HashMap<>(); + for (ShardId shard : shards) { + if (failedShardsCount-- > 0) { + shardData.put( + shard, + new GatewayStartedShard("alloc-" + allocationId++, false, null, new OpenSearchRejectedExecutionException()) + ); + } else { + shardData.put(shard, new GatewayStartedShard("alloc-" + allocationId++, false, null, null)); + } + } + return shardData; + } + + private void fillShards(Map shardAttributesMap, int numberOfShards) { + shardsInBatch = BatchTestUtil.setUpShards(numberOfShards); + for (ShardId shardId : shardsInBatch) { + ShardAttributes attr = new ShardAttributes(""); + shardAttributesMap.put(shardId, attr); + batchInfo.put( + shardId, + new ShardsBatchGatewayAllocator.ShardEntry(attr, randomShardRouting(shardId.getIndexName(), shardId.id())) + ); + } + } + + private ShardRouting randomShardRouting(String index, int shard) { + ShardRoutingState state = randomFrom(ShardRoutingState.values()); + return TestShardRouting.newShardRouting( + index, + shard, + state == ShardRoutingState.UNASSIGNED ? null : "1", + state == ShardRoutingState.RELOCATING ? "2" : null, + state != ShardRoutingState.UNASSIGNED && randomBoolean(), + state + ); + } +} 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 8721f40b2546b..53a4e90adb976 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 @@ -18,6 +18,7 @@ import org.opensearch.gateway.PrimaryShardBatchAllocator; import org.opensearch.gateway.ReplicaShardBatchAllocator; import org.opensearch.gateway.ShardsBatchGatewayAllocator; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper; import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; @@ -46,15 +47,15 @@ protected AsyncShardFetch.FetchResult> entry : knownAllocations.entrySet()) { String nodeId = entry.getKey(); Map shardsOnNode = entry.getValue(); - HashMap adaptedResponse = new HashMap<>(); + HashMap adaptedResponse = new HashMap<>(); for (ShardRouting shardRouting : eligibleShards) { ShardId shardId = shardRouting.shardId(); Set ignoreNodes = allocation.getIgnoreNodes(shardId); if (shardsOnNode.containsKey(shardId) && ignoreNodes.contains(nodeId) == false && currentNodes.nodeExists(nodeId)) { - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeShard = - new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard( + TransportNodesGatewayStartedShardHelper.GatewayStartedShard nodeShard = + new TransportNodesGatewayStartedShardHelper.GatewayStartedShard( shardsOnNode.get(shardId).allocationId().getId(), shardsOnNode.get(shardId).primary(), getReplicationCheckpoint(shardId, nodeId) From 3da0af3f6234e8f1ace8426b577accaa7e6361fa Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Fri, 12 Apr 2024 12:54:12 +0530 Subject: [PATCH 32/45] Fixed GatewayAllocatorTests Signed-off-by: Gaurav Chandani --- .../java/org/opensearch/gateway/GatewayAllocatorTests.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java index 604a5e62192c2..bb59a5792ec8c 100644 --- a/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java @@ -122,7 +122,7 @@ public void testCorrectnessOfBatch() { .values() .stream() .map(ShardsBatchGatewayAllocator.ShardsBatch::getBatchedShardRoutings) - .flatMap(Set::stream) + .flatMap(List::stream) .collect(Collectors.toSet()); primariesInAllBatches.forEach(shardRouting -> assertTrue(shardRouting.unassigned() && shardRouting.primary() == true)); @@ -130,7 +130,7 @@ public void testCorrectnessOfBatch() { .values() .stream() .map(ShardsBatchGatewayAllocator.ShardsBatch::getBatchedShardRoutings) - .flatMap(Set::stream) + .flatMap(List::stream) .collect(Collectors.toSet()); replicasInAllBatches.forEach(shardRouting -> assertTrue(shardRouting.unassigned() && shardRouting.primary() == false)); From dbd51b541098b0c419cc3d43f1f9218e77c8f86a Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Fri, 12 Apr 2024 16:21:44 +0530 Subject: [PATCH 33/45] Delete batch if empty when removing shards from batch Signed-off-by: Aman Khare --- .../gateway/ShardsBatchGatewayAllocator.java | 6 ++++-- ...ansportNodesListShardStoreMetadataBatch.java | 17 +++++++++++++++-- 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 0972f7b1398eb..204bb140c4031 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -251,7 +251,7 @@ else if (shardRouting.primary() == primary) { } }); - refreshShardBatches(currentBatches, batchedShardsToAssign); + refreshShardBatches(currentBatches, batchedShardsToAssign, primary); Iterator iterator = newShardsToBatch.iterator(); assert maxBatchSize > 0 : "Shards batch size must be greater than 0"; @@ -283,7 +283,7 @@ else if (shardRouting.primary() == primary) { return batchesToBeAssigned; } - private void refreshShardBatches(ConcurrentMap currentBatches, Set batchedShardsToAssign) { + private void refreshShardBatches(ConcurrentMap currentBatches, Set batchedShardsToAssign, boolean primary) { // cleanup shard from batches if they are not present in unassigned list from allocation object. This is // needed as AllocationService.reroute can also be called directly by API flows for example DeleteIndices. // So, as part of calling reroute, those shards will be removed from allocation object. It'll handle the @@ -297,6 +297,8 @@ private void refreshShardBatches(ConcurrentMap currentBatch batchEntry.getValue().clearShardFromCache(shardId); } } + ConcurrentMap batches = primary ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; + deleteBatchIfEmpty(batches, batchEntry.getValue().getBatchId()); } } diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java index 68dffa62f7b2c..22b03539cca74 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java @@ -329,7 +329,13 @@ public static class NodeStoreFilesMetadataBatch extends BaseNodeResponse { protected NodeStoreFilesMetadataBatch(StreamInput in) throws IOException { super(in); - this.nodeStoreFilesMetadataBatch = in.readMap(ShardId::new, NodeStoreFilesMetadata::new); + this.nodeStoreFilesMetadataBatch = in.readMap(ShardId::new, i -> { + if (i.readBoolean()) { + return new NodeStoreFilesMetadata(i); + } else { + return null; + } + }); } public NodeStoreFilesMetadataBatch(DiscoveryNode node, Map nodeStoreFilesMetadataBatch) { @@ -344,7 +350,14 @@ public Map getNodeStoreFilesMetadataBatch() { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeMap(nodeStoreFilesMetadataBatch, (o, k) -> k.writeTo(o), (o, v) -> v.writeTo(o)); + out.writeMap(nodeStoreFilesMetadataBatch, (o, k) -> k.writeTo(o), (o, v) -> { + if (v != null) { + o.writeBoolean(true); + v.writeTo(o); + } else { + o.writeBoolean(false); + } + }); } } From eff1d2e65343d22137c77d098a62cc85b5c40358 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Fri, 12 Apr 2024 16:33:47 +0530 Subject: [PATCH 34/45] Fix index deletion test Signed-off-by: Aman Khare Signed-off-by: Shivansh Arora --- .../gateway/RecoveryFromGatewayIT.java | 47 +++++++++++++++++++ 1 file changed, 47 insertions(+) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index dce96a6579586..c2204c37d2385 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -1190,6 +1190,53 @@ public void testShardStoreFetchCorruptedIndexUsingBatchAction() throws Exception assertNodeStoreFilesMetadataSuccessCase(nodeStoreFilesMetadata.get(shardId2), shardId2); } + public void testDeleteRedIndexInBatchMode() throws Exception { + internalCluster().startClusterManagerOnlyNodes( + 1, + Settings.builder().put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.getKey(), true).build() + ); + List dataOnlyNodes = internalCluster().startDataOnlyNodes(2, Settings.builder().put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.getKey(), true).build()); + createIndex( + "test", + Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 2).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0).build() + ); + createIndex( + "test1", + Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 2).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0).build() + ); + createIndex( + "test2", + Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 2).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0).build() + ); + createIndex( + "testg", + Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).build() + ); + + ensureGreen("test", "test1", "test2", "testg"); + internalCluster().stopRandomDataNode(); + ensureStableCluster(2); + + ShardsBatchGatewayAllocator gatewayAllocator = internalCluster().getInstance( + ShardsBatchGatewayAllocator.class, + internalCluster().getClusterManagerName() + ); + ensureRed("test", "test1", "test2"); + + assertTrue(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.get(internalCluster().clusterService().getSettings())); + + logger.info("--> Now do a reroute so batches are created"); // to avoid any race condition in test + ClusterRerouteResponse clusterRerouteResponse = client().admin().cluster().prepareReroute().setRetryFailed(true).get(); + assertTrue(clusterRerouteResponse.isAcknowledged()); + + AcknowledgedResponse deleteIndexResponse = client().admin().indices().prepareDelete("test").get(); + assertTrue(deleteIndexResponse.isAcknowledged()); + + ensureYellow("testg"); + IndicesExistsResponse indexExistResponse = client().admin().indices().prepareExists("test").get(); + assertFalse(indexExistResponse.isExists()); + } + private void prepareIndices(String[] indices, int numberOfPrimaryShards, int numberOfReplicaShards) { for (String index : indices) { createIndex( From 6f52f9f119d9d8e6de9d0efd7208e85657e5b698 Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Mon, 15 Apr 2024 10:45:50 +0530 Subject: [PATCH 35/45] Spotless Apply Signed-off-by: Gaurav Chandani --- .../java/org/opensearch/gateway/RecoveryFromGatewayIT.java | 5 ++++- .../cluster/routing/allocation/AllocationService.java | 6 +++--- .../org/opensearch/gateway/ShardsBatchGatewayAllocator.java | 6 +++++- 3 files changed, 12 insertions(+), 5 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index c2204c37d2385..bc0557ddc2afa 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -1195,7 +1195,10 @@ public void testDeleteRedIndexInBatchMode() throws Exception { 1, Settings.builder().put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.getKey(), true).build() ); - List dataOnlyNodes = internalCluster().startDataOnlyNodes(2, Settings.builder().put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.getKey(), true).build()); + List dataOnlyNodes = internalCluster().startDataOnlyNodes( + 2, + Settings.builder().put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.getKey(), true).build() + ); createIndex( "test", Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 2).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0).build() 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 a934ad506c435..371f025471f44 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 @@ -670,9 +670,9 @@ private void applyStartedShards(RoutingAllocation routingAllocation, List currentBatches, Set batchedShardsToAssign, boolean primary) { + private void refreshShardBatches( + ConcurrentMap currentBatches, + Set batchedShardsToAssign, + boolean primary + ) { // cleanup shard from batches if they are not present in unassigned list from allocation object. This is // needed as AllocationService.reroute can also be called directly by API flows for example DeleteIndices. // So, as part of calling reroute, those shards will be removed from allocation object. It'll handle the From 7298a0838ded14b4ea4c586cf70dc222e4d9445b Mon Sep 17 00:00:00 2001 From: Gaurav Chandani Date: Tue, 16 Apr 2024 21:09:59 +0530 Subject: [PATCH 36/45] Refactored to reuse duplicate code Signed-off-by: Shivansh Arora --- .../routing/allocation/AllocationService.java | 41 ++--- .../gateway/ShardsBatchGatewayAllocator.java | 140 ++++++++---------- 2 files changed, 67 insertions(+), 114 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 371f025471f44..d6b364887b560 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 @@ -55,7 +55,6 @@ import org.opensearch.cluster.routing.allocation.command.AllocationCommands; import org.opensearch.cluster.routing.allocation.decider.AllocationDeciders; import org.opensearch.cluster.routing.allocation.decider.Decision; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.settings.Settings; import org.opensearch.gateway.GatewayAllocator; import org.opensearch.gateway.PriorityComparator; @@ -201,9 +200,9 @@ private ClusterState buildResult(ClusterState oldState, RoutingAllocation alloca if (restoreInProgress != null) { RestoreInProgress updatedRestoreInProgress = allocation.updateRestoreInfoWithRoutingChanges(restoreInProgress); if (updatedRestoreInProgress != restoreInProgress) { - ImmutableOpenMap.Builder customsBuilder = ImmutableOpenMap.builder(allocation.getCustoms()); + final Map customsBuilder = new HashMap<>(allocation.getCustoms()); customsBuilder.put(RestoreInProgress.TYPE, updatedRestoreInProgress); - newStateBuilder.customs(customsBuilder.build()); + newStateBuilder.customs(customsBuilder); } } return newStateBuilder.build(); @@ -575,11 +574,7 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { If we do not have any custom allocator set then we will be using ShardsBatchGatewayAllocator Currently AllocationService will not run any custom Allocator that implements allocateAllUnassignedShards */ - ExistingShardsAllocator allocator = existingShardsAllocators.get(ShardsBatchGatewayAllocator.ALLOCATOR_NAME); - allocator.allocateAllUnassignedShards(allocation, true); - allocator.afterPrimariesBeforeReplicas(allocation); - // Replicas Assignment - allocator.allocateAllUnassignedShards(allocation, false); + allocateAllUnassignedShards(allocation); return; } logger.warn("Falling back to single shard assignment since batch mode disable or multiple custom allocators set"); @@ -605,30 +600,12 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { } } - /** - * Verify if all unassigned shards are allocated by the same allocator, if yes then return the allocator, else - * return null - * @param allocation {@link RoutingAllocation} - * @return {@link ExistingShardsAllocator} or null - */ - private ExistingShardsAllocator getAndVerifySameAllocatorForAllUnassignedShards(RoutingAllocation allocation) { - // if there is a single Allocator set in Allocation Service then use it for all shards - if (existingShardsAllocators.size() == 1) { - return existingShardsAllocators.values().iterator().next(); - } - RoutingNodes.UnassignedShards unassignedShards = allocation.routingNodes().unassigned(); - RoutingNodes.UnassignedShards.UnassignedIterator iterator = unassignedShards.iterator(); - ExistingShardsAllocator currentAllocatorForShard = null; - if (unassignedShards.size() > 0) { - currentAllocatorForShard = getAllocatorForShard(iterator.next(), allocation); - while (iterator.hasNext()) { - ExistingShardsAllocator allocatorForShard = getAllocatorForShard(iterator.next(), allocation); - if (currentAllocatorForShard.getClass().getName().equals(allocatorForShard.getClass().getName()) == false) { - return null; - } - } - } - return currentAllocatorForShard; + private void allocateAllUnassignedShards(RoutingAllocation allocation) { + ExistingShardsAllocator allocator = existingShardsAllocators.get(ShardsBatchGatewayAllocator.ALLOCATOR_NAME); + allocator.allocateAllUnassignedShards(allocation, true); + allocator.afterPrimariesBeforeReplicas(allocation); + // Replicas Assignment + allocator.allocateAllUnassignedShards(allocation, false); } private void disassociateDeadNodes(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 e5367883d9434..7683ab6ed75dc 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -480,48 +480,12 @@ protected AsyncShardFetch.FetchResult inEligibleShards, RoutingAllocation allocation ) { - ShardRouting shardRouting = eligibleShards.iterator().hasNext() ? eligibleShards.iterator().next() : null; - shardRouting = shardRouting == null && inEligibleShards.iterator().hasNext() - ? inEligibleShards.iterator().next() - : shardRouting; - if (shardRouting == null) { - return new AsyncShardFetch.FetchResult<>(null, Collections.emptyMap()); - } - String batchId = getBatchId(shardRouting, shardRouting.primary()); - if (batchId == null) { - logger.debug("Shard {} has no batch id", shardRouting); - throw new IllegalStateException("Shard " + shardRouting + " has no batch id. Shard should batched before fetching"); - } - - if (batchIdToStartedShardBatch.containsKey(batchId) == false) { - logger.debug("Batch {} has no started shard batch", batchId); - throw new IllegalStateException("Batch " + batchId + " has no started shard batch"); - } - - ShardsBatch shardsBatch = batchIdToStartedShardBatch.get(batchId); - // remove in eligible shards which allocator is not responsible for - inEligibleShards.forEach(sr -> safelyRemoveShardFromBatch(sr, sr.primary())); - - if (shardsBatch.getBatchedShards().isEmpty() && eligibleShards.isEmpty()) { - logger.debug("Batch {} is empty", batchId); - return new AsyncShardFetch.FetchResult<>(null, Collections.emptyMap()); - } - - Map> shardToIgnoreNodes = new HashMap<>(); - - for (ShardId shardId : shardsBatch.asyncBatch.shardAttributesMap.keySet()) { - shardToIgnoreNodes.put(shardId, allocation.getIgnoreNodes(shardId)); - } - AsyncShardBatchFetch asyncFetcher = shardsBatch.getAsyncFetcher(); - AsyncShardFetch.FetchResult shardBatchState = asyncFetcher.fetchData( - allocation.nodes(), - shardToIgnoreNodes - ); - - if (shardBatchState.hasData()) { - shardBatchState.processAllocation(allocation); - } - return (AsyncShardFetch.FetchResult) shardBatchState; + return (AsyncShardFetch.FetchResult< + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch>) fetchDataAndCleanIneligibleShards( + eligibleShards, + inEligibleShards, + allocation + ); } } @@ -534,46 +498,12 @@ protected AsyncShardFetch.FetchResult inEligibleShards, RoutingAllocation allocation ) { - // get batch id for anyone given shard. We are assuming all shards will have same batchId - ShardRouting shardRouting = eligibleShards.iterator().hasNext() ? eligibleShards.iterator().next() : null; - shardRouting = shardRouting == null && inEligibleShards.iterator().hasNext() - ? inEligibleShards.iterator().next() - : shardRouting; - if (shardRouting == null) { - return new AsyncShardFetch.FetchResult<>(null, Collections.emptyMap()); - } - String batchId = getBatchId(shardRouting, shardRouting.primary()); - if (batchId == null) { - logger.debug("Shard {} has no batch id", shardRouting); - throw new IllegalStateException("Shard " + shardRouting + " has no batch id. Shard should batched before fetching"); - } - - if (batchIdToStoreShardBatch.containsKey(batchId) == false) { - logger.debug("Batch {} has no store shard batch", batchId); - throw new IllegalStateException("Batch " + batchId + " has no shard store batch"); - } - - ShardsBatch shardsBatch = batchIdToStoreShardBatch.get(batchId); - // remove in eligible shards which allocator is not responsible for - inEligibleShards.forEach(sr -> safelyRemoveShardFromBatch(sr, sr.primary())); - - if (shardsBatch.getBatchedShards().isEmpty() && eligibleShards.isEmpty()) { - logger.debug("Batch {} is empty", batchId); - return new AsyncShardFetch.FetchResult<>(null, Collections.emptyMap()); - } - Map> shardToIgnoreNodes = new HashMap<>(); - for (ShardId shardId : shardsBatch.asyncBatch.shardAttributesMap.keySet()) { - shardToIgnoreNodes.put(shardId, allocation.getIgnoreNodes(shardId)); - } - AsyncShardBatchFetch asyncFetcher = shardsBatch.getAsyncFetcher(); - AsyncShardFetch.FetchResult shardBatchStores = asyncFetcher.fetchData( - allocation.nodes(), - shardToIgnoreNodes - ); - if (shardBatchStores.hasData()) { - shardBatchStores.processAllocation(allocation); - } - return (AsyncShardFetch.FetchResult) shardBatchStores; + return (AsyncShardFetch.FetchResult< + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch>) fetchDataAndCleanIneligibleShards( + eligibleShards, + inEligibleShards, + allocation + ); } @Override @@ -583,6 +513,52 @@ protected boolean hasInitiatedFetching(ShardRouting shard) { } } + AsyncShardFetch.FetchResult fetchDataAndCleanIneligibleShards( + List eligibleShards, + List inEligibleShards, + RoutingAllocation allocation + ) { + // get batch id for anyone given shard. We are assuming all shards will have same batchId + ShardRouting shardRouting = eligibleShards.iterator().hasNext() ? eligibleShards.iterator().next() : null; + shardRouting = shardRouting == null && inEligibleShards.iterator().hasNext() ? inEligibleShards.iterator().next() : shardRouting; + if (shardRouting == null) { + return new AsyncShardFetch.FetchResult<>(null, Collections.emptyMap()); + } + String batchId = getBatchId(shardRouting, shardRouting.primary()); + if (batchId == null) { + logger.debug("Shard {} has no batch id", shardRouting); + throw new IllegalStateException("Shard " + shardRouting + " has no batch id. Shard should batched before fetching"); + } + ConcurrentMap batches = shardRouting.primary() ? batchIdToStartedShardBatch : batchIdToStoreShardBatch; + if (batches.containsKey(batchId) == false) { + logger.debug("Batch {} has no shards batch", batchId); + throw new IllegalStateException("Batch " + batchId + " has no shards batch"); + } + + ShardsBatch shardsBatch = batches.get(batchId); + // remove in eligible shards which allocator is not responsible for + inEligibleShards.forEach(sr -> safelyRemoveShardFromBatch(sr, sr.primary())); + + if (shardsBatch.getBatchedShards().isEmpty() && eligibleShards.isEmpty()) { + logger.debug("Batch {} is empty", batchId); + return new AsyncShardFetch.FetchResult<>(null, Collections.emptyMap()); + } + Map> shardToIgnoreNodes = new HashMap<>(); + for (ShardId shardId : shardsBatch.asyncBatch.shardAttributesMap.keySet()) { + shardToIgnoreNodes.put(shardId, allocation.getIgnoreNodes(shardId)); + } + AsyncShardBatchFetch asyncFetcher = shardsBatch.getAsyncFetcher(); + AsyncShardFetch.FetchResult fetchResult = asyncFetcher.fetchData( + allocation.nodes(), + shardToIgnoreNodes + ); + if (fetchResult.hasData()) { + fetchResult.processAllocation(allocation); + } + + return fetchResult; + } + /** * Holds information about a batch of shards to be allocated. * Async fetcher is used to fetch the data for the batch. From 1afa3f4cf66ad8623bbd9ffb4af74a0565127577 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Mon, 22 Apr 2024 21:16:39 +0530 Subject: [PATCH 37/45] Keep initializing shards in batch Signed-off-by: Shivansh Arora --- .../org/opensearch/cluster/routing/RoutingNode.java | 4 ++++ .../gateway/ShardsBatchGatewayAllocator.java | 10 ++++++++++ 2 files changed, 14 insertions(+) diff --git a/server/src/main/java/org/opensearch/cluster/routing/RoutingNode.java b/server/src/main/java/org/opensearch/cluster/routing/RoutingNode.java index 413ddff72f7a5..6914709391f3c 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RoutingNode.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RoutingNode.java @@ -202,6 +202,10 @@ public int size() { return shards.size(); } + public Collection getInitializingShards() { + return initializingShards; + } + /** * Add a new shard to this node * @param shard Shard to create on this Node diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 7683ab6ed75dc..54a65f52321d6 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -251,6 +251,16 @@ else if (shardRouting.primary() == primary) { } }); + allocation.routingNodes().forEach( + routingNode -> + routingNode.getInitializingShards().forEach( + shardRouting -> { + if (currentBatchedShards.containsKey(shardRouting.shardId()) && shardRouting.primary() == primary) { + batchedShardsToAssign.add(shardRouting.shardId()); + } + }) + ); + refreshShardBatches(currentBatches, batchedShardsToAssign, primary); Iterator iterator = newShardsToBatch.iterator(); From 3b5751ec8d040673b4778f8fe6a91ba708703e45 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Tue, 23 Apr 2024 10:38:12 +0530 Subject: [PATCH 38/45] Spotless apply Signed-off-by: Shivansh Arora --- .../gateway/ShardsBatchGatewayAllocator.java | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 54a65f52321d6..cab815a65761f 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -251,15 +251,11 @@ else if (shardRouting.primary() == primary) { } }); - allocation.routingNodes().forEach( - routingNode -> - routingNode.getInitializingShards().forEach( - shardRouting -> { - if (currentBatchedShards.containsKey(shardRouting.shardId()) && shardRouting.primary() == primary) { - batchedShardsToAssign.add(shardRouting.shardId()); - } - }) - ); + allocation.routingNodes().forEach(routingNode -> routingNode.getInitializingShards().forEach(shardRouting -> { + if (currentBatchedShards.containsKey(shardRouting.shardId()) && shardRouting.primary() == primary) { + batchedShardsToAssign.add(shardRouting.shardId()); + } + })); refreshShardBatches(currentBatches, batchedShardsToAssign, primary); From 8ae90fc2e18e665097c23b904a1e2b737b516abd Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Tue, 23 Apr 2024 15:17:15 +0530 Subject: [PATCH 39/45] Update ensureAsyncFetchStorePrimaryRecency and refresh batch flow Signed-off-by: Shivansh Arora --- .../gateway/AsyncShardBatchFetch.java | 261 ++++++++++ .../opensearch/gateway/AsyncShardFetch.java | 460 +++++++----------- .../gateway/AsyncShardFetchCache.java | 319 ++++++++++++ .../gateway/ReplicaShardBatchAllocator.java | 188 +++++++ .../gateway/ShardsBatchGatewayAllocator.java | 31 +- 5 files changed, 968 insertions(+), 291 deletions(-) create mode 100644 server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java create mode 100644 server/src/main/java/org/opensearch/gateway/AsyncShardFetchCache.java create mode 100644 server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java new file mode 100644 index 0000000000000..9f4cee99465a5 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java @@ -0,0 +1,261 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway; + +import org.apache.logging.log4j.Logger; +import org.opensearch.action.support.nodes.BaseNodeResponse; +import org.opensearch.action.support.nodes.BaseNodesResponse; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.logging.Loggers; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.indices.store.ShardAttributes; + +import java.lang.reflect.Array; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.function.Predicate; + +import reactor.util.annotation.NonNull; + +/** + * Implementation of AsyncShardFetch with batching support. This class is responsible for executing the fetch + * part using the base class {@link AsyncShardFetch}. Other functionalities needed for a batch are only written here. + * This separation also takes care of the extra generic type V which is only needed for batch + * transport actions like {@link TransportNodesListGatewayStartedShardsBatch} and + * {@link org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch}. + * + * @param Response type of the transport action. + * @param Data type of shard level response. + * + * @opensearch.internal + */ +public abstract class AsyncShardBatchFetch extends AsyncShardFetch { + + @SuppressWarnings("unchecked") + AsyncShardBatchFetch( + Logger logger, + String type, + Map shardAttributesMap, + AsyncShardFetch.Lister, T> action, + String batchId, + Class clazz, + V emptyShardResponse, + Predicate emptyShardResponsePredicate, + ShardBatchResponseFactory responseFactory + ) { + super( + logger, + type, + shardAttributesMap, + action, + batchId, + new ShardBatchCache<>( + logger, + type, + shardAttributesMap, + "BatchID=[" + batchId + "]", + clazz, + emptyShardResponse, + emptyShardResponsePredicate, + responseFactory + ) + ); + } + + /** + * Remove a shard from the cache maintaining a full batch of shards. This is needed to clear the shard once it's + * assigned or failed. + * + * @param shardId shardId to be removed from the batch. + */ + public synchronized void clearShard(ShardId shardId) { + this.shardAttributesMap.remove(shardId); + this.cache.deleteShard(shardId); + } + + /** + * Clear the cache for a given node and shardId. + * + * @param nodeId node id to be removed from the batch. + * @param shardId shard id to be removed from the batch. + */ + public synchronized void clearCache(String nodeId, ShardId shardId) { + this.cache.cleanCacheForNodeForShardId(nodeId, shardId); + } + + /** + * Cache implementation of transport actions returning batch of shards related data in the response. + * Store node level responses of transport actions like {@link TransportNodesListGatewayStartedShardsBatch} or + * {@link org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch} with memory efficient caching + * approach. This cache class is not thread safe, all of its methods are being called from + * {@link AsyncShardFetch} class which has synchronized blocks present to handle multiple threads. + * + * @param Response type of transport action. + * @param Data type of shard level response. + */ + static class ShardBatchCache extends AsyncShardFetchCache { + private final Map> cache; + private final Map shardIdToArray; + private final int batchSize; + private final Class shardResponseClass; + private final ShardBatchResponseFactory responseFactory; + private final V emptyResponse; + private final Predicate emptyShardResponsePredicate; + private final Logger logger; + + public ShardBatchCache( + Logger logger, + String type, + Map shardAttributesMap, + String logKey, + Class clazz, + V emptyResponse, + Predicate emptyShardResponsePredicate, + ShardBatchResponseFactory responseFactory + ) { + super(Loggers.getLogger(logger, "_" + logKey), type); + this.batchSize = shardAttributesMap.size(); + this.emptyShardResponsePredicate = emptyShardResponsePredicate; + cache = new HashMap<>(); + shardIdToArray = new HashMap<>(); + fillShardIdKeys(shardAttributesMap.keySet()); + this.shardResponseClass = clazz; + this.emptyResponse = emptyResponse; + this.logger = logger; + this.responseFactory = responseFactory; + } + + @Override + @NonNull + public Map getCache() { + return cache; + } + + @Override + public void deleteShard(ShardId shardId) { + if (shardIdToArray.containsKey(shardId)) { + Integer shardIdIndex = shardIdToArray.remove(shardId); + for (String nodeId : cache.keySet()) { + cache.get(nodeId).clearShard(shardIdIndex); + } + } + } + + @Override + public void cleanCacheForNodeForShardId(String nodeId, ShardId shardId) { + if (shardIdToArray.containsKey(shardId)) { + Integer shardIdIndex = shardIdToArray.remove(shardId); + cache.get(nodeId).clearShard(shardIdIndex); + } + } + + @Override + public void initData(DiscoveryNode node) { + cache.put(node.getId(), new NodeEntry<>(node.getId(), shardResponseClass, batchSize, emptyShardResponsePredicate)); + } + + /** + * Put the response received from data nodes into the cache. + * Get shard level data from batch, then filter out if any shards received failures. + * After that complete storing the data at node level and mark fetching as done. + * + * @param node node from which we got the response. + * @param response shard metadata coming from node. + */ + @Override + public void putData(DiscoveryNode node, T response) { + NodeEntry nodeEntry = cache.get(node.getId()); + Map batchResponse = responseFactory.getShardBatchData(response); + nodeEntry.doneFetching(batchResponse, shardIdToArray); + } + + @Override + public T getData(DiscoveryNode node) { + return this.responseFactory.getNewResponse(node, getBatchData(cache.get(node.getId()))); + } + + private HashMap getBatchData(NodeEntry nodeEntry) { + V[] nodeShardEntries = nodeEntry.getData(); + boolean[] emptyResponses = nodeEntry.getEmptyShardResponse(); + HashMap shardData = new HashMap<>(); + for (Map.Entry shardIdEntry : shardIdToArray.entrySet()) { + ShardId shardId = shardIdEntry.getKey(); + Integer arrIndex = shardIdEntry.getValue(); + if (emptyResponses[arrIndex]) { + shardData.put(shardId, emptyResponse); + } else if (nodeShardEntries[arrIndex] != null) { + // ignore null responses here + shardData.put(shardId, nodeShardEntries[arrIndex]); + } + } + return shardData; + } + + private void fillShardIdKeys(Set shardIds) { + int shardIdIndex = 0; + for (ShardId shardId : shardIds) { + this.shardIdToArray.putIfAbsent(shardId, shardIdIndex++); + } + } + + /** + * A node entry, holding the state of the fetched data for a specific shard + * for a giving node. + */ + static class NodeEntry extends BaseNodeEntry { + private final V[] shardData; + private final boolean[] emptyShardResponse; // we can not rely on null entries of the shardData array, + // those null entries means that we need to ignore those entries. Empty responses on the other hand are + // actually needed in allocation/explain API response. So instead of storing full empty response object + // in cache, it's better to just store a boolean and create that object on the fly just before + // decision-making. + private final Predicate emptyShardResponsePredicate; + + NodeEntry(String nodeId, Class clazz, int batchSize, Predicate emptyShardResponsePredicate) { + super(nodeId); + this.shardData = (V[]) Array.newInstance(clazz, batchSize); + this.emptyShardResponse = new boolean[batchSize]; + this.emptyShardResponsePredicate = emptyShardResponsePredicate; + } + + void doneFetching(Map shardDataFromNode, Map shardIdKey) { + fillShardData(shardDataFromNode, shardIdKey); + super.doneFetching(); + } + + void clearShard(Integer shardIdIndex) { + this.shardData[shardIdIndex] = null; + emptyShardResponse[shardIdIndex] = false; + } + + V[] getData() { + return this.shardData; + } + + boolean[] getEmptyShardResponse() { + return emptyShardResponse; + } + + private void fillShardData(Map shardDataFromNode, Map shardIdKey) { + for (Map.Entry shardData : shardDataFromNode.entrySet()) { + if (shardData.getValue() != null) { + ShardId shardId = shardData.getKey(); + if (emptyShardResponsePredicate.test(shardData.getValue())) { + this.emptyShardResponse[shardIdKey.get(shardId)] = true; + this.shardData[shardIdKey.get(shardId)] = null; + } else { + this.shardData[shardIdKey.get(shardId)] = shardData.getValue(); + } + } + } + } + } + } +} diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java index b5c4d00228caa..5b9571a73bba9 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java @@ -31,12 +31,7 @@ package org.opensearch.gateway; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.opensearch.OpenSearchTimeoutException; -import org.opensearch.ExceptionsHelper; -import org.opensearch.action.ActionListener; import org.opensearch.action.FailedNodeException; import org.opensearch.action.support.nodes.BaseNodeResponse; import org.opensearch.action.support.nodes.BaseNodesResponse; @@ -45,30 +40,31 @@ import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.common.Nullable; import org.opensearch.common.lease.Releasable; -import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; -import org.opensearch.index.shard.ShardId; -import org.opensearch.transport.ReceiveTimeoutTransportException; +import org.opensearch.common.logging.Loggers; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.indices.store.ShardAttributes; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; -import static java.util.Collections.emptySet; -import static java.util.Collections.unmodifiableSet; +import reactor.util.annotation.NonNull; + +import static java.util.Collections.emptyMap; +import static java.util.Collections.unmodifiableMap; /** * Allows to asynchronously fetch shard related data from other nodes for allocation, without blocking * the cluster update thread. *

- * The async fetch logic maintains a map of which nodes are being fetched from in an async manner, - * and once the results are back, it makes sure to schedule a reroute to make sure those results will - * be taken into account. + * The async fetch logic maintains a cache {@link AsyncShardFetchCache} which is filled in async manner when nodes respond back. + * It also schedules a reroute to make sure those results will be taken into account. * * @opensearch.internal */ @@ -78,18 +74,19 @@ public abstract class AsyncShardFetch implements Rel * An action that lists the relevant shard data that needs to be fetched. */ public interface Lister, NodeResponse extends BaseNodeResponse> { - void list(ShardId shardId, @Nullable String customDataPath, DiscoveryNode[] nodes, ActionListener listener); + void list(Map shardAttributesMap, DiscoveryNode[] nodes, ActionListener listener); + } protected final Logger logger; protected final String type; - protected final ShardId shardId; - protected final String customDataPath; + protected final Map shardAttributesMap; private final Lister, T> action; - private final Map> cache = new HashMap<>(); - private final Set nodesToIgnore = new HashSet<>(); + protected final AsyncShardFetchCache cache; private final AtomicLong round = new AtomicLong(); private boolean closed; + final String reroutingKey; + private final Map> shardToIgnoreNodes = new HashMap<>(); @SuppressWarnings("unchecked") protected AsyncShardFetch( @@ -101,9 +98,37 @@ protected AsyncShardFetch( ) { this.logger = logger; this.type = type; - this.shardId = Objects.requireNonNull(shardId); - this.customDataPath = Objects.requireNonNull(customDataPath); + shardAttributesMap = new HashMap<>(); + shardAttributesMap.put(shardId, new ShardAttributes(customDataPath)); this.action = (Lister, T>) action; + this.reroutingKey = "ShardId=[" + shardId.toString() + "]"; + cache = new ShardCache<>(logger, reroutingKey, type); + } + + /** + * Added to fetch a batch of shards from nodes + * + * @param logger Logger + * @param type type of action + * @param shardAttributesMap Map of {@link ShardId} to {@link ShardAttributes} to perform fetching on them a + * @param action Transport Action + * @param batchId For the given ShardAttributesMap, we expect them to tie with a single batch id for logging and later identification + */ + @SuppressWarnings("unchecked") + protected AsyncShardFetch( + Logger logger, + String type, + Map shardAttributesMap, + Lister, T> action, + String batchId, + AsyncShardFetchCache cache + ) { + this.logger = logger; + this.type = type; + this.shardAttributesMap = shardAttributesMap; + this.action = (Lister, T>) action; + this.reroutingKey = "BatchID=[" + batchId + "]"; + this.cache = cache; } @Override @@ -111,19 +136,6 @@ public synchronized void close() { this.closed = true; } - /** - * Returns the number of async fetches that are currently ongoing. - */ - public synchronized int getNumberOfInFlightFetches() { - int count = 0; - for (NodeEntry nodeEntry : cache.values()) { - if (nodeEntry.isFetching()) { - count++; - } - } - return count; - } - /** * Fetches the data for the relevant shard. If there any ongoing async fetches going on, or new ones have * been initiated by this call, the result will have no data. @@ -131,63 +143,71 @@ public synchronized int getNumberOfInFlightFetches() { * The ignoreNodes are nodes that are supposed to be ignored for this round, since fetching is async, we need * to keep them around and make sure we add them back when all the responses are fetched and returned. */ - public synchronized FetchResult fetchData(DiscoveryNodes nodes, Set ignoreNodes) { + public synchronized FetchResult fetchData(DiscoveryNodes nodes, Map> ignoreNodes) { if (closed) { - throw new IllegalStateException(shardId + ": can't fetch data on closed async fetch"); + throw new IllegalStateException(reroutingKey + ": can't fetch data on closed async fetch"); } - nodesToIgnore.addAll(ignoreNodes); - fillShardCacheWithDataNodes(cache, nodes); - List> nodesToFetch = findNodesToFetch(cache); - if (nodesToFetch.isEmpty() == false) { + + if (shardAttributesMap.size() == 1) { + // we will do assertions here on ignoreNodes + if (ignoreNodes.size() > 1) { + throw new IllegalStateException( + "Fetching Shard Data, " + reroutingKey + "Can only have atmost one shard" + "for non-batch mode" + ); + } + if (ignoreNodes.size() == 1) { + if (shardAttributesMap.containsKey(ignoreNodes.keySet().iterator().next()) == false) { + throw new IllegalStateException("Shard Id must be same as initialized in AsyncShardFetch. Expecting = " + reroutingKey); + } + } + } + + // add the nodes to ignore to the list of nodes to ignore for each shard + for (Map.Entry> ignoreNodesEntry : ignoreNodes.entrySet()) { + Set ignoreNodesSet = shardToIgnoreNodes.getOrDefault(ignoreNodesEntry.getKey(), new HashSet<>()); + ignoreNodesSet.addAll(ignoreNodesEntry.getValue()); + shardToIgnoreNodes.put(ignoreNodesEntry.getKey(), ignoreNodesSet); + } + + cache.fillShardCacheWithDataNodes(nodes); + List nodeIds = cache.findNodesToFetch(); + if (nodeIds.isEmpty() == false) { // mark all node as fetching and go ahead and async fetch them // use a unique round id to detect stale responses in processAsyncFetch final long fetchingRound = round.incrementAndGet(); - for (NodeEntry nodeEntry : nodesToFetch) { - nodeEntry.markAsFetching(fetchingRound); - } - DiscoveryNode[] discoNodesToFetch = nodesToFetch.stream() - .map(NodeEntry::getNodeId) - .map(nodes::get) - .toArray(DiscoveryNode[]::new); + cache.markAsFetching(nodeIds, fetchingRound); + DiscoveryNode[] discoNodesToFetch = nodeIds.stream().map(nodes::get).toArray(DiscoveryNode[]::new); asyncFetch(discoNodesToFetch, fetchingRound); } // if we are still fetching, return null to indicate it - if (hasAnyNodeFetching(cache)) { - return new FetchResult<>(shardId, null, emptySet()); + if (cache.hasAnyNodeFetching()) { + return new FetchResult<>(null, emptyMap()); } else { // nothing to fetch, yay, build the return value - Map fetchData = new HashMap<>(); Set failedNodes = new HashSet<>(); - for (Iterator>> it = cache.entrySet().iterator(); it.hasNext();) { - Map.Entry> entry = it.next(); - String nodeId = entry.getKey(); - NodeEntry nodeEntry = entry.getValue(); - - DiscoveryNode node = nodes.get(nodeId); - if (node != null) { - if (nodeEntry.isFailed()) { - // if its failed, remove it from the list of nodes, so if this run doesn't work - // we try again next round to fetch it again - it.remove(); - failedNodes.add(nodeEntry.getNodeId()); - } else { - if (nodeEntry.getValue() != null) { - fetchData.put(node, nodeEntry.getValue()); - } - } - } - } - Set allIgnoreNodes = unmodifiableSet(new HashSet<>(nodesToIgnore)); + Map fetchData = cache.getCacheData(nodes, failedNodes); + + Map> allIgnoreNodesMap = unmodifiableMap(new HashMap<>(shardToIgnoreNodes)); // clear the nodes to ignore, we had a successful run in fetching everything we can // we need to try them if another full run is needed - nodesToIgnore.clear(); + shardToIgnoreNodes.clear(); // if at least one node failed, make sure to have a protective reroute // here, just case this round won't find anything, and we need to retry fetching data - if (failedNodes.isEmpty() == false || allIgnoreNodes.isEmpty() == false) { - reroute(shardId, "nodes failed [" + failedNodes.size() + "], ignored [" + allIgnoreNodes.size() + "]"); + + if (failedNodes.isEmpty() == false + || allIgnoreNodesMap.values().stream().anyMatch(ignoreNodeSet -> ignoreNodeSet.isEmpty() == false)) { + reroute( + reroutingKey, + "nodes failed [" + + failedNodes.size() + + "], ignored [" + + allIgnoreNodesMap.values().stream().mapToInt(Set::size).sum() + + "]" + ); } - return new FetchResult<>(shardId, fetchData, allIgnoreNodes); + + return new FetchResult<>(fetchData, allIgnoreNodesMap); } } @@ -200,87 +220,28 @@ public synchronized FetchResult fetchData(DiscoveryNodes nodes, Set i protected synchronized void processAsyncFetch(List responses, List failures, long fetchingRound) { if (closed) { // we are closed, no need to process this async fetch at all - logger.trace("{} ignoring fetched [{}] results, already closed", shardId, type); + logger.trace("{} ignoring fetched [{}] results, already closed", reroutingKey, type); return; } - logger.trace("{} processing fetched [{}] results", shardId, type); + logger.trace("{} processing fetched [{}] results", reroutingKey, type); if (responses != null) { - for (T response : responses) { - NodeEntry nodeEntry = cache.get(response.getNode().getId()); - if (nodeEntry != null) { - if (nodeEntry.getFetchingRound() != fetchingRound) { - assert nodeEntry.getFetchingRound() > fetchingRound : "node entries only replaced by newer rounds"; - logger.trace( - "{} received response for [{}] from node {} for an older fetching round (expected: {} but was: {})", - shardId, - nodeEntry.getNodeId(), - type, - nodeEntry.getFetchingRound(), - fetchingRound - ); - } else if (nodeEntry.isFailed()) { - logger.trace( - "{} node {} has failed for [{}] (failure [{}])", - shardId, - nodeEntry.getNodeId(), - type, - nodeEntry.getFailure() - ); - } else { - // if the entry is there, for the right fetching round and not marked as failed already, process it - logger.trace("{} marking {} as done for [{}], result is [{}]", shardId, nodeEntry.getNodeId(), type, response); - nodeEntry.doneFetching(response); - } - } - } + cache.processResponses(responses, fetchingRound); } if (failures != null) { - for (FailedNodeException failure : failures) { - logger.trace("{} processing failure {} for [{}]", shardId, failure, type); - NodeEntry nodeEntry = cache.get(failure.nodeId()); - if (nodeEntry != null) { - if (nodeEntry.getFetchingRound() != fetchingRound) { - assert nodeEntry.getFetchingRound() > fetchingRound : "node entries only replaced by newer rounds"; - logger.trace( - "{} received failure for [{}] from node {} for an older fetching round (expected: {} but was: {})", - shardId, - nodeEntry.getNodeId(), - type, - nodeEntry.getFetchingRound(), - fetchingRound - ); - } else if (nodeEntry.isFailed() == false) { - // if the entry is there, for the right fetching round and not marked as failed already, process it - Throwable unwrappedCause = ExceptionsHelper.unwrapCause(failure.getCause()); - // if the request got rejected or timed out, we need to try it again next time... - if (unwrappedCause instanceof OpenSearchRejectedExecutionException - || unwrappedCause instanceof ReceiveTimeoutTransportException - || unwrappedCause instanceof OpenSearchTimeoutException) { - nodeEntry.restartFetching(); - } else { - logger.warn( - () -> new ParameterizedMessage( - "{}: failed to list shard for {} on node [{}]", - shardId, - type, - failure.nodeId() - ), - failure - ); - nodeEntry.doneFetching(failure.getCause()); - } - } - } - } + cache.processFailures(failures, fetchingRound); } - reroute(shardId, "post_response"); + reroute(reroutingKey, "post_response"); + } + + public synchronized int getNumberOfInFlightFetches() { + return cache.getInflightFetches(); } /** * Implement this in order to scheduled another round that causes a call to fetch data. */ - protected abstract void reroute(ShardId shardId, String reason); + protected abstract void reroute(String reroutingKey, String reason); /** * Clear cache for node, ensuring next fetch will fetch a fresh copy. @@ -289,55 +250,13 @@ synchronized void clearCacheForNode(String nodeId) { cache.remove(nodeId); } - /** - * Fills the shard fetched data with new (data) nodes and a fresh NodeEntry, and removes from - * it nodes that are no longer part of the state. - */ - private void fillShardCacheWithDataNodes(Map> shardCache, DiscoveryNodes nodes) { - // verify that all current data nodes are there - for (ObjectObjectCursor cursor : nodes.getDataNodes()) { - DiscoveryNode node = cursor.value; - if (shardCache.containsKey(node.getId()) == false) { - shardCache.put(node.getId(), new NodeEntry(node.getId())); - } - } - // remove nodes that are not longer part of the data nodes set - shardCache.keySet().removeIf(nodeId -> !nodes.nodeExists(nodeId)); - } - - /** - * Finds all the nodes that need to be fetched. Those are nodes that have no - * data, and are not in fetch mode. - */ - private List> findNodesToFetch(Map> shardCache) { - List> nodesToFetch = new ArrayList<>(); - for (NodeEntry nodeEntry : shardCache.values()) { - if (nodeEntry.hasData() == false && nodeEntry.isFetching() == false) { - nodesToFetch.add(nodeEntry); - } - } - return nodesToFetch; - } - - /** - * Are there any nodes that are fetching data? - */ - private boolean hasAnyNodeFetching(Map> shardCache) { - for (NodeEntry nodeEntry : shardCache.values()) { - if (nodeEntry.isFetching()) { - return true; - } - } - return false; - } - /** * Async fetches data for the provided shard with the set of nodes that need to be fetched from. */ // visible for testing void asyncFetch(final DiscoveryNode[] nodes, long fetchingRound) { - logger.trace("{} fetching [{}] from {}", shardId, type, nodes); - action.list(shardId, customDataPath, nodes, new ActionListener>() { + logger.trace("{} fetching [{}] from {}", reroutingKey, type, nodes); + action.list(shardAttributesMap, nodes, new ActionListener>() { @Override public void onResponse(BaseNodesResponse response) { processAsyncFetch(response.getNodes(), response.failures(), fetchingRound); @@ -354,20 +273,89 @@ public void onFailure(Exception e) { }); } + /** + * Cache implementation of transport actions returning single shard related data in the response. + * Store node level responses of transport actions like {@link TransportNodesListGatewayStartedShards} or + * {@link TransportNodesListShardStoreMetadata}. + * + * @param Response type of transport action. + */ + static class ShardCache extends AsyncShardFetchCache { + + private final Map> cache; + + public ShardCache(Logger logger, String logKey, String type) { + super(Loggers.getLogger(logger, "_" + logKey), type); + cache = new HashMap<>(); + } + + @Override + public void initData(DiscoveryNode node) { + cache.put(node.getId(), new NodeEntry<>(node.getId())); + } + + @Override + public void putData(DiscoveryNode node, K response) { + cache.get(node.getId()).doneFetching(response); + } + + @Override + public K getData(DiscoveryNode node) { + return cache.get(node.getId()).getValue(); + } + + @NonNull + @Override + public Map getCache() { + return cache; + } + + @Override + public void deleteShard(ShardId shardId) { + cache.clear(); // single shard cache can clear the full map + } + + @Override + public void cleanCacheForNodeForShardId(String nodeId, ShardId shardId) { + cache.remove(nodeId); // non batch cache only has one entry per node + } + + /** + * A node entry, holding the state of the fetched data for a specific shard + * for a giving node. + */ + static class NodeEntry extends AsyncShardFetchCache.BaseNodeEntry { + @Nullable + private U value; + + void doneFetching(U value) { + super.doneFetching(); + this.value = value; + } + + NodeEntry(String nodeId) { + super(nodeId); + } + + U getValue() { + return value; + } + + } + } + /** * The result of a fetch operation. Make sure to first check {@link #hasData()} before * fetching the actual data. */ public static class FetchResult { - private final ShardId shardId; private final Map data; - private final Set ignoreNodes; + private final Map> ignoredShardToNodes; - public FetchResult(ShardId shardId, Map data, Set ignoreNodes) { - this.shardId = shardId; + public FetchResult(Map data, Map> ignoreNodes) { this.data = data; - this.ignoreNodes = ignoreNodes; + this.ignoredShardToNodes = ignoreNodes; } /** @@ -391,88 +379,14 @@ public Map getData() { * Process any changes needed to the allocation based on this fetch result. */ public void processAllocation(RoutingAllocation allocation) { - for (String ignoreNode : ignoreNodes) { - allocation.addIgnoreShardForNode(shardId, ignoreNode); + for (Map.Entry> entry : ignoredShardToNodes.entrySet()) { + ShardId shardId = entry.getKey(); + Set ignoreNodes = entry.getValue(); + if (ignoreNodes.isEmpty() == false) { + ignoreNodes.forEach(nodeId -> allocation.addIgnoreShardForNode(shardId, nodeId)); + } } - } - } - - /** - * A node entry, holding the state of the fetched data for a specific shard - * for a giving node. - */ - static class NodeEntry { - private final String nodeId; - private boolean fetching; - @Nullable - private T value; - private boolean valueSet; - private Throwable failure; - private long fetchingRound; - - NodeEntry(String nodeId) { - this.nodeId = nodeId; - } - - String getNodeId() { - return this.nodeId; - } - - boolean isFetching() { - return fetching; - } - - void markAsFetching(long fetchingRound) { - assert fetching == false : "double marking a node as fetching"; - this.fetching = true; - this.fetchingRound = fetchingRound; - } - - void doneFetching(T value) { - assert fetching : "setting value but not in fetching mode"; - assert failure == null : "setting value when failure already set"; - this.valueSet = true; - this.value = value; - this.fetching = false; - } - - void doneFetching(Throwable failure) { - assert fetching : "setting value but not in fetching mode"; - assert valueSet == false : "setting failure when already set value"; - assert failure != null : "setting failure can't be null"; - this.failure = failure; - this.fetching = false; - } - - void restartFetching() { - assert fetching : "restarting fetching, but not in fetching mode"; - assert valueSet == false : "value can't be set when restarting fetching"; - assert failure == null : "failure can't be set when restarting fetching"; - this.fetching = false; - } - - boolean isFailed() { - return failure != null; - } - - boolean hasData() { - return valueSet || failure != null; - } - - Throwable getFailure() { - assert hasData() : "getting failure when data has not been fetched"; - return failure; - } - - @Nullable - T getValue() { - assert failure == null : "trying to fetch value, but its marked as failed, check isFailed"; - assert valueSet : "value is not set, hasn't been fetched yet"; - return value; - } - long getFetchingRound() { - return fetchingRound; } } } diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardFetchCache.java b/server/src/main/java/org/opensearch/gateway/AsyncShardFetchCache.java new file mode 100644 index 0000000000000..8e37137d4aa24 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardFetchCache.java @@ -0,0 +1,319 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway; + +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.opensearch.ExceptionsHelper; +import org.opensearch.OpenSearchTimeoutException; +import org.opensearch.action.FailedNodeException; +import org.opensearch.action.support.nodes.BaseNodeResponse; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.transport.ReceiveTimeoutTransportException; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import reactor.util.annotation.NonNull; + +/** + * AsyncShardFetchCache will operate on the node level cache which is map of String and BaseNodeEntry. initData, + * putData and getData needs to be called for all the nodes. This class is responsible for managing the flow for all + * the nodes. + * It'll also give useful insights like how many ongoing fetches are happening, how many nodes are left for fetch or + * mark some node in fetching mode. All of these functionalities require checking the cache information and respond + * accordingly. + *

+ * initData : how to initialize an entry of shard cache for a node. + * putData : how to store the response of transport action in the cache. + * getData : how to get the stored data for any shard allocators like {@link PrimaryShardAllocator} or + * {@link ReplicaShardAllocator} + * deleteShard : how to clean up the stored data from cache for a shard. + * + * @param Response type of transport action which has the data to be stored in the cache. + * + * @opensearch.internal + */ +public abstract class AsyncShardFetchCache { + + private final Logger logger; + private final String type; + + protected AsyncShardFetchCache(Logger logger, String type) { + this.logger = logger; + this.type = type; + } + + abstract void initData(DiscoveryNode node); + + abstract void putData(DiscoveryNode node, K response); + + abstract K getData(DiscoveryNode node); + + @NonNull + abstract Map getCache(); + + /** + * Cleanup cached data for this shard once it's started. Cleanup only happens at shard level. Node entries will + * automatically be cleaned up once shards are assigned. + * + * @param shardId for which we need to free up the cached data. + */ + abstract void deleteShard(ShardId shardId); + + abstract void cleanCacheForNodeForShardId(String nodeId, ShardId shardId); + + /** + * Returns the number of fetches that are currently ongoing. + */ + int getInflightFetches() { + int count = 0; + for (BaseNodeEntry nodeEntry : getCache().values()) { + if (nodeEntry.isFetching()) { + count++; + } + } + return count; + } + + /** + * Fills the shard fetched data with new (data) nodes and a fresh NodeEntry, and removes from + * it nodes that are no longer part of the state. + */ + void fillShardCacheWithDataNodes(DiscoveryNodes nodes) { + // verify that all current data nodes are there + for (final DiscoveryNode node : nodes.getDataNodes().values()) { + if (getCache().containsKey(node.getId()) == false) { + initData(node); + } + } + // remove nodes that are not longer part of the data nodes set + getCache().keySet().removeIf(nodeId -> !nodes.nodeExists(nodeId)); + } + + /** + * Finds all the nodes that need to be fetched. Those are nodes that have no + * data, and are not in fetch mode. + */ + List findNodesToFetch() { + List nodesToFetch = new ArrayList<>(); + for (BaseNodeEntry nodeEntry : getCache().values()) { + if (nodeEntry.hasData() == false && nodeEntry.isFetching() == false) { + nodesToFetch.add(nodeEntry.getNodeId()); + } + } + return nodesToFetch; + } + + /** + * Are there any nodes that are fetching data? + */ + boolean hasAnyNodeFetching() { + for (BaseNodeEntry nodeEntry : getCache().values()) { + if (nodeEntry.isFetching()) { + return true; + } + } + return false; + } + + /** + * Get the data from cache, ignore the failed entries. Use getData functional interface to get the data, as + * different implementations may have different ways to populate the data from cache. + * + * @param nodes Discovery nodes for which we need to return the cache data. + * @param failedNodes return failedNodes with the nodes where fetch has failed. + * @return Map of cache data for every DiscoveryNode. + */ + Map getCacheData(DiscoveryNodes nodes, Set failedNodes) { + Map fetchData = new HashMap<>(); + for (Iterator> it = getCache().entrySet().iterator(); it.hasNext();) { + Map.Entry entry = (Map.Entry) it.next(); + String nodeId = entry.getKey(); + BaseNodeEntry nodeEntry = entry.getValue(); + + DiscoveryNode node = nodes.get(nodeId); + if (node != null) { + if (nodeEntry.isFailed()) { + // if its failed, remove it from the list of nodes, so if this run doesn't work + // we try again next round to fetch it again + it.remove(); + failedNodes.add(nodeEntry.getNodeId()); + } else { + K nodeResponse = getData(node); + if (nodeResponse != null) { + fetchData.put(node, nodeResponse); + } + } + } + } + return fetchData; + } + + void processResponses(List responses, long fetchingRound) { + for (K response : responses) { + BaseNodeEntry nodeEntry = getCache().get(response.getNode().getId()); + if (nodeEntry != null) { + if (validateNodeResponse(nodeEntry, fetchingRound)) { + // if the entry is there, for the right fetching round and not marked as failed already, process it + logger.trace("marking {} as done for [{}], result is [{}]", nodeEntry.getNodeId(), type, response); + putData(response.getNode(), response); + } + } + } + } + + private boolean validateNodeResponse(BaseNodeEntry nodeEntry, long fetchingRound) { + if (nodeEntry.getFetchingRound() != fetchingRound) { + assert nodeEntry.getFetchingRound() > fetchingRound : "node entries only replaced by newer rounds"; + logger.trace( + "received response for [{}] from node {} for an older fetching round (expected: {} but was: {})", + nodeEntry.getNodeId(), + type, + nodeEntry.getFetchingRound(), + fetchingRound + ); + return false; + } else if (nodeEntry.isFailed()) { + logger.trace("node {} has failed for [{}] (failure [{}])", nodeEntry.getNodeId(), type, nodeEntry.getFailure()); + return false; + } + return true; + } + + private void handleNodeFailure(BaseNodeEntry nodeEntry, FailedNodeException failure, long fetchingRound) { + if (nodeEntry.getFetchingRound() != fetchingRound) { + assert nodeEntry.getFetchingRound() > fetchingRound : "node entries only replaced by newer rounds"; + logger.trace( + "received failure for [{}] from node {} for an older fetching round (expected: {} but was: {})", + nodeEntry.getNodeId(), + type, + nodeEntry.getFetchingRound(), + fetchingRound + ); + } else if (nodeEntry.isFailed() == false) { + // if the entry is there, for the right fetching round and not marked as failed already, process it + Throwable unwrappedCause = ExceptionsHelper.unwrapCause(failure.getCause()); + // if the request got rejected or timed out, we need to try it again next time... + if (retryableException(unwrappedCause)) { + nodeEntry.restartFetching(); + } else { + logger.warn(() -> new ParameterizedMessage("failed to list shard for {} on node [{}]", type, failure.nodeId()), failure); + nodeEntry.doneFetching(failure.getCause()); + } + } + } + + boolean retryableException(Throwable unwrappedCause) { + return unwrappedCause instanceof OpenSearchRejectedExecutionException + || unwrappedCause instanceof ReceiveTimeoutTransportException + || unwrappedCause instanceof OpenSearchTimeoutException; + } + + void processFailures(List failures, long fetchingRound) { + for (FailedNodeException failure : failures) { + logger.trace("processing failure {} for [{}]", failure, type); + BaseNodeEntry nodeEntry = getCache().get(failure.nodeId()); + if (nodeEntry != null) { + handleNodeFailure(nodeEntry, failure, fetchingRound); + } + } + } + + /** + * Common function for removing whole node entry. + * + * @param nodeId nodeId to be cleaned. + */ + void remove(String nodeId) { + this.getCache().remove(nodeId); + } + + void markAsFetching(List nodeIds, long fetchingRound) { + for (String nodeId : nodeIds) { + getCache().get(nodeId).markAsFetching(fetchingRound); + } + } + + /** + * A node entry, holding only node level fetching related information. + * Actual metadata of shard is stored in child classes. + */ + static class BaseNodeEntry { + private final String nodeId; + private boolean fetching; + private boolean valueSet; + private Throwable failure; + private long fetchingRound; + + BaseNodeEntry(String nodeId) { + this.nodeId = nodeId; + } + + String getNodeId() { + return this.nodeId; + } + + boolean isFetching() { + return fetching; + } + + void markAsFetching(long fetchingRound) { + assert fetching == false : "double marking a node as fetching"; + this.fetching = true; + this.fetchingRound = fetchingRound; + } + + void doneFetching() { + assert fetching : "setting value but not in fetching mode"; + assert failure == null : "setting value when failure already set"; + this.valueSet = true; + this.fetching = false; + } + + void doneFetching(Throwable failure) { + assert fetching : "setting value but not in fetching mode"; + assert valueSet == false : "setting failure when already set value"; + assert failure != null : "setting failure can't be null"; + this.failure = failure; + this.fetching = false; + } + + void restartFetching() { + assert fetching : "restarting fetching, but not in fetching mode"; + assert valueSet == false : "value can't be set when restarting fetching"; + assert failure == null : "failure can't be set when restarting fetching"; + this.fetching = false; + } + + boolean isFailed() { + return failure != null; + } + + boolean hasData() { + return valueSet || failure != null; + } + + Throwable getFailure() { + assert hasData() : "getting failure when data has not been fetched"; + return failure; + } + + long getFetchingRound() { + return fetchingRound; + } + } +} diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java new file mode 100644 index 0000000000000..be7867b7823f6 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java @@ -0,0 +1,188 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway; + +import org.apache.logging.log4j.Logger; +import org.opensearch.cluster.node.DiscoveryNode; +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.NodeAllocationResult; +import org.opensearch.cluster.routing.allocation.RoutingAllocation; +import org.opensearch.cluster.routing.allocation.decider.Decision; +import org.opensearch.common.collect.Tuple; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.gateway.AsyncShardFetch.FetchResult; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Allocates replica shards in a batch mode + * + * @opensearch.internal + */ +public abstract class ReplicaShardBatchAllocator extends ReplicaShardAllocator { + + /** + * Process existing recoveries of replicas and see if we need to cancel them if we find a better + * match. Today, a better match is one that can perform a no-op recovery while the previous recovery + * has to copy segment files. + * + * @param allocation the overall routing allocation + * @param shardBatches a list of shard batches to check for existing recoveries + */ + public void processExistingRecoveries(RoutingAllocation allocation, List> shardBatches) { + List shardCancellationActions = new ArrayList<>(); + // iterate through the batches, each batch needs to be processed together as fetch call should be made for shards from same batch + for (List shardBatch : shardBatches) { + List eligibleShards = new ArrayList<>(); + List ineligibleShards = new ArrayList<>(); + // iterate over shards to check for match for each of those + for (ShardRouting shard : shardBatch) { + if (shard != null && !shard.primary()) { + // need to iterate over all the nodes to find matching shard + if (shouldSkipFetchForRecovery(shard)) { + // shard should just be skipped for fetchData, no need to remove from batch + continue; + } + eligibleShards.add(shard); + } + } + AsyncShardFetch.FetchResult shardState = fetchData(eligibleShards, ineligibleShards, allocation); + if (!shardState.hasData()) { + logger.trace("{}: fetching new stores for initializing shard batch", eligibleShards); + continue; // still fetching + } + for (ShardRouting shard : eligibleShards) { + Map nodeShardStores = convertToNodeStoreFilesMetadataMap(shard, shardState); + + Runnable cancellationAction = cancelExistingRecoveryForBetterMatch(shard, allocation, nodeShardStores); + if (cancellationAction != null) { + shardCancellationActions.add(cancellationAction); + } + } + } + for (Runnable action : shardCancellationActions) { + action.run(); + } + } + + abstract protected FetchResult fetchData( + List eligibleShards, + List ineligibleShards, + RoutingAllocation allocation + ); + + @Override + protected FetchResult fetchData( + ShardRouting shard, + RoutingAllocation allocation + ) { + logger.error("fetchData for single shard called via batch allocator"); + throw new IllegalStateException("ReplicaShardBatchAllocator should only be used for a batch of shards"); + } + + @Override + public AllocateUnassignedDecision makeAllocationDecision(ShardRouting unassignedShard, RoutingAllocation allocation, Logger logger) { + return makeAllocationDecision(Collections.singletonList(unassignedShard), allocation, logger).get(unassignedShard); + } + + @Override + public HashMap makeAllocationDecision( + List shards, + RoutingAllocation allocation, + Logger logger + ) { + HashMap shardAllocationDecisions = new HashMap<>(); + final boolean explain = allocation.debugDecision(); + List eligibleShards = new ArrayList<>(); + List ineligibleShards = new ArrayList<>(); + HashMap>> nodeAllocationDecisions = new HashMap<>(); + for (ShardRouting shard : shards) { + if (!isResponsibleFor(shard)) { + // this allocator n is not responsible for allocating this shard + ineligibleShards.add(shard); + shardAllocationDecisions.put(shard, AllocateUnassignedDecision.NOT_TAKEN); + continue; + } + + Tuple> result = canBeAllocatedToAtLeastOneNode(shard, allocation); + Decision allocationDecision = result.v1(); + if (allocationDecision.type() != Decision.Type.YES && (!explain || !hasInitiatedFetching(shard))) { + // only return early if we are not in explain mode, or we are in explain mode but we have not + // yet attempted to fetch any shard data + logger.trace("{}: ignoring allocation, can't be allocated on any node", shard); + shardAllocationDecisions.put( + shard, + AllocateUnassignedDecision.no( + UnassignedInfo.AllocationStatus.fromDecision(allocationDecision.type()), + result.v2() != null ? new ArrayList<>(result.v2().values()) : null + ) + ); + continue; + } + // storing the nodeDecisions in nodeAllocationDecisions if the decision is not YES + // so that we don't have to compute the decisions again + nodeAllocationDecisions.put(shard, result); + + eligibleShards.add(shard); + } + + // Do not call fetchData if there are no eligible shards + if (eligibleShards.isEmpty()) { + return shardAllocationDecisions; + } + // only fetch data for eligible shards + final FetchResult shardsState = fetchData(eligibleShards, ineligibleShards, allocation); + + for (ShardRouting unassignedShard : eligibleShards) { + Tuple> result = nodeAllocationDecisions.get(unassignedShard); + shardAllocationDecisions.put( + unassignedShard, + getAllocationDecision( + unassignedShard, + allocation, + convertToNodeStoreFilesMetadataMap(unassignedShard, shardsState), + result, + logger + ) + ); + } + return shardAllocationDecisions; + } + + private Map convertToNodeStoreFilesMetadataMap( + ShardRouting unassignedShard, + FetchResult data + ) { + if (!data.hasData()) { + return null; + } + + Map map = new HashMap<>(); + + data.getData().forEach((discoveryNode, value) -> { + Map batch = value.getNodeStoreFilesMetadataBatch(); + NodeStoreFilesMetadata metadata = batch.get(unassignedShard.shardId()); + if (metadata != null) { + map.put(discoveryNode, metadata.storeFilesMetadata()); + } + }); + + return map; + } +} diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index cab815a65761f..82dd6fa832ca5 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -254,6 +254,9 @@ else if (shardRouting.primary() == primary) { allocation.routingNodes().forEach(routingNode -> routingNode.getInitializingShards().forEach(shardRouting -> { if (currentBatchedShards.containsKey(shardRouting.shardId()) && shardRouting.primary() == primary) { batchedShardsToAssign.add(shardRouting.shardId()); + // Set updated shard routing in batch if it already exists + String batchId = currentBatchedShards.get(shardRouting.shardId()); + currentBatches.get(batchId).batchInfo.get(shardRouting.shardId()).setShardRouting(shardRouting); } })); @@ -410,10 +413,6 @@ private void ensureAsyncFetchStorePrimaryRecency(RoutingAllocation allocation) { Sets.difference(newEphemeralIds, lastSeenEphemeralIds) ) ); - // ToDo : Validate that we don't need below call for batch allocation - // storeShardBatchLookup.values().forEach(batch -> - // clearCacheForBatchPrimary(batchIdToStoreShardBatch.get(batch), allocation) - // ); batchIdToStoreShardBatch.values().forEach(batch -> clearCacheForBatchPrimary(batch, allocation)); // recalc to also (lazily) clear out old nodes. @@ -422,20 +421,16 @@ private void ensureAsyncFetchStorePrimaryRecency(RoutingAllocation allocation) { } private static void clearCacheForBatchPrimary(ShardsBatch batch, RoutingAllocation allocation) { - // We're not running below code because for removing a node from cache we need all replica's primaries - // to be assigned on same node. This was easy in single shard case and we're saving a call for a node - // if primary was already assigned for a replica. But here we don't keep track of per shard data in cache - // so it's not feasible to do any removal of node entry just based on single shard. - // ONLY run if single shard is present in the batch, to maintain backward compatibility - if (batch.getBatchedShards().size() == 1) { - List primaries = batch.getBatchedShards() - .stream() - .map(allocation.routingNodes()::activePrimary) - .filter(Objects::nonNull) - .collect(Collectors.toList()); - AsyncShardFetch fetch = batch.getAsyncFetcher(); - primaries.forEach(node -> fetch.clearCacheForNode(node.currentNodeId())); - } + // We need to clear the cache for the primary shard to ensure we do not cancel recoveries based on excessively + // stale data. We do this by clearing the cache of primary shards on nodes for all the active primaries of + // replicas in the current batch. + List primaries = batch.getBatchedShards() + .stream() + .map(allocation.routingNodes()::activePrimary) + .filter(Objects::nonNull) + .collect(Collectors.toList()); + AsyncShardBatchFetch fetch = batch.getAsyncFetcher(); + primaries.forEach(shardRouting -> fetch.clearCache(shardRouting.currentNodeId(), shardRouting.shardId())); } private boolean hasNewNodes(DiscoveryNodes nodes) { From 16d6c5966e5c97724b061e89ea08b60a7d008049 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Tue, 23 Apr 2024 20:28:12 +0530 Subject: [PATCH 40/45] Revert ensureAsyncFetchStorePrimaryRecency update Signed-off-by: Shivansh Arora --- .../gateway/AsyncShardBatchFetch.java | 18 ------------------ .../opensearch/gateway/AsyncShardFetch.java | 5 ----- .../gateway/AsyncShardFetchCache.java | 2 -- .../gateway/ShardsBatchGatewayAllocator.java | 10 +++++++--- 4 files changed, 7 insertions(+), 28 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java index 9f4cee99465a5..4f39a39cea678 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java @@ -80,16 +80,6 @@ public synchronized void clearShard(ShardId shardId) { this.cache.deleteShard(shardId); } - /** - * Clear the cache for a given node and shardId. - * - * @param nodeId node id to be removed from the batch. - * @param shardId shard id to be removed from the batch. - */ - public synchronized void clearCache(String nodeId, ShardId shardId) { - this.cache.cleanCacheForNodeForShardId(nodeId, shardId); - } - /** * Cache implementation of transport actions returning batch of shards related data in the response. * Store node level responses of transport actions like {@link TransportNodesListGatewayStartedShardsBatch} or @@ -148,14 +138,6 @@ public void deleteShard(ShardId shardId) { } } - @Override - public void cleanCacheForNodeForShardId(String nodeId, ShardId shardId) { - if (shardIdToArray.containsKey(shardId)) { - Integer shardIdIndex = shardIdToArray.remove(shardId); - cache.get(nodeId).clearShard(shardIdIndex); - } - } - @Override public void initData(DiscoveryNode node) { cache.put(node.getId(), new NodeEntry<>(node.getId(), shardResponseClass, batchSize, emptyShardResponsePredicate)); diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java index 5b9571a73bba9..b664dd573ce67 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java @@ -315,11 +315,6 @@ public void deleteShard(ShardId shardId) { cache.clear(); // single shard cache can clear the full map } - @Override - public void cleanCacheForNodeForShardId(String nodeId, ShardId shardId) { - cache.remove(nodeId); // non batch cache only has one entry per node - } - /** * A node entry, holding the state of the fetched data for a specific shard * for a giving node. diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardFetchCache.java b/server/src/main/java/org/opensearch/gateway/AsyncShardFetchCache.java index 8e37137d4aa24..2a4e6181467b0 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardFetchCache.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardFetchCache.java @@ -74,8 +74,6 @@ protected AsyncShardFetchCache(Logger logger, String type) { */ abstract void deleteShard(ShardId shardId); - abstract void cleanCacheForNodeForShardId(String nodeId, ShardId shardId); - /** * Returns the number of fetches that are currently ongoing. */ diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 82dd6fa832ca5..d2de78ffac965 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -407,6 +407,7 @@ private void ensureAsyncFetchStorePrimaryRecency(RoutingAllocation allocation) { // drops out, we fetch the shard data, then some indexing happens and then the node rejoins the cluster again. There are other // ways we could decide to cancel a recovery based on stale data (e.g. changing allocation filters or a primary failure) but // making the wrong decision here is not catastrophic so we only need to cover the common case. + logger.trace( () -> new ParameterizedMessage( "new nodes {} found, clearing primary async-fetch-store cache", @@ -422,15 +423,18 @@ private void ensureAsyncFetchStorePrimaryRecency(RoutingAllocation allocation) { private static void clearCacheForBatchPrimary(ShardsBatch batch, RoutingAllocation allocation) { // We need to clear the cache for the primary shard to ensure we do not cancel recoveries based on excessively - // stale data. We do this by clearing the cache of primary shards on nodes for all the active primaries of - // replicas in the current batch. + // stale data. We do this by clearing the cache of nodes for all the active primaries of replicas in the current batch. + // Although this flow can be optimized by only clearing the cache for the primary shard but currently + // when we want to fetch data we do for complete node, for doing this a new fetch flow will also handle just + // fetching the data for a single shard on the node and fill that up in our cache + // Opened issue #13352 - to track the improvement List primaries = batch.getBatchedShards() .stream() .map(allocation.routingNodes()::activePrimary) .filter(Objects::nonNull) .collect(Collectors.toList()); AsyncShardBatchFetch fetch = batch.getAsyncFetcher(); - primaries.forEach(shardRouting -> fetch.clearCache(shardRouting.currentNodeId(), shardRouting.shardId())); + primaries.forEach(shardRouting -> fetch.clearCacheForNode(shardRouting.currentNodeId())); } private boolean hasNewNodes(DiscoveryNodes nodes) { From 0693b9a2c83dd3774d1faba9c0244daf11f68e16 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Tue, 23 Apr 2024 20:29:21 +0530 Subject: [PATCH 41/45] Enable batch mode temporarily for gradle check run Signed-off-by: Shivansh Arora --- .../cluster/routing/allocation/ExistingShardsAllocator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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..7ced7a31c1b27 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 @@ -77,7 +77,7 @@ public interface ExistingShardsAllocator { */ Setting EXISTING_SHARDS_ALLOCATOR_BATCH_MODE = Setting.boolSetting( "cluster.allocator.existing_shards_allocator.batch_enabled", - false, + true, Setting.Property.NodeScope ); From 1ff45883ac191c8259ee246ed893f7938941f4d9 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Thu, 25 Apr 2024 17:42:53 +0530 Subject: [PATCH 42/45] Disabled batch mode by default Signed-off-by: Shivansh Arora --- .../cluster/routing/allocation/ExistingShardsAllocator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 7ced7a31c1b27..fb2a37237f8b6 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 @@ -77,7 +77,7 @@ public interface ExistingShardsAllocator { */ Setting EXISTING_SHARDS_ALLOCATOR_BATCH_MODE = Setting.boolSetting( "cluster.allocator.existing_shards_allocator.batch_enabled", - true, + false, Setting.Property.NodeScope ); From 3f103c16f21637a8a0005291ccbf6c02aa8e78ed Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Thu, 25 Apr 2024 23:24:49 +0530 Subject: [PATCH 43/45] Spotless apply Signed-off-by: Shivansh Arora --- .../java/org/opensearch/gateway/RecoveryFromGatewayIT.java | 1 - .../java/org/opensearch/common/settings/ClusterSettings.java | 3 +-- server/src/main/java/org/opensearch/node/Node.java | 2 +- .../java/org/opensearch/test/gateway/TestGatewayAllocator.java | 2 -- 4 files changed, 2 insertions(+), 6 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 27a9d93340bed..bc0557ddc2afa 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -81,7 +81,6 @@ import org.opensearch.test.InternalSettingsPlugin; import org.opensearch.test.InternalTestCluster; import org.opensearch.test.InternalTestCluster.RestartCallback; -import org.opensearch.test.InternalTestCluster; import org.opensearch.test.OpenSearchIntegTestCase; import org.opensearch.test.OpenSearchIntegTestCase.ClusterScope; import org.opensearch.test.OpenSearchIntegTestCase.Scope; 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 63419c34b6dcf..ded844b3a7f18 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -81,7 +81,6 @@ import org.opensearch.cluster.service.ClusterManagerService; import org.opensearch.cluster.service.ClusterManagerTaskThrottler; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.cluster.service.ClusterManagerService; import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.cache.CacheType; import org.opensearch.common.cache.settings.CacheSettings; @@ -104,8 +103,8 @@ import org.opensearch.gateway.DanglingIndicesState; import org.opensearch.gateway.GatewayService; import org.opensearch.gateway.PersistedClusterStateService; -import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.gateway.ShardsBatchGatewayAllocator; +import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.http.HttpTransportSettings; import org.opensearch.index.IndexModule; import org.opensearch.index.IndexSettings; diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 24d704ecc1bb9..b4f2a303c7024 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -136,8 +136,8 @@ import org.opensearch.gateway.GatewayService; import org.opensearch.gateway.MetaStateService; import org.opensearch.gateway.PersistedClusterStateService; -import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.gateway.ShardsBatchGatewayAllocator; +import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.http.HttpServerTransport; import org.opensearch.identity.IdentityService; import org.opensearch.index.IndexModule; diff --git a/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java b/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java index d302a77e7abdb..b9f52a62f823a 100644 --- a/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java +++ b/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java @@ -44,8 +44,6 @@ import org.opensearch.gateway.ReplicaShardAllocator; import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper; import org.opensearch.gateway.TransportNodesListGatewayStartedShards.NodeGatewayStartedShards; -import org.opensearch.gateway.TransportNodesListGatewayStartedBatchShards; -import org.opensearch.index.shard.ShardId; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata.NodeStoreFilesMetadata; From 154d13b8c3ecc73a94cece44713dae93dd0c3791 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Fri, 26 Apr 2024 00:08:09 +0530 Subject: [PATCH 44/45] Fix for missed merge conflict Signed-off-by: Shivansh Arora --- .../main/java/org/opensearch/cluster/ClusterModule.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/cluster/ClusterModule.java b/server/src/main/java/org/opensearch/cluster/ClusterModule.java index 57c44aa107e5b..aa9101090b6d5 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterModule.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterModule.java @@ -154,7 +154,13 @@ public ClusterModule( this.shardsAllocator = createShardsAllocator(settings, clusterService.getClusterSettings(), clusterPlugins); this.clusterService = clusterService; this.indexNameExpressionResolver = new IndexNameExpressionResolver(threadContext); - this.allocationService = new AllocationService(allocationDeciders, shardsAllocator, clusterInfoService, snapshotsInfoService); + this.allocationService = new AllocationService( + allocationDeciders, + shardsAllocator, + clusterInfoService, + snapshotsInfoService, + settings + ); } public static List getNamedWriteables() { From 080c3bab7247a48bc3660688eb36646c03373541 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Fri, 26 Apr 2024 07:44:56 +0530 Subject: [PATCH 45/45] Added changelog Signed-off-by: Shivansh Arora --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index f2fbf895dafcf..6e6e03f460fb7 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -25,6 +25,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Streaming Indexing] Ensure support of the new transport by security plugin ([#13174](https://github.com/opensearch-project/OpenSearch/pull/13174)) - Add cluster setting to dynamically configure the buckets for filter rewrite optimization. ([#13179](https://github.com/opensearch-project/OpenSearch/pull/13179)) - [Remote Store] Add capability of doing refresh as determined by the translog ([#12992](https://github.com/opensearch-project/OpenSearch/pull/12992)) +- Batch mode for async fetching shard information in GatewayAllocator for unassigned shards ([#8746](https://github.com/opensearch-project/OpenSearch/pull/8746)) ### Dependencies - Bump `org.apache.commons:commons-configuration2` from 2.10.0 to 2.10.1 ([#12896](https://github.com/opensearch-project/OpenSearch/pull/12896))