From 0d0e5c2b04b12fcf4dcfd70dcab3b1169e240661 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Tue, 25 Jul 2023 07:01:53 +0530 Subject: [PATCH 01/15] Created new ReplicaShardBatchAllocator to be used instead of ReplicaShardAllocator for batch calls Signed-off-by: Shivansh Arora --- .../opensearch/gateway/GatewayAllocator.java | 429 ++++++++++++++- .../gateway/ReplicaShardAllocator.java | 8 +- .../gateway/ReplicaShardBatchAllocator.java | 518 ++++++++++++++++++ 3 files changed, 946 insertions(+), 9 deletions(-) create mode 100644 server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java diff --git a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java index 5a20112b19219..503aa7274c5ee 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java @@ -42,21 +42,29 @@ 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.util.concurrent.ConcurrentCollections; import org.opensearch.common.util.set.Sets; import org.opensearch.common.lease.Releasables; -import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.shard.ShardId; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch; import java.util.Collections; +import java.util.HashMap; +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; @@ -73,29 +81,45 @@ 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; private final PrimaryShardAllocator primaryShardAllocator; private final ReplicaShardAllocator replicaShardAllocator; + private final PrimaryShardBatchAllocator primaryBatchShardAllocator; + private final ReplicaShardBatchAllocator replicaBatchShardAllocator; + private final TransportNodesListGatewayStartedShardsBatch 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(); + 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( RerouteService rerouteService, TransportNodesListGatewayStartedShards startedAction, - TransportNodesListShardStoreMetadata storeAction + TransportNodesListShardStoreMetadata storeAction, + TransportNodesListGatewayStartedShardsBatch batchStartedAction, + TransportNodesListShardStoreMetadataBatch batchStoreAction ) { 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(); } @Override @@ -104,6 +128,10 @@ public void cleanCaches() { asyncFetchStarted.clear(); Releasables.close(asyncFetchStore.values()); asyncFetchStore.clear(); + batchIdToStartedShardBatch.clear(); + batchIdToStoreShardBatch.clear(); + startedShardBatchLookup.clear(); + storeShardBatchLookup.clear(); } // for tests @@ -111,6 +139,10 @@ protected GatewayAllocator() { this.rerouteService = null; this.primaryShardAllocator = null; this.replicaShardAllocator = null; + this.batchStartedAction = null; + this.primaryBatchShardAllocator = null; + this.batchStoreAction = null; + this.replicaBatchShardAllocator = null; } @Override @@ -130,6 +162,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); } } @@ -138,6 +171,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()); } } @@ -145,15 +179,30 @@ 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) { - assert replicaShardAllocator != null; - if (allocation.routingNodes().hasInactiveShards()) { - // cancel existing recoveries if we have a better match - replicaShardAllocator.processExistingRecoveries(allocation); + // ToDo: fetch from settings + boolean batchMode = true; + if (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); + } } } @@ -168,6 +217,99 @@ public void allocateUnassigned( innerAllocatedUnassigned(allocation, primaryShardAllocator, replicaShardAllocator, shardRouting, unassignedAllocationHandler); } + @Override + public void allocateUnassignedBatch(final RoutingAllocation allocation, boolean primary) { + // create batches for unassigned shards + createBatches(allocation, primary); + + assert primaryBatchShardAllocator != null; + assert replicaBatchShardAllocator != null; + if (primary) { + 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 = 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.shardId()) == false) && (shardRouting.primary() == primary)) { + assert shardRouting.unassigned(); + 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) { + 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, 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 shard to be removed + */ + 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, @@ -216,7 +358,13 @@ private void ensureAsyncFetchStorePrimaryRecency(RoutingAllocation allocation) { Sets.difference(newEphemeralIds, lastSeenEphemeralIds) ) ); + asyncFetchStore.values().forEach(fetch -> clearCacheForPrimary(fetch, allocation)); + storeShardBatchLookup.values().forEach(batch -> + clearCacheForBatchPrimary(batchIdToStoreShardBatch.get(batch), allocation) + ); + + // recalc to also (lazily) clear out old nodes. this.lastSeenEphemeralIds = newEphemeralIds; } @@ -232,6 +380,18 @@ private static void clearCacheForPrimary( } } + private static void clearCacheForBatchPrimary( + ShardsBatch batch, + RoutingAllocation allocation + ) { + List primaries = batch.getBatchedShards().stream() + .map(allocation.routingNodes()::activePrimary) + .filter(Objects::nonNull) + .collect(Collectors.toList()); + AsyncBatchShardFetch 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) { @@ -268,6 +428,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; @@ -303,6 +489,59 @@ 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 AsyncBatchShardFetch.FetchResult<>(null, Collections.emptyMap()); + } + + String batchId = startedShardBatchLookup.getOrDefault(shardRouting.shardId(), null); + 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(GatewayAllocator.this::safelyRemoveShardFromBatch); + + if (shardsBatch.getBatchedShards().isEmpty() && shardsEligibleForFetch.isEmpty()) { + logger.debug("Batch {} is empty", batchId); + return new AsyncBatchShardFetch.FetchResult<>(null, Collections.emptyMap()); + } + + Map> shardToIgnoreNodes = new HashMap<>(); + + for(ShardId shardId : shardsBatch.asyncBatch.shardsToCustomDataPathMap.keySet()){ + shardToIgnoreNodes.put(shardId, allocation.getIgnoreNodes(shardId)); + } + AsyncBatchShardFetch asyncFetcher = shardsBatch.getAsyncFetcher(); + AsyncBatchShardFetch.FetchResult shardBatchState = asyncFetcher.fetchData( + allocation.nodes(), + shardToIgnoreNodes + ); + + if (shardBatchState.hasData()) { + shardBatchState.processAllocation(allocation); + } + return (AsyncBatchShardFetch.FetchResult) shardBatchState; + } + + } + class InternalReplicaShardAllocator extends ReplicaShardAllocator { private final TransportNodesListShardStoreMetadata storeAction; @@ -341,4 +580,180 @@ protected boolean hasInitiatedFetching(ShardRouting shard) { return asyncFetchStore.get(shard.shardId()) != null; } } + + class InternalReplicaBatchShardAllocator extends ReplicaShardBatchAllocator { + + @Override + @SuppressWarnings("unchecked") + protected AsyncBatchShardFetch.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 AsyncBatchShardFetch.FetchResult<>(null, Collections.emptyMap()); + } + + String batchId = storeShardBatchLookup.getOrDefault(shardRouting.shardId(), null); + 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(GatewayAllocator.this::safelyRemoveShardFromBatch); + + if (shardsBatch.getBatchedShards().isEmpty() && shardsEligibleForFetch.isEmpty()) { + logger.debug("Batch {} is empty", batchId); + return new AsyncBatchShardFetch.FetchResult<>(null, Collections.emptyMap()); + } + Map> shardToIgnoreNodes = new HashMap<>(); + for (ShardId shardId : shardsBatch.asyncBatch.shardsToCustomDataPathMap.keySet()) { + shardToIgnoreNodes.put(shardId, allocation.getIgnoreNodes(shardId)); + } + AsyncBatchShardFetch asyncFetcher = shardsBatch.getAsyncFetcher(); + AsyncBatchShardFetch.FetchResult shardBatchStores = asyncFetcher.fetchData( + allocation.nodes(), + shardToIgnoreNodes + ); + if (shardBatchStores.hasData()) { + shardBatchStores.processAllocation(allocation); + } + return (AsyncBatchShardFetch.FetchResult) shardBatchStores; + } + + @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. + */ + private class ShardsBatch { + 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 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"; + } + + Set getBatchedShardRoutings() { + return batchInfo.values().stream().map(ShardBatchEntry::getShardRouting).collect(Collectors.toSet()); + } + + Set getBatchedShards() { + return batchInfo.keySet(); + } + + public String getBatchId() { + 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; + } + } + } diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java index 5216dd2fcb4b5..c8d335c698a13 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java @@ -336,7 +336,7 @@ private static Tuple> canBeAllocated * Takes the store info for nodes that have a shard store and adds them to the node decisions, * leaving the node explanations untouched for those nodes that do not have any store information. */ - private static List augmentExplanationsWithStoreInfo( + public static List augmentExplanationsWithStoreInfo( Map nodeDecisions, Map withShardStores ) { @@ -499,7 +499,7 @@ private static boolean canPerformOperationBasedRecovery( */ protected abstract boolean hasInitiatedFetching(ShardRouting shard); - private static class MatchingNode { + public static class MatchingNode { static final Comparator COMPARATOR = Comparator.comparing(m -> m.isNoopRecovery) .thenComparing(m -> m.retainingSeqNo) .thenComparing(m -> m.matchingBytes); @@ -556,5 +556,9 @@ boolean canPerformNoopRecovery(DiscoveryNode node) { public boolean hasAnyData() { return matchingNodes.isEmpty() == false; } + + public Map getNodeDecisions() { + return this.nodeDecisions; + } } } 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..1ed28b281f710 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java @@ -0,0 +1,518 @@ +/* + * 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.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.routing.RoutingNode; +import org.opensearch.cluster.routing.RoutingNodes; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.UnassignedInfo; +import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; +import org.opensearch.cluster.routing.allocation.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.common.unit.ByteSizeValue; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.gateway.AsyncBatchShardFetch.FetchResult; +import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.opensearch.cluster.routing.UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING; + +public abstract class ReplicaShardBatchAllocator extends BaseGatewayShardAllocator { + + /** + * 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. + */ + public void processExistingRecoveries(RoutingAllocation allocation, List> shardBatches) { + Metadata metadata = allocation.metadata(); + RoutingNodes routingNodes = allocation.routingNodes(); + List shardCancellationActions = new ArrayList<>(); + for (Set shardBatch : shardBatches) { + Set eligibleFetchShards = new HashSet<>(); + Set ineligibleShards = new HashSet<>(); + for (ShardRouting shard : shardBatch) { + if (shard.primary()) { + ineligibleShards.add(shard); + continue; + } + if (shard.initializing() == false) { + ineligibleShards.add(shard); + continue; + } + if (shard.relocatingNodeId() != null) { + ineligibleShards.add(shard); + continue; + } + + // if we are allocating a replica because of index creation, no need to go and find a copy, there isn't one... + if (shard.unassignedInfo() != null && shard.unassignedInfo().getReason() == UnassignedInfo.Reason.INDEX_CREATED) { + ineligibleShards.add(shard); + continue; + } + eligibleFetchShards.add(shard); + } + AsyncBatchShardFetch.FetchResult shardState = fetchData(eligibleFetchShards, ineligibleShards, allocation); + if (shardState.hasData()) { + logger.trace("{}: fetching new stores for initializing shard batch", eligibleFetchShards); + continue; // still fetching + } + for (ShardRouting shard: eligibleFetchShards) { + ShardRouting primaryShard = allocation.routingNodes().activePrimary(shard.shardId()); + assert primaryShard != null : "the replica shard can be allocated on at least one node, so there must be an active primary"; + assert primaryShard.currentNodeId() != null; + final DiscoveryNode primaryNode = allocation.nodes().get(primaryShard.currentNodeId()); + final TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata primaryStore= findStore(primaryNode, shardState, shard); + if (primaryStore == null) { + // if we can't find the primary data, it is probably because the primary shard is corrupted (and listing failed) + // just let the recovery find it out, no need to do anything about it for the initializing shard + logger.trace("{}: no primary shard store found or allocated, letting actual allocation figure it out", shard); + continue; + } + ReplicaShardAllocator.MatchingNodes matchingNodes = findMatchingNodes(shard, allocation, true, primaryNode, primaryStore, shardState, false); + if (matchingNodes.getNodeWithHighestMatch() != null) { + DiscoveryNode currentNode = allocation.nodes().get(shard.currentNodeId()); + DiscoveryNode nodeWithHighestMatch = matchingNodes.getNodeWithHighestMatch(); + // current node will not be in matchingNodes as it is filtered away by SameShardAllocationDecider + if (currentNode.equals(nodeWithHighestMatch) == false + && matchingNodes.canPerformNoopRecovery(nodeWithHighestMatch) + && canPerformOperationBasedRecovery(primaryStore, shardState, currentNode, shard) == false) { + // we found a better match that can perform noop recovery, cancel the existing allocation. + logger.debug( + "cancelling allocation of replica on [{}], can perform a noop recovery on node [{}]", + currentNode, + nodeWithHighestMatch + ); + final Set failedNodeIds = shard.unassignedInfo() == null + ? Collections.emptySet() + : shard.unassignedInfo().getFailedNodeIds(); + UnassignedInfo unassignedInfo = new UnassignedInfo( + UnassignedInfo.Reason.REALLOCATED_REPLICA, + "existing allocation of replica to [" + + currentNode + + "] cancelled, can perform a noop recovery on [" + + nodeWithHighestMatch + + "]", + null, + 0, + allocation.getCurrentNanoTime(), + System.currentTimeMillis(), + false, + UnassignedInfo.AllocationStatus.NO_ATTEMPT, + failedNodeIds + ); + // don't cancel shard in the loop as it will cause a ConcurrentModificationException + shardCancellationActions.add( + () -> routingNodes.failShard( + logger, + shard, + unassignedInfo, + metadata.getIndexSafe(shard.index()), + allocation.changes() + ) + ); + } + } + } + } + for (Runnable action : shardCancellationActions) { + action.run(); + } + } + + private static boolean isResponsibleFor(final ShardRouting shard) { + return !shard.primary() // must be a replica + && shard.unassigned() // must be unassigned + // if we are allocating a replica because of index creation, no need to go and find a copy, there isn't one... + && shard.unassignedInfo().getReason() != UnassignedInfo.Reason.INDEX_CREATED; + } + + abstract protected FetchResult fetchData(Set shardEligibleForFetch, + Set inEligibleShards, + RoutingAllocation allocation); + + @Override + public AllocateUnassignedDecision makeAllocationDecision(ShardRouting unassignedShard, RoutingAllocation allocation, Logger logger) { + return null; + } + + @Override + public HashMap makeAllocationDecision(Set shards, RoutingAllocation allocation, Logger logger) { + HashMap shardAllocationDecisions = new HashMap<>(); + final boolean explain = allocation.debugDecision(); + final RoutingNodes routingNodes = allocation.routingNodes(); + Set shardsEligibleForFetch = new HashSet<>(); + Set shardsNotEligibleForFetch = new HashSet<>(); + HashMap>> nodeAllocationDecisions = new HashMap<>(); + for(ShardRouting shard : shards) { + if (!isResponsibleFor(shard)) { + // this allocator n is not responsible for allocating this shard + shardsNotEligibleForFetch.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); + shardsNotEligibleForFetch.add(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 + // ToDo: Check if we need to store or computing again will be cheaper/better + nodeAllocationDecisions.put(shard, result); + + shardsEligibleForFetch.add(shard); + } + + // only fetch data for eligible shards + final FetchResult shardsState = fetchData(shardsEligibleForFetch, shardsNotEligibleForFetch, allocation); + + // ToDo: Analyze if we need to create hashmaps here or sequential is better +// Map primaryNodesMap = shardsEligibleForFetch.stream() +// .map(x -> routingNodes.activePrimary(x.shardId())) +// .filter(Objects::nonNull) +// .filter(node -> node.currentNodeId() != null) +// .collect(Collectors.toMap(Function.identity(), node -> allocation.nodes().get(node.currentNodeId()))); +// +// Map primaryStoreMap = findStoresBatch(primaryNodesMap, shardsState); + + for (ShardRouting unassignedShard : shardsEligibleForFetch) { + if (!shardsState.hasData()) { + logger.trace("{}: ignoring allocation, still fetching shard stores", unassignedShard); + allocation.setHasPendingAsyncFetch(); + List nodeDecisions = null; + if (explain) { + nodeDecisions = buildDecisionsForAllNodes(unassignedShard, allocation); + } + shardAllocationDecisions.put(unassignedShard, + AllocateUnassignedDecision.no(UnassignedInfo.AllocationStatus.FETCHING_SHARD_DATA, nodeDecisions)); + continue; + } + Tuple> result = nodeAllocationDecisions.get(unassignedShard); + ShardRouting primaryShard = routingNodes.activePrimary(unassignedShard.shardId()); + if (primaryShard == null) { + assert explain : "primary should only be null here if we are in explain mode, so we didn't " + + "exit early when canBeAllocatedToAtLeastOneNode didn't return a YES decision"; + shardAllocationDecisions.put(unassignedShard, AllocateUnassignedDecision.no( + UnassignedInfo.AllocationStatus.fromDecision(result.v1().type()), + result.v2() != null ? new ArrayList<>(result.v2().values()) : null + )); + continue; + } + assert primaryShard.currentNodeId() != null; + final DiscoveryNode primaryNode = allocation.nodes().get(primaryShard.currentNodeId()); + final TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata primaryStore = findStore(primaryNode, shardsState, unassignedShard); + if (primaryStore == null) { + // if we can't find the primary data, it is probably because the primary shard is corrupted (and listing failed) + // we want to let the replica be allocated in order to expose the actual problem with the primary that the replica + // will try and recover from + // Note, this is the existing behavior, as exposed in running CorruptFileTest#testNoPrimaryData + logger.trace("{}: no primary shard store found or allocated, letting actual allocation figure it out", unassignedShard); + shardAllocationDecisions.put(unassignedShard, AllocateUnassignedDecision.NOT_TAKEN); + } + + // find the matching nodes + ReplicaShardAllocator.MatchingNodes matchingNodes = findMatchingNodes( + unassignedShard, + allocation, + false, + primaryNode, + primaryStore, + shardsState, + explain + ); + + assert explain == false || matchingNodes.getNodeDecisions() != null : "in explain mode, we must have individual node decisions"; + + List nodeDecisions = ReplicaShardAllocator.augmentExplanationsWithStoreInfo(result.v2(), matchingNodes.getNodeDecisions()); + if (result.v1().type() != Decision.Type.YES) { + shardAllocationDecisions.put(unassignedShard, AllocateUnassignedDecision.no(UnassignedInfo.AllocationStatus.fromDecision(result.v1().type()), nodeDecisions)); + continue; + } else if (matchingNodes.getNodeWithHighestMatch() != null) { + RoutingNode nodeWithHighestMatch = allocation.routingNodes().node(matchingNodes.getNodeWithHighestMatch().getId()); + // we only check on THROTTLE since we checked before on NO + Decision decision = allocation.deciders().canAllocate(unassignedShard, nodeWithHighestMatch, allocation); + if (decision.type() == Decision.Type.THROTTLE) { + logger.debug( + "[{}][{}]: throttling allocation [{}] to [{}] in order to reuse its unallocated persistent store", + unassignedShard.index(), + unassignedShard.id(), + unassignedShard, + nodeWithHighestMatch.node() + ); + // we are throttling this, as we have enough other shards to allocate to this node, so ignore it for now + shardAllocationDecisions.put(unassignedShard, AllocateUnassignedDecision.throttle(nodeDecisions)); + continue; + } else { + logger.debug( + "[{}][{}]: allocating [{}] to [{}] in order to reuse its unallocated persistent store", + unassignedShard.index(), + unassignedShard.id(), + unassignedShard, + nodeWithHighestMatch.node() + ); + // we found a match + shardAllocationDecisions.put(unassignedShard, AllocateUnassignedDecision.yes(nodeWithHighestMatch.node(), null, nodeDecisions, true)); + continue; + } + } else if (matchingNodes.hasAnyData() == false && unassignedShard.unassignedInfo().isDelayed()) { + // if we didn't manage to find *any* data (regardless of matching sizes), and the replica is + // unassigned due to a node leaving, so we delay allocation of this replica to see if the + // node with the shard copy will rejoin so we can re-use the copy it has + logger.debug("{}: allocation of [{}] is delayed", unassignedShard.shardId(), unassignedShard); + long remainingDelayMillis = 0L; + long totalDelayMillis = 0L; + if (explain) { + UnassignedInfo unassignedInfo = unassignedShard.unassignedInfo(); + Metadata metadata = allocation.metadata(); + IndexMetadata indexMetadata = metadata.index(unassignedShard.index()); + totalDelayMillis = INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(indexMetadata.getSettings()).getMillis(); + long remainingDelayNanos = unassignedInfo.getRemainingDelay(System.nanoTime(), indexMetadata.getSettings()); + remainingDelayMillis = TimeValue.timeValueNanos(remainingDelayNanos).millis(); + } + shardAllocationDecisions.put(unassignedShard, AllocateUnassignedDecision.delayed(remainingDelayMillis, totalDelayMillis, nodeDecisions)); + } + + shardAllocationDecisions.put(unassignedShard, AllocateUnassignedDecision.NOT_TAKEN); + } + return shardAllocationDecisions; + } + + private ReplicaShardAllocator.MatchingNodes findMatchingNodes( + ShardRouting shard, + RoutingAllocation allocation, + boolean noMatchFailedNodes, + DiscoveryNode primaryNode, + TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata primaryStore, + FetchResult data, + boolean explain + ) { + Map matchingNodes = new HashMap<>(); + Map nodeDecisions = explain ? new HashMap<>() : null; + for (Map.Entry nodeStoreEntry : data.getData().entrySet()) { + DiscoveryNode discoNode = nodeStoreEntry.getKey(); + if (noMatchFailedNodes + && shard.unassignedInfo() != null + && shard.unassignedInfo().getFailedNodeIds().contains(discoNode.getId())) { + continue; + } + TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata storeFilesMetadata = nodeStoreEntry.getValue() + .getNodeStoreFilesMetadataBatch().get(shard.shardId()).storeFilesMetadata(); + // we don't have any files at all, it is an empty index + if (storeFilesMetadata.isEmpty()) { + continue; + } + + RoutingNode node = allocation.routingNodes().node(discoNode.getId()); + if (node == null) { + continue; + } + + // check if we can allocate on that node... + // we only check for NO, since if this node is THROTTLING and it has enough "same data" + // then we will try and assign it next time + Decision decision = allocation.deciders().canAllocate(shard, node, allocation); + ReplicaShardAllocator.MatchingNode matchingNode = null; + if (explain) { + matchingNode = computeMatchingNode(primaryNode, primaryStore, discoNode, storeFilesMetadata); + NodeAllocationResult.ShardStoreInfo shardStoreInfo = new NodeAllocationResult.ShardStoreInfo(matchingNode.matchingBytes); + nodeDecisions.put(node.nodeId(), new NodeAllocationResult(discoNode, shardStoreInfo, decision)); + } + + if (decision.type() == Decision.Type.NO) { + continue; + } + + if (matchingNode == null) { + matchingNode = computeMatchingNode(primaryNode, primaryStore, discoNode, storeFilesMetadata); + } + matchingNodes.put(discoNode, matchingNode); + if (logger.isTraceEnabled()) { + if (matchingNode.isNoopRecovery) { + logger.trace("{}: node [{}] can perform a noop recovery", shard, discoNode.getName()); + } else if (matchingNode.retainingSeqNo >= 0) { + logger.trace( + "{}: node [{}] can perform operation-based recovery with retaining sequence number [{}]", + shard, + discoNode.getName(), + matchingNode.retainingSeqNo + ); + } else { + logger.trace( + "{}: node [{}] has [{}/{}] bytes of re-usable data", + shard, + discoNode.getName(), + new ByteSizeValue(matchingNode.matchingBytes), + matchingNode.matchingBytes + ); + } + } + } + + return new ReplicaShardAllocator.MatchingNodes(matchingNodes, nodeDecisions); + } + + private static ReplicaShardAllocator.MatchingNode computeMatchingNode( + DiscoveryNode primaryNode, + TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata primaryStore, + DiscoveryNode replicaNode, + TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata replicaStore + ) { + final long retainingSeqNoForPrimary = primaryStore.getPeerRecoveryRetentionLeaseRetainingSeqNo(primaryNode); + final long retainingSeqNoForReplica = primaryStore.getPeerRecoveryRetentionLeaseRetainingSeqNo(replicaNode); + final boolean isNoopRecovery = (retainingSeqNoForReplica >= retainingSeqNoForPrimary && retainingSeqNoForPrimary >= 0) + || hasMatchingSyncId(primaryStore, replicaStore); + final long matchingBytes = computeMatchingBytes(primaryStore, replicaStore); + return new ReplicaShardAllocator.MatchingNode(matchingBytes, retainingSeqNoForReplica, isNoopRecovery); + } + + private static boolean hasMatchingSyncId( + TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata primaryStore, + TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata replicaStore + ) { + String primarySyncId = primaryStore.syncId(); + return primarySyncId != null && primarySyncId.equals(replicaStore.syncId()); + } + + private static long computeMatchingBytes( + TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata primaryStore, + TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata storeFilesMetadata + ) { + long sizeMatched = 0; + for (StoreFileMetadata storeFileMetadata : storeFilesMetadata) { + String metadataFileName = storeFileMetadata.name(); + if (primaryStore.fileExists(metadataFileName) && primaryStore.file(metadataFileName).isSame(storeFileMetadata)) { + sizeMatched += storeFileMetadata.length(); + } + } + return sizeMatched; + } + + /** + * Determines if the shard can be allocated on at least one node based on the allocation deciders. + * + * Returns the best allocation decision for allocating the shard on any node (i.e. YES if at least one + * node decided YES, THROTTLE if at least one node decided THROTTLE, and NO if none of the nodes decided + * YES or THROTTLE). If in explain mode, also returns the node-level explanations as the second element + * in the returned tuple. + */ + private static Tuple> canBeAllocatedToAtLeastOneNode( + ShardRouting shard, + RoutingAllocation allocation + ) { + Decision madeDecision = Decision.NO; + final boolean explain = allocation.debugDecision(); + Map nodeDecisions = explain ? new HashMap<>() : null; + for (final DiscoveryNode cursor : allocation.nodes().getDataNodes().values()) { + RoutingNode node = allocation.routingNodes().node(cursor.getId()); + if (node == null) { + continue; + } + // if we can't allocate it on a node, ignore it, for example, this handles + // cases for only allocating a replica after a primary + Decision decision = allocation.deciders().canAllocate(shard, node, allocation); + if (decision.type() == Decision.Type.YES && madeDecision.type() != Decision.Type.YES) { + if (explain) { + madeDecision = decision; + } else { + return Tuple.tuple(decision, null); + } + } else if (madeDecision.type() == Decision.Type.NO && decision.type() == Decision.Type.THROTTLE) { + madeDecision = decision; + } + if (explain) { + nodeDecisions.put(node.nodeId(), new NodeAllocationResult(node.node(), null, decision)); + } + } + return Tuple.tuple(madeDecision, nodeDecisions); + } + + protected abstract AsyncShardFetch.FetchResult fetchData(ShardRouting shard, RoutingAllocation allocation); + + protected abstract boolean hasInitiatedFetching(ShardRouting shard); + +// private static Map findStoresBatch(Map shardToNodeMap, +// FetchResult data) { +// Map shardStores = new HashMap<>(); +// shardToNodeMap.entrySet().forEach(entry -> { +// NodeStoreFilesMetadataBatch nodeFilesStore = data.getData().get(entry.getValue()); +// if (nodeFilesStore == null) { +// shardStores.put(entry.getKey(), null); +// } else { +// shardStores.put(entry.getKey(), nodeFilesStore.getNodeStoreFilesMetadataBatch().get(entry.getKey().shardId())); +// } +// }); +// return shardStores; +// } + + private static TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata findStore( + DiscoveryNode node, + FetchResult data, + ShardRouting shard + ) { + NodeStoreFilesMetadataBatch nodeFilesStore = data.getData().get(node); + if (nodeFilesStore == null) { + return null; + } + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeFileStoreMetadata = nodeFilesStore.getNodeStoreFilesMetadataBatch().get(shard.shardId()); + if (nodeFileStoreMetadata.getStoreFileFetchException() != null) { + // Do we need to throw an exception here, to handle this case differently? + return null; + } + return nodeFileStoreMetadata.storeFilesMetadata(); + } + + private static boolean canPerformOperationBasedRecovery( + TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata primaryStore, + FetchResult data, + DiscoveryNode targetNode, + ShardRouting shard + ) { + final NodeStoreFilesMetadataBatch nodeFilesStore = data.getData().get(targetNode); + if (nodeFilesStore == null) { + return false; + } + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeFileStoreMetadata = nodeFilesStore.getNodeStoreFilesMetadataBatch().get(shard.shardId()); + if (nodeFileStoreMetadata.getStoreFileFetchException() != null) { + return false; + } + TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata targetNodeStore = nodeFileStoreMetadata.storeFilesMetadata(); + if (targetNodeStore == null || targetNodeStore.isEmpty()) { + return false; + } + if (hasMatchingSyncId(primaryStore, targetNodeStore)) { + return true; + } + return primaryStore.getPeerRecoveryRetentionLeaseRetainingSeqNo(targetNode) >= 0; + } +} From 8b4747218fc6fe62cdbc28532e22765b2489ed94 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Fri, 28 Jul 2023 18:22:38 +0530 Subject: [PATCH 02/15] fixed import path Signed-off-by: Shivansh Arora --- .../src/main/java/org/opensearch/gateway/GatewayAllocator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java index 503aa7274c5ee..6c8a042e6b85d 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java @@ -54,7 +54,7 @@ import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.common.util.set.Sets; import org.opensearch.common.lease.Releasables; -import org.opensearch.index.shard.ShardId; +import org.opensearch.core.index.shard.ShardId; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch; From aa68e295f5821f0daeb36eab2a89efba9552fa0f Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Tue, 22 Aug 2023 13:27:19 +0530 Subject: [PATCH 03/15] Cleaned up code and removed unnecessary ineligible shards Signed-off-by: Shivansh Arora --- .../gateway/ReplicaShardBatchAllocator.java | 47 ++++--------------- 1 file changed, 8 insertions(+), 39 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java index 1ed28b281f710..1d49b6e122477 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java @@ -21,11 +21,10 @@ import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.cluster.routing.allocation.decider.Decision; import org.opensearch.common.collect.Tuple; -import org.opensearch.common.unit.ByteSizeValue; +import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.common.unit.TimeValue; import org.opensearch.gateway.AsyncBatchShardFetch.FetchResult; import org.opensearch.index.store.StoreFileMetadata; -import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch; @@ -59,32 +58,29 @@ public void processExistingRecoveries(RoutingAllocation allocation, List shardState = fetchData(eligibleFetchShards, ineligibleShards, allocation); - if (shardState.hasData()) { + AsyncBatchShardFetch.FetchResult shardState = fetchData(eligibleFetchShards, ineligibleShards, allocation); + if (shardState.hasData() == false) { logger.trace("{}: fetching new stores for initializing shard batch", eligibleFetchShards); continue; // still fetching } - for (ShardRouting shard: eligibleFetchShards) { + for (ShardRouting shard : eligibleFetchShards) { ShardRouting primaryShard = allocation.routingNodes().activePrimary(shard.shardId()); assert primaryShard != null : "the replica shard can be allocated on at least one node, so there must be an active primary"; assert primaryShard.currentNodeId() != null; final DiscoveryNode primaryNode = allocation.nodes().get(primaryShard.currentNodeId()); - final TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata primaryStore= findStore(primaryNode, shardState, shard); + final TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata primaryStore = findStore(primaryNode, shardState, shard); if (primaryStore == null) { // if we can't find the primary data, it is probably because the primary shard is corrupted (and listing failed) // just let the recovery find it out, no need to do anything about it for the initializing shard @@ -166,7 +162,7 @@ public HashMap makeAllocationDecision( Set shardsEligibleForFetch = new HashSet<>(); Set shardsNotEligibleForFetch = new HashSet<>(); HashMap>> nodeAllocationDecisions = new HashMap<>(); - for(ShardRouting shard : shards) { + for (ShardRouting shard : shards) { if (!isResponsibleFor(shard)) { // this allocator n is not responsible for allocating this shard shardsNotEligibleForFetch.add(shard); @@ -176,11 +172,10 @@ public HashMap makeAllocationDecision( Tuple> result = canBeAllocatedToAtLeastOneNode(shard, allocation); Decision allocationDecision = result.v1(); - if (allocationDecision.type() != Decision.Type.YES && (!explain || !hasInitiatedFetching(shard))){ + 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); - shardsNotEligibleForFetch.add(shard); shardAllocationDecisions.put(shard, AllocateUnassignedDecision.no(UnassignedInfo.AllocationStatus.fromDecision(allocationDecision.type()), result.v2() != null ? new ArrayList<>(result.v2().values()) : null)); @@ -188,7 +183,6 @@ public HashMap makeAllocationDecision( } // storing the nodeDecisions in nodeAllocationDecisions if the decision is not YES // so that we don't have to compute the decisions again - // ToDo: Check if we need to store or computing again will be cheaper/better nodeAllocationDecisions.put(shard, result); shardsEligibleForFetch.add(shard); @@ -197,15 +191,6 @@ public HashMap makeAllocationDecision( // only fetch data for eligible shards final FetchResult shardsState = fetchData(shardsEligibleForFetch, shardsNotEligibleForFetch, allocation); - // ToDo: Analyze if we need to create hashmaps here or sequential is better -// Map primaryNodesMap = shardsEligibleForFetch.stream() -// .map(x -> routingNodes.activePrimary(x.shardId())) -// .filter(Objects::nonNull) -// .filter(node -> node.currentNodeId() != null) -// .collect(Collectors.toMap(Function.identity(), node -> allocation.nodes().get(node.currentNodeId()))); -// -// Map primaryStoreMap = findStoresBatch(primaryNodesMap, shardsState); - for (ShardRouting unassignedShard : shardsEligibleForFetch) { if (!shardsState.hasData()) { logger.trace("{}: ignoring allocation, still fetching shard stores", unassignedShard); @@ -420,7 +405,7 @@ private static long computeMatchingBytes( /** * Determines if the shard can be allocated on at least one node based on the allocation deciders. - * + *

* Returns the best allocation decision for allocating the shard on any node (i.e. YES if at least one * node decided YES, THROTTLE if at least one node decided THROTTLE, and NO if none of the nodes decided * YES or THROTTLE). If in explain mode, also returns the node-level explanations as the second element @@ -457,24 +442,8 @@ private static Tuple> canBeAllocated return Tuple.tuple(madeDecision, nodeDecisions); } - protected abstract AsyncShardFetch.FetchResult fetchData(ShardRouting shard, RoutingAllocation allocation); - protected abstract boolean hasInitiatedFetching(ShardRouting shard); -// private static Map findStoresBatch(Map shardToNodeMap, -// FetchResult data) { -// Map shardStores = new HashMap<>(); -// shardToNodeMap.entrySet().forEach(entry -> { -// NodeStoreFilesMetadataBatch nodeFilesStore = data.getData().get(entry.getValue()); -// if (nodeFilesStore == null) { -// shardStores.put(entry.getKey(), null); -// } else { -// shardStores.put(entry.getKey(), nodeFilesStore.getNodeStoreFilesMetadataBatch().get(entry.getKey().shardId())); -// } -// }); -// return shardStores; -// } - private static TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata findStore( DiscoveryNode node, FetchResult data, From e3354b4b66c502477f317c1bc6a45b0ee5c7e4d0 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Tue, 22 Aug 2023 13:38:27 +0530 Subject: [PATCH 04/15] Removed GatewayAllocator from PR Signed-off-by: Shivansh Arora --- .../opensearch/gateway/GatewayAllocator.java | 427 +----------------- 1 file changed, 6 insertions(+), 421 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java index 6c8a042e6b85d..5a20112b19219 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java @@ -42,29 +42,21 @@ 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.util.concurrent.ConcurrentCollections; import org.opensearch.common.util.set.Sets; import org.opensearch.common.lease.Releasables; import org.opensearch.core.index.shard.ShardId; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; -import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; -import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch; import java.util.Collections; -import java.util.HashMap; -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; @@ -81,45 +73,29 @@ 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; private final PrimaryShardAllocator primaryShardAllocator; private final ReplicaShardAllocator replicaShardAllocator; - private final PrimaryShardBatchAllocator primaryBatchShardAllocator; - private final ReplicaShardBatchAllocator replicaBatchShardAllocator; - private final TransportNodesListGatewayStartedShardsBatch 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(); - 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( RerouteService rerouteService, TransportNodesListGatewayStartedShards startedAction, - TransportNodesListShardStoreMetadata storeAction, - TransportNodesListGatewayStartedShardsBatch batchStartedAction, - TransportNodesListShardStoreMetadataBatch batchStoreAction + 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(); } @Override @@ -128,10 +104,6 @@ public void cleanCaches() { asyncFetchStarted.clear(); Releasables.close(asyncFetchStore.values()); asyncFetchStore.clear(); - batchIdToStartedShardBatch.clear(); - batchIdToStoreShardBatch.clear(); - startedShardBatchLookup.clear(); - storeShardBatchLookup.clear(); } // for tests @@ -139,10 +111,6 @@ protected GatewayAllocator() { this.rerouteService = null; this.primaryShardAllocator = null; this.replicaShardAllocator = null; - this.batchStartedAction = null; - this.primaryBatchShardAllocator = null; - this.batchStoreAction = null; - this.replicaBatchShardAllocator = null; } @Override @@ -162,7 +130,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())); - safelyRemoveShardFromBatch(startedShard); } } @@ -171,7 +138,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())); - safelyRemoveShardFromBatch(failedShard.getRoutingEntry()); } } @@ -179,30 +145,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) { - // ToDo: fetch from settings - boolean batchMode = true; - if (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); } } @@ -217,99 +168,6 @@ public void allocateUnassigned( innerAllocatedUnassigned(allocation, primaryShardAllocator, replicaShardAllocator, shardRouting, unassignedAllocationHandler); } - @Override - public void allocateUnassignedBatch(final RoutingAllocation allocation, boolean primary) { - // create batches for unassigned shards - createBatches(allocation, primary); - - assert primaryBatchShardAllocator != null; - assert replicaBatchShardAllocator != null; - if (primary) { - 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 = 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.shardId()) == false) && (shardRouting.primary() == primary)) { - assert shardRouting.unassigned(); - 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) { - 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, 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 shard to be removed - */ - 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, @@ -358,13 +216,7 @@ private void ensureAsyncFetchStorePrimaryRecency(RoutingAllocation allocation) { Sets.difference(newEphemeralIds, lastSeenEphemeralIds) ) ); - asyncFetchStore.values().forEach(fetch -> clearCacheForPrimary(fetch, allocation)); - storeShardBatchLookup.values().forEach(batch -> - clearCacheForBatchPrimary(batchIdToStoreShardBatch.get(batch), allocation) - ); - - // recalc to also (lazily) clear out old nodes. this.lastSeenEphemeralIds = newEphemeralIds; } @@ -380,18 +232,6 @@ private static void clearCacheForPrimary( } } - private static void clearCacheForBatchPrimary( - ShardsBatch batch, - RoutingAllocation allocation - ) { - List primaries = batch.getBatchedShards().stream() - .map(allocation.routingNodes()::activePrimary) - .filter(Objects::nonNull) - .collect(Collectors.toList()); - AsyncBatchShardFetch 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) { @@ -428,32 +268,6 @@ 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; @@ -489,59 +303,6 @@ 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 AsyncBatchShardFetch.FetchResult<>(null, Collections.emptyMap()); - } - - String batchId = startedShardBatchLookup.getOrDefault(shardRouting.shardId(), null); - 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(GatewayAllocator.this::safelyRemoveShardFromBatch); - - if (shardsBatch.getBatchedShards().isEmpty() && shardsEligibleForFetch.isEmpty()) { - logger.debug("Batch {} is empty", batchId); - return new AsyncBatchShardFetch.FetchResult<>(null, Collections.emptyMap()); - } - - Map> shardToIgnoreNodes = new HashMap<>(); - - for(ShardId shardId : shardsBatch.asyncBatch.shardsToCustomDataPathMap.keySet()){ - shardToIgnoreNodes.put(shardId, allocation.getIgnoreNodes(shardId)); - } - AsyncBatchShardFetch asyncFetcher = shardsBatch.getAsyncFetcher(); - AsyncBatchShardFetch.FetchResult shardBatchState = asyncFetcher.fetchData( - allocation.nodes(), - shardToIgnoreNodes - ); - - if (shardBatchState.hasData()) { - shardBatchState.processAllocation(allocation); - } - return (AsyncBatchShardFetch.FetchResult) shardBatchState; - } - - } - class InternalReplicaShardAllocator extends ReplicaShardAllocator { private final TransportNodesListShardStoreMetadata storeAction; @@ -580,180 +341,4 @@ protected boolean hasInitiatedFetching(ShardRouting shard) { return asyncFetchStore.get(shard.shardId()) != null; } } - - class InternalReplicaBatchShardAllocator extends ReplicaShardBatchAllocator { - - @Override - @SuppressWarnings("unchecked") - protected AsyncBatchShardFetch.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 AsyncBatchShardFetch.FetchResult<>(null, Collections.emptyMap()); - } - - String batchId = storeShardBatchLookup.getOrDefault(shardRouting.shardId(), null); - 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(GatewayAllocator.this::safelyRemoveShardFromBatch); - - if (shardsBatch.getBatchedShards().isEmpty() && shardsEligibleForFetch.isEmpty()) { - logger.debug("Batch {} is empty", batchId); - return new AsyncBatchShardFetch.FetchResult<>(null, Collections.emptyMap()); - } - Map> shardToIgnoreNodes = new HashMap<>(); - for (ShardId shardId : shardsBatch.asyncBatch.shardsToCustomDataPathMap.keySet()) { - shardToIgnoreNodes.put(shardId, allocation.getIgnoreNodes(shardId)); - } - AsyncBatchShardFetch asyncFetcher = shardsBatch.getAsyncFetcher(); - AsyncBatchShardFetch.FetchResult shardBatchStores = asyncFetcher.fetchData( - allocation.nodes(), - shardToIgnoreNodes - ); - if (shardBatchStores.hasData()) { - shardBatchStores.processAllocation(allocation); - } - return (AsyncBatchShardFetch.FetchResult) shardBatchStores; - } - - @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. - */ - private class ShardsBatch { - 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 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"; - } - - Set getBatchedShardRoutings() { - return batchInfo.values().stream().map(ShardBatchEntry::getShardRouting).collect(Collectors.toSet()); - } - - Set getBatchedShards() { - return batchInfo.keySet(); - } - - public String getBatchId() { - 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 4facee856b5e1c574297d57f4f9dbb883790a060 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Thu, 24 Aug 2023 03:36:17 +0530 Subject: [PATCH 05/15] Added RSBA UTs Signed-off-by: Shivansh Arora --- .../gateway/ReplicaShardBatchAllocator.java | 5 +- .../ReplicaShardBatchAllocatorTest.java | 746 ++++++++++++++++++ 2 files changed, 749 insertions(+), 2 deletions(-) create mode 100644 server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTest.java diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java index 1d49b6e122477..65b714257c274 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java @@ -224,6 +224,7 @@ public HashMap makeAllocationDecision( // Note, this is the existing behavior, as exposed in running CorruptFileTest#testNoPrimaryData logger.trace("{}: no primary shard store found or allocated, letting actual allocation figure it out", unassignedShard); shardAllocationDecisions.put(unassignedShard, AllocateUnassignedDecision.NOT_TAKEN); + continue; } // find the matching nodes @@ -257,7 +258,6 @@ public HashMap makeAllocationDecision( ); // we are throttling this, as we have enough other shards to allocate to this node, so ignore it for now shardAllocationDecisions.put(unassignedShard, AllocateUnassignedDecision.throttle(nodeDecisions)); - continue; } else { logger.debug( "[{}][{}]: allocating [{}] to [{}] in order to reuse its unallocated persistent store", @@ -268,8 +268,8 @@ public HashMap makeAllocationDecision( ); // we found a match shardAllocationDecisions.put(unassignedShard, AllocateUnassignedDecision.yes(nodeWithHighestMatch.node(), null, nodeDecisions, true)); - continue; } + continue; } else if (matchingNodes.hasAnyData() == false && unassignedShard.unassignedInfo().isDelayed()) { // if we didn't manage to find *any* data (regardless of matching sizes), and the replica is // unassigned due to a node leaving, so we delay allocation of this replica to see if the @@ -286,6 +286,7 @@ public HashMap makeAllocationDecision( remainingDelayMillis = TimeValue.timeValueNanos(remainingDelayNanos).millis(); } shardAllocationDecisions.put(unassignedShard, AllocateUnassignedDecision.delayed(remainingDelayMillis, totalDelayMillis, nodeDecisions)); + continue; } shardAllocationDecisions.put(unassignedShard, AllocateUnassignedDecision.NOT_TAKEN); diff --git a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTest.java b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTest.java new file mode 100644 index 0000000000000..0999dd9f8467b --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTest.java @@ -0,0 +1,746 @@ +/* + * 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 com.carrotsearch.randomizedtesting.generators.RandomPicks; +import org.junit.Before; +import org.opensearch.Version; +import org.opensearch.cluster.ClusterInfo; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.OpenSearchAllocationTestCase; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.IndexShardRoutingTable; +import org.opensearch.cluster.routing.RecoverySource; +import org.opensearch.cluster.routing.RoutingNode; +import org.opensearch.cluster.routing.RoutingNodes; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.ShardRoutingState; +import org.opensearch.cluster.routing.TestShardRouting; +import org.opensearch.cluster.routing.UnassignedInfo; +import org.opensearch.cluster.routing.allocation.RoutingAllocation; +import org.opensearch.cluster.routing.allocation.decider.AllocationDecider; +import org.opensearch.cluster.routing.allocation.decider.AllocationDeciders; +import org.opensearch.cluster.routing.allocation.decider.Decision; +import org.opensearch.cluster.routing.allocation.decider.SameShardAllocationDecider; +import org.opensearch.common.Nullable; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.set.Sets; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.engine.Engine; +import org.opensearch.index.seqno.ReplicationTracker; +import org.opensearch.index.seqno.RetentionLease; +import org.opensearch.index.store.Store; +import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch; +import org.opensearch.snapshots.SnapshotShardSizeInfo; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +import static java.util.Collections.unmodifiableMap; +import static org.hamcrest.Matchers.*; + +public class ReplicaShardBatchAllocatorTest extends OpenSearchAllocationTestCase { + private static final org.apache.lucene.util.Version MIN_SUPPORTED_LUCENE_VERSION = org.opensearch.Version.CURRENT + .minimumIndexCompatibilityVersion().luceneVersion; + private final ShardId shardId = new ShardId("test", "_na_", 0); + private final DiscoveryNode node1 = newNode("node1"); + private final DiscoveryNode node2 = newNode("node2"); + private final DiscoveryNode node3 = newNode("node3"); + + private TestBatchAllocator testBatchAllocator; + + @Before + public void buildTestAllocator() { + this.testBatchAllocator = new TestBatchAllocator(); + } + + private void allocateAllUnassignedBatch(final RoutingAllocation allocation) { + final RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); + Set shardToBatch = new HashSet<>(); + while (iterator.hasNext()) { + shardToBatch.add(iterator.next()); + } + testBatchAllocator.allocateUnassignedBatch(shardToBatch, allocation); + } + + /** + * Verifies that when we are still fetching data in an async manner, the replica shard moves to ignore unassigned. + */ + public void testNoAsyncFetchData() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); + testBatchAllocator.clean(); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); + assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); + } + + /** + * Verifies that on index creation, we don't fetch data for any shards, but keep the replica shard unassigned to let + * the shard allocator to allocate it. There isn't a copy around to find anyhow. + */ + public void testAsyncFetchWithNoShardOnIndexCreation() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica( + yesAllocationDeciders(), + Settings.EMPTY, + UnassignedInfo.Reason.INDEX_CREATED + ); + testBatchAllocator.clean(); + allocateAllUnassignedBatch(allocation); + assertThat(testBatchAllocator.getFetchDataCalledAndClean(), equalTo(true)); + assertThat(testBatchAllocator.getShardEligibleFetchDataCountAndClean(), equalTo(0)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(1)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).get(0).shardId(), equalTo(shardId)); + } + + /** + * Verifies that for anything but index creation, fetch data ends up being called, since we need to go and try + * and find a better copy for the shard. + */ + public void testAsyncFetchOnAnythingButIndexCreation() { + UnassignedInfo.Reason reason = RandomPicks.randomFrom( + random(), + EnumSet.complementOf(EnumSet.of(UnassignedInfo.Reason.INDEX_CREATED)) + ); + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders(), Settings.EMPTY, reason); + testBatchAllocator.clean(); + allocateAllUnassignedBatch(allocation); + assertThat("failed with reason " + reason, testBatchAllocator.getFetchDataCalledAndClean(), equalTo(true)); + assertThat("failed with reason" + reason, testBatchAllocator.getShardEligibleFetchDataCountAndClean(), equalTo(1)); + } + + /** + * Verifies that when there is a full match (syncId and files) we allocate it to matching node. + */ + public void testSimpleFullMatchAllocation() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); + DiscoveryNode nodeToMatch = randomBoolean() ? node2 : node3; + testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) + .addData(nodeToMatch, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); + assertThat( + allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), + equalTo(nodeToMatch.getId()) + ); + } + + /** + * Verifies that when there is a sync id match but no files match, we allocate it to matching node. + */ + public void testSyncIdMatch() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); + DiscoveryNode nodeToMatch = randomBoolean() ? node2 : node3; + testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) + .addData(nodeToMatch, "MATCH", null, new StoreFileMetadata("file1", 10, "NO_MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); + assertThat( + allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), + equalTo(nodeToMatch.getId()) + ); + } + + /** + * Verifies that when there is no sync id match but files match, we allocate it to matching node. + */ + public void testFileChecksumMatch() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); + DiscoveryNode nodeToMatch = randomBoolean() ? node2 : node3; + testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) + .addData(nodeToMatch, "NO_MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); + assertThat( + allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), + equalTo(nodeToMatch.getId()) + ); + } + + public void testPreferCopyWithHighestMatchingOperations() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); + long retainingSeqNoOnPrimary = randomLongBetween(1, Integer.MAX_VALUE); + long retainingSeqNoForNode2 = randomLongBetween(0, retainingSeqNoOnPrimary - 1); + // Rarely use a seqNo above retainingSeqNoOnPrimary, which could in theory happen when primary fails and comes back quickly. + long retainingSeqNoForNode3 = randomLongBetween(retainingSeqNoForNode2 + 1, retainingSeqNoOnPrimary + 100); + List retentionLeases = Arrays.asList( + newRetentionLease(node1, retainingSeqNoOnPrimary), + newRetentionLease(node2, retainingSeqNoForNode2), + newRetentionLease(node3, retainingSeqNoForNode3) + ); + testBatchAllocator.addData( + node1, + retentionLeases, + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData(node2, "NOT_MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + testBatchAllocator.addData(node3, randomSyncId(), null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); + assertThat( + allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), + equalTo(node3.getId()) + ); + } + + public void testCancelRecoveryIfFoundCopyWithNoopRetentionLease() { + final UnassignedInfo unassignedInfo; + final Set failedNodes; + if (randomBoolean()) { + failedNodes = Collections.emptySet(); + unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.CLUSTER_RECOVERED, null); + } else { + failedNodes = new HashSet<>(randomSubsetOf(Arrays.asList("node-4", "node-5", "node-6"))); + unassignedInfo = new UnassignedInfo( + UnassignedInfo.Reason.ALLOCATION_FAILED, + null, + null, + randomIntBetween(1, 10), + System.nanoTime(), + System.currentTimeMillis(), + false, + UnassignedInfo.AllocationStatus.NO_ATTEMPT, + failedNodes + ); + } + RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders(), unassignedInfo); + long retainingSeqNo = randomLongBetween(1, Long.MAX_VALUE); + testBatchAllocator.addData( + node1, + Arrays.asList(newRetentionLease(node1, retainingSeqNo), newRetentionLease(node3, retainingSeqNo)), + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData(node2, "NO_MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + testBatchAllocator.addData(node3, randomSyncId(), null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + Collection replicaShards = allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED); + Set shardRoutingBatch = new HashSet<>(replicaShards); + List> shardBatchList = Collections.singletonList(new HashSet<>(allocation + .routingNodes() + .shardsWithState(ShardRoutingState.INITIALIZING))); + + testBatchAllocator.processExistingRecoveries(allocation, shardBatchList); + assertThat(allocation.routingNodesChanged(), equalTo(true)); + List unassignedShards = allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED); + assertThat(unassignedShards, hasSize(1)); + assertThat(unassignedShards.get(0).shardId(), equalTo(shardId)); + assertThat(unassignedShards.get(0).unassignedInfo().getNumFailedAllocations(), equalTo(0)); + assertThat(unassignedShards.get(0).unassignedInfo().getFailedNodeIds(), equalTo(failedNodes)); + } + + public void testNotCancellingRecoveryIfCurrentRecoveryHasRetentionLease() { + RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders()); + List peerRecoveryRetentionLeasesOnPrimary = new ArrayList<>(); + long retainingSeqNo = randomLongBetween(1, Long.MAX_VALUE); + peerRecoveryRetentionLeasesOnPrimary.add(newRetentionLease(node1, retainingSeqNo)); + peerRecoveryRetentionLeasesOnPrimary.add(newRetentionLease(node2, randomLongBetween(1, retainingSeqNo))); + if (randomBoolean()) { + peerRecoveryRetentionLeasesOnPrimary.add(newRetentionLease(node3, randomLongBetween(0, retainingSeqNo))); + } + testBatchAllocator.addData( + node1, + peerRecoveryRetentionLeasesOnPrimary, + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData(node2, randomSyncId(), null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + testBatchAllocator.addData(node3, randomSyncId(), null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + testBatchAllocator.processExistingRecoveries(allocation, Collections.singletonList(new HashSet<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING)))); + assertThat(allocation.routingNodesChanged(), equalTo(false)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(0)); + } + + public void testNotCancelIfPrimaryDoesNotHaveValidRetentionLease() { + RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders()); + testBatchAllocator.addData( + node1, + Collections.singletonList(newRetentionLease(node3, randomNonNegativeLong())), + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData(node2, "NOT_MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + testBatchAllocator.addData(node3, "NOT_MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + testBatchAllocator.processExistingRecoveries(allocation, Collections.singletonList(new HashSet<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING)))); + assertThat(allocation.routingNodesChanged(), equalTo(false)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(0)); + } + + public void testIgnoreRetentionLeaseIfCopyIsEmpty() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); + long retainingSeqNo = randomLongBetween(1, Long.MAX_VALUE); + List retentionLeases = new ArrayList<>(); + retentionLeases.add(newRetentionLease(node1, retainingSeqNo)); + retentionLeases.add(newRetentionLease(node2, randomLongBetween(0, retainingSeqNo))); + if (randomBoolean()) { + retentionLeases.add(newRetentionLease(node3, randomLongBetween(0, retainingSeqNo))); + } + testBatchAllocator.addData( + node1, + retentionLeases, + randomSyncId(), + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData(node2, null, null); // has retention lease but store is empty + testBatchAllocator.addData(node3, randomSyncId(), null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); + assertThat( + allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), + equalTo(node3.getId()) + ); + } + + /** + * When we can't find primary data, but still find replica data, we go ahead and keep it unassigned + * to be allocated. This is today behavior, which relies on a primary corruption identified with + * adding a replica and having that replica actually recover and cause the corruption to be identified + * See CorruptFileTest# + */ + public void testNoPrimaryData() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); + testBatchAllocator.addData(node2, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(1)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).get(0).shardId(), equalTo(shardId)); + } + + /** + * Verifies that when there is primary data, but no data at all on other nodes, the shard keeps + * unassigned to be allocated later on. + */ + public void testNoDataForReplicaOnAnyNode() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); + testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(1)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).get(0).shardId(), equalTo(shardId)); + } + + /** + * Verifies that when there is primary data, but no matching data at all on other nodes, the shard keeps + * unassigned to be allocated later on. + */ + public void testNoMatchingFilesForReplicaOnAnyNode() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); + testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) + .addData(node2, "NO_MATCH", null, new StoreFileMetadata("file1", 10, "NO_MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(1)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).get(0).shardId(), equalTo(shardId)); + } + + /** + * When there is no decision or throttle decision across all nodes for the shard, make sure the shard + * moves to the ignore unassigned list. + */ + public void testNoOrThrottleDecidersRemainsInUnassigned() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica( + randomBoolean() ? noAllocationDeciders() : throttleAllocationDeciders() + ); + testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) + .addData(node2, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); + assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); + } + + /** + * Tests when the node to allocate to due to matching is being throttled, we move the shard to ignored + * to wait till throttling on it is done. + */ + public void testThrottleWhenAllocatingToMatchingNode() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica( + new AllocationDeciders( + Arrays.asList( + new TestAllocateDecision(Decision.YES), + new SameShardAllocationDecider( + Settings.EMPTY, + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + ), + new AllocationDecider() { + @Override + public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) { + if (node.node().equals(node2)) { + return Decision.THROTTLE; + } + return Decision.YES; + } + } + ) + ) + ); + testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) + .addData(node2, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); + assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); + } + + public void testDelayedAllocation() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica( + yesAllocationDeciders(), + Settings.builder().put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), TimeValue.timeValueHours(1)).build(), + UnassignedInfo.Reason.NODE_LEFT + ); + testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + if (randomBoolean()) { + // we sometime return empty list of files, make sure we test this as well + testBatchAllocator.addData(node2, null, null); + } + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(false)); + assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); + assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); + + allocation = onePrimaryOnNode1And1Replica( + yesAllocationDeciders(), + Settings.builder().put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), TimeValue.timeValueHours(1)).build(), + UnassignedInfo.Reason.NODE_LEFT + ); + testBatchAllocator.addData(node2, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); + assertThat( + allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), + equalTo(node2.getId()) + ); + } + + public void testCancelRecoveryBetterSyncId() { + RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders()); + testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) + .addData(node2, "NO_MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) + .addData(node3, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + testBatchAllocator.processExistingRecoveries(allocation, Collections.singletonList( + new HashSet<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING)))); + assertThat(allocation.routingNodesChanged(), equalTo(true)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(1)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).get(0).shardId(), equalTo(shardId)); + } + + public void testNotCancellingRecoveryIfSyncedOnExistingRecovery() { + final UnassignedInfo unassignedInfo; + if (randomBoolean()) { + unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.CLUSTER_RECOVERED, null); + } else { + unassignedInfo = new UnassignedInfo( + UnassignedInfo.Reason.ALLOCATION_FAILED, + null, + null, + randomIntBetween(1, 10), + System.nanoTime(), + System.currentTimeMillis(), + false, + UnassignedInfo.AllocationStatus.NO_ATTEMPT, + Collections.singleton("node-4") + ); + } + RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders(), unassignedInfo); + List retentionLeases = new ArrayList<>(); + if (randomBoolean()) { + long retainingSeqNoOnPrimary = randomLongBetween(0, Long.MAX_VALUE); + retentionLeases.add(newRetentionLease(node1, retainingSeqNoOnPrimary)); + if (randomBoolean()) { + retentionLeases.add(newRetentionLease(node2, randomLongBetween(0, retainingSeqNoOnPrimary))); + } + if (randomBoolean()) { + retentionLeases.add(newRetentionLease(node3, randomLongBetween(0, retainingSeqNoOnPrimary))); + } + } + testBatchAllocator.addData( + node1, + retentionLeases, + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData(node2, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + testBatchAllocator.addData(node3, randomSyncId(), null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + testBatchAllocator.processExistingRecoveries(allocation, Collections.singletonList( + new HashSet<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING)) + )); + assertThat(allocation.routingNodesChanged(), equalTo(false)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(0)); + } + + public void testNotCancellingRecovery() { + RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders()); + testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) + .addData(node2, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + testBatchAllocator.processExistingRecoveries(allocation, Collections.singletonList( + new HashSet<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING)) + )); + assertThat(allocation.routingNodesChanged(), equalTo(false)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(0)); + } + + public void testDoNotCancelForBrokenNode() { + Set failedNodes = new HashSet<>(); + failedNodes.add(node3.getId()); + if (randomBoolean()) { + failedNodes.add("node4"); + } + UnassignedInfo unassignedInfo = new UnassignedInfo( + UnassignedInfo.Reason.ALLOCATION_FAILED, + null, + null, + randomIntBetween(failedNodes.size(), 10), + System.nanoTime(), + System.currentTimeMillis(), + false, + UnassignedInfo.AllocationStatus.NO_ATTEMPT, + failedNodes + ); + RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders(), unassignedInfo); + long retainingSeqNoOnPrimary = randomLongBetween(0, Long.MAX_VALUE); + List retentionLeases = Arrays.asList( + newRetentionLease(node1, retainingSeqNoOnPrimary), + newRetentionLease(node3, retainingSeqNoOnPrimary) + ); + testBatchAllocator.addData( + node1, + retentionLeases, + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ) + .addData(node2, randomSyncId(), null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) + .addData(node3, randomSyncId(), null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + testBatchAllocator.processExistingRecoveries(allocation, Collections.singletonList( + new HashSet<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING)) + )); + assertThat(allocation.routingNodesChanged(), equalTo(false)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED), empty()); + } + + private RoutingAllocation onePrimaryOnNode1And1Replica(AllocationDeciders deciders) { + return onePrimaryOnNode1And1Replica(deciders, Settings.EMPTY, UnassignedInfo.Reason.CLUSTER_RECOVERED); + } + + private RoutingAllocation onePrimaryOnNode1And1Replica(AllocationDeciders deciders, Settings settings, UnassignedInfo.Reason reason) { + ShardRouting primaryShard = TestShardRouting.newShardRouting(shardId, node1.getId(), true, ShardRoutingState.STARTED); + IndexMetadata.Builder indexMetadata = IndexMetadata.builder(shardId.getIndexName()) + .settings(settings(Version.CURRENT).put(settings)) + .numberOfShards(1) + .numberOfReplicas(1) + .putInSyncAllocationIds(0, Sets.newHashSet(primaryShard.allocationId().getId())); + Metadata metadata = Metadata.builder().put(indexMetadata).build(); + // mark shard as delayed if reason is NODE_LEFT + boolean delayed = reason == UnassignedInfo.Reason.NODE_LEFT + && UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(settings).nanos() > 0; + int failedAllocations = reason == UnassignedInfo.Reason.ALLOCATION_FAILED ? 1 : 0; + RoutingTable routingTable = RoutingTable.builder() + .add( + IndexRoutingTable.builder(shardId.getIndex()) + .addIndexShard( + new IndexShardRoutingTable.Builder(shardId).addShard(primaryShard) + .addShard( + ShardRouting.newUnassigned( + shardId, + false, + RecoverySource.PeerRecoverySource.INSTANCE, + new UnassignedInfo( + reason, + null, + null, + failedAllocations, + System.nanoTime(), + System.currentTimeMillis(), + delayed, + UnassignedInfo.AllocationStatus.NO_ATTEMPT, + Collections.emptySet() + ) + ) + ) + .build() + ) + ) + .build(); + ClusterState state = ClusterState.builder(org.opensearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTable) + .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) + .build(); + return new RoutingAllocation( + deciders, + new RoutingNodes(state, false), + state, + ClusterInfo.EMPTY, + SnapshotShardSizeInfo.EMPTY, + System.nanoTime() + ); + } + + private RoutingAllocation onePrimaryOnNode1And1ReplicaRecovering(AllocationDeciders deciders, UnassignedInfo unassignedInfo) { + ShardRouting primaryShard = TestShardRouting.newShardRouting(shardId, node1.getId(), true, ShardRoutingState.STARTED); + Metadata metadata = Metadata.builder() + .put( + IndexMetadata.builder(shardId.getIndexName()) + .settings(settings(Version.CURRENT)) + .numberOfShards(1) + .numberOfReplicas(1) + .putInSyncAllocationIds(0, Sets.newHashSet(primaryShard.allocationId().getId())) + ) + .build(); + RoutingTable routingTable = RoutingTable.builder() + .add( + IndexRoutingTable.builder(shardId.getIndex()) + .addIndexShard( + new IndexShardRoutingTable.Builder(shardId).addShard(primaryShard) + .addShard( + TestShardRouting.newShardRouting( + shardId, + node2.getId(), + null, + false, + ShardRoutingState.INITIALIZING, + unassignedInfo + ) + ) + .build() + ) + ) + .build(); + ClusterState state = ClusterState.builder(org.opensearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTable) + .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) + .build(); + return new RoutingAllocation( + deciders, + new RoutingNodes(state, false), + state, + ClusterInfo.EMPTY, + SnapshotShardSizeInfo.EMPTY, + System.nanoTime() + ); + } + + private RoutingAllocation onePrimaryOnNode1And1ReplicaRecovering(AllocationDeciders deciders) { + return onePrimaryOnNode1And1ReplicaRecovering(deciders, new UnassignedInfo(UnassignedInfo.Reason.CLUSTER_RECOVERED, null)); + } + + static RetentionLease newRetentionLease(DiscoveryNode node, long retainingSeqNo) { + return new RetentionLease( + ReplicationTracker.getPeerRecoveryRetentionLeaseId(node.getId()), + retainingSeqNo, + randomNonNegativeLong(), + ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE + ); + } + + static String randomSyncId() { + return randomFrom("MATCH", "NOT_MATCH", null); + } + + class TestBatchAllocator extends ReplicaShardBatchAllocator { + private Map data = null; + private AtomicBoolean fetchDataCalled = new AtomicBoolean(false); + private AtomicInteger eligibleShardFetchDataCount = new AtomicInteger(0); + + public void clean() { + data = null; + } + + public boolean getFetchDataCalledAndClean() { + return fetchDataCalled.getAndSet(false); + } + + public int getShardEligibleFetchDataCountAndClean() { + return eligibleShardFetchDataCount.getAndSet(0); + } + + public TestBatchAllocator addData(DiscoveryNode node, String syncId, @Nullable Exception storeFileFetchException, StoreFileMetadata ...files) { + return addData(node, Collections.emptyList(), syncId, storeFileFetchException, files); + } + + public TestBatchAllocator addData( + DiscoveryNode node, + List peerRecoveryRetentionLeases, + String syncId, + @Nullable Exception storeFileFetchException, + StoreFileMetadata ...files) { + if (data == null) { + data = new HashMap<>(); + } + Map filesAsMap = new HashMap<>(); + for (StoreFileMetadata file : files) { + filesAsMap.put(file.name(), file); + } + Map commitData = new HashMap<>(); + if (syncId != null) { + commitData.put(Engine.SYNC_COMMIT_ID, syncId); + } + data.put( + node, + new TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata( + new TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata( + shardId, + new Store.MetadataSnapshot(unmodifiableMap(filesAsMap), unmodifiableMap(commitData), randomInt()), + peerRecoveryRetentionLeases + ), + storeFileFetchException + ) + ); + return this; + } + + @Override + protected AsyncBatchShardFetch.FetchResult fetchData(Set shardEligibleForFetch, Set inEligibleShards, RoutingAllocation allocation) { + fetchDataCalled.set(true); + eligibleShardFetchDataCount.set(shardEligibleForFetch.size()); + Map tData = null; + if (data != null) { + tData = new HashMap<>(); + for (Map.Entry entry : data.entrySet()) { + Map shardData = Map.of( + shardId, + entry.getValue() + ); + tData.put( + entry.getKey(), + new TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch(entry.getKey(), shardData) + ); + } + } + return new AsyncBatchShardFetch.FetchResult<>(tData, Collections.>emptyMap()); + } + + @Override + protected boolean hasInitiatedFetching(ShardRouting shard) { + return fetchDataCalled.get(); + } + } +} From 11cc223896aae9bf8cb46b848e8083f6154ebdb6 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Wed, 4 Oct 2023 05:10:52 +0530 Subject: [PATCH 06/15] RSBA after RSA refactor Signed-off-by: Shivansh Arora --- .../gateway/ReplicaShardBatchAllocator.java | 409 ++---------------- .../ReplicaShardBatchAllocatorTest.java | 5 +- 2 files changed, 51 insertions(+), 363 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java index 65b714257c274..f0cc831a4af0f 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java @@ -9,7 +9,6 @@ package org.opensearch.gateway; import org.apache.logging.log4j.Logger; -import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.routing.RoutingNode; @@ -21,11 +20,9 @@ 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.common.unit.ByteSizeValue; -import org.opensearch.common.unit.TimeValue; -import org.opensearch.gateway.AsyncBatchShardFetch.FetchResult; -import org.opensearch.index.store.StoreFileMetadata; -import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; +import org.opensearch.gateway.AsyncShardFetch.FetchResult; +import org.opensearch.indices.store.StoreFilesMetadata; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch; import java.util.ArrayList; @@ -35,10 +32,14 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; -import static org.opensearch.cluster.routing.UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING; - -public abstract class ReplicaShardBatchAllocator extends BaseGatewayShardAllocator { +/** + * 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 @@ -46,29 +47,28 @@ public abstract class ReplicaShardBatchAllocator extends BaseGatewayShardAllocat * has to copy segment files. */ public void processExistingRecoveries(RoutingAllocation allocation, List> shardBatches) { - Metadata metadata = allocation.metadata(); RoutingNodes routingNodes = allocation.routingNodes(); List shardCancellationActions = new ArrayList<>(); for (Set shardBatch : shardBatches) { Set eligibleFetchShards = new HashSet<>(); Set ineligibleShards = new HashSet<>(); for (ShardRouting shard : shardBatch) { - if (shard.primary()) { - ineligibleShards.add(shard); - continue; - } - if (shard.initializing() == false) { - continue; - } - if (shard.relocatingNodeId() != null) { - continue; - } - - // if we are allocating a replica because of index creation, no need to go and find a copy, there isn't one... - if (shard.unassignedInfo() != null && shard.unassignedInfo().getReason() == UnassignedInfo.Reason.INDEX_CREATED) { - continue; + shardMatched = false; + for (RoutingNode routingNode : routingNodes) { + if (routingNode.getByShardId(shard.shardId()) != null) { + ShardRouting shardFromRoutingNode = routingNode.getByShardId(shard.shardId()); + if (!shardFromRoutingNode.primary()){ + shardMatched = true; + if (shouldSkipFetchForRecovery(shardFromRoutingNode)) { + continue; + } + eligibleFetchShards.add(shardFromRoutingNode); + } + } + if (shardMatched){ + break; + } } - eligibleFetchShards.add(shard); } AsyncBatchShardFetch.FetchResult shardState = fetchData(eligibleFetchShards, ineligibleShards, allocation); if (shardState.hasData() == false) { @@ -76,60 +76,11 @@ public void processExistingRecoveries(RoutingAllocation allocation, List failedNodeIds = shard.unassignedInfo() == null - ? Collections.emptySet() - : shard.unassignedInfo().getFailedNodeIds(); - UnassignedInfo unassignedInfo = new UnassignedInfo( - UnassignedInfo.Reason.REALLOCATED_REPLICA, - "existing allocation of replica to [" - + currentNode - + "] cancelled, can perform a noop recovery on [" - + nodeWithHighestMatch - + "]", - null, - 0, - allocation.getCurrentNanoTime(), - System.currentTimeMillis(), - false, - UnassignedInfo.AllocationStatus.NO_ATTEMPT, - failedNodeIds - ); - // don't cancel shard in the loop as it will cause a ConcurrentModificationException - shardCancellationActions.add( - () -> routingNodes.failShard( - logger, - shard, - unassignedInfo, - metadata.getIndexSafe(shard.index()), - allocation.changes() - ) - ); - } + Map nodeShardStores = getNodeShardStores(shard, shardState); + + Runnable cancellationAction = getShardCancellationAction(shard, allocation, nodeShardStores); + if (cancellationAction != null) { + shardCancellationActions.add(cancellationAction); } } } @@ -138,17 +89,16 @@ && canPerformOperationBasedRecovery(primaryStore, shardState, currentNode, shard } } - private static boolean isResponsibleFor(final ShardRouting shard) { - return !shard.primary() // must be a replica - && shard.unassigned() // must be unassigned - // if we are allocating a replica because of index creation, no need to go and find a copy, there isn't one... - && shard.unassignedInfo().getReason() != UnassignedInfo.Reason.INDEX_CREATED; - } abstract protected FetchResult fetchData(Set shardEligibleForFetch, Set inEligibleShards, RoutingAllocation allocation); + @Override + protected FetchResult fetchData(ShardRouting shard, RoutingAllocation allocation) { + return null; + } + @Override public AllocateUnassignedDecision makeAllocationDecision(ShardRouting unassignedShard, RoutingAllocation allocation, Logger logger) { return null; @@ -188,6 +138,10 @@ public HashMap makeAllocationDecision( shardsEligibleForFetch.add(shard); } + // Do not call fetchData if there are no eligible shards + if (shardsEligibleForFetch.isEmpty()) { + return shardAllocationDecisions; + } // only fetch data for eligible shards final FetchResult shardsState = fetchData(shardsEligibleForFetch, shardsNotEligibleForFetch, allocation); @@ -204,285 +158,18 @@ public HashMap makeAllocationDecision( continue; } Tuple> result = nodeAllocationDecisions.get(unassignedShard); - ShardRouting primaryShard = routingNodes.activePrimary(unassignedShard.shardId()); - if (primaryShard == null) { - assert explain : "primary should only be null here if we are in explain mode, so we didn't " - + "exit early when canBeAllocatedToAtLeastOneNode didn't return a YES decision"; - shardAllocationDecisions.put(unassignedShard, AllocateUnassignedDecision.no( - UnassignedInfo.AllocationStatus.fromDecision(result.v1().type()), - result.v2() != null ? new ArrayList<>(result.v2().values()) : null - )); - continue; - } - assert primaryShard.currentNodeId() != null; - final DiscoveryNode primaryNode = allocation.nodes().get(primaryShard.currentNodeId()); - final TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata primaryStore = findStore(primaryNode, shardsState, unassignedShard); - if (primaryStore == null) { - // if we can't find the primary data, it is probably because the primary shard is corrupted (and listing failed) - // we want to let the replica be allocated in order to expose the actual problem with the primary that the replica - // will try and recover from - // Note, this is the existing behavior, as exposed in running CorruptFileTest#testNoPrimaryData - logger.trace("{}: no primary shard store found or allocated, letting actual allocation figure it out", unassignedShard); - shardAllocationDecisions.put(unassignedShard, AllocateUnassignedDecision.NOT_TAKEN); - continue; - } - - // find the matching nodes - ReplicaShardAllocator.MatchingNodes matchingNodes = findMatchingNodes( - unassignedShard, - allocation, - false, - primaryNode, - primaryStore, - shardsState, - explain - ); - - assert explain == false || matchingNodes.getNodeDecisions() != null : "in explain mode, we must have individual node decisions"; - - List nodeDecisions = ReplicaShardAllocator.augmentExplanationsWithStoreInfo(result.v2(), matchingNodes.getNodeDecisions()); - if (result.v1().type() != Decision.Type.YES) { - shardAllocationDecisions.put(unassignedShard, AllocateUnassignedDecision.no(UnassignedInfo.AllocationStatus.fromDecision(result.v1().type()), nodeDecisions)); - continue; - } else if (matchingNodes.getNodeWithHighestMatch() != null) { - RoutingNode nodeWithHighestMatch = allocation.routingNodes().node(matchingNodes.getNodeWithHighestMatch().getId()); - // we only check on THROTTLE since we checked before on NO - Decision decision = allocation.deciders().canAllocate(unassignedShard, nodeWithHighestMatch, allocation); - if (decision.type() == Decision.Type.THROTTLE) { - logger.debug( - "[{}][{}]: throttling allocation [{}] to [{}] in order to reuse its unallocated persistent store", - unassignedShard.index(), - unassignedShard.id(), - unassignedShard, - nodeWithHighestMatch.node() - ); - // we are throttling this, as we have enough other shards to allocate to this node, so ignore it for now - shardAllocationDecisions.put(unassignedShard, AllocateUnassignedDecision.throttle(nodeDecisions)); - } else { - logger.debug( - "[{}][{}]: allocating [{}] to [{}] in order to reuse its unallocated persistent store", - unassignedShard.index(), - unassignedShard.id(), - unassignedShard, - nodeWithHighestMatch.node() - ); - // we found a match - shardAllocationDecisions.put(unassignedShard, AllocateUnassignedDecision.yes(nodeWithHighestMatch.node(), null, nodeDecisions, true)); - } - continue; - } else if (matchingNodes.hasAnyData() == false && unassignedShard.unassignedInfo().isDelayed()) { - // if we didn't manage to find *any* data (regardless of matching sizes), and the replica is - // unassigned due to a node leaving, so we delay allocation of this replica to see if the - // node with the shard copy will rejoin so we can re-use the copy it has - logger.debug("{}: allocation of [{}] is delayed", unassignedShard.shardId(), unassignedShard); - long remainingDelayMillis = 0L; - long totalDelayMillis = 0L; - if (explain) { - UnassignedInfo unassignedInfo = unassignedShard.unassignedInfo(); - Metadata metadata = allocation.metadata(); - IndexMetadata indexMetadata = metadata.index(unassignedShard.index()); - totalDelayMillis = INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(indexMetadata.getSettings()).getMillis(); - long remainingDelayNanos = unassignedInfo.getRemainingDelay(System.nanoTime(), indexMetadata.getSettings()); - remainingDelayMillis = TimeValue.timeValueNanos(remainingDelayNanos).millis(); - } - shardAllocationDecisions.put(unassignedShard, AllocateUnassignedDecision.delayed(remainingDelayMillis, totalDelayMillis, nodeDecisions)); - continue; - } - - shardAllocationDecisions.put(unassignedShard, AllocateUnassignedDecision.NOT_TAKEN); + shardAllocationDecisions.put(unassignedShard, getAllocationDecision(unassignedShard, allocation, getNodeShardStores(unassignedShard, shardsState), result, logger)); } return shardAllocationDecisions; } - private ReplicaShardAllocator.MatchingNodes findMatchingNodes( - ShardRouting shard, - RoutingAllocation allocation, - boolean noMatchFailedNodes, - DiscoveryNode primaryNode, - TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata primaryStore, - FetchResult data, - boolean explain - ) { - Map matchingNodes = new HashMap<>(); - Map nodeDecisions = explain ? new HashMap<>() : null; - for (Map.Entry nodeStoreEntry : data.getData().entrySet()) { - DiscoveryNode discoNode = nodeStoreEntry.getKey(); - if (noMatchFailedNodes - && shard.unassignedInfo() != null - && shard.unassignedInfo().getFailedNodeIds().contains(discoNode.getId())) { - continue; - } - TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata storeFilesMetadata = nodeStoreEntry.getValue() - .getNodeStoreFilesMetadataBatch().get(shard.shardId()).storeFilesMetadata(); - // we don't have any files at all, it is an empty index - if (storeFilesMetadata.isEmpty()) { - continue; - } - - RoutingNode node = allocation.routingNodes().node(discoNode.getId()); - if (node == null) { - continue; - } - - // check if we can allocate on that node... - // we only check for NO, since if this node is THROTTLING and it has enough "same data" - // then we will try and assign it next time - Decision decision = allocation.deciders().canAllocate(shard, node, allocation); - ReplicaShardAllocator.MatchingNode matchingNode = null; - if (explain) { - matchingNode = computeMatchingNode(primaryNode, primaryStore, discoNode, storeFilesMetadata); - NodeAllocationResult.ShardStoreInfo shardStoreInfo = new NodeAllocationResult.ShardStoreInfo(matchingNode.matchingBytes); - nodeDecisions.put(node.nodeId(), new NodeAllocationResult(discoNode, shardStoreInfo, decision)); - } - - if (decision.type() == Decision.Type.NO) { - continue; - } - - if (matchingNode == null) { - matchingNode = computeMatchingNode(primaryNode, primaryStore, discoNode, storeFilesMetadata); - } - matchingNodes.put(discoNode, matchingNode); - if (logger.isTraceEnabled()) { - if (matchingNode.isNoopRecovery) { - logger.trace("{}: node [{}] can perform a noop recovery", shard, discoNode.getName()); - } else if (matchingNode.retainingSeqNo >= 0) { - logger.trace( - "{}: node [{}] can perform operation-based recovery with retaining sequence number [{}]", - shard, - discoNode.getName(), - matchingNode.retainingSeqNo - ); - } else { - logger.trace( - "{}: node [{}] has [{}/{}] bytes of re-usable data", - shard, - discoNode.getName(), - new ByteSizeValue(matchingNode.matchingBytes), - matchingNode.matchingBytes - ); - } - } - } - - return new ReplicaShardAllocator.MatchingNodes(matchingNodes, nodeDecisions); - } - - private static ReplicaShardAllocator.MatchingNode computeMatchingNode( - DiscoveryNode primaryNode, - TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata primaryStore, - DiscoveryNode replicaNode, - TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata replicaStore - ) { - final long retainingSeqNoForPrimary = primaryStore.getPeerRecoveryRetentionLeaseRetainingSeqNo(primaryNode); - final long retainingSeqNoForReplica = primaryStore.getPeerRecoveryRetentionLeaseRetainingSeqNo(replicaNode); - final boolean isNoopRecovery = (retainingSeqNoForReplica >= retainingSeqNoForPrimary && retainingSeqNoForPrimary >= 0) - || hasMatchingSyncId(primaryStore, replicaStore); - final long matchingBytes = computeMatchingBytes(primaryStore, replicaStore); - return new ReplicaShardAllocator.MatchingNode(matchingBytes, retainingSeqNoForReplica, isNoopRecovery); - } - - private static boolean hasMatchingSyncId( - TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata primaryStore, - TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata replicaStore - ) { - String primarySyncId = primaryStore.syncId(); - return primarySyncId != null && primarySyncId.equals(replicaStore.syncId()); - } - - private static long computeMatchingBytes( - TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata primaryStore, - TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata storeFilesMetadata - ) { - long sizeMatched = 0; - for (StoreFileMetadata storeFileMetadata : storeFilesMetadata) { - String metadataFileName = storeFileMetadata.name(); - if (primaryStore.fileExists(metadataFileName) && primaryStore.file(metadataFileName).isSame(storeFileMetadata)) { - sizeMatched += storeFileMetadata.length(); - } - } - return sizeMatched; - } - - /** - * Determines if the shard can be allocated on at least one node based on the allocation deciders. - *

- * Returns the best allocation decision for allocating the shard on any node (i.e. YES if at least one - * node decided YES, THROTTLE if at least one node decided THROTTLE, and NO if none of the nodes decided - * YES or THROTTLE). If in explain mode, also returns the node-level explanations as the second element - * in the returned tuple. - */ - private static Tuple> canBeAllocatedToAtLeastOneNode( - ShardRouting shard, - RoutingAllocation allocation - ) { - Decision madeDecision = Decision.NO; - final boolean explain = allocation.debugDecision(); - Map nodeDecisions = explain ? new HashMap<>() : null; - for (final DiscoveryNode cursor : allocation.nodes().getDataNodes().values()) { - RoutingNode node = allocation.routingNodes().node(cursor.getId()); - if (node == null) { - continue; - } - // if we can't allocate it on a node, ignore it, for example, this handles - // cases for only allocating a replica after a primary - Decision decision = allocation.deciders().canAllocate(shard, node, allocation); - if (decision.type() == Decision.Type.YES && madeDecision.type() != Decision.Type.YES) { - if (explain) { - madeDecision = decision; - } else { - return Tuple.tuple(decision, null); - } - } else if (madeDecision.type() == Decision.Type.NO && decision.type() == Decision.Type.THROTTLE) { - madeDecision = decision; - } - if (explain) { - nodeDecisions.put(node.nodeId(), new NodeAllocationResult(node.node(), null, decision)); - } - } - return Tuple.tuple(madeDecision, nodeDecisions); - } - - protected abstract boolean hasInitiatedFetching(ShardRouting shard); - - private static TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata findStore( - DiscoveryNode node, - FetchResult data, - ShardRouting shard - ) { - NodeStoreFilesMetadataBatch nodeFilesStore = data.getData().get(node); - if (nodeFilesStore == null) { - return null; - } - TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeFileStoreMetadata = nodeFilesStore.getNodeStoreFilesMetadataBatch().get(shard.shardId()); - if (nodeFileStoreMetadata.getStoreFileFetchException() != null) { - // Do we need to throw an exception here, to handle this case differently? - return null; - } - return nodeFileStoreMetadata.storeFilesMetadata(); - } - - private static boolean canPerformOperationBasedRecovery( - TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata primaryStore, - FetchResult data, - DiscoveryNode targetNode, - ShardRouting shard - ) { - final NodeStoreFilesMetadataBatch nodeFilesStore = data.getData().get(targetNode); - if (nodeFilesStore == null) { - return false; - } - TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeFileStoreMetadata = nodeFilesStore.getNodeStoreFilesMetadataBatch().get(shard.shardId()); - if (nodeFileStoreMetadata.getStoreFileFetchException() != null) { - return false; - } - TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata targetNodeStore = nodeFileStoreMetadata.storeFilesMetadata(); - if (targetNodeStore == null || targetNodeStore.isEmpty()) { - return false; - } - if (hasMatchingSyncId(primaryStore, targetNodeStore)) { - return true; - } - return primaryStore.getPeerRecoveryRetentionLeaseRetainingSeqNo(targetNode) >= 0; + private Map getNodeShardStores(ShardRouting unassignedShard, FetchResult data) { + assert data.hasData(); + return new HashMap<>( + data.getData().entrySet().stream().collect(Collectors.toMap( + Map.Entry::getKey, + entry -> entry.getValue().getNodeStoreFilesMetadataBatch().get(unassignedShard.shardId()).storeFilesMetadata() + )) + ); } } diff --git a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTest.java b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTest.java index 0999dd9f8467b..8563869682d89 100644 --- a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTest.java +++ b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTest.java @@ -44,6 +44,7 @@ import org.opensearch.index.seqno.RetentionLease; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.indices.store.StoreFilesMetadata; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch; import org.opensearch.snapshots.SnapshotShardSizeInfo; @@ -112,7 +113,7 @@ public void testAsyncFetchWithNoShardOnIndexCreation() { ); testBatchAllocator.clean(); allocateAllUnassignedBatch(allocation); - assertThat(testBatchAllocator.getFetchDataCalledAndClean(), equalTo(true)); + assertThat(testBatchAllocator.getFetchDataCalledAndClean(), equalTo(false)); assertThat(testBatchAllocator.getShardEligibleFetchDataCountAndClean(), equalTo(0)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(1)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).get(0).shardId(), equalTo(shardId)); @@ -706,7 +707,7 @@ public TestBatchAllocator addData( data.put( node, new TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata( - new TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata( + new StoreFilesMetadata( shardId, new Store.MetadataSnapshot(unmodifiableMap(filesAsMap), unmodifiableMap(commitData), randomInt()), peerRecoveryRetentionLeases From d8a29d600a098659e7038c929797d3ad90e1e4b1 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Wed, 4 Oct 2023 05:17:12 +0530 Subject: [PATCH 07/15] Made some funcitons private Signed-off-by: Shivansh Arora --- .../java/org/opensearch/gateway/ReplicaShardAllocator.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java index c8d335c698a13..36b0da9867b35 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java @@ -336,7 +336,7 @@ private static Tuple> canBeAllocated * Takes the store info for nodes that have a shard store and adds them to the node decisions, * leaving the node explanations untouched for those nodes that do not have any store information. */ - public static List augmentExplanationsWithStoreInfo( + private static List augmentExplanationsWithStoreInfo( Map nodeDecisions, Map withShardStores ) { @@ -499,7 +499,7 @@ private static boolean canPerformOperationBasedRecovery( */ protected abstract boolean hasInitiatedFetching(ShardRouting shard); - public static class MatchingNode { + private static class MatchingNode { static final Comparator COMPARATOR = Comparator.comparing(m -> m.isNoopRecovery) .thenComparing(m -> m.retainingSeqNo) .thenComparing(m -> m.matchingBytes); From 30c57e0ca0d554b8b399206d1a84fcc8c7b5bed9 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Mon, 5 Feb 2024 18:26:19 +0530 Subject: [PATCH 08/15] Use List instead of Set, adding more comments Signed-off-by: Shivansh Arora --- .../gateway/ReplicaShardBatchAllocator.java | 153 +++++++++----- .../ReplicaShardBatchAllocatorTest.java | 198 +++++++++++++----- 2 files changed, 243 insertions(+), 108 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java index f0cc831a4af0f..e51e19827ccbb 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java @@ -20,10 +20,13 @@ 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.StoreFilesMetadata; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; +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; @@ -45,38 +48,48 @@ 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) { + public void processExistingRecoveries(RoutingAllocation allocation, List> shardBatches) { RoutingNodes routingNodes = allocation.routingNodes(); List shardCancellationActions = new ArrayList<>(); - for (Set shardBatch : shardBatches) { - Set eligibleFetchShards = new HashSet<>(); + // 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) { + Set eligibleShards = new HashSet<>(); Set ineligibleShards = new HashSet<>(); + boolean shardMatched; + // iterate over shards to check for match for each of those for (ShardRouting shard : shardBatch) { shardMatched = false; + // need to iterate over all the nodes to find matching shard for (RoutingNode routingNode : routingNodes) { - if (routingNode.getByShardId(shard.shardId()) != null) { - ShardRouting shardFromRoutingNode = routingNode.getByShardId(shard.shardId()); - if (!shardFromRoutingNode.primary()){ - shardMatched = true; - if (shouldSkipFetchForRecovery(shardFromRoutingNode)) { - continue; - } - eligibleFetchShards.add(shardFromRoutingNode); + ShardRouting shardFromRoutingNode = routingNode.getByShardId(shard.shardId()); + if (shardFromRoutingNode != null && !shardFromRoutingNode.primary()) { + shardMatched = true; + if (shouldSkipFetchForRecovery(shardFromRoutingNode)) { + ineligibleShards.add(shardFromRoutingNode); + continue; } + eligibleShards.add(shardFromRoutingNode); } - if (shardMatched){ + if (shardMatched) { break; } } } - AsyncBatchShardFetch.FetchResult shardState = fetchData(eligibleFetchShards, ineligibleShards, allocation); - if (shardState.hasData() == false) { - logger.trace("{}: fetching new stores for initializing shard batch", eligibleFetchShards); + 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 : eligibleFetchShards) { - Map nodeShardStores = getNodeShardStores(shard, shardState); + for (ShardRouting shard : eligibleShards) { + Map nodeShardStores = convertToNodeStoreFilesMetadataMap(shard, shardState); Runnable cancellationAction = getShardCancellationAction(shard, allocation, nodeShardStores); if (cancellationAction != null) { @@ -89,33 +102,41 @@ public void processExistingRecoveries(RoutingAllocation allocation, List fetchData(Set shardEligibleForFetch, - Set inEligibleShards, - RoutingAllocation allocation); + abstract protected FetchResult fetchData( + Set eligibleShards, + Set ineligibleShards, + RoutingAllocation allocation + ); @Override - protected FetchResult fetchData(ShardRouting shard, RoutingAllocation allocation) { - return null; + 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 null; + return makeAllocationDecision(Collections.singletonList(unassignedShard), allocation, logger).get(unassignedShard); } @Override - public HashMap makeAllocationDecision(Set shards, RoutingAllocation allocation, Logger logger) { + public HashMap makeAllocationDecision( + List shards, + RoutingAllocation allocation, + Logger logger + ) { HashMap shardAllocationDecisions = new HashMap<>(); final boolean explain = allocation.debugDecision(); - final RoutingNodes routingNodes = allocation.routingNodes(); - Set shardsEligibleForFetch = new HashSet<>(); - Set shardsNotEligibleForFetch = new HashSet<>(); + Set eligibleShards = new HashSet<>(); + Set ineligibleShards = new HashSet<>(); HashMap>> nodeAllocationDecisions = new HashMap<>(); for (ShardRouting shard : shards) { if (!isResponsibleFor(shard)) { // this allocator n is not responsible for allocating this shard - shardsNotEligibleForFetch.add(shard); + ineligibleShards.add(shard); shardAllocationDecisions.put(shard, AllocateUnassignedDecision.NOT_TAKEN); continue; } @@ -126,50 +147,66 @@ public HashMap makeAllocationDecision( // 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)); + 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); - shardsEligibleForFetch.add(shard); + eligibleShards.add(shard); } // Do not call fetchData if there are no eligible shards - if (shardsEligibleForFetch.isEmpty()) { + if (eligibleShards.isEmpty()) { return shardAllocationDecisions; } // only fetch data for eligible shards - final FetchResult shardsState = fetchData(shardsEligibleForFetch, shardsNotEligibleForFetch, allocation); - - for (ShardRouting unassignedShard : shardsEligibleForFetch) { - if (!shardsState.hasData()) { - logger.trace("{}: ignoring allocation, still fetching shard stores", unassignedShard); - allocation.setHasPendingAsyncFetch(); - List nodeDecisions = null; - if (explain) { - nodeDecisions = buildDecisionsForAllNodes(unassignedShard, allocation); - } - shardAllocationDecisions.put(unassignedShard, - AllocateUnassignedDecision.no(UnassignedInfo.AllocationStatus.FETCHING_SHARD_DATA, nodeDecisions)); - continue; - } + final FetchResult shardsState = fetchData( + eligibleShards, + ineligibleShards, + allocation + ); + + for (ShardRouting unassignedShard : eligibleShards) { Tuple> result = nodeAllocationDecisions.get(unassignedShard); - shardAllocationDecisions.put(unassignedShard, getAllocationDecision(unassignedShard, allocation, getNodeShardStores(unassignedShard, shardsState), result, logger)); + shardAllocationDecisions.put( + unassignedShard, + getAllocationDecision( + unassignedShard, + allocation, + convertToNodeStoreFilesMetadataMap(unassignedShard, shardsState), + result, + logger + ) + ); } return shardAllocationDecisions; } + private Map convertToNodeStoreFilesMetadataMap( + ShardRouting unassignedShard, + FetchResult data) { - private Map getNodeShardStores(ShardRouting unassignedShard, FetchResult data) { - assert data.hasData(); - return new HashMap<>( - data.getData().entrySet().stream().collect(Collectors.toMap( - Map.Entry::getKey, - entry -> entry.getValue().getNodeStoreFilesMetadataBatch().get(unassignedShard.shardId()).storeFilesMetadata() - )) - ); + if(!data.hasData()) { + return new HashMap<>(); + } + + Map map = new HashMap<>(); + + data.getData().forEach((key, value) -> { + Map batch = value.getNodeStoreFilesMetadataBatch(); + NodeStoreFilesMetadata metadata = batch.get(unassignedShard.shardId()); + if (metadata != null && metadata.getStoreFileFetchException() == null) { + map.put(key, metadata.storeFilesMetadata()); + } + }); + + return map; } } diff --git a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTest.java b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTest.java index 8563869682d89..c76d0e5449bd7 100644 --- a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTest.java +++ b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTest.java @@ -9,7 +9,7 @@ package org.opensearch.gateway; import com.carrotsearch.randomizedtesting.generators.RandomPicks; -import org.junit.Before; + import org.opensearch.Version; import org.opensearch.cluster.ClusterInfo; import org.opensearch.cluster.ClusterState; @@ -44,10 +44,11 @@ import org.opensearch.index.seqno.RetentionLease; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; -import org.opensearch.indices.store.StoreFilesMetadata; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper; import org.opensearch.snapshots.SnapshotShardSizeInfo; +import org.junit.Before; import java.util.ArrayList; import java.util.Arrays; @@ -56,15 +57,16 @@ import java.util.EnumSet; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; -import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; import static java.util.Collections.unmodifiableMap; -import static org.hamcrest.Matchers.*; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; public class ReplicaShardBatchAllocatorTest extends OpenSearchAllocationTestCase { private static final org.apache.lucene.util.Version MIN_SUPPORTED_LUCENE_VERSION = org.opensearch.Version.CURRENT @@ -83,7 +85,7 @@ public void buildTestAllocator() { private void allocateAllUnassignedBatch(final RoutingAllocation allocation) { final RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); - Set shardToBatch = new HashSet<>(); + List shardToBatch = new ArrayList<>(); while (iterator.hasNext()) { shardToBatch.add(iterator.next()); } @@ -201,8 +203,18 @@ public void testPreferCopyWithHighestMatchingOperations() { null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) ); - testBatchAllocator.addData(node2, "NOT_MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); - testBatchAllocator.addData(node3, randomSyncId(), null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + testBatchAllocator.addData( + node2, + "NOT_MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData( + node3, + randomSyncId(), + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); allocateAllUnassignedBatch(allocation); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); assertThat( @@ -240,13 +252,23 @@ public void testCancelRecoveryIfFoundCopyWithNoopRetentionLease() { null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) ); - testBatchAllocator.addData(node2, "NO_MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); - testBatchAllocator.addData(node3, randomSyncId(), null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + testBatchAllocator.addData( + node2, + "NO_MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData( + node3, + randomSyncId(), + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); Collection replicaShards = allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED); - Set shardRoutingBatch = new HashSet<>(replicaShards); - List> shardBatchList = Collections.singletonList(new HashSet<>(allocation - .routingNodes() - .shardsWithState(ShardRoutingState.INITIALIZING))); + List shardRoutingBatch = new ArrayList<>(replicaShards); + List> shardBatchList = Collections.singletonList( + new ArrayList<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING)) + ); testBatchAllocator.processExistingRecoveries(allocation, shardBatchList); assertThat(allocation.routingNodesChanged(), equalTo(true)); @@ -273,9 +295,22 @@ public void testNotCancellingRecoveryIfCurrentRecoveryHasRetentionLease() { null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) ); - testBatchAllocator.addData(node2, randomSyncId(), null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); - testBatchAllocator.addData(node3, randomSyncId(), null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); - testBatchAllocator.processExistingRecoveries(allocation, Collections.singletonList(new HashSet<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING)))); + testBatchAllocator.addData( + node2, + randomSyncId(), + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData( + node3, + randomSyncId(), + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.processExistingRecoveries( + allocation, + Collections.singletonList(new ArrayList<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING))) + ); assertThat(allocation.routingNodesChanged(), equalTo(false)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(0)); } @@ -289,9 +324,22 @@ public void testNotCancelIfPrimaryDoesNotHaveValidRetentionLease() { null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) ); - testBatchAllocator.addData(node2, "NOT_MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); - testBatchAllocator.addData(node3, "NOT_MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); - testBatchAllocator.processExistingRecoveries(allocation, Collections.singletonList(new HashSet<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING)))); + testBatchAllocator.addData( + node2, + "NOT_MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData( + node3, + "NOT_MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.processExistingRecoveries( + allocation, + Collections.singletonList(new ArrayList<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING))) + ); assertThat(allocation.routingNodesChanged(), equalTo(false)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(0)); } @@ -313,7 +361,12 @@ public void testIgnoreRetentionLeaseIfCopyIsEmpty() { new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) ); testBatchAllocator.addData(node2, null, null); // has retention lease but store is empty - testBatchAllocator.addData(node3, randomSyncId(), null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + testBatchAllocator.addData( + node3, + randomSyncId(), + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); allocateAllUnassignedBatch(allocation); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); assertThat( @@ -330,7 +383,12 @@ public void testIgnoreRetentionLeaseIfCopyIsEmpty() { */ public void testNoPrimaryData() { RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); - testBatchAllocator.addData(node2, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + testBatchAllocator.addData( + node2, + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); allocateAllUnassignedBatch(allocation); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(1)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).get(0).shardId(), equalTo(shardId)); @@ -342,7 +400,12 @@ public void testNoPrimaryData() { */ public void testNoDataForReplicaOnAnyNode() { RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); - testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + testBatchAllocator.addData( + node1, + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); allocateAllUnassignedBatch(allocation); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(1)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).get(0).shardId(), equalTo(shardId)); @@ -414,7 +477,12 @@ public void testDelayedAllocation() { Settings.builder().put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), TimeValue.timeValueHours(1)).build(), UnassignedInfo.Reason.NODE_LEFT ); - testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + testBatchAllocator.addData( + node1, + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); if (randomBoolean()) { // we sometime return empty list of files, make sure we test this as well testBatchAllocator.addData(node2, null, null); @@ -429,7 +497,12 @@ public void testDelayedAllocation() { Settings.builder().put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), TimeValue.timeValueHours(1)).build(), UnassignedInfo.Reason.NODE_LEFT ); - testBatchAllocator.addData(node2, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + testBatchAllocator.addData( + node2, + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); allocateAllUnassignedBatch(allocation); assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); @@ -444,8 +517,10 @@ public void testCancelRecoveryBetterSyncId() { testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) .addData(node2, "NO_MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) .addData(node3, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); - testBatchAllocator.processExistingRecoveries(allocation, Collections.singletonList( - new HashSet<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING)))); + testBatchAllocator.processExistingRecoveries( + allocation, + Collections.singletonList(new ArrayList<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING))) + ); assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(1)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).get(0).shardId(), equalTo(shardId)); @@ -487,11 +562,22 @@ public void testNotCancellingRecoveryIfSyncedOnExistingRecovery() { null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) ); - testBatchAllocator.addData(node2, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); - testBatchAllocator.addData(node3, randomSyncId(), null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); - testBatchAllocator.processExistingRecoveries(allocation, Collections.singletonList( - new HashSet<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING)) - )); + testBatchAllocator.addData( + node2, + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData( + node3, + randomSyncId(), + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.processExistingRecoveries( + allocation, + Collections.singletonList(new ArrayList<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING))) + ); assertThat(allocation.routingNodesChanged(), equalTo(false)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(0)); } @@ -500,9 +586,10 @@ public void testNotCancellingRecovery() { RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders()); testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) .addData(node2, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); - testBatchAllocator.processExistingRecoveries(allocation, Collections.singletonList( - new HashSet<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING)) - )); + testBatchAllocator.processExistingRecoveries( + allocation, + Collections.singletonList(new ArrayList<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING))) + ); assertThat(allocation.routingNodesChanged(), equalTo(false)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(0)); } @@ -531,17 +618,18 @@ public void testDoNotCancelForBrokenNode() { newRetentionLease(node3, retainingSeqNoOnPrimary) ); testBatchAllocator.addData( - node1, - retentionLeases, - "MATCH", - null, - new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) - ) + node1, + retentionLeases, + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ) .addData(node2, randomSyncId(), null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) .addData(node3, randomSyncId(), null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); - testBatchAllocator.processExistingRecoveries(allocation, Collections.singletonList( - new HashSet<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING)) - )); + testBatchAllocator.processExistingRecoveries( + allocation, + Collections.singletonList(new ArrayList<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING))) + ); assertThat(allocation.routingNodesChanged(), equalTo(false)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED), empty()); } @@ -683,7 +771,12 @@ public int getShardEligibleFetchDataCountAndClean() { return eligibleShardFetchDataCount.getAndSet(0); } - public TestBatchAllocator addData(DiscoveryNode node, String syncId, @Nullable Exception storeFileFetchException, StoreFileMetadata ...files) { + public TestBatchAllocator addData( + DiscoveryNode node, + String syncId, + @Nullable Exception storeFileFetchException, + StoreFileMetadata... files + ) { return addData(node, Collections.emptyList(), syncId, storeFileFetchException, files); } @@ -692,7 +785,8 @@ public TestBatchAllocator addData( List peerRecoveryRetentionLeases, String syncId, @Nullable Exception storeFileFetchException, - StoreFileMetadata ...files) { + StoreFileMetadata... files + ) { if (data == null) { data = new HashMap<>(); } @@ -707,7 +801,7 @@ public TestBatchAllocator addData( data.put( node, new TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata( - new StoreFilesMetadata( + new TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata( shardId, new Store.MetadataSnapshot(unmodifiableMap(filesAsMap), unmodifiableMap(commitData), randomInt()), peerRecoveryRetentionLeases @@ -719,9 +813,13 @@ public TestBatchAllocator addData( } @Override - protected AsyncBatchShardFetch.FetchResult fetchData(Set shardEligibleForFetch, Set inEligibleShards, RoutingAllocation allocation) { + protected AsyncShardFetch.FetchResult fetchData( + Set eligibleShards, + Set ineligibleShards, + RoutingAllocation allocation + ) { fetchDataCalled.set(true); - eligibleShardFetchDataCount.set(shardEligibleForFetch.size()); + eligibleShardFetchDataCount.set(eligibleShards.size()); Map tData = null; if (data != null) { tData = new HashMap<>(); @@ -736,7 +834,7 @@ protected AsyncBatchShardFetch.FetchResult fetchDat ); } } - return new AsyncBatchShardFetch.FetchResult<>(tData, Collections.>emptyMap()); + return new AsyncShardFetch.FetchResult<>(tData, Collections.>emptyMap()); } @Override From 20c7a65993a16e2e59f5d90527f8b43e42fd4eb1 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Wed, 28 Feb 2024 17:02:18 +0530 Subject: [PATCH 09/15] changed fetchData signature Signed-off-by: Shivansh Arora --- .../gateway/ReplicaShardBatchAllocator.java | 16 ++++++---------- .../gateway/ReplicaShardBatchAllocatorTest.java | 4 ++-- 2 files changed, 8 insertions(+), 12 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java index e51e19827ccbb..2b767c439b1a4 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java @@ -23,7 +23,6 @@ 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; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; @@ -31,11 +30,8 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; /** * Allocates replica shards in a batch mode @@ -57,8 +53,8 @@ public void processExistingRecoveries(RoutingAllocation allocation, 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) { - Set eligibleShards = new HashSet<>(); - Set ineligibleShards = new HashSet<>(); + List eligibleShards = new ArrayList<>(); + List ineligibleShards = new ArrayList<>(); boolean shardMatched; // iterate over shards to check for match for each of those for (ShardRouting shard : shardBatch) { @@ -103,8 +99,8 @@ public void processExistingRecoveries(RoutingAllocation allocation, List fetchData( - Set eligibleShards, - Set ineligibleShards, + List eligibleShards, + List ineligibleShards, RoutingAllocation allocation ); @@ -130,8 +126,8 @@ public HashMap makeAllocationDecision( ) { HashMap shardAllocationDecisions = new HashMap<>(); final boolean explain = allocation.debugDecision(); - Set eligibleShards = new HashSet<>(); - Set ineligibleShards = new HashSet<>(); + List eligibleShards = new ArrayList<>(); + List ineligibleShards = new ArrayList<>(); HashMap>> nodeAllocationDecisions = new HashMap<>(); for (ShardRouting shard : shards) { if (!isResponsibleFor(shard)) { diff --git a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTest.java b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTest.java index c76d0e5449bd7..a96463bf4317e 100644 --- a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTest.java +++ b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTest.java @@ -814,8 +814,8 @@ public TestBatchAllocator addData( @Override protected AsyncShardFetch.FetchResult fetchData( - Set eligibleShards, - Set ineligibleShards, + List eligibleShards, + List ineligibleShards, RoutingAllocation allocation ) { fetchDataCalled.set(true); From 6568d633602b076c97c835cd1f944fb7aabd9b53 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Wed, 28 Feb 2024 17:09:25 +0530 Subject: [PATCH 10/15] removed redundant code Signed-off-by: Shivansh Arora --- .../java/org/opensearch/gateway/ReplicaShardAllocator.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java index 36b0da9867b35..5216dd2fcb4b5 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java @@ -556,9 +556,5 @@ boolean canPerformNoopRecovery(DiscoveryNode node) { public boolean hasAnyData() { return matchingNodes.isEmpty() == false; } - - public Map getNodeDecisions() { - return this.nodeDecisions; - } } } From 891bb1f90560ab20be17a79148a47b6863c191da Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Thu, 14 Mar 2024 19:11:24 +0530 Subject: [PATCH 11/15] fix method reference Signed-off-by: Shivansh Arora --- .../java/org/opensearch/gateway/ReplicaShardBatchAllocator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java index 2b767c439b1a4..58ab7c539db85 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java @@ -87,7 +87,7 @@ public void processExistingRecoveries(RoutingAllocation allocation, List nodeShardStores = convertToNodeStoreFilesMetadataMap(shard, shardState); - Runnable cancellationAction = getShardCancellationAction(shard, allocation, nodeShardStores); + Runnable cancellationAction = cancelExistingRecoveryForBetterMatch(shard, allocation, nodeShardStores); if (cancellationAction != null) { shardCancellationActions.add(cancellationAction); } From 5639a694f3caa71d868117984019c9480a1e899b Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Mon, 18 Mar 2024 17:33:20 +0530 Subject: [PATCH 12/15] apply spotless Signed-off-by: Shivansh Arora --- .../gateway/ReplicaShardBatchAllocator.java | 21 +++++++------------ ...a => ReplicaShardBatchAllocatorTests.java} | 2 +- 2 files changed, 8 insertions(+), 15 deletions(-) rename server/src/test/java/org/opensearch/gateway/{ReplicaShardBatchAllocatorTest.java => ReplicaShardBatchAllocatorTests.java} (99%) diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java index 58ab7c539db85..e4588e5a72dd9 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java @@ -9,7 +9,6 @@ package org.opensearch.gateway; import org.apache.logging.log4j.Logger; -import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.routing.RoutingNode; import org.opensearch.cluster.routing.RoutingNodes; @@ -75,11 +74,7 @@ public void processExistingRecoveries(RoutingAllocation allocation, List shardState = fetchData( - eligibleShards, - ineligibleShards, - allocation - ); + AsyncShardFetch.FetchResult shardState = fetchData(eligibleShards, ineligibleShards, allocation); if (!shardState.hasData()) { logger.trace("{}: fetching new stores for initializing shard batch", eligibleShards); continue; // still fetching @@ -164,11 +159,7 @@ public HashMap makeAllocationDecision( return shardAllocationDecisions; } // only fetch data for eligible shards - final FetchResult shardsState = fetchData( - eligibleShards, - ineligibleShards, - allocation - ); + final FetchResult shardsState = fetchData(eligibleShards, ineligibleShards, allocation); for (ShardRouting unassignedShard : eligibleShards) { Tuple> result = nodeAllocationDecisions.get(unassignedShard); @@ -185,11 +176,13 @@ public HashMap makeAllocationDecision( } return shardAllocationDecisions; } + private Map convertToNodeStoreFilesMetadataMap( - ShardRouting unassignedShard, - FetchResult data) { + ShardRouting unassignedShard, + FetchResult data + ) { - if(!data.hasData()) { + if (!data.hasData()) { return new HashMap<>(); } diff --git a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTest.java b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java similarity index 99% rename from server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTest.java rename to server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java index a96463bf4317e..584341089fe69 100644 --- a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTest.java +++ b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java @@ -68,7 +68,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; -public class ReplicaShardBatchAllocatorTest extends OpenSearchAllocationTestCase { +public class ReplicaShardBatchAllocatorTests extends OpenSearchAllocationTestCase { private static final org.apache.lucene.util.Version MIN_SUPPORTED_LUCENE_VERSION = org.opensearch.Version.CURRENT .minimumIndexCompatibilityVersion().luceneVersion; private final ShardId shardId = new ShardId("test", "_na_", 0); From b252e719da010cf396e0bce3c446d40d44e1cfa9 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Mon, 18 Mar 2024 20:08:31 +0530 Subject: [PATCH 13/15] fix test Signed-off-by: Shivansh Arora --- .../org/opensearch/gateway/ReplicaShardBatchAllocator.java | 3 +-- .../opensearch/gateway/ReplicaShardBatchAllocatorTests.java | 6 +++++- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java index e4588e5a72dd9..92bd7787fde48 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java @@ -181,9 +181,8 @@ private Map convertToNodeStoreFilesMetadataMa ShardRouting unassignedShard, FetchResult data ) { - if (!data.hasData()) { - return new HashMap<>(); + return null; } Map map = new HashMap<>(); diff --git a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java index 584341089fe69..464038c93228b 100644 --- a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java @@ -834,7 +834,11 @@ protected AsyncShardFetch.FetchResult fetchData( ); } } - return new AsyncShardFetch.FetchResult<>(tData, Collections.>emptyMap()); + return new AsyncShardFetch.FetchResult<>(tData, new HashMap<>() { + { + put(shardId, Collections.emptySet()); + } + }); } @Override From 6e99e18c8ac9f5fb0b102ce562197164c3034717 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Tue, 19 Mar 2024 14:38:47 +0530 Subject: [PATCH 14/15] Skip iterating over Routing Nodes Signed-off-by: Shivansh Arora --- .../gateway/ReplicaShardBatchAllocator.java | 22 +++++-------------- 1 file changed, 6 insertions(+), 16 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java index 92bd7787fde48..ba46525683b46 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java @@ -10,7 +10,6 @@ import org.apache.logging.log4j.Logger; import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.cluster.routing.RoutingNode; import org.opensearch.cluster.routing.RoutingNodes; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.UnassignedInfo; @@ -54,24 +53,15 @@ public void processExistingRecoveries(RoutingAllocation allocation, List shardBatch : shardBatches) { List eligibleShards = new ArrayList<>(); List ineligibleShards = new ArrayList<>(); - boolean shardMatched; // iterate over shards to check for match for each of those for (ShardRouting shard : shardBatch) { - shardMatched = false; - // need to iterate over all the nodes to find matching shard - for (RoutingNode routingNode : routingNodes) { - ShardRouting shardFromRoutingNode = routingNode.getByShardId(shard.shardId()); - if (shardFromRoutingNode != null && !shardFromRoutingNode.primary()) { - shardMatched = true; - if (shouldSkipFetchForRecovery(shardFromRoutingNode)) { - ineligibleShards.add(shardFromRoutingNode); - continue; - } - eligibleShards.add(shardFromRoutingNode); - } - if (shardMatched) { - break; + if (shard != null && !shard.primary()) { + // need to iterate over all the nodes to find matching shard + if (shouldSkipFetchForRecovery(shard)) { + ineligibleShards.add(shard); + continue; } + eligibleShards.add(shard); } } AsyncShardFetch.FetchResult shardState = fetchData(eligibleShards, ineligibleShards, allocation); From 0d23917611d6e0d721b134489669862bceb1b568 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Tue, 19 Mar 2024 23:00:14 +0530 Subject: [PATCH 15/15] remove null check Signed-off-by: Shivansh Arora --- .../opensearch/gateway/ReplicaShardBatchAllocator.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java index ba46525683b46..3459f1591b633 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java @@ -10,7 +10,6 @@ import org.apache.logging.log4j.Logger; import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.cluster.routing.RoutingNodes; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.UnassignedInfo; import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; @@ -47,7 +46,6 @@ public abstract class ReplicaShardBatchAllocator extends ReplicaShardAllocator { * @param shardBatches a list of shard batches to check for existing recoveries */ public void processExistingRecoveries(RoutingAllocation allocation, List> shardBatches) { - RoutingNodes routingNodes = allocation.routingNodes(); 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) { @@ -177,11 +175,11 @@ private Map convertToNodeStoreFilesMetadataMa Map map = new HashMap<>(); - data.getData().forEach((key, value) -> { + data.getData().forEach((discoveryNode, value) -> { Map batch = value.getNodeStoreFilesMetadataBatch(); NodeStoreFilesMetadata metadata = batch.get(unassignedShard.shardId()); - if (metadata != null && metadata.getStoreFileFetchException() == null) { - map.put(key, metadata.storeFilesMetadata()); + if (metadata != null) { + map.put(discoveryNode, metadata.storeFilesMetadata()); } });