From e87bfddc0c43ffe1d2fb99fdfc4c9fd6ab51560e Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Thu, 27 Jul 2023 14:07:05 +0530 Subject: [PATCH 01/29] Add PrimaryShardBatchAllocator to take allocation decisions for a batch of shards Signed-off-by: Aman Khare --- .../gateway/PrimaryShardBatchAllocator.java | 562 ++++++++++++++++++ 1 file changed, 562 insertions(+) create mode 100644 server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java new file mode 100644 index 0000000000000..36c81100793cf --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java @@ -0,0 +1,562 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ +package org.opensearch.gateway; + +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.routing.RecoverySource; +import org.opensearch.cluster.routing.RoutingNode; +import org.opensearch.cluster.routing.RoutingNodes; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.UnassignedInfo; +import org.opensearch.cluster.routing.UnassignedInfo.AllocationStatus; +import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; +import org.opensearch.cluster.routing.allocation.NodeAllocationResult; +import org.opensearch.cluster.routing.allocation.NodeAllocationResult.ShardStoreInfo; +import org.opensearch.cluster.routing.allocation.RoutingAllocation; +import org.opensearch.cluster.routing.allocation.decider.Decision; +import org.opensearch.cluster.routing.allocation.decider.Decision.Type; +import org.opensearch.env.ShardLockObtainFailedException; +import org.opensearch.gateway.AsyncBatchShardFetch.FetchResult; +import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch; +import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * PrimaryShardBatchAllocator is similar to {@link org.opensearch.gateway.PrimaryShardAllocator} only difference is + * that it can allocate multiple unassigned primary shards wherein PrimaryShardAllocator can only allocate single + * unassigned shard. + * The primary shard batch allocator allocates multiple unassigned primary shards to nodes that hold + * valid copies of the unassigned primaries. It does this by iterating over all unassigned + * primary shards in the routing table and fetching shard metadata from each node in the cluster + * that holds a copy of the shard. The shard metadata from each node is compared against the + * set of valid allocation IDs and for all valid shard copies (if any), the primary shard batch allocator + * executes the allocation deciders to chose a copy to assign the primary shard to. + *

+ * Note that the PrimaryShardBatchAllocator does *not* allocate primaries on index creation + * (see {@link org.opensearch.cluster.routing.allocation.allocator.BalancedShardsAllocator}), + * nor does it allocate primaries when a primary shard failed and there is a valid replica + * copy that can immediately be promoted to primary, as this takes place in {@link RoutingNodes#failShard}. + * + * @opensearch.internal + */ +public abstract class PrimaryShardBatchAllocator extends BaseGatewayShardAllocator { + /** + * Is the allocator responsible for allocating the given {@link ShardRouting}? + */ + private static boolean isResponsibleFor(final ShardRouting shard) { + return shard.primary() // must be primary + && shard.unassigned() // must be unassigned + // only handle either an existing store or a snapshot recovery + && (shard.recoverySource().getType() == RecoverySource.Type.EXISTING_STORE + || shard.recoverySource().getType() == RecoverySource.Type.SNAPSHOT); + } + + abstract protected FetchResult fetchData(Set shardsEligibleForFetch, + Set inEligibleShards, + RoutingAllocation allocation); + + @Override + public AllocateUnassignedDecision makeAllocationDecision(ShardRouting unassignedShard, + RoutingAllocation allocation, + Logger logger) { + + return makeAllocationDecision(new HashSet<>(Collections.singletonList(unassignedShard)), + allocation, logger).get(unassignedShard); + } + + /** + * Build allocation decisions for all the shards present in the batch identified by batchId. + * @param shards set of shards given for allocation + * @param allocation current allocation of all the shards + * @param logger logger used for logging + * @return shard to allocation decision map + */ + @Override + public HashMap makeAllocationDecision(Set shards, + RoutingAllocation allocation, + Logger logger) { + HashMap shardAllocationDecisions = new HashMap<>(); + final boolean explain = allocation.debugDecision(); + Set shardsEligibleForFetch = new HashSet<>(); + Set shardsNotEligibleForFetch = new HashSet<>(); + // identify ineligible shards + for (ShardRouting shard : shards) { + AllocateUnassignedDecision decision = skipSnapshotRestore(shard, allocation); + if (decision != null) { + shardsNotEligibleForFetch.add(shard); + shardAllocationDecisions.put(shard, decision); + } else { + shardsEligibleForFetch.add(shard); + } + } + // only fetch data for eligible shards + final FetchResult shardsState = fetchData(shardsEligibleForFetch, shardsNotEligibleForFetch, allocation); + // Note : shardsState contain the Data, there key is DiscoveryNode but value is Map so to get one shard level data (from all the nodes), we'll traverse the map + // and construct the nodeShardState along the way before making any allocation decision. As metadata for a + // particular shard is needed from all the discovery nodes. + + // process the received data + for (ShardRouting unassignedShard : shardsEligibleForFetch) { + if (shardsState.hasData() == false) { + // if fetching is not done, add that no decision in the resultant map + allocation.setHasPendingAsyncFetch(); + List nodeDecisions = null; + if (explain) { + nodeDecisions = buildDecisionsForAllNodes(unassignedShard, allocation); + } + shardAllocationDecisions.put(unassignedShard, + AllocateUnassignedDecision.no(AllocationStatus.FETCHING_SHARD_DATA, + nodeDecisions)); + } else { + Map nodeResponses = shardsState.getData(); + Map shardData = new HashMap<>(); + // build data for a shard from all the nodes + for (Map.Entry nodeEntry : nodeResponses.entrySet()) { + shardData.put(nodeEntry.getKey(), + nodeEntry.getValue().getNodeGatewayStartedShardsBatch().get(unassignedShard.shardId())); + } + // get allocation decision for this shard + shardAllocationDecisions.put(unassignedShard, getAllocationDecision(unassignedShard, allocation, + shardData, logger)); + } + } + return shardAllocationDecisions; + } + + /** + * Below code is very similar to {@link org.opensearch.gateway.PrimaryShardAllocator} class makeAllocationDecision, + * only difference is that NodeGatewayStartedShards object doesn't have the DiscoveryNode object as + * BaseNodeResponse. So, DiscoveryNode reference is passed in Map so + * corresponding DiscoveryNode object can be used for rest of the implementation. Also, DiscoveryNode object + * reference is added in DecidedNode class to achieve same use case of accessing corresponding DiscoveryNode object. + * @param unassignedShard unassigned shard routing + * @param allocation routing allocation object + * @param shardState shard metadata fetched from all data nodes + * @param logger logger + * @return allocation decision taken for this shard + */ + private AllocateUnassignedDecision getAllocationDecision(ShardRouting unassignedShard, RoutingAllocation allocation, + Map shardState, + Logger logger) { + final boolean explain = allocation.debugDecision(); + // don't create a new IndexSetting object for every shard as this could cause a lot of garbage + // on cluster restart if we allocate a boat load of shards + final IndexMetadata indexMetadata = allocation.metadata().getIndexSafe(unassignedShard.index()); + final Set inSyncAllocationIds = indexMetadata.inSyncAllocationIds(unassignedShard.id()); + final boolean snapshotRestore = unassignedShard.recoverySource().getType() == RecoverySource.Type.SNAPSHOT; + + assert inSyncAllocationIds.isEmpty() == false; + // use in-sync allocation ids to select nodes + final PrimaryShardBatchAllocator.NodeShardsResult nodeShardsResult = buildNodeShardsResult( + unassignedShard, + snapshotRestore, + allocation.getIgnoreNodes(unassignedShard.shardId()), + inSyncAllocationIds, + shardState, + logger + ); + final boolean enoughAllocationsFound = nodeShardsResult.orderedAllocationCandidates.size() > 0; + logger.debug( + "[{}][{}]: found {} allocation candidates of {} based on allocation ids: [{}]", + unassignedShard.index(), + unassignedShard.id(), + nodeShardsResult.orderedAllocationCandidates.size(), + unassignedShard, + inSyncAllocationIds + ); + + if (enoughAllocationsFound == false) { + if (snapshotRestore) { + // let BalancedShardsAllocator take care of allocating this shard + logger.debug( + "[{}][{}]: missing local data, will restore from [{}]", + unassignedShard.index(), + unassignedShard.id(), + unassignedShard.recoverySource() + ); + return AllocateUnassignedDecision.NOT_TAKEN; + } else { + // We have a shard that was previously allocated, but we could not find a valid shard copy to allocate the primary. + // We could just be waiting for the node that holds the primary to start back up, in which case the allocation for + // this shard will be picked up when the node joins and we do another allocation reroute + logger.debug( + "[{}][{}]: not allocating, number_of_allocated_shards_found [{}]", + unassignedShard.index(), + unassignedShard.id(), + nodeShardsResult.allocationsFound + ); + return AllocateUnassignedDecision.no( + AllocationStatus.NO_VALID_SHARD_COPY, + explain ? buildNodeDecisions(null, shardState, inSyncAllocationIds) : null + ); + } + } + + NodesToAllocate nodesToAllocate = buildNodesToAllocate( + allocation, + nodeShardsResult.orderedAllocationCandidates, + unassignedShard, + false + ); + DiscoveryNode node = null; + String allocationId = null; + boolean throttled = false; + if (nodesToAllocate.yesNodeShards.isEmpty() == false) { + DecidedNode decidedNode = nodesToAllocate.yesNodeShards.get(0); + logger.debug( + "[{}][{}]: allocating [{}] to [{}] on primary allocation", + unassignedShard.index(), + unassignedShard.id(), + unassignedShard, + decidedNode.getNode() + ); + node = decidedNode.getNode(); + allocationId = decidedNode.nodeShardState.allocationId(); + } else if (nodesToAllocate.throttleNodeShards.isEmpty() && !nodesToAllocate.noNodeShards.isEmpty()) { + // The deciders returned a NO decision for all nodes with shard copies, so we check if primary shard + // can be force-allocated to one of the nodes. + nodesToAllocate = buildNodesToAllocate(allocation, nodeShardsResult.orderedAllocationCandidates, unassignedShard, true); + if (nodesToAllocate.yesNodeShards.isEmpty() == false) { + final DecidedNode decidedNode = nodesToAllocate.yesNodeShards.get(0); + final NodeGatewayStartedShards nodeShardState = decidedNode.nodeShardState; + logger.debug( + "[{}][{}]: allocating [{}] to [{}] on forced primary allocation", + unassignedShard.index(), + unassignedShard.id(), + unassignedShard, + decidedNode.getNode() + ); + node = decidedNode.getNode(); + allocationId = nodeShardState.allocationId(); + } else if (nodesToAllocate.throttleNodeShards.isEmpty() == false) { + logger.debug( + "[{}][{}]: throttling allocation [{}] to [{}] on forced primary allocation", + unassignedShard.index(), + unassignedShard.id(), + unassignedShard, + nodesToAllocate.throttleNodeShards + ); + throttled = true; + } else { + logger.debug( + "[{}][{}]: forced primary allocation denied [{}]", + unassignedShard.index(), + unassignedShard.id(), + unassignedShard + ); + } + } else { + // we are throttling this, since we are allowed to allocate to this node but there are enough allocations + // taking place on the node currently, ignore it for now + logger.debug( + "[{}][{}]: throttling allocation [{}] to [{}] on primary allocation", + unassignedShard.index(), + unassignedShard.id(), + unassignedShard, + nodesToAllocate.throttleNodeShards + ); + throttled = true; + } + + List nodeResults = null; + if (explain) { + nodeResults = buildNodeDecisions(nodesToAllocate, shardState, inSyncAllocationIds); + } + if (allocation.hasPendingAsyncFetch()) { + return AllocateUnassignedDecision.no(AllocationStatus.FETCHING_SHARD_DATA, nodeResults); + } else if (node != null) { + return AllocateUnassignedDecision.yes(node, allocationId, nodeResults, false); + } else if (throttled) { + return AllocateUnassignedDecision.throttle(nodeResults); + } else { + return AllocateUnassignedDecision.no(AllocationStatus.DECIDERS_NO, nodeResults, true); + } + } + + /** + * Skip doing fetchData call for a shard if recovery mode is snapshot. Also do not take decision if allocator is + * not responsible for this particular shard. + * @param unassignedShard unassigned shard routing + * @param allocation routing allocation object + * @return allocation decision taken for this shard + */ + private AllocateUnassignedDecision skipSnapshotRestore(ShardRouting unassignedShard, RoutingAllocation allocation) { + if (isResponsibleFor(unassignedShard) == false) { + // this allocator is not responsible for allocating this shard + return AllocateUnassignedDecision.NOT_TAKEN; + } + final boolean explain = allocation.debugDecision(); + if (unassignedShard.recoverySource().getType() == RecoverySource.Type.SNAPSHOT + && allocation.snapshotShardSizeInfo().getShardSize(unassignedShard) == null) { + List nodeDecisions = null; + if (explain) { + nodeDecisions = buildDecisionsForAllNodes(unassignedShard, allocation); + } + return AllocateUnassignedDecision.no(UnassignedInfo.AllocationStatus.FETCHING_SHARD_DATA, nodeDecisions); + } + return null; + } + + /** + * Builds a map of nodes to the corresponding allocation decisions for those nodes. + */ + private static List buildNodeDecisions( + NodesToAllocate nodesToAllocate, + Map fetchedShardData, + Set inSyncAllocationIds + ) { + List nodeResults = new ArrayList<>(); + Map ineligibleShards; + if (nodesToAllocate != null) { + final Set discoNodes = new HashSet<>(); + nodeResults.addAll( + Stream.of(nodesToAllocate.yesNodeShards, nodesToAllocate.throttleNodeShards, nodesToAllocate.noNodeShards) + .flatMap(Collection::stream) + .map(dnode -> { + discoNodes.add(dnode.getNode()); + return new NodeAllocationResult( + dnode.getNode(), + shardStoreInfo(dnode.nodeShardState, inSyncAllocationIds), + dnode.decision + ); + }) + .collect(Collectors.toList()) + ); + + ineligibleShards = fetchedShardData.entrySet() + .stream() + .filter(shardData -> discoNodes.contains(shardData.getKey()) == false) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } else { + // there were no shard copies that were eligible for being assigned the allocation, + // so all fetched shard data are ineligible shards + ineligibleShards = fetchedShardData; + } + + nodeResults.addAll( + ineligibleShards.entrySet().stream() + .map(shardData -> new NodeAllocationResult(shardData.getKey(), shardStoreInfo(shardData.getValue(), + inSyncAllocationIds), null)) + .collect(Collectors.toList()) + ); + + return nodeResults; + } + + private static ShardStoreInfo shardStoreInfo(NodeGatewayStartedShards nodeShardState, Set inSyncAllocationIds) { + final Exception storeErr = nodeShardState.storeException(); + final boolean inSync = nodeShardState.allocationId() != null && inSyncAllocationIds.contains(nodeShardState.allocationId()); + return new ShardStoreInfo(nodeShardState.allocationId(), inSync, storeErr); + } + + private static final Comparator NO_STORE_EXCEPTION_FIRST_COMPARATOR = Comparator.comparing( + (NodeGatewayStartedShards state) -> state.storeException() == null + ).reversed(); + private static final Comparator PRIMARY_FIRST_COMPARATOR = Comparator.comparing( + NodeGatewayStartedShards::primary + ).reversed(); + + private static final Comparator HIGHEST_REPLICATION_CHECKPOINT_FIRST_COMPARATOR = Comparator.comparing( + NodeGatewayStartedShards::replicationCheckpoint, + Comparator.nullsLast(Comparator.naturalOrder()) + ); + + /** + * Builds a list of nodes. If matchAnyShard is set to false, only nodes that have an allocation id matching + * inSyncAllocationIds are added to the list. Otherwise, any node that has a shard is added to the list, but + * entries with matching allocation id are always at the front of the list. + */ + protected static NodeShardsResult buildNodeShardsResult( + ShardRouting shard, + boolean matchAnyShard, + Set ignoreNodes, + Set inSyncAllocationIds, + Map shardState, + Logger logger + ) { + /** + * Orders the active shards copies based on below comparators + * 1. No store exception i.e. shard copy is readable + * 2. Prefer previous primary shard + * 3. Prefer shard copy with the highest replication checkpoint. It is NO-OP for doc rep enabled indices. + */ + final Comparator comparator; // allocation preference + if (matchAnyShard) { + // prefer shards with matching allocation ids + Comparator matchingAllocationsFirst = Comparator.comparing( + (NodeGatewayStartedShards state) -> inSyncAllocationIds.contains(state.allocationId()) + ).reversed(); + comparator = matchingAllocationsFirst.thenComparing(NO_STORE_EXCEPTION_FIRST_COMPARATOR) + .thenComparing(PRIMARY_FIRST_COMPARATOR) + .thenComparing(HIGHEST_REPLICATION_CHECKPOINT_FIRST_COMPARATOR); + } else { + comparator = NO_STORE_EXCEPTION_FIRST_COMPARATOR.thenComparing(PRIMARY_FIRST_COMPARATOR) + .thenComparing(HIGHEST_REPLICATION_CHECKPOINT_FIRST_COMPARATOR); + } + // TreeMap will sort the entries based on key, comparator is assigned above + TreeMap shardStatesToNode = new TreeMap<>(comparator); + int numberOfAllocationsFound = 0; + for (Map.Entry nodeShardStateEntry : shardState.entrySet()) { + DiscoveryNode node = nodeShardStateEntry.getKey(); + NodeGatewayStartedShards nodeShardState = nodeShardStateEntry.getValue(); + String allocationId = nodeShardState.allocationId(); + + if (ignoreNodes.contains(node.getId())) { + continue; + } + + if (nodeShardState.storeException() == null) { + if (allocationId == null) { + logger.trace("[{}] on node [{}] has no shard state information", shard, node); + } else { + logger.trace("[{}] on node [{}] has allocation id [{}]", shard, node, allocationId); + } + } else { + final String finalAllocationId = allocationId; + if (nodeShardState.storeException() instanceof ShardLockObtainFailedException) { + logger.trace( + () -> new ParameterizedMessage( + "[{}] on node [{}] has allocation id [{}] but the store can not be " + + "opened as it's locked, treating as valid shard", + shard, + node, + finalAllocationId + ), + nodeShardState.storeException() + ); + } else { + logger.trace( + () -> new ParameterizedMessage( + "[{}] on node [{}] has allocation id [{}] but the store can not be " + "opened, treating as no allocation id", + shard, + node, + finalAllocationId + ), + nodeShardState.storeException() + ); + allocationId = null; + } + } + + if (allocationId != null) { + assert nodeShardState.storeException() == null || nodeShardState.storeException() instanceof ShardLockObtainFailedException + : "only allow store that can be opened or that throws a ShardLockObtainFailedException while being opened but got a " + + "store throwing " + + nodeShardState.storeException(); + numberOfAllocationsFound++; + if (matchAnyShard || inSyncAllocationIds.contains(nodeShardState.allocationId())) { + shardStatesToNode.put(nodeShardState, node); + } + } + } + + if (logger.isTraceEnabled()) { + logger.trace( + "{} candidates for allocation: {}", + shard, + shardState.keySet().stream().map(DiscoveryNode::getName).collect(Collectors.joining(", ")) + ); + } + return new NodeShardsResult(shardStatesToNode, numberOfAllocationsFound); + } + + /** + * Split the list of node shard states into groups yes/no/throttle based on allocation deciders + */ + private static NodesToAllocate buildNodesToAllocate( + RoutingAllocation allocation, + TreeMap shardStateToNode, + ShardRouting shardRouting, + boolean forceAllocate + ) { + List yesNodeShards = new ArrayList<>(); + List throttledNodeShards = new ArrayList<>(); + List noNodeShards = new ArrayList<>(); + for (Map.Entry nodeShardState : shardStateToNode.entrySet()) { + RoutingNode node = allocation.routingNodes().node(nodeShardState.getValue().getId()); + if (node == null) { + continue; + } + + Decision decision = forceAllocate + ? allocation.deciders().canForceAllocatePrimary(shardRouting, node, allocation) + : allocation.deciders().canAllocate(shardRouting, node, allocation); + DecidedNode decidedNode = new DecidedNode(nodeShardState.getKey(), decision, nodeShardState.getValue()); + if (decision.type() == Type.THROTTLE) { + throttledNodeShards.add(decidedNode); + } else if (decision.type() == Type.NO) { + noNodeShards.add(decidedNode); + } else { + yesNodeShards.add(decidedNode); + } + } + return new NodesToAllocate( + Collections.unmodifiableList(yesNodeShards), + Collections.unmodifiableList(throttledNodeShards), + Collections.unmodifiableList(noNodeShards) + ); + } + + private static class NodeShardsResult { + final TreeMap orderedAllocationCandidates; + final int allocationsFound; + + NodeShardsResult(TreeMap orderedAllocationCandidates, int allocationsFound) { + this.orderedAllocationCandidates = orderedAllocationCandidates; + this.allocationsFound = allocationsFound; + } + } + + static class NodesToAllocate { + final List yesNodeShards; + final List throttleNodeShards; + final List noNodeShards; + + NodesToAllocate(List yesNodeShards, List throttleNodeShards, List noNodeShards) { + this.yesNodeShards = yesNodeShards; + this.throttleNodeShards = throttleNodeShards; + this.noNodeShards = noNodeShards; + } + } + + /** + * This class encapsulates the shard state retrieved from a node and the decision that was made + * by the allocator for allocating to the node that holds the shard copy. + */ + private static class DecidedNode { + final NodeGatewayStartedShards nodeShardState; + final Decision decision; + final DiscoveryNode node; + + private DecidedNode(NodeGatewayStartedShards nodeShardState, Decision decision, DiscoveryNode node) { + this.nodeShardState = nodeShardState; + this.decision = decision; + this.node = node; + } + + public DiscoveryNode getNode() { + return node; + } + } +} From 6c383034184cafa6de55c251ca5bd32662a0947f Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Thu, 21 Sep 2023 16:29:01 +0530 Subject: [PATCH 02/29] Add unit tests and refactor PrimaryShardBatchAllocator Signed-off-by: Aman Khare --- .../gateway/PrimaryShardBatchAllocator.java | 540 +++--------------- .../PrimaryShardBatchAllocatorTests.java | 339 +++++++++++ 2 files changed, 410 insertions(+), 469 deletions(-) create mode 100644 server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java index 36c81100793cf..0dfd8d5718e54 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java @@ -8,38 +8,23 @@ package org.opensearch.gateway; import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.cluster.routing.RecoverySource; -import org.opensearch.cluster.routing.RoutingNode; import org.opensearch.cluster.routing.RoutingNodes; import org.opensearch.cluster.routing.ShardRouting; -import org.opensearch.cluster.routing.UnassignedInfo; import org.opensearch.cluster.routing.UnassignedInfo.AllocationStatus; import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.opensearch.cluster.routing.allocation.NodeAllocationResult; -import org.opensearch.cluster.routing.allocation.NodeAllocationResult.ShardStoreInfo; import org.opensearch.cluster.routing.allocation.RoutingAllocation; -import org.opensearch.cluster.routing.allocation.decider.Decision; -import org.opensearch.cluster.routing.allocation.decider.Decision.Type; -import org.opensearch.env.ShardLockObtainFailedException; -import org.opensearch.gateway.AsyncBatchShardFetch.FetchResult; +import org.opensearch.gateway.AsyncShardFetch.FetchResult; import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch; import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards; -import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.TreeMap; -import java.util.stream.Collectors; -import java.util.stream.Stream; /** * PrimaryShardBatchAllocator is similar to {@link org.opensearch.gateway.PrimaryShardAllocator} only difference is @@ -59,49 +44,48 @@ * * @opensearch.internal */ -public abstract class PrimaryShardBatchAllocator extends BaseGatewayShardAllocator { - /** - * Is the allocator responsible for allocating the given {@link ShardRouting}? - */ - private static boolean isResponsibleFor(final ShardRouting shard) { - return shard.primary() // must be primary - && shard.unassigned() // must be unassigned - // only handle either an existing store or a snapshot recovery - && (shard.recoverySource().getType() == RecoverySource.Type.EXISTING_STORE - || shard.recoverySource().getType() == RecoverySource.Type.SNAPSHOT); - } +public abstract class PrimaryShardBatchAllocator extends PrimaryShardAllocator { - abstract protected FetchResult fetchData(Set shardsEligibleForFetch, - Set inEligibleShards, - RoutingAllocation allocation); + abstract protected FetchResult fetchData( + Set shardsEligibleForFetch, + Set inEligibleShards, + RoutingAllocation allocation + ); + + protected FetchResult fetchData( + ShardRouting shard, + RoutingAllocation allocation + ) { + return null; + } @Override - public AllocateUnassignedDecision makeAllocationDecision(ShardRouting unassignedShard, - RoutingAllocation allocation, - Logger logger) { + public AllocateUnassignedDecision makeAllocationDecision(ShardRouting unassignedShard, RoutingAllocation allocation, Logger logger) { - return makeAllocationDecision(new HashSet<>(Collections.singletonList(unassignedShard)), - allocation, logger).get(unassignedShard); + return makeAllocationDecision(new HashSet<>(Collections.singletonList(unassignedShard)), allocation, logger).get(unassignedShard); } /** - * Build allocation decisions for all the shards present in the batch identified by batchId. - * @param shards set of shards given for allocation + * Build allocation decisions for all the shards given to this allocator.. + * + * @param shards set of shards given for allocation * @param allocation current allocation of all the shards - * @param logger logger used for logging + * @param logger logger used for logging * @return shard to allocation decision map */ @Override - public HashMap makeAllocationDecision(Set shards, - RoutingAllocation allocation, - Logger logger) { + public HashMap makeAllocationDecision( + Set shards, + RoutingAllocation allocation, + Logger logger + ) { HashMap shardAllocationDecisions = new HashMap<>(); final boolean explain = allocation.debugDecision(); Set shardsEligibleForFetch = new HashSet<>(); Set shardsNotEligibleForFetch = new HashSet<>(); // identify ineligible shards for (ShardRouting shard : shards) { - AllocateUnassignedDecision decision = skipSnapshotRestore(shard, allocation); + AllocateUnassignedDecision decision = getInEligibleShardDecision(shard, allocation); if (decision != null) { shardsNotEligibleForFetch.add(shard); shardAllocationDecisions.put(shard, decision); @@ -109,12 +93,16 @@ public HashMap makeAllocationDecision( shardsEligibleForFetch.add(shard); } } + // Do not call fetchData if there are no eligible shards + if (shardsEligibleForFetch.size() == 0) { + return shardAllocationDecisions; + } // only fetch data for eligible shards - final FetchResult shardsState = fetchData(shardsEligibleForFetch, shardsNotEligibleForFetch, allocation); - // Note : shardsState contain the Data, there key is DiscoveryNode but value is Map so to get one shard level data (from all the nodes), we'll traverse the map - // and construct the nodeShardState along the way before making any allocation decision. As metadata for a - // particular shard is needed from all the discovery nodes. + final FetchResult shardsState = fetchData( + shardsEligibleForFetch, + shardsNotEligibleForFetch, + allocation + ); // process the received data for (ShardRouting unassignedShard : shardsEligibleForFetch) { @@ -125,438 +113,52 @@ public HashMap makeAllocationDecision( if (explain) { nodeDecisions = buildDecisionsForAllNodes(unassignedShard, allocation); } - shardAllocationDecisions.put(unassignedShard, - AllocateUnassignedDecision.no(AllocationStatus.FETCHING_SHARD_DATA, - nodeDecisions)); - } else { - Map nodeResponses = shardsState.getData(); - Map shardData = new HashMap<>(); - // build data for a shard from all the nodes - for (Map.Entry nodeEntry : nodeResponses.entrySet()) { - shardData.put(nodeEntry.getKey(), - nodeEntry.getValue().getNodeGatewayStartedShardsBatch().get(unassignedShard.shardId())); - } - // get allocation decision for this shard - shardAllocationDecisions.put(unassignedShard, getAllocationDecision(unassignedShard, allocation, - shardData, logger)); - } - } - return shardAllocationDecisions; - } - - /** - * Below code is very similar to {@link org.opensearch.gateway.PrimaryShardAllocator} class makeAllocationDecision, - * only difference is that NodeGatewayStartedShards object doesn't have the DiscoveryNode object as - * BaseNodeResponse. So, DiscoveryNode reference is passed in Map so - * corresponding DiscoveryNode object can be used for rest of the implementation. Also, DiscoveryNode object - * reference is added in DecidedNode class to achieve same use case of accessing corresponding DiscoveryNode object. - * @param unassignedShard unassigned shard routing - * @param allocation routing allocation object - * @param shardState shard metadata fetched from all data nodes - * @param logger logger - * @return allocation decision taken for this shard - */ - private AllocateUnassignedDecision getAllocationDecision(ShardRouting unassignedShard, RoutingAllocation allocation, - Map shardState, - Logger logger) { - final boolean explain = allocation.debugDecision(); - // don't create a new IndexSetting object for every shard as this could cause a lot of garbage - // on cluster restart if we allocate a boat load of shards - final IndexMetadata indexMetadata = allocation.metadata().getIndexSafe(unassignedShard.index()); - final Set inSyncAllocationIds = indexMetadata.inSyncAllocationIds(unassignedShard.id()); - final boolean snapshotRestore = unassignedShard.recoverySource().getType() == RecoverySource.Type.SNAPSHOT; - - assert inSyncAllocationIds.isEmpty() == false; - // use in-sync allocation ids to select nodes - final PrimaryShardBatchAllocator.NodeShardsResult nodeShardsResult = buildNodeShardsResult( - unassignedShard, - snapshotRestore, - allocation.getIgnoreNodes(unassignedShard.shardId()), - inSyncAllocationIds, - shardState, - logger - ); - final boolean enoughAllocationsFound = nodeShardsResult.orderedAllocationCandidates.size() > 0; - logger.debug( - "[{}][{}]: found {} allocation candidates of {} based on allocation ids: [{}]", - unassignedShard.index(), - unassignedShard.id(), - nodeShardsResult.orderedAllocationCandidates.size(), - unassignedShard, - inSyncAllocationIds - ); - - if (enoughAllocationsFound == false) { - if (snapshotRestore) { - // let BalancedShardsAllocator take care of allocating this shard - logger.debug( - "[{}][{}]: missing local data, will restore from [{}]", - unassignedShard.index(), - unassignedShard.id(), - unassignedShard.recoverySource() - ); - return AllocateUnassignedDecision.NOT_TAKEN; - } else { - // We have a shard that was previously allocated, but we could not find a valid shard copy to allocate the primary. - // We could just be waiting for the node that holds the primary to start back up, in which case the allocation for - // this shard will be picked up when the node joins and we do another allocation reroute - logger.debug( - "[{}][{}]: not allocating, number_of_allocated_shards_found [{}]", - unassignedShard.index(), - unassignedShard.id(), - nodeShardsResult.allocationsFound - ); - return AllocateUnassignedDecision.no( - AllocationStatus.NO_VALID_SHARD_COPY, - explain ? buildNodeDecisions(null, shardState, inSyncAllocationIds) : null - ); - } - } - - NodesToAllocate nodesToAllocate = buildNodesToAllocate( - allocation, - nodeShardsResult.orderedAllocationCandidates, - unassignedShard, - false - ); - DiscoveryNode node = null; - String allocationId = null; - boolean throttled = false; - if (nodesToAllocate.yesNodeShards.isEmpty() == false) { - DecidedNode decidedNode = nodesToAllocate.yesNodeShards.get(0); - logger.debug( - "[{}][{}]: allocating [{}] to [{}] on primary allocation", - unassignedShard.index(), - unassignedShard.id(), - unassignedShard, - decidedNode.getNode() - ); - node = decidedNode.getNode(); - allocationId = decidedNode.nodeShardState.allocationId(); - } else if (nodesToAllocate.throttleNodeShards.isEmpty() && !nodesToAllocate.noNodeShards.isEmpty()) { - // The deciders returned a NO decision for all nodes with shard copies, so we check if primary shard - // can be force-allocated to one of the nodes. - nodesToAllocate = buildNodesToAllocate(allocation, nodeShardsResult.orderedAllocationCandidates, unassignedShard, true); - if (nodesToAllocate.yesNodeShards.isEmpty() == false) { - final DecidedNode decidedNode = nodesToAllocate.yesNodeShards.get(0); - final NodeGatewayStartedShards nodeShardState = decidedNode.nodeShardState; - logger.debug( - "[{}][{}]: allocating [{}] to [{}] on forced primary allocation", - unassignedShard.index(), - unassignedShard.id(), - unassignedShard, - decidedNode.getNode() - ); - node = decidedNode.getNode(); - allocationId = nodeShardState.allocationId(); - } else if (nodesToAllocate.throttleNodeShards.isEmpty() == false) { - logger.debug( - "[{}][{}]: throttling allocation [{}] to [{}] on forced primary allocation", - unassignedShard.index(), - unassignedShard.id(), + shardAllocationDecisions.put( unassignedShard, - nodesToAllocate.throttleNodeShards + AllocateUnassignedDecision.no(AllocationStatus.FETCHING_SHARD_DATA, nodeDecisions) ); - throttled = true; } else { - logger.debug( - "[{}][{}]: forced primary allocation denied [{}]", - unassignedShard.index(), - unassignedShard.id(), - unassignedShard - ); - } - } else { - // we are throttling this, since we are allowed to allocate to this node but there are enough allocations - // taking place on the node currently, ignore it for now - logger.debug( - "[{}][{}]: throttling allocation [{}] to [{}] on primary allocation", - unassignedShard.index(), - unassignedShard.id(), - unassignedShard, - nodesToAllocate.throttleNodeShards - ); - throttled = true; - } - - List nodeResults = null; - if (explain) { - nodeResults = buildNodeDecisions(nodesToAllocate, shardState, inSyncAllocationIds); - } - if (allocation.hasPendingAsyncFetch()) { - return AllocateUnassignedDecision.no(AllocationStatus.FETCHING_SHARD_DATA, nodeResults); - } else if (node != null) { - return AllocateUnassignedDecision.yes(node, allocationId, nodeResults, false); - } else if (throttled) { - return AllocateUnassignedDecision.throttle(nodeResults); - } else { - return AllocateUnassignedDecision.no(AllocationStatus.DECIDERS_NO, nodeResults, true); - } - } - - /** - * Skip doing fetchData call for a shard if recovery mode is snapshot. Also do not take decision if allocator is - * not responsible for this particular shard. - * @param unassignedShard unassigned shard routing - * @param allocation routing allocation object - * @return allocation decision taken for this shard - */ - private AllocateUnassignedDecision skipSnapshotRestore(ShardRouting unassignedShard, RoutingAllocation allocation) { - if (isResponsibleFor(unassignedShard) == false) { - // this allocator is not responsible for allocating this shard - return AllocateUnassignedDecision.NOT_TAKEN; - } - final boolean explain = allocation.debugDecision(); - if (unassignedShard.recoverySource().getType() == RecoverySource.Type.SNAPSHOT - && allocation.snapshotShardSizeInfo().getShardSize(unassignedShard) == null) { - List nodeDecisions = null; - if (explain) { - nodeDecisions = buildDecisionsForAllNodes(unassignedShard, allocation); - } - return AllocateUnassignedDecision.no(UnassignedInfo.AllocationStatus.FETCHING_SHARD_DATA, nodeDecisions); - } - return null; - } - /** - * Builds a map of nodes to the corresponding allocation decisions for those nodes. - */ - private static List buildNodeDecisions( - NodesToAllocate nodesToAllocate, - Map fetchedShardData, - Set inSyncAllocationIds - ) { - List nodeResults = new ArrayList<>(); - Map ineligibleShards; - if (nodesToAllocate != null) { - final Set discoNodes = new HashSet<>(); - nodeResults.addAll( - Stream.of(nodesToAllocate.yesNodeShards, nodesToAllocate.throttleNodeShards, nodesToAllocate.noNodeShards) - .flatMap(Collection::stream) - .map(dnode -> { - discoNodes.add(dnode.getNode()); - return new NodeAllocationResult( - dnode.getNode(), - shardStoreInfo(dnode.nodeShardState, inSyncAllocationIds), - dnode.decision - ); - }) - .collect(Collectors.toList()) - ); - - ineligibleShards = fetchedShardData.entrySet() - .stream() - .filter(shardData -> discoNodes.contains(shardData.getKey()) == false) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - } else { - // there were no shard copies that were eligible for being assigned the allocation, - // so all fetched shard data are ineligible shards - ineligibleShards = fetchedShardData; - } - - nodeResults.addAll( - ineligibleShards.entrySet().stream() - .map(shardData -> new NodeAllocationResult(shardData.getKey(), shardStoreInfo(shardData.getValue(), - inSyncAllocationIds), null)) - .collect(Collectors.toList()) - ); - - return nodeResults; - } - - private static ShardStoreInfo shardStoreInfo(NodeGatewayStartedShards nodeShardState, Set inSyncAllocationIds) { - final Exception storeErr = nodeShardState.storeException(); - final boolean inSync = nodeShardState.allocationId() != null && inSyncAllocationIds.contains(nodeShardState.allocationId()); - return new ShardStoreInfo(nodeShardState.allocationId(), inSync, storeErr); - } - - private static final Comparator NO_STORE_EXCEPTION_FIRST_COMPARATOR = Comparator.comparing( - (NodeGatewayStartedShards state) -> state.storeException() == null - ).reversed(); - private static final Comparator PRIMARY_FIRST_COMPARATOR = Comparator.comparing( - NodeGatewayStartedShards::primary - ).reversed(); - - private static final Comparator HIGHEST_REPLICATION_CHECKPOINT_FIRST_COMPARATOR = Comparator.comparing( - NodeGatewayStartedShards::replicationCheckpoint, - Comparator.nullsLast(Comparator.naturalOrder()) - ); - - /** - * Builds a list of nodes. If matchAnyShard is set to false, only nodes that have an allocation id matching - * inSyncAllocationIds are added to the list. Otherwise, any node that has a shard is added to the list, but - * entries with matching allocation id are always at the front of the list. - */ - protected static NodeShardsResult buildNodeShardsResult( - ShardRouting shard, - boolean matchAnyShard, - Set ignoreNodes, - Set inSyncAllocationIds, - Map shardState, - Logger logger - ) { - /** - * Orders the active shards copies based on below comparators - * 1. No store exception i.e. shard copy is readable - * 2. Prefer previous primary shard - * 3. Prefer shard copy with the highest replication checkpoint. It is NO-OP for doc rep enabled indices. - */ - final Comparator comparator; // allocation preference - if (matchAnyShard) { - // prefer shards with matching allocation ids - Comparator matchingAllocationsFirst = Comparator.comparing( - (NodeGatewayStartedShards state) -> inSyncAllocationIds.contains(state.allocationId()) - ).reversed(); - comparator = matchingAllocationsFirst.thenComparing(NO_STORE_EXCEPTION_FIRST_COMPARATOR) - .thenComparing(PRIMARY_FIRST_COMPARATOR) - .thenComparing(HIGHEST_REPLICATION_CHECKPOINT_FIRST_COMPARATOR); - } else { - comparator = NO_STORE_EXCEPTION_FIRST_COMPARATOR.thenComparing(PRIMARY_FIRST_COMPARATOR) - .thenComparing(HIGHEST_REPLICATION_CHECKPOINT_FIRST_COMPARATOR); - } - // TreeMap will sort the entries based on key, comparator is assigned above - TreeMap shardStatesToNode = new TreeMap<>(comparator); - int numberOfAllocationsFound = 0; - for (Map.Entry nodeShardStateEntry : shardState.entrySet()) { - DiscoveryNode node = nodeShardStateEntry.getKey(); - NodeGatewayStartedShards nodeShardState = nodeShardStateEntry.getValue(); - String allocationId = nodeShardState.allocationId(); - - if (ignoreNodes.contains(node.getId())) { - continue; - } - - if (nodeShardState.storeException() == null) { - if (allocationId == null) { - logger.trace("[{}] on node [{}] has no shard state information", shard, node); - } else { - logger.trace("[{}] on node [{}] has allocation id [{}]", shard, node, allocationId); - } - } else { - final String finalAllocationId = allocationId; - if (nodeShardState.storeException() instanceof ShardLockObtainFailedException) { - logger.trace( - () -> new ParameterizedMessage( - "[{}] on node [{}] has allocation id [{}] but the store can not be " - + "opened as it's locked, treating as valid shard", - shard, - node, - finalAllocationId - ), - nodeShardState.storeException() - ); - } else { - logger.trace( - () -> new ParameterizedMessage( - "[{}] on node [{}] has allocation id [{}] but the store can not be " + "opened, treating as no allocation id", - shard, - node, - finalAllocationId - ), - nodeShardState.storeException() - ); - allocationId = null; - } - } - - if (allocationId != null) { - assert nodeShardState.storeException() == null || nodeShardState.storeException() instanceof ShardLockObtainFailedException - : "only allow store that can be opened or that throws a ShardLockObtainFailedException while being opened but got a " - + "store throwing " - + nodeShardState.storeException(); - numberOfAllocationsFound++; - if (matchAnyShard || inSyncAllocationIds.contains(nodeShardState.allocationId())) { - shardStatesToNode.put(nodeShardState, node); - } + NodeShardStates nodeShardStates = adaptToNodeShardStates(unassignedShard, shardsState); + // get allocation decision for this shard + shardAllocationDecisions.put(unassignedShard, getAllocationDecision(unassignedShard, allocation, nodeShardStates, logger)); } } - - if (logger.isTraceEnabled()) { - logger.trace( - "{} candidates for allocation: {}", - shard, - shardState.keySet().stream().map(DiscoveryNode::getName).collect(Collectors.joining(", ")) - ); - } - return new NodeShardsResult(shardStatesToNode, numberOfAllocationsFound); + return shardAllocationDecisions; } /** - * Split the list of node shard states into groups yes/no/throttle based on allocation deciders + * shardsState contain the Data, there key is DiscoveryNode but value is Map so to get one shard level data (from all the nodes), we'll traverse the map + * and construct the nodeShardState along the way before making any allocation decision. As metadata for a + * particular shard is needed from all the discovery nodes. + * + * @param unassignedShard unassigned shard + * @param shardsState fetch data result for the whole batch + * @return shard state returned from each node */ - private static NodesToAllocate buildNodesToAllocate( - RoutingAllocation allocation, - TreeMap shardStateToNode, - ShardRouting shardRouting, - boolean forceAllocate + private static NodeShardStates adaptToNodeShardStates( + ShardRouting unassignedShard, + FetchResult shardsState ) { - List yesNodeShards = new ArrayList<>(); - List throttledNodeShards = new ArrayList<>(); - List noNodeShards = new ArrayList<>(); - for (Map.Entry nodeShardState : shardStateToNode.entrySet()) { - RoutingNode node = allocation.routingNodes().node(nodeShardState.getValue().getId()); - if (node == null) { - continue; - } - - Decision decision = forceAllocate - ? allocation.deciders().canForceAllocatePrimary(shardRouting, node, allocation) - : allocation.deciders().canAllocate(shardRouting, node, allocation); - DecidedNode decidedNode = new DecidedNode(nodeShardState.getKey(), decision, nodeShardState.getValue()); - if (decision.type() == Type.THROTTLE) { - throttledNodeShards.add(decidedNode); - } else if (decision.type() == Type.NO) { - noNodeShards.add(decidedNode); - } else { - yesNodeShards.add(decidedNode); - } - } - return new NodesToAllocate( - Collections.unmodifiableList(yesNodeShards), - Collections.unmodifiableList(throttledNodeShards), - Collections.unmodifiableList(noNodeShards) - ); - } - - private static class NodeShardsResult { - final TreeMap orderedAllocationCandidates; - final int allocationsFound; - - NodeShardsResult(TreeMap orderedAllocationCandidates, int allocationsFound) { - this.orderedAllocationCandidates = orderedAllocationCandidates; - this.allocationsFound = allocationsFound; - } - } - - static class NodesToAllocate { - final List yesNodeShards; - final List throttleNodeShards; - final List noNodeShards; - - NodesToAllocate(List yesNodeShards, List throttleNodeShards, List noNodeShards) { - this.yesNodeShards = yesNodeShards; - this.throttleNodeShards = throttleNodeShards; - this.noNodeShards = noNodeShards; - } - } - - /** - * This class encapsulates the shard state retrieved from a node and the decision that was made - * by the allocator for allocating to the node that holds the shard copy. - */ - private static class DecidedNode { - final NodeGatewayStartedShards nodeShardState; - final Decision decision; - final DiscoveryNode node; - - private DecidedNode(NodeGatewayStartedShards nodeShardState, Decision decision, DiscoveryNode node) { - this.nodeShardState = nodeShardState; - this.decision = decision; - this.node = node; - } - - public DiscoveryNode getNode() { - return node; - } + NodeShardStates nodeShardStates = new NodeShardStates(); + Map nodeResponses = shardsState.getData(); + + // build data for a shard from all the nodes + nodeResponses.forEach((node, nodeGatewayStartedShardsBatch) -> { + NodeGatewayStartedShards shardData = nodeGatewayStartedShardsBatch.getNodeGatewayStartedShardsBatch() + .get(unassignedShard.shardId()); + nodeShardStates.getNodeShardStates() + .add( + new NodeShardState( + node, + shardData.allocationId(), + shardData.primary(), + shardData.replicationCheckpoint(), + shardData.storeException() + ) + ); + }); + return nodeShardStates; } } diff --git a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java new file mode 100644 index 0000000000000..b8703192f09b3 --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java @@ -0,0 +1,339 @@ +/* + * 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.lucene.codecs.Codec; +import org.junit.Before; +import org.opensearch.Version; +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.OpenSearchAllocationTestCase; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.RoutingNodes; +import org.opensearch.cluster.routing.RoutingTable; +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.AllocationDecision; +import org.opensearch.cluster.routing.allocation.RoutingAllocation; +import org.opensearch.cluster.routing.allocation.decider.AllocationDeciders; +import org.opensearch.common.Nullable; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.set.Sets; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.env.Environment; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.codec.CodecService; +import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.test.IndexSettingsModule; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.opensearch.cluster.routing.UnassignedInfo.Reason.CLUSTER_RECOVERED; + +public class PrimaryShardBatchAllocatorTests extends OpenSearchAllocationTestCase { + + private final ShardId shardId = new ShardId("test", "_na_", 0); + private static Set shardsInBatch; + private final DiscoveryNode node1 = newNode("node1"); + private final DiscoveryNode node2 = newNode("node2"); + private final DiscoveryNode node3 = newNode("node3"); + private TestBatchAllocator batchAllocator; + + public static void setUpShards(int numberOfShards) { + shardsInBatch = new HashSet<>(); + for (int shardNumber = 0; shardNumber < numberOfShards; shardNumber++) { + ShardId shardId = new ShardId("test", "_na_", shardNumber); + shardsInBatch.add(shardId); + } + } + + @Before + public void buildTestAllocator() { + this.batchAllocator = new TestBatchAllocator(); + } + + private void allocateAllUnassigned(final RoutingAllocation allocation) { + final RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); + while (iterator.hasNext()) { + batchAllocator.allocateUnassigned(iterator.next(), allocation, iterator); + } + } + + private void allocateAllUnassignedBatch(final RoutingAllocation allocation) { + final RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); + Set shardsToBatch = new HashSet<>(); + while (iterator.hasNext()) { + shardsToBatch.add(iterator.next()); + } + batchAllocator.allocateUnassignedBatch(shardsToBatch, allocation); + } + + public void testMakeAllocationDecisionDataFetching() { + final RoutingAllocation allocation = routingAllocationWithOnePrimary(noAllocationDeciders(), CLUSTER_RECOVERED, "allocId1"); + + Set shards = new HashSet<>(); + allocateAllUnassignedBatch(allocation); + ShardRouting shard = allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).primaryShard(); + shards.add(shard); + HashMap allDecisions = batchAllocator.makeAllocationDecision(shards, allocation, logger); + // verify we get decisions for all the shards + assertEquals(shards.size(), allDecisions.size()); + assertEquals(shards, allDecisions.keySet()); + assertEquals(AllocationDecision.AWAITING_INFO, allDecisions.get(shard).getAllocationDecision()); + } + + public void testMakeAllocationDecisionForReplicaShard() { + final RoutingAllocation allocation = routingAllocationWithOnePrimary(noAllocationDeciders(), CLUSTER_RECOVERED, "allocId1"); + + List replicaShards = allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).replicaShards(); + Set shards = new HashSet<>(replicaShards); + HashMap allDecisions = batchAllocator.makeAllocationDecision(shards, allocation, logger); + // verify we get decisions for all the shards + assertEquals(shards.size(), allDecisions.size()); + assertEquals(shards, allDecisions.keySet()); + assertEquals(false, allDecisions.get(replicaShards.get(0)).isDecisionTaken()); + } + + public void testMakeAllocationDecisionDataFetched() { + final RoutingAllocation allocation = routingAllocationWithOnePrimary(noAllocationDeciders(), CLUSTER_RECOVERED, "allocId1"); + + Set shards = new HashSet<>(); + ShardRouting shard = allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).primaryShard(); + shards.add(shard); + batchAllocator.addData(node1, "allocId1", true, new ReplicationCheckpoint(shardId, 20, 101, 1, Codec.getDefault().getName())); + HashMap allDecisions = batchAllocator.makeAllocationDecision(shards, allocation, logger); + // verify we get decisions for all the shards + assertEquals(shards.size(), allDecisions.size()); + assertEquals(shards, allDecisions.keySet()); + assertEquals(AllocationDecision.YES, allDecisions.get(shard).getAllocationDecision()); + } + + public void testMakeAllocationDecisionDataFetchedMultipleShards() { + setUpShards(2); + final RoutingAllocation allocation = routingAllocationWithMultiplePrimaries( + noAllocationDeciders(), + CLUSTER_RECOVERED, + 2, + 0, + "allocId-0", + "allocId-1" + ); + Set shards = new HashSet<>(); + for (ShardId shardId : shardsInBatch) { + ShardRouting shard = allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).primaryShard(); + allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).primaryShard().recoverySource(); + shards.add(shard); + batchAllocator.addShardData( + node1, + "allocId-" + shardId.id(), + shardId, + true, + new ReplicationCheckpoint(shardId, 20, 101, 1, Codec.getDefault().getName()), + null + ); + } + HashMap allDecisions = batchAllocator.makeAllocationDecision(shards, allocation, logger); + // verify we get decisions for all the shards + assertEquals(shards.size(), allDecisions.size()); + assertEquals(shards, allDecisions.keySet()); + for (ShardRouting shard : shards) { + assertEquals(AllocationDecision.YES, allDecisions.get(shard).getAllocationDecision()); + } + } + + private RoutingAllocation routingAllocationWithOnePrimary( + AllocationDeciders deciders, + UnassignedInfo.Reason reason, + String... activeAllocationIds + ) { + Metadata metadata = Metadata.builder() + .put( + IndexMetadata.builder(shardId.getIndexName()) + .settings(settings(Version.CURRENT)) + .numberOfShards(1) + .numberOfReplicas(1) + .putInSyncAllocationIds(shardId.id(), Sets.newHashSet(activeAllocationIds)) + ) + .build(); + RoutingTable.Builder routingTableBuilder = RoutingTable.builder(); + switch (reason) { + + case INDEX_CREATED: + routingTableBuilder.addAsNew(metadata.index(shardId.getIndex())); + break; + case CLUSTER_RECOVERED: + routingTableBuilder.addAsRecovery(metadata.index(shardId.getIndex())); + break; + case INDEX_REOPENED: + routingTableBuilder.addAsFromCloseToOpen(metadata.index(shardId.getIndex())); + break; + default: + throw new IllegalArgumentException("can't do " + reason + " for you. teach me"); + } + ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTableBuilder.build()) + .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) + .build(); + return new RoutingAllocation(deciders, new RoutingNodes(state, false), state, null, null, System.nanoTime()); + } + + private RoutingAllocation routingAllocationWithMultiplePrimaries( + AllocationDeciders deciders, + UnassignedInfo.Reason reason, + int numberOfShards, + int replicas, + String... activeAllocationIds + ) { + Iterator shardIterator = shardsInBatch.iterator(); + Metadata metadata = Metadata.builder() + .put( + IndexMetadata.builder(shardId.getIndexName()) + .settings(settings(Version.CURRENT)) + .numberOfShards(numberOfShards) + .numberOfReplicas(replicas) + .putInSyncAllocationIds(shardIterator.next().id(), Sets.newHashSet(activeAllocationIds[0])) + .putInSyncAllocationIds(shardIterator.next().id(), Sets.newHashSet(activeAllocationIds[1])) + ) + .build(); + + RoutingTable.Builder routingTableBuilder = RoutingTable.builder(); + for (ShardId shardIdFromBatch : shardsInBatch) { + switch (reason) { + case INDEX_CREATED: + routingTableBuilder.addAsNew(metadata.index(shardIdFromBatch.getIndex())); + break; + case CLUSTER_RECOVERED: + routingTableBuilder.addAsRecovery(metadata.index(shardIdFromBatch.getIndex())); + break; + case INDEX_REOPENED: + routingTableBuilder.addAsFromCloseToOpen(metadata.index(shardIdFromBatch.getIndex())); + break; + default: + throw new IllegalArgumentException("can't do " + reason + " for you. teach me"); + } + } + ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTableBuilder.build()) + .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) + .build(); + return new RoutingAllocation(deciders, new RoutingNodes(state, false), state, null, null, System.nanoTime()); + } + + class TestBatchAllocator extends PrimaryShardBatchAllocator { + + private Map data; + + public TestBatchAllocator clear() { + data = null; + return this; + } + + public TestBatchAllocator addData( + DiscoveryNode node, + String allocationId, + boolean primary, + ReplicationCheckpoint replicationCheckpoint + ) { + return addData(node, allocationId, primary, replicationCheckpoint, null); + } + + public TestBatchAllocator addData(DiscoveryNode node, String allocationId, boolean primary) { + Settings nodeSettings = Settings.builder().put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()).build(); + IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test", nodeSettings); + return addData( + node, + allocationId, + primary, + ReplicationCheckpoint.empty(shardId, new CodecService(null, indexSettings, null).codec("default").getName()), + null + ); + } + + public TestBatchAllocator addData(DiscoveryNode node, String allocationId, boolean primary, @Nullable Exception storeException) { + Settings nodeSettings = Settings.builder().put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()).build(); + IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test", nodeSettings); + return addData( + node, + allocationId, + primary, + ReplicationCheckpoint.empty(shardId, new CodecService(null, indexSettings, null).codec("default").getName()), + storeException + ); + } + + public TestBatchAllocator addData( + DiscoveryNode node, + String allocationId, + boolean primary, + ReplicationCheckpoint replicationCheckpoint, + @Nullable Exception storeException + ) { + if (data == null) { + data = new HashMap<>(); + } + Map shardData = Map.of( + shardId, + new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards( + allocationId, + primary, + replicationCheckpoint, + storeException + ) + ); + data.put(node, new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch(node, shardData)); + return this; + } + + public TestBatchAllocator addShardData( + DiscoveryNode node, + String allocationId, + ShardId shardId, + boolean primary, + ReplicationCheckpoint replicationCheckpoint, + @Nullable Exception storeException + ) { + if (data == null) { + data = new HashMap<>(); + } + Map shardData = new HashMap<>(); + shardData.put( + shardId, + new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards( + allocationId, + primary, + replicationCheckpoint, + storeException + ) + ); + if (data.get(node) != null) shardData.putAll(data.get(node).getNodeGatewayStartedShardsBatch()); + data.put(node, new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch(node, shardData)); + return this; + } + + @Override + protected AsyncShardFetch.FetchResult fetchData( + Set shardsEligibleForFetch, + Set inEligibleShards, + RoutingAllocation allocation + ) { + return new AsyncShardFetch.FetchResult<>(data, Collections.>emptyMap()); + } + } +} From 1a6894006232fd50a41cf56458d0a36b965c3098 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Mon, 4 Dec 2023 15:56:38 +0530 Subject: [PATCH 03/29] Throw exception for single shard calls Signed-off-by: Aman Khare --- .../gateway/PrimaryShardBatchAllocator.java | 56 ++++++++++--------- 1 file changed, 31 insertions(+), 25 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java index 0dfd8d5718e54..4cb313e7c9927 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java @@ -9,6 +9,7 @@ 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.AllocationStatus; @@ -16,9 +17,10 @@ import org.opensearch.cluster.routing.allocation.NodeAllocationResult; import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.gateway.AsyncShardFetch.FetchResult; -import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch; -import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards; +import org.opensearch.gateway.TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShards; +import org.opensearch.gateway.TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShardsBatch; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -56,7 +58,8 @@ protected FetchResult makeAllocationDecision( ) { HashMap shardAllocationDecisions = new HashMap<>(); final boolean explain = allocation.debugDecision(); - Set shardsEligibleForFetch = new HashSet<>(); - Set shardsNotEligibleForFetch = new HashSet<>(); + Set eligibleShards = new HashSet<>(); + Set inEligibleShards = new HashSet<>(); // identify ineligible shards for (ShardRouting shard : shards) { AllocateUnassignedDecision decision = getInEligibleShardDecision(shard, allocation); if (decision != null) { - shardsNotEligibleForFetch.add(shard); + inEligibleShards.add(shard); shardAllocationDecisions.put(shard, decision); } else { - shardsEligibleForFetch.add(shard); + eligibleShards.add(shard); } } // Do not call fetchData if there are no eligible shards - if (shardsEligibleForFetch.size() == 0) { + if (eligibleShards.isEmpty()) { return shardAllocationDecisions; } // only fetch data for eligible shards final FetchResult shardsState = fetchData( - shardsEligibleForFetch, - shardsNotEligibleForFetch, + eligibleShards, + inEligibleShards, allocation ); + // Note : shardsState contain the Data, there key is DiscoveryNode but value is Map so to get one shard level data (from all the nodes), we'll traverse the map + // and construct the nodeShardState along the way before making any allocation decision. As metadata for a + // particular shard is needed from all the discovery nodes. // process the received data - for (ShardRouting unassignedShard : shardsEligibleForFetch) { + for (ShardRouting unassignedShard : eligibleShards) { if (shardsState.hasData() == false) { // if fetching is not done, add that no decision in the resultant map allocation.setHasPendingAsyncFetch(); @@ -119,7 +126,7 @@ public HashMap makeAllocationDecision( ); } else { - NodeShardStates nodeShardStates = adaptToNodeShardStates(unassignedShard, shardsState); + List nodeShardStates = adaptToNodeShardStates(unassignedShard, shardsState); // get allocation decision for this shard shardAllocationDecisions.put(unassignedShard, getAllocationDecision(unassignedShard, allocation, nodeShardStates, logger)); } @@ -137,27 +144,26 @@ public HashMap makeAllocationDecision( * @param shardsState fetch data result for the whole batch * @return shard state returned from each node */ - private static NodeShardStates adaptToNodeShardStates( + private static List adaptToNodeShardStates( ShardRouting unassignedShard, FetchResult shardsState ) { - NodeShardStates nodeShardStates = new NodeShardStates(); + List nodeShardStates = new ArrayList<>(); Map nodeResponses = shardsState.getData(); // build data for a shard from all the nodes nodeResponses.forEach((node, nodeGatewayStartedShardsBatch) -> { NodeGatewayStartedShards shardData = nodeGatewayStartedShardsBatch.getNodeGatewayStartedShardsBatch() .get(unassignedShard.shardId()); - nodeShardStates.getNodeShardStates() - .add( - new NodeShardState( - node, - shardData.allocationId(), - shardData.primary(), - shardData.replicationCheckpoint(), - shardData.storeException() - ) - ); + nodeShardStates.add( + new NodeShardState( + node, + shardData.allocationId(), + shardData.primary(), + shardData.replicationCheckpoint(), + shardData.storeException() + ) + ); }); return nodeShardStates; } From dc7eb43e37ecc91e1cf0b797b8bc98e9817bd8b8 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Wed, 13 Dec 2023 15:26:30 +0530 Subject: [PATCH 04/29] Modify according to transport PRs. Signed-off-by: Aman Khare --- .../gateway/PrimaryShardBatchAllocator.java | 62 +++++++------------ 1 file changed, 23 insertions(+), 39 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java index 4cb313e7c9927..6a13df1b688ba 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java @@ -5,19 +5,17 @@ * this file be licensed under the Apache-2.0 license or a * compatible open source license. */ + package org.opensearch.gateway; import org.apache.logging.log4j.Logger; import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.cluster.routing.RoutingNode; import org.opensearch.cluster.routing.RoutingNodes; import org.opensearch.cluster.routing.ShardRouting; -import org.opensearch.cluster.routing.UnassignedInfo.AllocationStatus; import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; -import org.opensearch.cluster.routing.allocation.NodeAllocationResult; import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.gateway.AsyncShardFetch.FetchResult; -import org.opensearch.gateway.TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShards; +import org.opensearch.gateway.TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShard; import org.opensearch.gateway.TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShardsBatch; import java.util.ArrayList; @@ -64,12 +62,11 @@ protected FetchResult(Collections.singletonList(unassignedShard)), allocation, logger).get(unassignedShard); } /** - * Build allocation decisions for all the shards given to this allocator. + * Build allocation decisions for all the shards present in the batch identified by batchId. * * @param shards set of shards given for allocation * @param allocation current allocation of all the shards @@ -83,7 +80,6 @@ public HashMap makeAllocationDecision( Logger logger ) { HashMap shardAllocationDecisions = new HashMap<>(); - final boolean explain = allocation.debugDecision(); Set eligibleShards = new HashSet<>(); Set inEligibleShards = new HashSet<>(); // identify ineligible shards @@ -101,42 +97,27 @@ public HashMap makeAllocationDecision( return shardAllocationDecisions; } // only fetch data for eligible shards - final FetchResult shardsState = fetchData( - eligibleShards, - inEligibleShards, - allocation - ); - // Note : shardsState contain the Data, there key is DiscoveryNode but value is Map so to get one shard level data (from all the nodes), we'll traverse the map - // and construct the nodeShardState along the way before making any allocation decision. As metadata for a - // particular shard is needed from all the discovery nodes. + final FetchResult shardsState = fetchData(eligibleShards, inEligibleShards, allocation); // process the received data for (ShardRouting unassignedShard : eligibleShards) { - if (shardsState.hasData() == false) { - // if fetching is not done, add that no decision in the resultant map - allocation.setHasPendingAsyncFetch(); - List nodeDecisions = null; - if (explain) { - nodeDecisions = buildDecisionsForAllNodes(unassignedShard, allocation); - } - shardAllocationDecisions.put( - unassignedShard, - AllocateUnassignedDecision.no(AllocationStatus.FETCHING_SHARD_DATA, nodeDecisions) - ); - } else { - - List nodeShardStates = adaptToNodeShardStates(unassignedShard, shardsState); - // get allocation decision for this shard - shardAllocationDecisions.put(unassignedShard, getAllocationDecision(unassignedShard, allocation, nodeShardStates, logger)); - } + List nodeShardStates = adaptToNodeShardStates( + unassignedShard, + shardsState + ); + // get allocation decision for this shard + shardAllocationDecisions.put(unassignedShard, getAllocationDecision(unassignedShard, allocation, nodeShardStates, logger)); } return shardAllocationDecisions; } /** - * shardsState contain the Data, there key is DiscoveryNode but value is Map so to get one shard level data (from all the nodes), we'll traverse the map + * Transforms {@link FetchResult} of {@link NodeGatewayStartedShardsBatch} to {@link List} of {@link TransportNodesListGatewayStartedShards.NodeGatewayStartedShards}. + *

+ * Returns null if {@link FetchResult} does not have any data. + *

+ * shardsState contain the Data, there key is DiscoveryNode but value is Map of ShardId + * and NodeGatewayStartedShardsBatch so to get one shard level data (from all the nodes), we'll traverse the map * and construct the nodeShardState along the way before making any allocation decision. As metadata for a * particular shard is needed from all the discovery nodes. * @@ -144,19 +125,22 @@ public HashMap makeAllocationDecision( * @param shardsState fetch data result for the whole batch * @return shard state returned from each node */ - private static List adaptToNodeShardStates( + private static List adaptToNodeShardStates( ShardRouting unassignedShard, FetchResult shardsState ) { - List nodeShardStates = new ArrayList<>(); + if (!shardsState.hasData()) { + return null; + } + List nodeShardStates = new ArrayList<>(); Map nodeResponses = shardsState.getData(); // build data for a shard from all the nodes nodeResponses.forEach((node, nodeGatewayStartedShardsBatch) -> { - NodeGatewayStartedShards shardData = nodeGatewayStartedShardsBatch.getNodeGatewayStartedShardsBatch() + NodeGatewayStartedShard shardData = nodeGatewayStartedShardsBatch.getNodeGatewayStartedShardsBatch() .get(unassignedShard.shardId()); nodeShardStates.add( - new NodeShardState( + new TransportNodesListGatewayStartedShards.NodeGatewayStartedShards( node, shardData.allocationId(), shardData.primary(), From e0c49437e89f02a8c5bb2126d7b3903682d1dfcc Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Tue, 27 Feb 2024 13:41:38 +0530 Subject: [PATCH 05/29] Use List instead of Set Signed-off-by: Aman Khare --- .../gateway/PrimaryShardBatchAllocator.java | 18 ++++----- .../PrimaryShardBatchAllocatorTests.java | 37 ++++++++++--------- 2 files changed, 27 insertions(+), 28 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java index 6a13df1b688ba..d073482405ec2 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java @@ -15,16 +15,14 @@ import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.gateway.AsyncShardFetch.FetchResult; -import org.opensearch.gateway.TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShard; -import org.opensearch.gateway.TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShardsBatch; +import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard; +import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch; 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; /** * PrimaryShardBatchAllocator is similar to {@link org.opensearch.gateway.PrimaryShardAllocator} only difference is @@ -47,8 +45,8 @@ public abstract class PrimaryShardBatchAllocator extends PrimaryShardAllocator { abstract protected FetchResult fetchData( - Set shardsEligibleForFetch, - Set inEligibleShards, + List eligibleShards, + List inEligibleShards, RoutingAllocation allocation ); @@ -62,7 +60,7 @@ protected FetchResult(Collections.singletonList(unassignedShard)), allocation, logger).get(unassignedShard); + return makeAllocationDecision(Collections.singletonList(unassignedShard), allocation, logger).get(unassignedShard); } /** @@ -75,13 +73,13 @@ public AllocateUnassignedDecision makeAllocationDecision(ShardRouting unassigned */ @Override public HashMap makeAllocationDecision( - Set shards, + List shards, RoutingAllocation allocation, Logger logger ) { HashMap shardAllocationDecisions = new HashMap<>(); - Set eligibleShards = new HashSet<>(); - Set inEligibleShards = new HashSet<>(); + List eligibleShards = new ArrayList<>(); + List inEligibleShards = new ArrayList<>(); // identify ineligible shards for (ShardRouting shard : shards) { AllocateUnassignedDecision decision = getInEligibleShardDecision(shard, allocation); diff --git a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java index b8703192f09b3..a95328b9ddc39 100644 --- a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java @@ -8,7 +8,6 @@ package org.opensearch.gateway; import org.apache.lucene.codecs.Codec; -import org.junit.Before; import org.opensearch.Version; import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.ClusterState; @@ -34,7 +33,9 @@ import org.opensearch.index.codec.CodecService; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.test.IndexSettingsModule; +import org.junit.Before; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -76,7 +77,7 @@ private void allocateAllUnassigned(final RoutingAllocation allocation) { private void allocateAllUnassignedBatch(final RoutingAllocation allocation) { final RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); - Set shardsToBatch = new HashSet<>(); + List shardsToBatch = new ArrayList<>(); while (iterator.hasNext()) { shardsToBatch.add(iterator.next()); } @@ -86,14 +87,14 @@ private void allocateAllUnassignedBatch(final RoutingAllocation allocation) { public void testMakeAllocationDecisionDataFetching() { final RoutingAllocation allocation = routingAllocationWithOnePrimary(noAllocationDeciders(), CLUSTER_RECOVERED, "allocId1"); - Set shards = new HashSet<>(); + List shards = new ArrayList<>(); allocateAllUnassignedBatch(allocation); ShardRouting shard = allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).primaryShard(); shards.add(shard); HashMap allDecisions = batchAllocator.makeAllocationDecision(shards, allocation, logger); // verify we get decisions for all the shards assertEquals(shards.size(), allDecisions.size()); - assertEquals(shards, allDecisions.keySet()); + assertEquals(shards, new ArrayList<>(allDecisions.keySet())); assertEquals(AllocationDecision.AWAITING_INFO, allDecisions.get(shard).getAllocationDecision()); } @@ -101,25 +102,25 @@ public void testMakeAllocationDecisionForReplicaShard() { final RoutingAllocation allocation = routingAllocationWithOnePrimary(noAllocationDeciders(), CLUSTER_RECOVERED, "allocId1"); List replicaShards = allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).replicaShards(); - Set shards = new HashSet<>(replicaShards); + List shards = new ArrayList<>(replicaShards); HashMap allDecisions = batchAllocator.makeAllocationDecision(shards, allocation, logger); // verify we get decisions for all the shards assertEquals(shards.size(), allDecisions.size()); - assertEquals(shards, allDecisions.keySet()); - assertEquals(false, allDecisions.get(replicaShards.get(0)).isDecisionTaken()); + assertEquals(shards, new ArrayList<>(allDecisions.keySet())); + assertFalse(allDecisions.get(replicaShards.get(0)).isDecisionTaken()); } public void testMakeAllocationDecisionDataFetched() { final RoutingAllocation allocation = routingAllocationWithOnePrimary(noAllocationDeciders(), CLUSTER_RECOVERED, "allocId1"); - Set shards = new HashSet<>(); + List shards = new ArrayList<>(); ShardRouting shard = allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).primaryShard(); shards.add(shard); batchAllocator.addData(node1, "allocId1", true, new ReplicationCheckpoint(shardId, 20, 101, 1, Codec.getDefault().getName())); HashMap allDecisions = batchAllocator.makeAllocationDecision(shards, allocation, logger); // verify we get decisions for all the shards assertEquals(shards.size(), allDecisions.size()); - assertEquals(shards, allDecisions.keySet()); + assertEquals(shards, new ArrayList<>(allDecisions.keySet())); assertEquals(AllocationDecision.YES, allDecisions.get(shard).getAllocationDecision()); } @@ -133,7 +134,7 @@ public void testMakeAllocationDecisionDataFetchedMultipleShards() { "allocId-0", "allocId-1" ); - Set shards = new HashSet<>(); + List shards = new ArrayList<>(); for (ShardId shardId : shardsInBatch) { ShardRouting shard = allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).primaryShard(); allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).primaryShard().recoverySource(); @@ -150,7 +151,7 @@ public void testMakeAllocationDecisionDataFetchedMultipleShards() { HashMap allDecisions = batchAllocator.makeAllocationDecision(shards, allocation, logger); // verify we get decisions for all the shards assertEquals(shards.size(), allDecisions.size()); - assertEquals(shards, allDecisions.keySet()); + assertEquals(new HashSet<>(shards), allDecisions.keySet()); for (ShardRouting shard : shards) { assertEquals(AllocationDecision.YES, allDecisions.get(shard).getAllocationDecision()); } @@ -185,7 +186,7 @@ private RoutingAllocation routingAllocationWithOnePrimary( default: throw new IllegalArgumentException("can't do " + reason + " for you. teach me"); } - ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + ClusterState state = ClusterState.builder(org.opensearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) .metadata(metadata) .routingTable(routingTableBuilder.build()) .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) @@ -288,9 +289,9 @@ public TestBatchAllocator addData( if (data == null) { data = new HashMap<>(); } - Map shardData = Map.of( + Map shardData = Map.of( shardId, - new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards( + new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard( allocationId, primary, replicationCheckpoint, @@ -312,10 +313,10 @@ public TestBatchAllocator addShardData( if (data == null) { data = new HashMap<>(); } - Map shardData = new HashMap<>(); + Map shardData = new HashMap<>(); shardData.put( shardId, - new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards( + new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard( allocationId, primary, replicationCheckpoint, @@ -329,8 +330,8 @@ public TestBatchAllocator addShardData( @Override protected AsyncShardFetch.FetchResult fetchData( - Set shardsEligibleForFetch, - Set inEligibleShards, + List shardsEligibleForFetch, + List inEligibleShards, RoutingAllocation allocation ) { return new AsyncShardFetch.FetchResult<>(data, Collections.>emptyMap()); From 446881804a54203b9dd7233ccbba53089a0c83ea Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Thu, 29 Feb 2024 17:21:02 +0530 Subject: [PATCH 06/29] Add ShardBatchCache to handle the responses of batch trasport calls Signed-off-by: Aman Khare --- .../gateway/AsyncShardBatchFetch.java | 57 ++++ .../opensearch/gateway/BaseShardResponse.java | 37 +++ .../opensearch/gateway/ShardBatchCache.java | 244 ++++++++++++++++++ 3 files changed, 338 insertions(+) create mode 100644 server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java create mode 100644 server/src/main/java/org/opensearch/gateway/BaseShardResponse.java create mode 100644 server/src/main/java/org/opensearch/gateway/ShardBatchCache.java diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java new file mode 100644 index 0000000000000..24ec4043531ee --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java @@ -0,0 +1,57 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway; + +import org.apache.logging.log4j.Logger; +import org.opensearch.action.support.nodes.BaseNodeResponse; +import org.opensearch.action.support.nodes.BaseNodesResponse; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.indices.store.ShardAttributes; + +import java.util.Map; +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.function.Supplier; + +/** + * Implementation of AsyncShardFetchAbstract with batching support. + * @param Response type of the transport action. + * @param Data type of shard level response. + */ +public abstract class AsyncShardBatchFetch + extends AsyncShardFetch{ + + @SuppressWarnings("unchecked") + AsyncShardBatchFetch( + Logger logger, + String type, + Map shardToCustomDataPath, + AsyncShardFetch.Lister, T> action, + String batchId, + Class clazz, + BiFunction, T> responseGetter, + Function> shardsBatchDataGetter, + Supplier emptyResponseBuilder + ) { + super(logger, type, shardToCustomDataPath, action, batchId); + this.shardCache = new ShardBatchCache<>(logger, type, shardToCustomDataPath, "BatchID=[" + batchId+ "]" + , clazz, responseGetter, shardsBatchDataGetter, emptyResponseBuilder); + } + + /** + * Remove a shard from the cache maintaining a full batch of shards. This is needed to clear the shard once it's + * assigned or failed. + * @param shardId shardId to be removed from the batch. + */ + public void clearShard(ShardId shardId) { + this.shardAttributesMap.remove(shardId); + this.cache.clearShardCache(shardId); + } +} diff --git a/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java b/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java new file mode 100644 index 0000000000000..7c4ddf28705cb --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java @@ -0,0 +1,37 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway; + +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.transport.TransportResponse; + +import java.io.IOException; + +/** + * Base response class for shard response. Provides necessary information about shard level response. Based on these + * functionalities, receiver decides if it needs to store the response or ignore it or retry the fetch. + * + * @opensearch.internal + */ +public abstract class BaseShardResponse extends TransportResponse { + public BaseShardResponse(){} + + public abstract boolean isEmpty(); + + public abstract Exception getException(); + + public BaseShardResponse(StreamInput in) throws IOException { + super(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + } +} diff --git a/server/src/main/java/org/opensearch/gateway/ShardBatchCache.java b/server/src/main/java/org/opensearch/gateway/ShardBatchCache.java new file mode 100644 index 0000000000000..c19232d0101ff --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/ShardBatchCache.java @@ -0,0 +1,244 @@ +/* + * 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.OpenSearchTimeoutException; +import org.opensearch.action.support.nodes.BaseNodeResponse; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.common.Nullable; +import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.indices.store.ShardAttributes; +import org.opensearch.transport.ReceiveTimeoutTransportException; + +import java.lang.reflect.Array; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.function.Supplier; + +/** + * Cache implementation of transport actions returning batch of shards related data in the response. + * + * @param Response type of transport action. + * @param Data type of shard level response. + */ +public class ShardBatchCache extends BaseShardCache { + private final Map> cache = new HashMap<>(); + private final Map shardIdToArray = new HashMap<>(); // used for mapping array index for a shard + private final AtomicInteger shardIdIndex = new AtomicInteger(); + private final int batchSize; + private final Class shardResponseClass; + private final BiFunction, T> responseConstructor; + private final Map arrayToShardId = new HashMap<>(); + private final Function> shardsBatchDataGetter; + private final Supplier emptyResponseBuilder; + private final Set failedShards; + + public ShardBatchCache(Logger logger, String type, + Map shardToCustomDataPath, String logKey, Class clazz, + BiFunction, T> responseGetter, Function> shardsBatchDataGetter, Supplier emptyResponseBuilder) { + super(logger, logKey, type); + this.batchSize = shardToCustomDataPath.size(); + fillShardIdKeys(shardToCustomDataPath.keySet()); + this.shardResponseClass = clazz; + this.responseConstructor = responseGetter; + this.shardsBatchDataGetter = shardsBatchDataGetter; + this.emptyResponseBuilder = emptyResponseBuilder; + failedShards = new HashSet<>(); + } + + @Override + public Map getCache() { + return cache; + } + + @Override + public void clearShardCache(ShardId shardId) { + if (shardIdToArray.containsKey(shardId)) { + Integer shardIdIndex = shardIdToArray.remove(shardId); + for (String nodeId : cache.keySet()) { + cache.get(nodeId).clearShard(shardIdIndex); + } + } + } + + @Override + public Map getCacheData(DiscoveryNodes nodes, Set failedNodes) { + refreshReverseIdMap(); + return super.getCacheData(nodes, failedNodes); + } + + /** + * Build a reverse map to get shardId from the array index, this will be used to construct the response which + * PrimaryShardBatchAllocator or ReplicaShardBatchAllocator are looking for. + */ + private void refreshReverseIdMap() { + arrayToShardId.clear(); + for (ShardId shardId : shardIdToArray.keySet()) { + arrayToShardId.putIfAbsent(shardIdToArray.get(shardId), shardId); + } + } + + @Override + public void initData(DiscoveryNode node) { + cache.put(node.getId(), new NodeEntry<>(node.getId(), shardResponseClass, batchSize)); + } + + /** + * Put the response received from data nodes into the cache. + * Get shard level data from batch, then filter out if any shards received failures. + * After that, complete storing the data at node level and mark fetching as done. + * @param node node from which we got the response. + * @param response shard metadata coming from node. + */ + @Override + public void putData(DiscoveryNode node, T response) { + NodeEntry nodeEntry = cache.get(node.getId()); + Map batchResponse = shardsBatchDataGetter.apply(response); + failedShards.addAll(filterFailedShards(batchResponse)); + nodeEntry.doneFetching(batchResponse, shardIdToArray); + } + + /** + * Return the shard for which we got unhandled exceptions. + * + * @param batchResponse response from one node for the batch. + * @return List of failed shards. + */ + private List filterFailedShards(Map batchResponse) { + logger.trace("filtering failed shards"); + List failedShards = new ArrayList<>(); + for (Iterator it = batchResponse.keySet().iterator(); it.hasNext(); ) { + ShardId shardId = it.next(); + if (batchResponse.get(shardId) != null) { + if (batchResponse.get(shardId).getException() != null) { + // handle per shard level exceptions, process other shards, only throw out this shard from + // the batch + Exception shardException = batchResponse.get(shardId).getException(); + // if the request got rejected or timed out, we need to try it again next time... + if (shardException instanceof OpenSearchRejectedExecutionException + || shardException instanceof ReceiveTimeoutTransportException + || shardException instanceof OpenSearchTimeoutException) { + logger.trace("got unhandled retryable exception for shard {} {}", shardId.toString(), + shardException.toString()); + failedShards.add(shardId); + // remove this failed entry. So, while storing the data, we don't need to re-process it. + it.remove(); + } + } + } + } + return failedShards; + } + + @Override + public T getData(DiscoveryNode node) { + return this.responseConstructor.apply(node, getBatchData(cache.get(node.getId()))); + } + + @Override + public List getFailedShards() { + List defectedShards = List.copyOf(failedShards); + failedShards.clear(); + return defectedShards; + } + + private HashMap getBatchData(NodeEntry nodeEntry) { + V[] nodeShardEntries = nodeEntry.getData(); + boolean[] emptyResponses = nodeEntry.getEmptyShardResponse(); + HashMap shardData = new HashMap<>(); + for (Integer shardIdIndex : shardIdToArray.values()) { + if (emptyResponses[shardIdIndex]) { + shardData.put(arrayToShardId.get(shardIdIndex), emptyResponseBuilder.get()); + } else if (nodeShardEntries[shardIdIndex] != null) { + // ignore null responses here + shardData.put(arrayToShardId.get(shardIdIndex), + nodeShardEntries[shardIdIndex]); + } + } + return shardData; + } + + private void fillShardIdKeys(Set shardIds) { + for (ShardId shardId : shardIds) { + this.shardIdToArray.putIfAbsent(shardId, shardIdIndex.getAndIncrement()); + } + this.shardIdToArray.keySet().removeIf(shardId -> { + if (!shardIds.contains(shardId)) { + clearShardCache(shardId); + return true; + } else { + return false; + } + }); + } + + /** + * A node entry, holding the state of the fetched data for a specific shard + * for a giving node. This will only store the data from TransportNodesListGatewayStartedShardsBatch or + * TransportNodesListShardStoreMetadataBatch transport actions. + */ + static class NodeEntry extends BaseShardCache.BaseNodeEntry { + @Nullable + private final V[] shardData; + private final boolean[] emptyShardResponse; + + NodeEntry(String nodeId, Class clazz, int batchSize) { + super(nodeId); + this.shardData = (V[]) Array.newInstance(clazz, batchSize); + this.emptyShardResponse = new boolean[batchSize]; + } + + void doneFetching(Map shardDataFromNode, Map shardIdKey) { + fillShardData(shardDataFromNode, shardIdKey); + super.doneFetching(); + } + + void clearShard(Integer shardIdIndex) { + this.shardData[shardIdIndex] = null; + } + + V[] getData() { + return this.shardData; + } + + boolean[] getEmptyShardResponse() { + return emptyShardResponse; + } + + + private void fillShardData(Map shardDataFromNode, Map shardIdKey) { + for (ShardId shardId : shardDataFromNode.keySet()) { + if (shardDataFromNode.get(shardId) != null) { + if (shardDataFromNode.get(shardId).isEmpty()) { + this.emptyShardResponse[shardIdKey.get(shardId)] = true; + this.shardData[shardIdKey.get(shardId)] = null; + } else if (shardDataFromNode.get(shardId).getException() == null) { + this.shardData[shardIdKey.get(shardId)] = shardDataFromNode.get(shardId); + } + //if exception is not null, we got unhandled failure for the shard which needs to be ignored + } + } + } + } + +} + + From 3bd85a438f70ac42bf00c88aeeac5e836dc81cb0 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Fri, 1 Mar 2024 12:52:42 +0530 Subject: [PATCH 07/29] Handle shard failures by triggereing a reroute Signed-off-by: Aman Khare --- .../gateway/AsyncShardBatchFetch.java | 55 ++++++++++++++++--- .../opensearch/gateway/AsyncShardFetch.java | 2 +- .../opensearch/gateway/ShardBatchCache.java | 26 ++++++--- 3 files changed, 67 insertions(+), 16 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java index 24ec4043531ee..c44d06b813c71 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java @@ -12,42 +12,83 @@ import org.opensearch.action.support.nodes.BaseNodeResponse; import org.opensearch.action.support.nodes.BaseNodesResponse; import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.core.index.shard.ShardId; import org.opensearch.indices.store.ShardAttributes; +import java.util.List; import java.util.Map; +import java.util.Set; import java.util.function.BiFunction; +import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; /** * Implementation of AsyncShardFetchAbstract with batching support. + * cache will be created using ShardBatchCache class as that can handle the caching strategy correctly for a + * batch of shards. Other necessary functions are also stored so cache can store or get the data for both primary + * and replicas. + * * @param Response type of the transport action. * @param Data type of shard level response. */ public abstract class AsyncShardBatchFetch - extends AsyncShardFetch{ + extends AsyncShardFetch { @SuppressWarnings("unchecked") AsyncShardBatchFetch( Logger logger, String type, - Map shardToCustomDataPath, + Map shardAttributesMap, AsyncShardFetch.Lister, T> action, String batchId, Class clazz, - BiFunction, T> responseGetter, + BiFunction, T> responseConstructor, Function> shardsBatchDataGetter, - Supplier emptyResponseBuilder + Supplier emptyResponseBuilder, + Consumer handleFailedShard ) { - super(logger, type, shardToCustomDataPath, action, batchId); - this.shardCache = new ShardBatchCache<>(logger, type, shardToCustomDataPath, "BatchID=[" + batchId+ "]" - , clazz, responseGetter, shardsBatchDataGetter, emptyResponseBuilder); + super(logger, type, shardAttributesMap, action, batchId); + this.cache = new ShardBatchCache<>(logger, type, shardAttributesMap, "BatchID=[" + batchId + "]" + , clazz, responseConstructor, shardsBatchDataGetter, emptyResponseBuilder, handleFailedShard); + } + + /** + * Fetch the data for a batch of shards, this uses the already written {@link AsyncShardFetch} fetchData method. + * Based on the shards failed in last round, it makes sure to trigger a reroute for them. + * + * @param nodes all the nodes where transport call should be sent + * @param ignoreNodes nodes to update based on failures received from transport actions + * @return data received from the transport actions + */ + public synchronized FetchResult fetchData(DiscoveryNodes nodes, Map> ignoreNodes) { + List failedShards = cleanUpFailedShards(); + if (failedShards.isEmpty() == false) { + // trigger a reroute if there are any shards failed, to make sure they're picked up in next run + logger.trace("triggering another reroute for failed shards in {}", reroutingKey); + reroute("shards-failed", "shards failed in " + reroutingKey); + } + return super.fetchData(nodes, ignoreNodes); + } + + /** + * Remove the shard from shardAttributesMap so we don't send it in next fetching round. + * + * @return return the failed shards so a reroute can be triggered. + */ + private List cleanUpFailedShards() { + List failedShards = cache.getFailedShards(); + if (failedShards != null && failedShards.isEmpty() == false) { + shardAttributesMap.keySet().removeIf(failedShards::contains); + } + return failedShards; } /** * Remove a shard from the cache maintaining a full batch of shards. This is needed to clear the shard once it's * assigned or failed. + * * @param shardId shardId to be removed from the batch. */ public void clearShard(ShardId shardId) { diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java index 50774f7e0cb1c..bcadc5be1d1e2 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java @@ -89,7 +89,7 @@ public interface Lister, N private final Map> cache = new HashMap<>(); private final AtomicLong round = new AtomicLong(); private boolean closed; - private final String reroutingKey; + protected final String reroutingKey; private final Map> shardToIgnoreNodes = new HashMap<>(); private final boolean enableBatchMode; diff --git a/server/src/main/java/org/opensearch/gateway/ShardBatchCache.java b/server/src/main/java/org/opensearch/gateway/ShardBatchCache.java index c19232d0101ff..5749bf5a32a86 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardBatchCache.java +++ b/server/src/main/java/org/opensearch/gateway/ShardBatchCache.java @@ -29,39 +29,48 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; +import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; /** - * Cache implementation of transport actions returning batch of shards related data in the response. + * Cache implementation of transport actions returning batch of shards data in the response. Cache uses a specific + * NodeEntry class that stores the data in array format. To keep the class generic for primary or replica, all + * functions are stored during object creation. * * @param Response type of transport action. * @param Data type of shard level response. */ public class ShardBatchCache extends BaseShardCache { - private final Map> cache = new HashMap<>(); - private final Map shardIdToArray = new HashMap<>(); // used for mapping array index for a shard - private final AtomicInteger shardIdIndex = new AtomicInteger(); + private final Map> cache; + private final Map shardIdToArray; // used for mapping array index for a shard + private final AtomicInteger shardIdIndex; private final int batchSize; private final Class shardResponseClass; private final BiFunction, T> responseConstructor; - private final Map arrayToShardId = new HashMap<>(); + private final Map arrayToShardId; private final Function> shardsBatchDataGetter; private final Supplier emptyResponseBuilder; private final Set failedShards; + private final Consumer handleFailedShard; public ShardBatchCache(Logger logger, String type, Map shardToCustomDataPath, String logKey, Class clazz, - BiFunction, T> responseGetter, Function> shardsBatchDataGetter, Supplier emptyResponseBuilder) { + BiFunction, T> responseConstructor, Function> shardsBatchDataGetter, Supplier emptyResponseBuilder, Consumer handleFailedShard) { super(logger, logKey, type); this.batchSize = shardToCustomDataPath.size(); fillShardIdKeys(shardToCustomDataPath.keySet()); this.shardResponseClass = clazz; - this.responseConstructor = responseGetter; + this.responseConstructor = responseConstructor; this.shardsBatchDataGetter = shardsBatchDataGetter; this.emptyResponseBuilder = emptyResponseBuilder; failedShards = new HashSet<>(); + cache = new HashMap<>(); + shardIdToArray = new HashMap<>(); + arrayToShardId = new HashMap<>(); + shardIdIndex = new AtomicInteger(); + this.handleFailedShard = handleFailedShard; } @Override @@ -139,6 +148,7 @@ private List filterFailedShards(Map batchResponse) { logger.trace("got unhandled retryable exception for shard {} {}", shardId.toString(), shardException.toString()); failedShards.add(shardId); + handleFailedShard.accept(shardId); // remove this failed entry. So, while storing the data, we don't need to re-process it. it.remove(); } From bb7d560c79ec1572590eeafc59266404d7d7a312 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Fri, 1 Mar 2024 14:21:57 +0530 Subject: [PATCH 08/29] Send unhandled exception as it is and send null for known exceptions Signed-off-by: Aman Khare --- .../gateway/AsyncShardBatchFetch.java | 16 ++++++++--- .../opensearch/gateway/BaseShardResponse.java | 5 ++-- .../opensearch/gateway/ShardBatchCache.java | 28 ++++++++++--------- ...ansportNodesGatewayStartedShardHelper.java | 4 ++- ...ortNodesListGatewayStartedShardsBatch.java | 13 +++++---- 5 files changed, 40 insertions(+), 26 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java index c44d06b813c71..7d9dcd909e3df 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java @@ -33,8 +33,7 @@ * @param Response type of the transport action. * @param Data type of shard level response. */ -public abstract class AsyncShardBatchFetch - extends AsyncShardFetch { +public abstract class AsyncShardBatchFetch extends AsyncShardFetch { @SuppressWarnings("unchecked") AsyncShardBatchFetch( @@ -50,8 +49,17 @@ public abstract class AsyncShardBatchFetch handleFailedShard ) { super(logger, type, shardAttributesMap, action, batchId); - this.cache = new ShardBatchCache<>(logger, type, shardAttributesMap, "BatchID=[" + batchId + "]" - , clazz, responseConstructor, shardsBatchDataGetter, emptyResponseBuilder, handleFailedShard); + this.cache = new ShardBatchCache<>( + logger, + type, + shardAttributesMap, + "BatchID=[" + batchId + "]", + clazz, + responseConstructor, + shardsBatchDataGetter, + emptyResponseBuilder, + handleFailedShard + ); } /** diff --git a/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java b/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java index 7c4ddf28705cb..5d5585678d0a5 100644 --- a/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java +++ b/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java @@ -21,7 +21,7 @@ * @opensearch.internal */ public abstract class BaseShardResponse extends TransportResponse { - public BaseShardResponse(){} + public BaseShardResponse() {} public abstract boolean isEmpty(); @@ -32,6 +32,5 @@ public BaseShardResponse(StreamInput in) throws IOException { } @Override - public void writeTo(StreamOutput out) throws IOException { - } + public void writeTo(StreamOutput out) throws IOException {} } diff --git a/server/src/main/java/org/opensearch/gateway/ShardBatchCache.java b/server/src/main/java/org/opensearch/gateway/ShardBatchCache.java index 5749bf5a32a86..77880092374bd 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardBatchCache.java +++ b/server/src/main/java/org/opensearch/gateway/ShardBatchCache.java @@ -54,10 +54,17 @@ public class ShardBatchCache failedShards; private final Consumer handleFailedShard; - public ShardBatchCache(Logger logger, String type, - Map shardToCustomDataPath, String logKey, Class clazz, - BiFunction, T> responseConstructor, Function> shardsBatchDataGetter, Supplier emptyResponseBuilder, Consumer handleFailedShard) { + public ShardBatchCache( + Logger logger, + String type, + Map shardToCustomDataPath, + String logKey, + Class clazz, + BiFunction, T> responseConstructor, + Function> shardsBatchDataGetter, + Supplier emptyResponseBuilder, + Consumer handleFailedShard + ) { super(logger, logKey, type); this.batchSize = shardToCustomDataPath.size(); fillShardIdKeys(shardToCustomDataPath.keySet()); @@ -134,7 +141,7 @@ public void putData(DiscoveryNode node, T response) { private List filterFailedShards(Map batchResponse) { logger.trace("filtering failed shards"); List failedShards = new ArrayList<>(); - for (Iterator it = batchResponse.keySet().iterator(); it.hasNext(); ) { + for (Iterator it = batchResponse.keySet().iterator(); it.hasNext();) { ShardId shardId = it.next(); if (batchResponse.get(shardId) != null) { if (batchResponse.get(shardId).getException() != null) { @@ -145,8 +152,7 @@ private List filterFailedShards(Map batchResponse) { if (shardException instanceof OpenSearchRejectedExecutionException || shardException instanceof ReceiveTimeoutTransportException || shardException instanceof OpenSearchTimeoutException) { - logger.trace("got unhandled retryable exception for shard {} {}", shardId.toString(), - shardException.toString()); + logger.trace("got unhandled retryable exception for shard {} {}", shardId.toString(), shardException.toString()); failedShards.add(shardId); handleFailedShard.accept(shardId); // remove this failed entry. So, while storing the data, we don't need to re-process it. @@ -179,8 +185,7 @@ private HashMap getBatchData(NodeEntry nodeEntry) { shardData.put(arrayToShardId.get(shardIdIndex), emptyResponseBuilder.get()); } else if (nodeShardEntries[shardIdIndex] != null) { // ignore null responses here - shardData.put(arrayToShardId.get(shardIdIndex), - nodeShardEntries[shardIdIndex]); + shardData.put(arrayToShardId.get(shardIdIndex), nodeShardEntries[shardIdIndex]); } } return shardData; @@ -233,7 +238,6 @@ boolean[] getEmptyShardResponse() { return emptyShardResponse; } - private void fillShardData(Map shardDataFromNode, Map shardIdKey) { for (ShardId shardId : shardDataFromNode.keySet()) { if (shardDataFromNode.get(shardId) != null) { @@ -243,12 +247,10 @@ private void fillShardData(Map shardDataFromNode, Map Date: Fri, 1 Mar 2024 14:52:34 +0530 Subject: [PATCH 09/29] Move common functionalities to BaseShardResponse Signed-off-by: Aman Khare --- .../gateway/RecoveryFromGatewayIT.java | 4 +- .../opensearch/gateway/BaseShardResponse.java | 26 +++++++-- ...ransportNodesListGatewayStartedShards.java | 2 +- ...ortNodesListGatewayStartedShardsBatch.java | 54 ++++++++++--------- 4 files changed, 55 insertions(+), 31 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 9da1336642a64..fac0ff00634dc 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -812,7 +812,7 @@ public void testShardFetchCorruptedShardsUsingBatchAction() throws Exception { .get(discoveryNodes[0].getId()) .getNodeGatewayStartedShardsBatch() .get(shardId); - assertNotNull(nodeGatewayStartedShards.storeException()); + assertNotNull(nodeGatewayStartedShards.getException()); assertNotNull(nodeGatewayStartedShards.allocationId()); assertTrue(nodeGatewayStartedShards.primary()); } @@ -820,7 +820,7 @@ public void testShardFetchCorruptedShardsUsingBatchAction() throws Exception { private void assertNodeGatewayStartedShardsHappyCase( TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeGatewayStartedShards ) { - assertNull(nodeGatewayStartedShards.storeException()); + assertNull(nodeGatewayStartedShards.getException()); assertNotNull(nodeGatewayStartedShards.allocationId()); assertTrue(nodeGatewayStartedShards.primary()); } diff --git a/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java b/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java index 5d5585678d0a5..0922abf2c942f 100644 --- a/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java +++ b/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java @@ -21,16 +21,34 @@ * @opensearch.internal */ public abstract class BaseShardResponse extends TransportResponse { - public BaseShardResponse() {} + + private Exception storeException; + + public BaseShardResponse(Exception storeException) { + this.storeException = storeException; + } public abstract boolean isEmpty(); - public abstract Exception getException(); + public Exception getException() { + return storeException; + } public BaseShardResponse(StreamInput in) throws IOException { - super(in); + if (in.readBoolean()) { + storeException = in.readException(); + } else { + storeException = null; + } } @Override - public void writeTo(StreamOutput out) throws IOException {} + public void writeTo(StreamOutput out) throws IOException { + if (storeException != null) { + out.writeBoolean(true); + out.writeException(storeException); + } else { + out.writeBoolean(false); + } + } } diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java index 0ba872aab9974..732cb17120d40 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java @@ -169,7 +169,7 @@ protected NodeGatewayStartedShards nodeOperation(NodeRequest request) { shardInfo.allocationId(), shardInfo.primary(), shardInfo.replicationCheckpoint(), - shardInfo.storeException() + shardInfo.getException() ); } catch (Exception e) { throw new OpenSearchException("failed to load started shards", e); diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java index 680f81658a623..86829cfbd12db 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java @@ -260,20 +260,20 @@ public void writeTo(StreamOutput out) throws IOException { * * @opensearch.internal */ - public static class NodeGatewayStartedShard { + public static class NodeGatewayStartedShard extends BaseShardResponse { private final String allocationId; private final boolean primary; - private final Exception storeException; private final ReplicationCheckpoint replicationCheckpoint; + @Override + public boolean isEmpty() { + return allocationId == null && primary == false && getException() == null && replicationCheckpoint == null; + } + public NodeGatewayStartedShard(StreamInput in) throws IOException { + super(in); allocationId = in.readOptionalString(); primary = in.readBoolean(); - if (in.readBoolean()) { - storeException = in.readException(); - } else { - storeException = null; - } if (in.readBoolean()) { replicationCheckpoint = new ReplicationCheckpoint(in); } else { @@ -291,10 +291,10 @@ public NodeGatewayStartedShard( ReplicationCheckpoint replicationCheckpoint, Exception storeException ) { + super(storeException); this.allocationId = allocationId; this.primary = primary; this.replicationCheckpoint = replicationCheckpoint; - this.storeException = storeException; } public String allocationId() { @@ -309,19 +309,10 @@ public ReplicationCheckpoint replicationCheckpoint() { return this.replicationCheckpoint; } - public Exception storeException() { - return this.storeException; - } - public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); out.writeOptionalString(allocationId); out.writeBoolean(primary); - if (storeException != null) { - out.writeBoolean(true); - out.writeException(storeException); - } else { - out.writeBoolean(false); - } if (replicationCheckpoint != null) { out.writeBoolean(true); replicationCheckpoint.writeTo(out); @@ -343,7 +334,7 @@ public boolean equals(Object o) { return primary == that.primary && Objects.equals(allocationId, that.allocationId) - && Objects.equals(storeException, that.storeException) + && Objects.equals(getException(), that.getException()) && Objects.equals(replicationCheckpoint, that.replicationCheckpoint); } @@ -351,7 +342,7 @@ public boolean equals(Object o) { public int hashCode() { int result = (allocationId != null ? allocationId.hashCode() : 0); result = 31 * result + (primary ? 1 : 0); - result = 31 * result + (storeException != null ? storeException.hashCode() : 0); + result = 31 * result + (getException() != null ? getException().hashCode() : 0); result = 31 * result + (replicationCheckpoint != null ? replicationCheckpoint.hashCode() : 0); return result; } @@ -360,8 +351,8 @@ public int hashCode() { public String toString() { StringBuilder buf = new StringBuilder(); buf.append("NodeGatewayStartedShards[").append("allocationId=").append(allocationId).append(",primary=").append(primary); - if (storeException != null) { - buf.append(",storeException=").append(storeException); + if (getException() != null) { + buf.append(",storeException=").append(getException()); } if (replicationCheckpoint != null) { buf.append(",ReplicationCheckpoint=").append(replicationCheckpoint.toString()); @@ -387,13 +378,28 @@ public Map getNodeGatewayStartedShardsBatch() public NodeGatewayStartedShardsBatch(StreamInput in) throws IOException { super(in); - this.nodeGatewayStartedShardsBatch = in.readMap(ShardId::new, NodeGatewayStartedShard::new); + this.nodeGatewayStartedShardsBatch = in.readMap(ShardId::new, + i -> { + if (i.readBoolean()) { + return new NodeGatewayStartedShard(i); + } else { + return null; + } + }); } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeMap(nodeGatewayStartedShardsBatch, (o, k) -> k.writeTo(o), (o, v) -> v.writeTo(o)); + out.writeMap(nodeGatewayStartedShardsBatch, (o, k) -> k.writeTo(o), + (o, v) -> { + if (v != null) { + o.writeBoolean(true); + v.writeTo(o); + } else { + o.writeBoolean(false); + } + }); } public NodeGatewayStartedShardsBatch(DiscoveryNode node, Map nodeGatewayStartedShardsBatch) { From d8c5a8dbf8ba383643027900745b88516dcd3d60 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Tue, 5 Mar 2024 23:39:52 +0530 Subject: [PATCH 10/29] Add unit test for ShardBatchCache implementation Signed-off-by: Aman Khare --- .../gateway/AsyncShardBatchFetch.java | 26 +- .../opensearch/gateway/ShardBatchCache.java | 29 +-- .../org/opensearch/gateway/BatchTestUtil.java | 25 ++ .../gateway/ShardBatchCacheTests.java | 231 ++++++++++++++++++ 4 files changed, 278 insertions(+), 33 deletions(-) create mode 100644 server/src/test/java/org/opensearch/gateway/BatchTestUtil.java create mode 100644 server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java index 7d9dcd909e3df..b787bae5e5057 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java @@ -16,6 +16,7 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.indices.store.ShardAttributes; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Set; @@ -35,6 +36,9 @@ */ public abstract class AsyncShardBatchFetch extends AsyncShardFetch { + private final Consumer removeShardFromBatch; + private final List failedShards; + @SuppressWarnings("unchecked") AsyncShardBatchFetch( Logger logger, @@ -49,6 +53,8 @@ public abstract class AsyncShardBatchFetch handleFailedShard ) { super(logger, type, shardAttributesMap, action, batchId); + this.removeShardFromBatch = handleFailedShard; + this.failedShards = new ArrayList<>(); this.cache = new ShardBatchCache<>( logger, type, @@ -71,7 +77,6 @@ public abstract class AsyncShardBatchFetch fetchData(DiscoveryNodes nodes, Map> ignoreNodes) { - List failedShards = cleanUpFailedShards(); if (failedShards.isEmpty() == false) { // trigger a reroute if there are any shards failed, to make sure they're picked up in next run logger.trace("triggering another reroute for failed shards in {}", reroutingKey); @@ -81,16 +86,15 @@ public synchronized FetchResult fetchData(DiscoveryNodes nodes, Map cleanUpFailedShards() { - List failedShards = cache.getFailedShards(); - if (failedShards != null && failedShards.isEmpty() == false) { - shardAttributesMap.keySet().removeIf(failedShards::contains); - } - return failedShards; + private void cleanUpFailedShard(ShardId shardId) { + shardAttributesMap.remove(shardId); + removeShardFromBatch.accept(shardId); + failedShards.add(shardId); } /** @@ -100,7 +104,7 @@ private List cleanUpFailedShards() { * @param shardId shardId to be removed from the batch. */ public void clearShard(ShardId shardId) { - this.shardAttributesMap.remove(shardId); - this.cache.clearShardCache(shardId); + shardAttributesMap.remove(shardId); + cache.deleteData(shardId); } } diff --git a/server/src/main/java/org/opensearch/gateway/ShardBatchCache.java b/server/src/main/java/org/opensearch/gateway/ShardBatchCache.java index 77880092374bd..336a5b5c94a42 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardBatchCache.java +++ b/server/src/main/java/org/opensearch/gateway/ShardBatchCache.java @@ -51,7 +51,6 @@ public class ShardBatchCache arrayToShardId; private final Function> shardsBatchDataGetter; private final Supplier emptyResponseBuilder; - private final Set failedShards; private final Consumer handleFailedShard; public ShardBatchCache( @@ -72,7 +71,6 @@ public ShardBatchCache( this.responseConstructor = responseConstructor; this.shardsBatchDataGetter = shardsBatchDataGetter; this.emptyResponseBuilder = emptyResponseBuilder; - failedShards = new HashSet<>(); cache = new HashMap<>(); shardIdToArray = new HashMap<>(); arrayToShardId = new HashMap<>(); @@ -86,7 +84,7 @@ public ShardBatchCache( } @Override - public void clearShardCache(ShardId shardId) { + public void deleteData(ShardId shardId) { if (shardIdToArray.containsKey(shardId)) { Integer shardIdIndex = shardIdToArray.remove(shardId); for (String nodeId : cache.keySet()) { @@ -128,7 +126,7 @@ public void initData(DiscoveryNode node) { public void putData(DiscoveryNode node, T response) { NodeEntry nodeEntry = cache.get(node.getId()); Map batchResponse = shardsBatchDataGetter.apply(response); - failedShards.addAll(filterFailedShards(batchResponse)); + filterFailedShards(batchResponse); nodeEntry.doneFetching(batchResponse, shardIdToArray); } @@ -136,11 +134,8 @@ public void putData(DiscoveryNode node, T response) { * Return the shard for which we got unhandled exceptions. * * @param batchResponse response from one node for the batch. - * @return List of failed shards. */ - private List filterFailedShards(Map batchResponse) { - logger.trace("filtering failed shards"); - List failedShards = new ArrayList<>(); + private void filterFailedShards(Map batchResponse) { for (Iterator it = batchResponse.keySet().iterator(); it.hasNext();) { ShardId shardId = it.next(); if (batchResponse.get(shardId) != null) { @@ -149,11 +144,9 @@ private List filterFailedShards(Map batchResponse) { // the batch Exception shardException = batchResponse.get(shardId).getException(); // if the request got rejected or timed out, we need to try it again next time... - if (shardException instanceof OpenSearchRejectedExecutionException - || shardException instanceof ReceiveTimeoutTransportException - || shardException instanceof OpenSearchTimeoutException) { - logger.trace("got unhandled retryable exception for shard {} {}", shardId.toString(), shardException.toString()); - failedShards.add(shardId); + if (retryableException(shardException)) { + logger.trace("got unhandled retryable exception for shard {} {}", shardId.toString(), + shardException.toString()); handleFailedShard.accept(shardId); // remove this failed entry. So, while storing the data, we don't need to re-process it. it.remove(); @@ -161,7 +154,6 @@ private List filterFailedShards(Map batchResponse) { } } } - return failedShards; } @Override @@ -169,13 +161,6 @@ public T getData(DiscoveryNode node) { return this.responseConstructor.apply(node, getBatchData(cache.get(node.getId()))); } - @Override - public List getFailedShards() { - List defectedShards = List.copyOf(failedShards); - failedShards.clear(); - return defectedShards; - } - private HashMap getBatchData(NodeEntry nodeEntry) { V[] nodeShardEntries = nodeEntry.getData(); boolean[] emptyResponses = nodeEntry.getEmptyShardResponse(); @@ -197,7 +182,7 @@ private void fillShardIdKeys(Set shardIds) { } this.shardIdToArray.keySet().removeIf(shardId -> { if (!shardIds.contains(shardId)) { - clearShardCache(shardId); + deleteData(shardId); return true; } else { return false; diff --git a/server/src/test/java/org/opensearch/gateway/BatchTestUtil.java b/server/src/test/java/org/opensearch/gateway/BatchTestUtil.java new file mode 100644 index 0000000000000..69f0cfeeb2c7d --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/BatchTestUtil.java @@ -0,0 +1,25 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway; + +import org.opensearch.core.index.shard.ShardId; + +import java.util.ArrayList; +import java.util.List; + +public class BatchTestUtil { + public static List setUpShards(int numberOfShards) { + List shards = new ArrayList<>(); + for (int shardNumber = 0; shardNumber < numberOfShards; shardNumber++) { + ShardId shardId = new ShardId("test", "_na_", shardNumber); + shards.add(shardId); + } + return shards; + } +} diff --git a/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java b/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java new file mode 100644 index 0000000000000..598de3d98f9b1 --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java @@ -0,0 +1,231 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway; + +import org.opensearch.cluster.OpenSearchAllocationTestCase; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.ShardRoutingState; +import org.opensearch.cluster.routing.TestShardRouting; +import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard; +import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch; +import org.opensearch.indices.store.ShardAttributes; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class ShardBatchCacheTests extends OpenSearchAllocationTestCase { + private static final String BATCH_ID = "b1"; + private final DiscoveryNode node1 = newNode("node1"); + private final DiscoveryNode node2 = newNode("node2"); + private final Map batchInfo = new HashMap<>(); + private ShardBatchCache shardCache; + private List shardsInBatch = new ArrayList<>(); + private static final int NUMBER_OF_SHARDS_DEFAULT = 10; + + private enum ResponseType { + NULL, + EMPTY, + FAILURE, + VALID + } + + public void setupShardBatchCache(String batchId, int numberOfShards) { + Map shardAttributesMap = new HashMap<>(); + fillShards(shardAttributesMap, numberOfShards); + this.shardCache = new ShardBatchCache<>( + logger, + "batch_shards_started", + shardAttributesMap, + "BatchID=[" + batchId + "]", + NodeGatewayStartedShard.class, + NodeGatewayStartedShardsBatch::new, + NodeGatewayStartedShardsBatch::getNodeGatewayStartedShardsBatch, + () -> new NodeGatewayStartedShard(null, false, null, null), + this::removeShard + ); + } + + public void testClearShardCache() { + setupShardBatchCache(BATCH_ID, NUMBER_OF_SHARDS_DEFAULT); + ShardId shard = shardsInBatch.iterator().next(); + this.shardCache.initData(node1); + this.shardCache.markAsFetching(List.of(node1.getId()), 1); + this.shardCache.putData(node1, new NodeGatewayStartedShardsBatch(node1, getPrimaryResponse(shardsInBatch, ResponseType.EMPTY))); + assertTrue( + this.shardCache.getCacheData(DiscoveryNodes.builder().add(node1).build(), null) + .get(node1) + .getNodeGatewayStartedShardsBatch() + .containsKey(shard) + ); + this.shardCache.deleteData(shard); + assertFalse( + this.shardCache.getCacheData(DiscoveryNodes.builder().add(node1).build(), null) + .get(node1) + .getNodeGatewayStartedShardsBatch() + .containsKey(shard) + ); + } + + public void testGetCacheData() { + setupShardBatchCache(BATCH_ID, NUMBER_OF_SHARDS_DEFAULT); + ShardId shard = shardsInBatch.iterator().next(); + this.shardCache.initData(node1); + this.shardCache.initData(node2); + this.shardCache.markAsFetching(List.of(node1.getId(), node2.getId()), 1); + this.shardCache.putData(node1, new NodeGatewayStartedShardsBatch(node1, getPrimaryResponse(shardsInBatch, + ResponseType.EMPTY))); + assertTrue( + this.shardCache.getCacheData(DiscoveryNodes.builder().add(node1).build(), null) + .get(node1) + .getNodeGatewayStartedShardsBatch() + .containsKey(shard) + ); + assertTrue( + this.shardCache.getCacheData(DiscoveryNodes.builder().add(node2).build(), null) + .get(node2) + .getNodeGatewayStartedShardsBatch() + .isEmpty() + ); + } + + public void testInitCacheData() { + setupShardBatchCache(BATCH_ID, NUMBER_OF_SHARDS_DEFAULT); + this.shardCache.initData(node1); + this.shardCache.initData(node2); + assertEquals(2, shardCache.getCache().size()); + + // test getData without fetch + assertTrue(shardCache.getData(node1).getNodeGatewayStartedShardsBatch().isEmpty()); + } + + public void testPutData() { + // test empty and non-empty responses + setupShardBatchCache(BATCH_ID, NUMBER_OF_SHARDS_DEFAULT); + ShardId shard = shardsInBatch.iterator().next(); + this.shardCache.initData(node1); + this.shardCache.initData(node2); + this.shardCache.markAsFetching(List.of(node1.getId(), node2.getId()), 1); + this.shardCache.putData(node1, new NodeGatewayStartedShardsBatch(node1, getPrimaryResponse(shardsInBatch, + ResponseType.VALID))); + this.shardCache.putData(node2, new NodeGatewayStartedShardsBatch(node1, getPrimaryResponse(shardsInBatch, ResponseType.EMPTY))); + + Map fetchData = shardCache.getCacheData( + DiscoveryNodes.builder().add(node1).add(node2).build(), + null + ); + assertEquals(2, fetchData.size()); + assertEquals(10, fetchData.get(node1).getNodeGatewayStartedShardsBatch().size()); + assertEquals("alloc-1", fetchData.get(node1).getNodeGatewayStartedShardsBatch().get(shard).allocationId()); + + assertEquals(10, fetchData.get(node2).getNodeGatewayStartedShardsBatch().size()); + assertTrue(fetchData.get(node2).getNodeGatewayStartedShardsBatch().get(shard).isEmpty()); + + // test GetData after fetch + assertEquals(10, shardCache.getData(node1).getNodeGatewayStartedShardsBatch().size()); + } + + public void testNullResponses() { + setupShardBatchCache(BATCH_ID, NUMBER_OF_SHARDS_DEFAULT); + this.shardCache.initData(node1); + this.shardCache.markAsFetching(List.of(node1.getId()), 1); + this.shardCache.putData(node1, new NodeGatewayStartedShardsBatch(node1, getPrimaryResponse(shardsInBatch, + ResponseType.NULL))); + + Map fetchData = shardCache.getCacheData( + DiscoveryNodes.builder().add(node1).build(), null); + assertTrue(fetchData.get(node1).getNodeGatewayStartedShardsBatch().isEmpty()); + } + + public void testFilterFailedShards() { + setupShardBatchCache(BATCH_ID, NUMBER_OF_SHARDS_DEFAULT); + this.shardCache.initData(node1); + this.shardCache.initData(node2); + this.shardCache.markAsFetching(List.of(node1.getId(), node2.getId()), 1); + this.shardCache.putData(node1, new NodeGatewayStartedShardsBatch(node1, + getFailedPrimaryResponse(shardsInBatch, 5))); + Map fetchData = shardCache.getCacheData( + DiscoveryNodes.builder().add(node1).add(node2).build(), null); + + assertEquals(5, batchInfo.size()); + assertEquals(2, fetchData.size()); + assertEquals(5, fetchData.get(node1).getNodeGatewayStartedShardsBatch().size()); + assertTrue(fetchData.get(node2).getNodeGatewayStartedShardsBatch().isEmpty()); + } + + private Map getPrimaryResponse(List shards, ResponseType responseType) { + int allocationId = 1; + Map shardData = new HashMap<>(); + for (ShardId shard : shards) { + switch (responseType) { + case NULL: + shardData.put(shard, null); + break; + case EMPTY: + shardData.put(shard, new NodeGatewayStartedShard(null, false, null, null)); + break; + case VALID: + shardData.put(shard, new NodeGatewayStartedShard("alloc-" + allocationId++, false, null, null)); + break; + default: + throw new AssertionError("unknown response type"); + } + } + return shardData; + } + + private Map getFailedPrimaryResponse(List shards, + int failedShardsCount) { + int allocationId = 1; + Map shardData = new HashMap<>(); + for (ShardId shard : shards) { + if (failedShardsCount-- > 0) { + shardData.put(shard, new NodeGatewayStartedShard("alloc-" + allocationId++, false, null, + new OpenSearchRejectedExecutionException())); + } else { + shardData.put(shard, new NodeGatewayStartedShard("alloc-" + allocationId++, false, null, + null)); + } + } + return shardData; + } + + public void removeShard(ShardId shardId) { + batchInfo.remove(shardId); + } + + private void fillShards(Map shardAttributesMap, int numberOfShards) { + shardsInBatch = BatchTestUtil.setUpShards(numberOfShards); + for (ShardId shardId : shardsInBatch) { + ShardAttributes attr = new ShardAttributes(""); + shardAttributesMap.put(shardId, attr); + batchInfo.put( + shardId, + new ShardsBatchGatewayAllocator.ShardEntry(attr, randomShardRouting(shardId.getIndexName(), shardId.id())) + ); + } + } + + private ShardRouting randomShardRouting(String index, int shard) { + ShardRoutingState state = randomFrom(ShardRoutingState.values()); + return TestShardRouting.newShardRouting( + index, + shard, + state == ShardRoutingState.UNASSIGNED ? null : "1", + state == ShardRoutingState.RELOCATING ? "2" : null, + state != ShardRoutingState.UNASSIGNED && randomBoolean(), + state + ); + } +} From 483d75fbac6d316980a0be9d4aa16e51bfc327cf Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Mon, 11 Mar 2024 23:27:04 +0530 Subject: [PATCH 11/29] Move ShardBatchCache as inner class of AsyncShardBatchFetch Signed-off-by: Aman Khare --- .../gateway/AsyncShardBatchFetch.java | 245 ++++++++++++++++-- .../opensearch/gateway/AsyncShardFetch.java | 8 +- .../opensearch/gateway/ShardBatchCache.java | 241 ----------------- 3 files changed, 224 insertions(+), 270 deletions(-) delete mode 100644 server/src/main/java/org/opensearch/gateway/ShardBatchCache.java diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java index b787bae5e5057..481f1835034f2 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java @@ -13,26 +13,34 @@ import org.opensearch.action.support.nodes.BaseNodesResponse; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.common.logging.Loggers; import org.opensearch.core.index.shard.ShardId; import org.opensearch.indices.store.ShardAttributes; +import java.lang.reflect.Array; import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; /** - * Implementation of AsyncShardFetchAbstract with batching support. - * cache will be created using ShardBatchCache class as that can handle the caching strategy correctly for a - * batch of shards. Other necessary functions are also stored so cache can store or get the data for both primary - * and replicas. + * Implementation of AsyncShardFetch with batching support. This class is responsible for executing the fetch + * part using the base class {@link AsyncShardFetch}. Other functionalities needed for a batch are only written here. + * Cleanup of failed shards is necessary in a batch and based on that a reroute should be triggered to take care of + * those in the next run. This separation also takes care of the extra generic type V which is only needed for batch + * transport actions like {@link TransportNodesListGatewayStartedShardsBatch}. * * @param Response type of the transport action. * @param Data type of shard level response. + * + * @opensearch.internal */ public abstract class AsyncShardBatchFetch extends AsyncShardFetch { @@ -47,7 +55,7 @@ public abstract class AsyncShardBatchFetch, T> action, String batchId, Class clazz, - BiFunction, T> responseConstructor, + BiFunction, T> responseGetter, Function> shardsBatchDataGetter, Supplier emptyResponseBuilder, Consumer handleFailedShard @@ -61,21 +69,13 @@ public abstract class AsyncShardBatchFetch fetchData(DiscoveryNodes nodes, Map> ignoreNodes) { if (failedShards.isEmpty() == false) { // trigger a reroute if there are any shards failed, to make sure they're picked up in next run @@ -85,13 +85,7 @@ public synchronized FetchResult fetchData(DiscoveryNodes nodes, Map Response type of transport action. + * @param Data type of shard level response. + */ + public static class ShardBatchCache extends AsyncShardFetchCache { + private final Map> cache = new HashMap<>(); + private final Map shardIdKey = new HashMap<>(); + private final AtomicInteger shardIdIndex = new AtomicInteger(); + private final int batchSize; + private final Class shardResponseClass; + private final BiFunction, T> responseConstructor; + private final Map shardIdReverseKey = new HashMap<>(); + private final Function> shardsBatchDataGetter; + private final Supplier emptyResponseBuilder; + private final Consumer handleFailedShard; + + public ShardBatchCache( + Logger logger, + String type, + Map shardAttributesMap, + String logKey, + Class clazz, + BiFunction, T> responseGetter, + Function> shardsBatchDataGetter, + Supplier emptyResponseBuilder, + Consumer handleFailedShard + ) { + super(Loggers.getLogger(logger, "_" + logKey), type); + this.batchSize = shardAttributesMap.size(); + fillShardIdKeys(shardAttributesMap.keySet()); + this.shardResponseClass = clazz; + this.responseConstructor = responseGetter; + this.shardsBatchDataGetter = shardsBatchDataGetter; + this.emptyResponseBuilder = emptyResponseBuilder; + this.handleFailedShard = handleFailedShard; + } + + @Override + public Map getCache() { + return cache; + } + + @Override + public void deleteShard(ShardId shardId) { + if (shardIdKey.containsKey(shardId)) { + Integer shardIdIndex = shardIdKey.remove(shardId); + for (String nodeId : cache.keySet()) { + cache.get(nodeId).clearShard(shardIdIndex); + } + } + } + + @Override + public Map getCacheData(DiscoveryNodes nodes, Set failedNodes) { + refreshReverseIdMap(); + return super.getCacheData(nodes, failedNodes); + } + + /** + * Build a reverse map to get shardId from the array index, this will be used to construct the response which + * PrimaryShardBatchAllocator or ReplicaShardBatchAllocator are looking for. + */ + private void refreshReverseIdMap() { + shardIdReverseKey.clear(); + for (ShardId shardId : shardIdKey.keySet()) { + shardIdReverseKey.putIfAbsent(shardIdKey.get(shardId), shardId); + } + } + + @Override + public void initData(DiscoveryNode node) { + cache.put(node.getId(), new NodeEntry<>(node.getId(), shardResponseClass, batchSize)); + } + + /** + * Put the response received from data nodes into the cache. + * Get shard level data from batch, then filter out if any shards received failures. + * After that complete storing the data at node level and mark fetching as done. + * + * @param node node from which we got the response. + * @param response shard metadata coming from node. + */ + @Override + public void putData(DiscoveryNode node, T response) { + NodeEntry nodeEntry = cache.get(node.getId()); + Map batchResponse = shardsBatchDataGetter.apply(response); + filterFailedShards(batchResponse); + nodeEntry.doneFetching(batchResponse, shardIdKey); + } + + /** + * Return the shard for which we got unhandled exceptions. + * + * @param batchResponse response from one node for the batch. + */ + private void filterFailedShards(Map batchResponse) { + logger.trace("filtering failed shards"); + for (Iterator it = batchResponse.keySet().iterator(); it.hasNext();) { + ShardId shardId = it.next(); + if (batchResponse.get(shardId) != null) { + if (batchResponse.get(shardId).getException() != null) { + // handle per shard level exceptions, process other shards, only throw out this shard from + // the batch + Exception shardException = batchResponse.get(shardId).getException(); + // if the request got rejected or timed out, we need to try it again next time... + if (retryableException(shardException)) { + logger.trace( + "got unhandled retryable exception for shard {} {}", + shardId.toString(), + shardException.toString() + ); + handleFailedShard.accept(shardId); + // remove this failed entry. So, while storing the data, we don't need to re-process it. + it.remove(); + } + } + } + } + } + + @Override + public T getData(DiscoveryNode node) { + return this.responseConstructor.apply(node, getBatchData(cache.get(node.getId()))); + } + + private HashMap getBatchData(NodeEntry nodeEntry) { + V[] nodeShardEntries = nodeEntry.getData(); + boolean[] emptyResponses = nodeEntry.getEmptyShardResponse(); + HashMap shardData = new HashMap<>(); + for (Integer shardIdIndex : shardIdKey.values()) { + if (emptyResponses[shardIdIndex]) { + shardData.put(shardIdReverseKey.get(shardIdIndex), emptyResponseBuilder.get()); + } else if (nodeShardEntries[shardIdIndex] != null) { + // ignore null responses here + shardData.put(shardIdReverseKey.get(shardIdIndex), nodeShardEntries[shardIdIndex]); + } + } + return shardData; + } + + private void fillShardIdKeys(Set shardIds) { + for (ShardId shardId : shardIds) { + this.shardIdKey.putIfAbsent(shardId, shardIdIndex.getAndIncrement()); + } + this.shardIdKey.keySet().removeIf(shardId -> { + if (!shardIds.contains(shardId)) { + deleteShard(shardId); + return true; + } else { + return false; + } + }); + } + + /** + * A node entry, holding the state of the fetched data for a specific shard + * for a giving node. + */ + static class NodeEntry extends BaseNodeEntry { + private final V[] shardData; + private final boolean[] emptyShardResponse; + + NodeEntry(String nodeId, Class clazz, int batchSize) { + super(nodeId); + this.shardData = (V[]) Array.newInstance(clazz, batchSize); + this.emptyShardResponse = new boolean[batchSize]; + } + + void doneFetching(Map shardDataFromNode, Map shardIdKey) { + fillShardData(shardDataFromNode, shardIdKey); + super.doneFetching(); + } + + void clearShard(Integer shardIdIndex) { + this.shardData[shardIdIndex] = null; + } + + V[] getData() { + return this.shardData; + } + + boolean[] getEmptyShardResponse() { + return emptyShardResponse; + } + + private void fillShardData(Map shardDataFromNode, Map shardIdKey) { + for (ShardId shardId : shardDataFromNode.keySet()) { + if (shardDataFromNode.get(shardId) != null) { + if (shardDataFromNode.get(shardId).isEmpty()) { + this.emptyShardResponse[shardIdKey.get(shardId)] = true; + this.shardData[shardIdKey.get(shardId)] = null; + } else if (shardDataFromNode.get(shardId).getException() == null) { + this.shardData[shardIdKey.get(shardId)] = shardDataFromNode.get(shardId); + } + // if exception is not null, we got unhandled failure for the shard which needs to be ignored + } + } + } + } } } diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java index bcadc5be1d1e2..5abcef37be42f 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java @@ -65,11 +65,9 @@ * Allows to asynchronously fetch shard related data from other nodes for allocation, without blocking * the cluster update thread. *

- * The async fetch logic maintains a map of which nodes are being fetched from in an async manner, - * and once the results are back, it makes sure to schedule a reroute to make sure those results will - * be taken into account. + * The async fetch logic maintains a cache {@link AsyncShardFetchCache} which is filled in async manner when nodes respond back. + * It also schedules a reroute to make sure those results will be taken into account. * - * It comes in two modes, to single fetch a shard or fetch a batch of shards. * @opensearch.internal */ public abstract class AsyncShardFetch implements Releasable { @@ -86,7 +84,7 @@ public interface Lister, N protected final String type; protected final Map shardAttributesMap; private final Lister, T> action; - private final Map> cache = new HashMap<>(); + final Map> cache = new HashMap<>(); private final AtomicLong round = new AtomicLong(); private boolean closed; protected final String reroutingKey; diff --git a/server/src/main/java/org/opensearch/gateway/ShardBatchCache.java b/server/src/main/java/org/opensearch/gateway/ShardBatchCache.java deleted file mode 100644 index 336a5b5c94a42..0000000000000 --- a/server/src/main/java/org/opensearch/gateway/ShardBatchCache.java +++ /dev/null @@ -1,241 +0,0 @@ -/* - * 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.OpenSearchTimeoutException; -import org.opensearch.action.support.nodes.BaseNodeResponse; -import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.common.Nullable; -import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; -import org.opensearch.core.index.shard.ShardId; -import org.opensearch.indices.store.ShardAttributes; -import org.opensearch.transport.ReceiveTimeoutTransportException; - -import java.lang.reflect.Array; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.BiFunction; -import java.util.function.Consumer; -import java.util.function.Function; -import java.util.function.Supplier; - -/** - * Cache implementation of transport actions returning batch of shards data in the response. Cache uses a specific - * NodeEntry class that stores the data in array format. To keep the class generic for primary or replica, all - * functions are stored during object creation. - * - * @param Response type of transport action. - * @param Data type of shard level response. - */ -public class ShardBatchCache extends BaseShardCache { - private final Map> cache; - private final Map shardIdToArray; // used for mapping array index for a shard - private final AtomicInteger shardIdIndex; - private final int batchSize; - private final Class shardResponseClass; - private final BiFunction, T> responseConstructor; - private final Map arrayToShardId; - private final Function> shardsBatchDataGetter; - private final Supplier emptyResponseBuilder; - private final Consumer handleFailedShard; - - public ShardBatchCache( - Logger logger, - String type, - Map shardToCustomDataPath, - String logKey, - Class clazz, - BiFunction, T> responseConstructor, - Function> shardsBatchDataGetter, - Supplier emptyResponseBuilder, - Consumer handleFailedShard - ) { - super(logger, logKey, type); - this.batchSize = shardToCustomDataPath.size(); - fillShardIdKeys(shardToCustomDataPath.keySet()); - this.shardResponseClass = clazz; - this.responseConstructor = responseConstructor; - this.shardsBatchDataGetter = shardsBatchDataGetter; - this.emptyResponseBuilder = emptyResponseBuilder; - cache = new HashMap<>(); - shardIdToArray = new HashMap<>(); - arrayToShardId = new HashMap<>(); - shardIdIndex = new AtomicInteger(); - this.handleFailedShard = handleFailedShard; - } - - @Override - public Map getCache() { - return cache; - } - - @Override - public void deleteData(ShardId shardId) { - if (shardIdToArray.containsKey(shardId)) { - Integer shardIdIndex = shardIdToArray.remove(shardId); - for (String nodeId : cache.keySet()) { - cache.get(nodeId).clearShard(shardIdIndex); - } - } - } - - @Override - public Map getCacheData(DiscoveryNodes nodes, Set failedNodes) { - refreshReverseIdMap(); - return super.getCacheData(nodes, failedNodes); - } - - /** - * Build a reverse map to get shardId from the array index, this will be used to construct the response which - * PrimaryShardBatchAllocator or ReplicaShardBatchAllocator are looking for. - */ - private void refreshReverseIdMap() { - arrayToShardId.clear(); - for (ShardId shardId : shardIdToArray.keySet()) { - arrayToShardId.putIfAbsent(shardIdToArray.get(shardId), shardId); - } - } - - @Override - public void initData(DiscoveryNode node) { - cache.put(node.getId(), new NodeEntry<>(node.getId(), shardResponseClass, batchSize)); - } - - /** - * Put the response received from data nodes into the cache. - * Get shard level data from batch, then filter out if any shards received failures. - * After that, complete storing the data at node level and mark fetching as done. - * @param node node from which we got the response. - * @param response shard metadata coming from node. - */ - @Override - public void putData(DiscoveryNode node, T response) { - NodeEntry nodeEntry = cache.get(node.getId()); - Map batchResponse = shardsBatchDataGetter.apply(response); - filterFailedShards(batchResponse); - nodeEntry.doneFetching(batchResponse, shardIdToArray); - } - - /** - * Return the shard for which we got unhandled exceptions. - * - * @param batchResponse response from one node for the batch. - */ - private void filterFailedShards(Map batchResponse) { - for (Iterator it = batchResponse.keySet().iterator(); it.hasNext();) { - ShardId shardId = it.next(); - if (batchResponse.get(shardId) != null) { - if (batchResponse.get(shardId).getException() != null) { - // handle per shard level exceptions, process other shards, only throw out this shard from - // the batch - Exception shardException = batchResponse.get(shardId).getException(); - // if the request got rejected or timed out, we need to try it again next time... - if (retryableException(shardException)) { - logger.trace("got unhandled retryable exception for shard {} {}", shardId.toString(), - shardException.toString()); - handleFailedShard.accept(shardId); - // remove this failed entry. So, while storing the data, we don't need to re-process it. - it.remove(); - } - } - } - } - } - - @Override - public T getData(DiscoveryNode node) { - return this.responseConstructor.apply(node, getBatchData(cache.get(node.getId()))); - } - - private HashMap getBatchData(NodeEntry nodeEntry) { - V[] nodeShardEntries = nodeEntry.getData(); - boolean[] emptyResponses = nodeEntry.getEmptyShardResponse(); - HashMap shardData = new HashMap<>(); - for (Integer shardIdIndex : shardIdToArray.values()) { - if (emptyResponses[shardIdIndex]) { - shardData.put(arrayToShardId.get(shardIdIndex), emptyResponseBuilder.get()); - } else if (nodeShardEntries[shardIdIndex] != null) { - // ignore null responses here - shardData.put(arrayToShardId.get(shardIdIndex), nodeShardEntries[shardIdIndex]); - } - } - return shardData; - } - - private void fillShardIdKeys(Set shardIds) { - for (ShardId shardId : shardIds) { - this.shardIdToArray.putIfAbsent(shardId, shardIdIndex.getAndIncrement()); - } - this.shardIdToArray.keySet().removeIf(shardId -> { - if (!shardIds.contains(shardId)) { - deleteData(shardId); - return true; - } else { - return false; - } - }); - } - - /** - * A node entry, holding the state of the fetched data for a specific shard - * for a giving node. This will only store the data from TransportNodesListGatewayStartedShardsBatch or - * TransportNodesListShardStoreMetadataBatch transport actions. - */ - static class NodeEntry extends BaseShardCache.BaseNodeEntry { - @Nullable - private final V[] shardData; - private final boolean[] emptyShardResponse; - - NodeEntry(String nodeId, Class clazz, int batchSize) { - super(nodeId); - this.shardData = (V[]) Array.newInstance(clazz, batchSize); - this.emptyShardResponse = new boolean[batchSize]; - } - - void doneFetching(Map shardDataFromNode, Map shardIdKey) { - fillShardData(shardDataFromNode, shardIdKey); - super.doneFetching(); - } - - void clearShard(Integer shardIdIndex) { - this.shardData[shardIdIndex] = null; - } - - V[] getData() { - return this.shardData; - } - - boolean[] getEmptyShardResponse() { - return emptyShardResponse; - } - - private void fillShardData(Map shardDataFromNode, Map shardIdKey) { - for (ShardId shardId : shardDataFromNode.keySet()) { - if (shardDataFromNode.get(shardId) != null) { - if (shardDataFromNode.get(shardId).isEmpty()) { - this.emptyShardResponse[shardIdKey.get(shardId)] = true; - this.shardData[shardIdKey.get(shardId)] = null; - } else if (shardDataFromNode.get(shardId).getException() == null) { - this.shardData[shardIdKey.get(shardId)] = shardDataFromNode.get(shardId); - } - // if exception is not null, we got unhandled failure for the shard which needs to be ignored - } - } - } - } - -} From afffde65e40a1b62faeb6a61886820c8dc9c80d6 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Mon, 11 Mar 2024 23:44:06 +0530 Subject: [PATCH 12/29] Merge remote-tracking branch 'origin/main' into shard-batch-cache Signed-off-by: Aman Khare --- .../gateway/AsyncShardBatchFetch.java | 4 ++++ .../opensearch/gateway/AsyncShardFetch.java | 2 +- ...ortNodesListGatewayStartedShardsBatch.java | 6 +++--- .../indices/store/ShardAttributes.java | 20 +++++++------------ .../gateway/ShardBatchCacheTests.java | 7 ++++--- 5 files changed, 19 insertions(+), 20 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java index 481f1835034f2..87cc9501ffa93 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java @@ -16,6 +16,7 @@ import org.opensearch.common.logging.Loggers; import org.opensearch.core.index.shard.ShardId; import org.opensearch.indices.store.ShardAttributes; +import reactor.util.annotation.NonNull; import java.lang.reflect.Array; import java.util.ArrayList; @@ -119,6 +120,7 @@ public static class ShardBatchCache> shardsBatchDataGetter; private final Supplier emptyResponseBuilder; private final Consumer handleFailedShard; + private final Logger logger; public ShardBatchCache( Logger logger, @@ -139,8 +141,10 @@ public ShardBatchCache( this.shardsBatchDataGetter = shardsBatchDataGetter; this.emptyResponseBuilder = emptyResponseBuilder; this.handleFailedShard = handleFailedShard; + this.logger = logger; } + @NonNull @Override public Map getCache() { return cache; diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java index 5abcef37be42f..09d7f31911d41 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java @@ -103,7 +103,7 @@ protected AsyncShardFetch( this.logger = logger; this.type = type; shardAttributesMap = new HashMap<>(); - shardAttributesMap.put(shardId, new ShardAttributes(shardId, customDataPath)); + shardAttributesMap.put(shardId, new ShardAttributes(customDataPath)); this.action = (Lister, T>) action; this.reroutingKey = "ShardId=[" + shardId.toString() + "]"; enableBatchMode = false; diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java index 86829cfbd12db..fc17022b1c3f5 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java @@ -136,8 +136,8 @@ protected NodesGatewayStartedShardsBatch newResponse( @Override protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { Map shardsOnNode = new HashMap<>(); - for (ShardAttributes shardAttr : request.shardAttributes.values()) { - final ShardId shardId = shardAttr.getShardId(); + for (Map.Entry shardAttr : request.shardAttributes.entrySet()) { + final ShardId shardId = shardAttr.getKey(); try { shardsOnNode.put( shardId, @@ -147,7 +147,7 @@ protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { namedXContentRegistry, nodeEnv, indicesService, - shardAttr.getCustomDataPath(), + shardAttr.getValue().getCustomDataPath(), settings, clusterService ) diff --git a/server/src/main/java/org/opensearch/indices/store/ShardAttributes.java b/server/src/main/java/org/opensearch/indices/store/ShardAttributes.java index 4ef4e91f7af8c..155d787ae8316 100644 --- a/server/src/main/java/org/opensearch/indices/store/ShardAttributes.java +++ b/server/src/main/java/org/opensearch/indices/store/ShardAttributes.java @@ -12,36 +12,26 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; -import org.opensearch.core.index.shard.ShardId; -import org.opensearch.gateway.AsyncShardFetch; import java.io.IOException; /** - * This class contains information about the shard that needs to be sent as part of request in Transport Action implementing - * {@link AsyncShardFetch.Lister} to fetch shard information in async manner + * This class contains Attributes related to Shards that are necessary for making the {@link org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch} transport requests * * @opensearch.internal */ public class ShardAttributes implements Writeable { - private final ShardId shardId; @Nullable private final String customDataPath; - public ShardAttributes(ShardId shardId, String customDataPath) { - this.shardId = shardId; + public ShardAttributes(String customDataPath) { this.customDataPath = customDataPath; } public ShardAttributes(StreamInput in) throws IOException { - shardId = new ShardId(in); customDataPath = in.readString(); } - public ShardId getShardId() { - return shardId; - } - /** * Returns the custom data path that is used to look up information for this shard. * Returns an empty string if no custom data path is used for this index. @@ -53,7 +43,11 @@ public String getCustomDataPath() { } public void writeTo(StreamOutput out) throws IOException { - shardId.writeTo(out); out.writeString(customDataPath); } + + @Override + public String toString() { + return "ShardAttributes{" + ", customDataPath='" + customDataPath + '\'' + '}'; + } } diff --git a/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java b/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java index 598de3d98f9b1..27ddae5bce9ed 100644 --- a/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java +++ b/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java @@ -29,8 +29,9 @@ public class ShardBatchCacheTests extends OpenSearchAllocationTestCase { private static final String BATCH_ID = "b1"; private final DiscoveryNode node1 = newNode("node1"); private final DiscoveryNode node2 = newNode("node2"); + // Compilation would pass once ShardsBatchGatewayAllocator is committed in main private final Map batchInfo = new HashMap<>(); - private ShardBatchCache shardCache; + private AsyncShardBatchFetch.ShardBatchCache shardCache; private List shardsInBatch = new ArrayList<>(); private static final int NUMBER_OF_SHARDS_DEFAULT = 10; @@ -44,7 +45,7 @@ private enum ResponseType { public void setupShardBatchCache(String batchId, int numberOfShards) { Map shardAttributesMap = new HashMap<>(); fillShards(shardAttributesMap, numberOfShards); - this.shardCache = new ShardBatchCache<>( + this.shardCache = new AsyncShardBatchFetch.ShardBatchCache<>( logger, "batch_shards_started", shardAttributesMap, @@ -69,7 +70,7 @@ public void testClearShardCache() { .getNodeGatewayStartedShardsBatch() .containsKey(shard) ); - this.shardCache.deleteData(shard); + this.shardCache.deleteShard(shard); assertFalse( this.shardCache.getCacheData(DiscoveryNodes.builder().add(node1).build(), null) .get(node1) From 29029672a4a8d4a150e53ca81b9698629a2a1308 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Tue, 12 Mar 2024 00:19:30 +0530 Subject: [PATCH 13/29] Use failed shards data only when all nodes fetching is done Signed-off-by: Aman Khare --- .../gateway/GatewayRecoveryTestUtils.java | 2 +- .../gateway/AsyncShardBatchFetch.java | 60 ++++++++++++------- .../opensearch/gateway/BaseShardResponse.java | 4 +- ...ansportNodesGatewayStartedShardHelper.java | 1 + ...ortNodesListGatewayStartedShardsBatch.java | 32 +++++----- .../gateway/AsyncShardFetchTests.java | 4 +- .../gateway/ShardBatchCacheTests.java | 48 +++++++-------- .../indices/store/ShardAttributesTests.java | 6 +- 8 files changed, 84 insertions(+), 73 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/GatewayRecoveryTestUtils.java b/server/src/internalClusterTest/java/org/opensearch/gateway/GatewayRecoveryTestUtils.java index 2b6a5b4ee6867..dc157681be6fa 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/GatewayRecoveryTestUtils.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/GatewayRecoveryTestUtils.java @@ -54,7 +54,7 @@ public static Map prepareRequestMap(String[] indices, ); for (int shardIdNum = 0; shardIdNum < primaryShardCount; shardIdNum++) { final ShardId shardId = new ShardId(index, shardIdNum); - shardIdShardAttributesMap.put(shardId, new ShardAttributes(shardId, customDataPath)); + shardIdShardAttributesMap.put(shardId, new ShardAttributes(customDataPath)); } } return shardIdShardAttributesMap; diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java index 87cc9501ffa93..e8484d9040ba0 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java @@ -16,7 +16,6 @@ import org.opensearch.common.logging.Loggers; import org.opensearch.core.index.shard.ShardId; import org.opensearch.indices.store.ShardAttributes; -import reactor.util.annotation.NonNull; import java.lang.reflect.Array; import java.util.ArrayList; @@ -25,12 +24,13 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; +import reactor.util.annotation.NonNull; + /** * Implementation of AsyncShardFetch with batching support. This class is responsible for executing the fetch * part using the base class {@link AsyncShardFetch}. Other functionalities needed for a batch are only written here. @@ -78,14 +78,27 @@ public abstract class AsyncShardBatchFetch fetchData(DiscoveryNodes nodes, Map> ignoreNodes) { - if (failedShards.isEmpty() == false) { - // trigger a reroute if there are any shards failed, to make sure they're picked up in next run - logger.trace("triggering another reroute for failed shards in {}", reroutingKey); - reroute("shards-failed", "shards failed in " + reroutingKey); + FetchResult result = super.fetchData(nodes, ignoreNodes); + if (result.hasData()) { + // trigger reroute for failed shards only when all nodes have completed fetching + if (failedShards.isEmpty() == false) { + // trigger a reroute if there are any shards failed, to make sure they're picked up in next run + logger.trace("triggering another reroute for failed shards in {}", reroutingKey); + reroute("shards-failed", "shards failed in " + reroutingKey); + failedShards.clear(); + } } - return super.fetchData(nodes, ignoreNodes); + return result; } + /** + * Remove the shard from shardAttributesMap so it's not sent in next asyncFetch. + * Call removeShardFromBatch method to remove the shardId from the batch object created in + * ShardsBatchGatewayAllocator. + * Add shardId to failedShards, so it can be used to trigger another reroute as part of upcoming fetchData call. + * + * @param shardId shardId to be cleaned up from batch and cache. + */ private void cleanUpFailedShards(ShardId shardId) { shardAttributesMap.remove(shardId); removeShardFromBatch.accept(shardId); @@ -110,13 +123,12 @@ public void clearShard(ShardId shardId) { * @param Data type of shard level response. */ public static class ShardBatchCache extends AsyncShardFetchCache { - private final Map> cache = new HashMap<>(); - private final Map shardIdKey = new HashMap<>(); - private final AtomicInteger shardIdIndex = new AtomicInteger(); + private final Map> cache; + private final Map shardIdToArray; private final int batchSize; private final Class shardResponseClass; private final BiFunction, T> responseConstructor; - private final Map shardIdReverseKey = new HashMap<>(); + private final Map arrayToShardId; private final Function> shardsBatchDataGetter; private final Supplier emptyResponseBuilder; private final Consumer handleFailedShard; @@ -135,6 +147,9 @@ public ShardBatchCache( ) { super(Loggers.getLogger(logger, "_" + logKey), type); this.batchSize = shardAttributesMap.size(); + cache = new HashMap<>(); + shardIdToArray = new HashMap<>(); + arrayToShardId = new HashMap<>(); fillShardIdKeys(shardAttributesMap.keySet()); this.shardResponseClass = clazz; this.responseConstructor = responseGetter; @@ -152,8 +167,8 @@ public ShardBatchCache( @Override public void deleteShard(ShardId shardId) { - if (shardIdKey.containsKey(shardId)) { - Integer shardIdIndex = shardIdKey.remove(shardId); + if (shardIdToArray.containsKey(shardId)) { + Integer shardIdIndex = shardIdToArray.remove(shardId); for (String nodeId : cache.keySet()) { cache.get(nodeId).clearShard(shardIdIndex); } @@ -171,9 +186,9 @@ public Map getCacheData(DiscoveryNodes nodes, Set fail * PrimaryShardBatchAllocator or ReplicaShardBatchAllocator are looking for. */ private void refreshReverseIdMap() { - shardIdReverseKey.clear(); - for (ShardId shardId : shardIdKey.keySet()) { - shardIdReverseKey.putIfAbsent(shardIdKey.get(shardId), shardId); + arrayToShardId.clear(); + for (ShardId shardId : shardIdToArray.keySet()) { + arrayToShardId.putIfAbsent(shardIdToArray.get(shardId), shardId); } } @@ -195,7 +210,7 @@ public void putData(DiscoveryNode node, T response) { NodeEntry nodeEntry = cache.get(node.getId()); Map batchResponse = shardsBatchDataGetter.apply(response); filterFailedShards(batchResponse); - nodeEntry.doneFetching(batchResponse, shardIdKey); + nodeEntry.doneFetching(batchResponse, shardIdToArray); } /** @@ -237,22 +252,23 @@ private HashMap getBatchData(NodeEntry nodeEntry) { V[] nodeShardEntries = nodeEntry.getData(); boolean[] emptyResponses = nodeEntry.getEmptyShardResponse(); HashMap shardData = new HashMap<>(); - for (Integer shardIdIndex : shardIdKey.values()) { + for (Integer shardIdIndex : shardIdToArray.values()) { if (emptyResponses[shardIdIndex]) { - shardData.put(shardIdReverseKey.get(shardIdIndex), emptyResponseBuilder.get()); + shardData.put(arrayToShardId.get(shardIdIndex), emptyResponseBuilder.get()); } else if (nodeShardEntries[shardIdIndex] != null) { // ignore null responses here - shardData.put(shardIdReverseKey.get(shardIdIndex), nodeShardEntries[shardIdIndex]); + shardData.put(arrayToShardId.get(shardIdIndex), nodeShardEntries[shardIdIndex]); } } return shardData; } private void fillShardIdKeys(Set shardIds) { + int shardIdIndex = 0; for (ShardId shardId : shardIds) { - this.shardIdKey.putIfAbsent(shardId, shardIdIndex.getAndIncrement()); + this.shardIdToArray.putIfAbsent(shardId, shardIdIndex++); } - this.shardIdKey.keySet().removeIf(shardId -> { + this.shardIdToArray.keySet().removeIf(shardId -> { if (!shardIds.contains(shardId)) { deleteShard(shardId); return true; diff --git a/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java b/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java index 0922abf2c942f..876d9632b5ed8 100644 --- a/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java +++ b/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java @@ -10,7 +10,6 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; -import org.opensearch.core.transport.TransportResponse; import java.io.IOException; @@ -20,7 +19,7 @@ * * @opensearch.internal */ -public abstract class BaseShardResponse extends TransportResponse { +public abstract class BaseShardResponse { private Exception storeException; @@ -42,7 +41,6 @@ public BaseShardResponse(StreamInput in) throws IOException { } } - @Override public void writeTo(StreamOutput out) throws IOException { if (storeException != null) { out.writeBoolean(true); diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java index cac88275ce0d9..bea98335d03e8 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java @@ -39,6 +39,7 @@ public class TransportNodesGatewayStartedShardHelper { public static final String INDEX_NOT_FOUND = "node doesn't have meta data for index"; + public static TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard getShardInfoOnLocalNode( Logger logger, final ShardId shardId, diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java index fc17022b1c3f5..10a9538a37d18 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java @@ -378,28 +378,26 @@ public Map getNodeGatewayStartedShardsBatch() public NodeGatewayStartedShardsBatch(StreamInput in) throws IOException { super(in); - this.nodeGatewayStartedShardsBatch = in.readMap(ShardId::new, - i -> { - if (i.readBoolean()) { - return new NodeGatewayStartedShard(i); - } else { - return null; - } - }); + this.nodeGatewayStartedShardsBatch = in.readMap(ShardId::new, i -> { + if (i.readBoolean()) { + return new NodeGatewayStartedShard(i); + } else { + return null; + } + }); } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeMap(nodeGatewayStartedShardsBatch, (o, k) -> k.writeTo(o), - (o, v) -> { - if (v != null) { - o.writeBoolean(true); - v.writeTo(o); - } else { - o.writeBoolean(false); - } - }); + out.writeMap(nodeGatewayStartedShardsBatch, (o, k) -> k.writeTo(o), (o, v) -> { + if (v != null) { + o.writeBoolean(true); + v.writeTo(o); + } else { + o.writeBoolean(false); + } + }); } public NodeGatewayStartedShardsBatch(DiscoveryNode node, Map nodeGatewayStartedShardsBatch) { diff --git a/server/src/test/java/org/opensearch/gateway/AsyncShardFetchTests.java b/server/src/test/java/org/opensearch/gateway/AsyncShardFetchTests.java index 4e5e9c71e1fe4..3502cc8996fa2 100644 --- a/server/src/test/java/org/opensearch/gateway/AsyncShardFetchTests.java +++ b/server/src/test/java/org/opensearch/gateway/AsyncShardFetchTests.java @@ -92,8 +92,8 @@ public void setUp() throws Exception { HashMap shardToCustomDataPath = new HashMap<>(); ShardId shardId0 = new ShardId("index1", "index_uuid1", 0); ShardId shardId1 = new ShardId("index2", "index_uuid2", 0); - shardToCustomDataPath.put(shardId0, new ShardAttributes(shardId0, "")); - shardToCustomDataPath.put(shardId1, new ShardAttributes(shardId1, "")); + shardToCustomDataPath.put(shardId0, new ShardAttributes("")); + shardToCustomDataPath.put(shardId1, new ShardAttributes("")); this.test = new TestFetch(threadPool, shardToCustomDataPath); } } diff --git a/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java b/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java index 27ddae5bce9ed..6922bafd5ef68 100644 --- a/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java +++ b/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java @@ -29,8 +29,8 @@ public class ShardBatchCacheTests extends OpenSearchAllocationTestCase { private static final String BATCH_ID = "b1"; private final DiscoveryNode node1 = newNode("node1"); private final DiscoveryNode node2 = newNode("node2"); - // Compilation would pass once ShardsBatchGatewayAllocator is committed in main - private final Map batchInfo = new HashMap<>(); + // Needs to be enabled once ShardsBatchGatewayAllocator is pushed + // private final Map batchInfo = new HashMap<>(); private AsyncShardBatchFetch.ShardBatchCache shardCache; private List shardsInBatch = new ArrayList<>(); private static final int NUMBER_OF_SHARDS_DEFAULT = 10; @@ -85,8 +85,7 @@ public void testGetCacheData() { this.shardCache.initData(node1); this.shardCache.initData(node2); this.shardCache.markAsFetching(List.of(node1.getId(), node2.getId()), 1); - this.shardCache.putData(node1, new NodeGatewayStartedShardsBatch(node1, getPrimaryResponse(shardsInBatch, - ResponseType.EMPTY))); + this.shardCache.putData(node1, new NodeGatewayStartedShardsBatch(node1, getPrimaryResponse(shardsInBatch, ResponseType.EMPTY))); assertTrue( this.shardCache.getCacheData(DiscoveryNodes.builder().add(node1).build(), null) .get(node1) @@ -118,8 +117,7 @@ public void testPutData() { this.shardCache.initData(node1); this.shardCache.initData(node2); this.shardCache.markAsFetching(List.of(node1.getId(), node2.getId()), 1); - this.shardCache.putData(node1, new NodeGatewayStartedShardsBatch(node1, getPrimaryResponse(shardsInBatch, - ResponseType.VALID))); + this.shardCache.putData(node1, new NodeGatewayStartedShardsBatch(node1, getPrimaryResponse(shardsInBatch, ResponseType.VALID))); this.shardCache.putData(node2, new NodeGatewayStartedShardsBatch(node1, getPrimaryResponse(shardsInBatch, ResponseType.EMPTY))); Map fetchData = shardCache.getCacheData( @@ -141,11 +139,12 @@ public void testNullResponses() { setupShardBatchCache(BATCH_ID, NUMBER_OF_SHARDS_DEFAULT); this.shardCache.initData(node1); this.shardCache.markAsFetching(List.of(node1.getId()), 1); - this.shardCache.putData(node1, new NodeGatewayStartedShardsBatch(node1, getPrimaryResponse(shardsInBatch, - ResponseType.NULL))); + this.shardCache.putData(node1, new NodeGatewayStartedShardsBatch(node1, getPrimaryResponse(shardsInBatch, ResponseType.NULL))); Map fetchData = shardCache.getCacheData( - DiscoveryNodes.builder().add(node1).build(), null); + DiscoveryNodes.builder().add(node1).build(), + null + ); assertTrue(fetchData.get(node1).getNodeGatewayStartedShardsBatch().isEmpty()); } @@ -154,12 +153,13 @@ public void testFilterFailedShards() { this.shardCache.initData(node1); this.shardCache.initData(node2); this.shardCache.markAsFetching(List.of(node1.getId(), node2.getId()), 1); - this.shardCache.putData(node1, new NodeGatewayStartedShardsBatch(node1, - getFailedPrimaryResponse(shardsInBatch, 5))); + this.shardCache.putData(node1, new NodeGatewayStartedShardsBatch(node1, getFailedPrimaryResponse(shardsInBatch, 5))); Map fetchData = shardCache.getCacheData( - DiscoveryNodes.builder().add(node1).add(node2).build(), null); + DiscoveryNodes.builder().add(node1).add(node2).build(), + null + ); - assertEquals(5, batchInfo.size()); + // assertEquals(5, batchInfo.size()); assertEquals(2, fetchData.size()); assertEquals(5, fetchData.get(node1).getNodeGatewayStartedShardsBatch().size()); assertTrue(fetchData.get(node2).getNodeGatewayStartedShardsBatch().isEmpty()); @@ -186,24 +186,24 @@ private Map getPrimaryResponse(List s return shardData; } - private Map getFailedPrimaryResponse(List shards, - int failedShardsCount) { + private Map getFailedPrimaryResponse(List shards, int failedShardsCount) { int allocationId = 1; Map shardData = new HashMap<>(); for (ShardId shard : shards) { if (failedShardsCount-- > 0) { - shardData.put(shard, new NodeGatewayStartedShard("alloc-" + allocationId++, false, null, - new OpenSearchRejectedExecutionException())); + shardData.put( + shard, + new NodeGatewayStartedShard("alloc-" + allocationId++, false, null, new OpenSearchRejectedExecutionException()) + ); } else { - shardData.put(shard, new NodeGatewayStartedShard("alloc-" + allocationId++, false, null, - null)); + shardData.put(shard, new NodeGatewayStartedShard("alloc-" + allocationId++, false, null, null)); } } return shardData; } public void removeShard(ShardId shardId) { - batchInfo.remove(shardId); + // batchInfo.remove(shardId); } private void fillShards(Map shardAttributesMap, int numberOfShards) { @@ -211,10 +211,10 @@ private void fillShards(Map shardAttributesMap, int nu for (ShardId shardId : shardsInBatch) { ShardAttributes attr = new ShardAttributes(""); shardAttributesMap.put(shardId, attr); - batchInfo.put( - shardId, - new ShardsBatchGatewayAllocator.ShardEntry(attr, randomShardRouting(shardId.getIndexName(), shardId.id())) - ); + // batchInfo.put( + // shardId, + // new ShardsBatchGatewayAllocator.ShardEntry(attr, randomShardRouting(shardId.getIndexName(), shardId.id())) + // ); } } diff --git a/server/src/test/java/org/opensearch/indices/store/ShardAttributesTests.java b/server/src/test/java/org/opensearch/indices/store/ShardAttributesTests.java index 7fa95fefe72fd..94834bab1d98b 100644 --- a/server/src/test/java/org/opensearch/indices/store/ShardAttributesTests.java +++ b/server/src/test/java/org/opensearch/indices/store/ShardAttributesTests.java @@ -28,13 +28,12 @@ public class ShardAttributesTests extends OpenSearchTestCase { String customDataPath = "/path/to/data"; public void testShardAttributesConstructor() { - ShardAttributes attributes = new ShardAttributes(shardId, customDataPath); - assertEquals(attributes.getShardId(), shardId); + ShardAttributes attributes = new ShardAttributes(customDataPath); assertEquals(attributes.getCustomDataPath(), customDataPath); } public void testSerialization() throws IOException { - ShardAttributes attributes1 = new ShardAttributes(shardId, customDataPath); + ShardAttributes attributes1 = new ShardAttributes(customDataPath); ByteArrayOutputStream bytes = new ByteArrayOutputStream(); StreamOutput output = new DataOutputStreamOutput(new DataOutputStream(bytes)); attributes1.writeTo(output); @@ -42,7 +41,6 @@ public void testSerialization() throws IOException { StreamInput input = new InputStreamStreamInput(new ByteArrayInputStream(bytes.toByteArray())); ShardAttributes attributes2 = new ShardAttributes(input); input.close(); - assertEquals(attributes1.getShardId(), attributes2.getShardId()); assertEquals(attributes1.getCustomDataPath(), attributes2.getCustomDataPath()); } From d26f8b2c45cb14ef84e3bb437b43530f6be3b9fc Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Tue, 12 Mar 2024 08:09:21 +0530 Subject: [PATCH 14/29] Modify description to use existing Transport Signed-off-by: Aman Khare --- .../java/org/opensearch/indices/store/ShardAttributes.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/indices/store/ShardAttributes.java b/server/src/main/java/org/opensearch/indices/store/ShardAttributes.java index 155d787ae8316..999acd02030ab 100644 --- a/server/src/main/java/org/opensearch/indices/store/ShardAttributes.java +++ b/server/src/main/java/org/opensearch/indices/store/ShardAttributes.java @@ -16,7 +16,8 @@ import java.io.IOException; /** - * This class contains Attributes related to Shards that are necessary for making the {@link org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch} transport requests + * This class contains Attributes related to Shards that are necessary for making the + * {@link org.opensearch.gateway.TransportNodesListGatewayStartedShards} transport requests * * @opensearch.internal */ From e1217c6a562845e6d336fe39919afd4adf17d6b3 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Thu, 14 Mar 2024 13:00:15 +0530 Subject: [PATCH 15/29] Fix DCO Signed-off-by: Aman Khare --- .../opensearch/gateway/AsyncShardFetch.java | 335 +++++------------- .../gateway/AsyncShardFetchCache.java | 313 ++++++++++++++++ 2 files changed, 402 insertions(+), 246 deletions(-) create mode 100644 server/src/main/java/org/opensearch/gateway/AsyncShardFetchCache.java diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java index 09d7f31911d41..cb2d1e88b1bc8 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java @@ -32,9 +32,6 @@ package org.opensearch.gateway; import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.opensearch.ExceptionsHelper; -import org.opensearch.OpenSearchTimeoutException; import org.opensearch.action.FailedNodeException; import org.opensearch.action.support.nodes.BaseNodeResponse; import org.opensearch.action.support.nodes.BaseNodesResponse; @@ -43,16 +40,15 @@ import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.common.Nullable; import org.opensearch.common.lease.Releasable; +import org.opensearch.common.logging.Loggers; import org.opensearch.core.action.ActionListener; -import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.core.index.shard.ShardId; import org.opensearch.indices.store.ShardAttributes; -import org.opensearch.transport.ReceiveTimeoutTransportException; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; @@ -84,14 +80,12 @@ public interface Lister, N protected final String type; protected final Map shardAttributesMap; private final Lister, T> action; - final Map> cache = new HashMap<>(); + protected AsyncShardFetchCache cache; private final AtomicLong round = new AtomicLong(); private boolean closed; - protected final String reroutingKey; + final String reroutingKey; private final Map> shardToIgnoreNodes = new HashMap<>(); - private final boolean enableBatchMode; - @SuppressWarnings("unchecked") protected AsyncShardFetch( Logger logger, @@ -106,17 +100,17 @@ protected AsyncShardFetch( shardAttributesMap.put(shardId, new ShardAttributes(customDataPath)); this.action = (Lister, T>) action; this.reroutingKey = "ShardId=[" + shardId.toString() + "]"; - enableBatchMode = false; + cache = new ShardCache<>(logger, reroutingKey, type); } /** * Added to fetch a batch of shards from nodes * - * @param logger Logger - * @param type type of action + * @param logger Logger + * @param type type of action * @param shardAttributesMap Map of {@link ShardId} to {@link ShardAttributes} to perform fetching on them a - * @param action Transport Action - * @param batchId For the given ShardAttributesMap, we expect them to tie with a single batch id for logging and later identification + * @param action Transport Action + * @param batchId For the given ShardAttributesMap, we expect them to tie with a single batch id for logging and later identification */ @SuppressWarnings("unchecked") protected AsyncShardFetch( @@ -131,7 +125,7 @@ protected AsyncShardFetch( this.shardAttributesMap = shardAttributesMap; this.action = (Lister, T>) action; this.reroutingKey = "BatchID=[" + batchId + "]"; - enableBatchMode = true; + cache = new ShardCache<>(logger, reroutingKey, type); } @Override @@ -139,19 +133,6 @@ public synchronized void close() { this.closed = true; } - /** - * Returns the number of async fetches that are currently ongoing. - */ - public synchronized int getNumberOfInFlightFetches() { - int count = 0; - for (NodeEntry nodeEntry : cache.values()) { - if (nodeEntry.isFetching()) { - count++; - } - } - return count; - } - /** * Fetches the data for the relevant shard. If there any ongoing async fetches going on, or new ones have * been initiated by this call, the result will have no data. @@ -164,7 +145,7 @@ public synchronized FetchResult fetchData(DiscoveryNodes nodes, Map 1) { throw new IllegalStateException( @@ -185,48 +166,24 @@ public synchronized FetchResult fetchData(DiscoveryNodes nodes, Map> nodesToFetch = findNodesToFetch(cache); - if (nodesToFetch.isEmpty() == false) { + cache.fillShardCacheWithDataNodes(nodes); + List nodeIds = cache.findNodesToFetch(); + if (nodeIds.isEmpty() == false) { // mark all node as fetching and go ahead and async fetch them // use a unique round id to detect stale responses in processAsyncFetch final long fetchingRound = round.incrementAndGet(); - for (NodeEntry nodeEntry : nodesToFetch) { - nodeEntry.markAsFetching(fetchingRound); - } - DiscoveryNode[] discoNodesToFetch = nodesToFetch.stream() - .map(NodeEntry::getNodeId) - .map(nodes::get) - .toArray(DiscoveryNode[]::new); + cache.markAsFetching(nodeIds, fetchingRound); + DiscoveryNode[] discoNodesToFetch = nodeIds.stream().map(nodes::get).toArray(DiscoveryNode[]::new); asyncFetch(discoNodesToFetch, fetchingRound); } // if we are still fetching, return null to indicate it - if (hasAnyNodeFetching(cache)) { + if (cache.hasAnyNodeFetching()) { return new FetchResult<>(null, emptyMap()); } else { // nothing to fetch, yay, build the return value - Map fetchData = new HashMap<>(); Set failedNodes = new HashSet<>(); - for (Iterator>> it = cache.entrySet().iterator(); it.hasNext();) { - Map.Entry> entry = it.next(); - String nodeId = entry.getKey(); - NodeEntry nodeEntry = entry.getValue(); - - DiscoveryNode node = nodes.get(nodeId); - if (node != null) { - if (nodeEntry.isFailed()) { - // if its failed, remove it from the list of nodes, so if this run doesn't work - // we try again next round to fetch it again - it.remove(); - failedNodes.add(nodeEntry.getNodeId()); - } else { - if (nodeEntry.getValue() != null) { - fetchData.put(node, nodeEntry.getValue()); - } - } - } - } + Map fetchData = cache.getCacheData(nodes, failedNodes); Map> allIgnoreNodesMap = unmodifiableMap(new HashMap<>(shardToIgnoreNodes)); // clear the nodes to ignore, we had a successful run in fetching everything we can @@ -266,77 +223,18 @@ protected synchronized void processAsyncFetch(List responses, List nodeEntry = cache.get(response.getNode().getId()); - if (nodeEntry != null) { - if (nodeEntry.getFetchingRound() != fetchingRound) { - assert nodeEntry.getFetchingRound() > fetchingRound : "node entries only replaced by newer rounds"; - logger.trace( - "{} received response for [{}] from node {} for an older fetching round (expected: {} but was: {})", - reroutingKey, - nodeEntry.getNodeId(), - type, - nodeEntry.getFetchingRound(), - fetchingRound - ); - } else if (nodeEntry.isFailed()) { - logger.trace( - "{} node {} has failed for [{}] (failure [{}])", - reroutingKey, - nodeEntry.getNodeId(), - type, - nodeEntry.getFailure() - ); - } else { - // if the entry is there, for the right fetching round and not marked as failed already, process it - logger.trace("{} marking {} as done for [{}], result is [{}]", reroutingKey, nodeEntry.getNodeId(), type, response); - nodeEntry.doneFetching(response); - } - } - } + cache.processResponses(responses, fetchingRound); } if (failures != null) { - for (FailedNodeException failure : failures) { - logger.trace("{} processing failure {} for [{}]", reroutingKey, failure, type); - NodeEntry nodeEntry = cache.get(failure.nodeId()); - if (nodeEntry != null) { - if (nodeEntry.getFetchingRound() != fetchingRound) { - assert nodeEntry.getFetchingRound() > fetchingRound : "node entries only replaced by newer rounds"; - logger.trace( - "{} received failure for [{}] from node {} for an older fetching round (expected: {} but was: {})", - reroutingKey, - nodeEntry.getNodeId(), - type, - nodeEntry.getFetchingRound(), - fetchingRound - ); - } else if (nodeEntry.isFailed() == false) { - // if the entry is there, for the right fetching round and not marked as failed already, process it - Throwable unwrappedCause = ExceptionsHelper.unwrapCause(failure.getCause()); - // if the request got rejected or timed out, we need to try it again next time... - if (unwrappedCause instanceof OpenSearchRejectedExecutionException - || unwrappedCause instanceof ReceiveTimeoutTransportException - || unwrappedCause instanceof OpenSearchTimeoutException) { - nodeEntry.restartFetching(); - } else { - logger.warn( - () -> new ParameterizedMessage( - "{}: failed to list shard for {} on node [{}]", - reroutingKey, - type, - failure.nodeId() - ), - failure - ); - nodeEntry.doneFetching(failure.getCause()); - } - } - } - } + cache.processFailures(failures, fetchingRound); } reroute(reroutingKey, "post_response"); } + public synchronized int getNumberOfInFlightFetches() { + return cache.getInflightFetches(); + } + /** * Implement this in order to scheduled another round that causes a call to fetch data. */ @@ -349,47 +247,6 @@ synchronized void clearCacheForNode(String nodeId) { cache.remove(nodeId); } - /** - * Fills the shard fetched data with new (data) nodes and a fresh NodeEntry, and removes from - * it nodes that are no longer part of the state. - */ - private void fillShardCacheWithDataNodes(Map> shardCache, DiscoveryNodes nodes) { - // verify that all current data nodes are there - for (final DiscoveryNode node : nodes.getDataNodes().values()) { - if (shardCache.containsKey(node.getId()) == false) { - shardCache.put(node.getId(), new NodeEntry(node.getId())); - } - } - // remove nodes that are not longer part of the data nodes set - shardCache.keySet().removeIf(nodeId -> !nodes.nodeExists(nodeId)); - } - - /** - * Finds all the nodes that need to be fetched. Those are nodes that have no - * data, and are not in fetch mode. - */ - private List> findNodesToFetch(Map> shardCache) { - List> nodesToFetch = new ArrayList<>(); - for (NodeEntry nodeEntry : shardCache.values()) { - if (nodeEntry.hasData() == false && nodeEntry.isFetching() == false) { - nodesToFetch.add(nodeEntry); - } - } - return nodesToFetch; - } - - /** - * Are there any nodes that are fetching data? - */ - private boolean hasAnyNodeFetching(Map> shardCache) { - for (NodeEntry nodeEntry : shardCache.values()) { - if (nodeEntry.isFetching()) { - return true; - } - } - return false; - } - /** * Async fetches data for the provided shard with the set of nodes that need to be fetched from. */ @@ -413,6 +270,71 @@ public void onFailure(Exception e) { }); } + /** + * Cache implementation of transport actions returning single shard related data in the response. + * Store node level responses of transport actions like {@link TransportNodesListGatewayStartedShards} or + * {@link TransportNodesListShardStoreMetadata}. + * + * @param Response type of transport action. + */ + static class ShardCache extends AsyncShardFetchCache { + + private final Map> cache; + + public ShardCache(Logger logger, String logKey, String type) { + super(Loggers.getLogger(logger, "_" + logKey), type); + cache = new HashMap<>(); + } + + @Override + public void initData(DiscoveryNode node) { + cache.put(node.getId(), new NodeEntry<>(node.getId())); + } + + @Override + public void putData(DiscoveryNode node, K response) { + cache.get(node.getId()).doneFetching(response); + } + + @Override + public K getData(DiscoveryNode node) { + return cache.get(node.getId()).getValue(); + } + + @Override + public Map getCache() { + return cache; + } + + @Override + public void deleteShard(ShardId shardId) { + cache.clear(); // single shard cache can clear the full map + } + + /** + * A node entry, holding the state of the fetched data for a specific shard + * for a giving node. + */ + static class NodeEntry extends AsyncShardFetchCache.BaseNodeEntry { + @Nullable + private U value; + + void doneFetching(U value) { + super.doneFetching(); + this.value = value; + } + + NodeEntry(String nodeId) { + super(nodeId); + } + + U getValue() { + return value; + } + + } + } + /** * The result of a fetch operation. Make sure to first check {@link #hasData()} before * fetching the actual data. @@ -458,83 +380,4 @@ public void processAllocation(RoutingAllocation allocation) { } } - - /** - * A node entry, holding the state of the fetched data for a specific shard - * for a giving node. - */ - static class NodeEntry { - private final String nodeId; - private boolean fetching; - @Nullable - private T value; - private boolean valueSet; - private Throwable failure; - private long fetchingRound; - - NodeEntry(String nodeId) { - this.nodeId = nodeId; - } - - String getNodeId() { - return this.nodeId; - } - - boolean isFetching() { - return fetching; - } - - void markAsFetching(long fetchingRound) { - assert fetching == false : "double marking a node as fetching"; - this.fetching = true; - this.fetchingRound = fetchingRound; - } - - void doneFetching(T value) { - assert fetching : "setting value but not in fetching mode"; - assert failure == null : "setting value when failure already set"; - this.valueSet = true; - this.value = value; - this.fetching = false; - } - - void doneFetching(Throwable failure) { - assert fetching : "setting value but not in fetching mode"; - assert valueSet == false : "setting failure when already set value"; - assert failure != null : "setting failure can't be null"; - this.failure = failure; - this.fetching = false; - } - - void restartFetching() { - assert fetching : "restarting fetching, but not in fetching mode"; - assert valueSet == false : "value can't be set when restarting fetching"; - assert failure == null : "failure can't be set when restarting fetching"; - this.fetching = false; - } - - boolean isFailed() { - return failure != null; - } - - boolean hasData() { - return valueSet || failure != null; - } - - Throwable getFailure() { - assert hasData() : "getting failure when data has not been fetched"; - return failure; - } - - @Nullable - T getValue() { - assert failure == null : "trying to fetch value, but its marked as failed, check isFailed"; - assert valueSet : "value is not set, hasn't been fetched yet"; - return value; - } - - long getFetchingRound() { - return fetchingRound; - } - } } diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardFetchCache.java b/server/src/main/java/org/opensearch/gateway/AsyncShardFetchCache.java new file mode 100644 index 0000000000000..fae362038d143 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardFetchCache.java @@ -0,0 +1,313 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway; + +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.opensearch.ExceptionsHelper; +import org.opensearch.OpenSearchTimeoutException; +import org.opensearch.action.FailedNodeException; +import org.opensearch.action.support.nodes.BaseNodeResponse; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.transport.ReceiveTimeoutTransportException; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * AsyncShardFetchCache will operate on the node level cache which is map of String and BaseNodeEntry. initData, + * putData and getData needs to be called for all the nodes. This class is responsible for managing the flow for all + * the nodes. + * It'll also give useful insights like how many ongoing fetches are happening, how many nodes are left for fetch or + * mark some node in fetching mode. All of these functionalities require checking the cache information and respond + * accordingly. + *

+ * initData : how to initialize an entry of shard cache for a node. + * putData : how to store the response of transport action in the cache. + * getData : how to get the stored data for any shard allocators like {@link PrimaryShardAllocator} or + * {@link ReplicaShardAllocator} + * deleteShard : how to clean up the stored data from cache for a shard. + * + * @param Response type of transport action which has the data to be stored in the cache. + * + * @opensearch.internal + */ +public abstract class AsyncShardFetchCache { + final Logger logger; + private final String type; + + protected AsyncShardFetchCache(Logger logger, String type) { + this.logger = logger; + this.type = type; + } + + abstract void initData(DiscoveryNode node); + + abstract void putData(DiscoveryNode node, K response); + + abstract K getData(DiscoveryNode node); + + abstract Map getCache(); + + /** + * Cleanup cached data for this shard once it's started. Cleanup only happens at shard level. Node entries will + * automatically be cleaned up once shards are assigned. + * + * @param shardId for which we need to free up the cached data. + */ + abstract void deleteShard(ShardId shardId); + + /** + * Returns the number of fetches that are currently ongoing. + */ + int getInflightFetches() { + int count = 0; + for (BaseNodeEntry nodeEntry : getCache().values()) { + if (nodeEntry.isFetching()) { + count++; + } + } + return count; + } + + /** + * Fills the shard fetched data with new (data) nodes and a fresh NodeEntry, and removes from + * it nodes that are no longer part of the state. + */ + void fillShardCacheWithDataNodes(DiscoveryNodes nodes) { + // verify that all current data nodes are there + for (final DiscoveryNode node : nodes.getDataNodes().values()) { + if (getCache().containsKey(node.getId()) == false) { + initData(node); + } + } + // remove nodes that are not longer part of the data nodes set + getCache().keySet().removeIf(nodeId -> !nodes.nodeExists(nodeId)); + } + + /** + * Finds all the nodes that need to be fetched. Those are nodes that have no + * data, and are not in fetch mode. + */ + List findNodesToFetch() { + List nodesToFetch = new ArrayList<>(); + for (BaseNodeEntry nodeEntry : getCache().values()) { + if (nodeEntry.hasData() == false && nodeEntry.isFetching() == false) { + nodesToFetch.add(nodeEntry.getNodeId()); + } + } + return nodesToFetch; + } + + /** + * Are there any nodes that are fetching data? + */ + boolean hasAnyNodeFetching() { + for (BaseNodeEntry nodeEntry : getCache().values()) { + if (nodeEntry.isFetching()) { + return true; + } + } + return false; + } + + /** + * Get the data from cache, ignore the failed entries. Use getData functional interface to get the data, as + * different implementations may have different ways to populate the data from cache. + * + * @param nodes Discovery nodes for which we need to return the cache data. + * @param failedNodes return failedNodes with the nodes where fetch has failed. + * @return Map of cache data for every DiscoveryNode. + */ + Map getCacheData(DiscoveryNodes nodes, Set failedNodes) { + Map fetchData = new HashMap<>(); + for (Iterator> it = getCache().entrySet().iterator(); it.hasNext();) { + Map.Entry entry = (Map.Entry) it.next(); + String nodeId = entry.getKey(); + BaseNodeEntry nodeEntry = entry.getValue(); + + DiscoveryNode node = nodes.get(nodeId); + if (node != null) { + if (nodeEntry.isFailed()) { + // if its failed, remove it from the list of nodes, so if this run doesn't work + // we try again next round to fetch it again + it.remove(); + failedNodes.add(nodeEntry.getNodeId()); + } else { + K nodeResponse = getData(node); + if (nodeResponse != null) { + fetchData.put(node, nodeResponse); + } + } + } + } + return fetchData; + } + + void processResponses(List responses, long fetchingRound) { + for (K response : responses) { + BaseNodeEntry nodeEntry = getCache().get(response.getNode().getId()); + if (nodeEntry != null) { + if (validateNodeResponse(nodeEntry, fetchingRound)) { + // if the entry is there, for the right fetching round and not marked as failed already, process it + logger.trace("marking {} as done for [{}], result is [{}]", nodeEntry.getNodeId(), type, response); + putData(response.getNode(), response); + } + } + } + } + + private boolean validateNodeResponse(BaseNodeEntry nodeEntry, long fetchingRound) { + if (nodeEntry.getFetchingRound() != fetchingRound) { + assert nodeEntry.getFetchingRound() > fetchingRound : "node entries only replaced by newer rounds"; + logger.trace( + "received response for [{}] from node {} for an older fetching round (expected: {} but was: {})", + nodeEntry.getNodeId(), + type, + nodeEntry.getFetchingRound(), + fetchingRound + ); + return false; + } else if (nodeEntry.isFailed()) { + logger.trace("node {} has failed for [{}] (failure [{}])", nodeEntry.getNodeId(), type, nodeEntry.getFailure()); + return false; + } + return true; + } + + private void handleNodeFailure(BaseNodeEntry nodeEntry, FailedNodeException failure, long fetchingRound) { + if (nodeEntry.getFetchingRound() != fetchingRound) { + assert nodeEntry.getFetchingRound() > fetchingRound : "node entries only replaced by newer rounds"; + logger.trace( + "received failure for [{}] from node {} for an older fetching round (expected: {} but was: {})", + nodeEntry.getNodeId(), + type, + nodeEntry.getFetchingRound(), + fetchingRound + ); + } else if (nodeEntry.isFailed() == false) { + // if the entry is there, for the right fetching round and not marked as failed already, process it + Throwable unwrappedCause = ExceptionsHelper.unwrapCause(failure.getCause()); + // if the request got rejected or timed out, we need to try it again next time... + if (retryableException(unwrappedCause)) { + nodeEntry.restartFetching(); + } else { + logger.warn(() -> new ParameterizedMessage("failed to list shard for {} on node [{}]", type, failure.nodeId()), failure); + nodeEntry.doneFetching(failure.getCause()); + } + } + } + + boolean retryableException(Throwable unwrappedCause) { + return unwrappedCause instanceof OpenSearchRejectedExecutionException + || unwrappedCause instanceof ReceiveTimeoutTransportException + || unwrappedCause instanceof OpenSearchTimeoutException; + } + + void processFailures(List failures, long fetchingRound) { + for (FailedNodeException failure : failures) { + logger.trace("processing failure {} for [{}]", failure, type); + BaseNodeEntry nodeEntry = getCache().get(failure.nodeId()); + if (nodeEntry != null) { + handleNodeFailure(nodeEntry, failure, fetchingRound); + } + } + } + + /** + * Common function for removing whole node entry. + * + * @param nodeId nodeId to be cleaned. + */ + void remove(String nodeId) { + this.getCache().remove(nodeId); + } + + void markAsFetching(List nodeIds, long fetchingRound) { + for (String nodeId : nodeIds) { + getCache().get(nodeId).markAsFetching(fetchingRound); + } + } + + /** + * A node entry, holding only node level fetching related information. + * Actual metadata of shard is stored in child classes. + */ + static class BaseNodeEntry { + private final String nodeId; + private boolean fetching; + private boolean valueSet; + private Throwable failure; + private long fetchingRound; + + BaseNodeEntry(String nodeId) { + this.nodeId = nodeId; + } + + String getNodeId() { + return this.nodeId; + } + + boolean isFetching() { + return fetching; + } + + void markAsFetching(long fetchingRound) { + assert fetching == false : "double marking a node as fetching"; + this.fetching = true; + this.fetchingRound = fetchingRound; + } + + void doneFetching() { + assert fetching : "setting value but not in fetching mode"; + assert failure == null : "setting value when failure already set"; + this.valueSet = true; + this.fetching = false; + } + + void doneFetching(Throwable failure) { + assert fetching : "setting value but not in fetching mode"; + assert valueSet == false : "setting failure when already set value"; + assert failure != null : "setting failure can't be null"; + this.failure = failure; + this.fetching = false; + } + + void restartFetching() { + assert fetching : "restarting fetching, but not in fetching mode"; + assert valueSet == false : "value can't be set when restarting fetching"; + assert failure == null : "failure can't be set when restarting fetching"; + this.fetching = false; + } + + boolean isFailed() { + return failure != null; + } + + boolean hasData() { + return valueSet || failure != null; + } + + Throwable getFailure() { + assert hasData() : "getting failure when data has not been fetched"; + return failure; + } + + long getFetchingRound() { + return fetchingRound; + } + } +} From aa3f82dfdd6a8e8564a15a067ce11fe8a0297001 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Fri, 15 Mar 2024 16:27:47 +0530 Subject: [PATCH 16/29] Move GatewayShardStarted in helper class to avoid dependency on TransportNodesListGatewayStartedShards Signed-off-by: Aman Khare --- .../gateway/RecoveryFromGatewayIT.java | 25 +-- .../gateway/PrimaryShardAllocator.java | 65 ++++---- .../gateway/PrimaryShardBatchAllocator.java | 18 +-- ...ansportNodesGatewayStartedShardHelper.java | 149 +++++++++++++++++- ...ransportNodesListGatewayStartedShards.java | 3 +- ...ortNodesListGatewayStartedShardsBatch.java | 132 +--------------- .../PrimaryShardBatchAllocatorTests.java | 8 +- 7 files changed, 216 insertions(+), 184 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 6c248a32c9928..3128b55c17029 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -56,6 +56,7 @@ import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; import org.opensearch.env.NodeEnvironment; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayShardStarted; import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; import org.opensearch.index.MergePolicyProvider; @@ -764,11 +765,11 @@ public void testSingleShardFetchUsingBatchAction() { ); final Index index = resolveIndex(indexName); final ShardId shardId = new ShardId(index, 0); - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() + GatewayShardStarted gatewayShardStarted = response.getNodesMap() .get(searchShardsResponse.getNodes()[0].getId()) .getNodeGatewayStartedShardsBatch() .get(shardId); - assertNodeGatewayStartedShardsHappyCase(nodeGatewayStartedShards); + assertNodeGatewayStartedShardsHappyCase(gatewayShardStarted); } public void testShardFetchMultiNodeMultiIndexesUsingBatchAction() { @@ -792,11 +793,11 @@ public void testShardFetchMultiNodeMultiIndexesUsingBatchAction() { ShardId shardId = clusterSearchShardsGroup.getShardId(); assertEquals(1, clusterSearchShardsGroup.getShards().length); String nodeId = clusterSearchShardsGroup.getShards()[0].currentNodeId(); - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() + GatewayShardStarted gatewayShardStarted = response.getNodesMap() .get(nodeId) .getNodeGatewayStartedShardsBatch() .get(shardId); - assertNodeGatewayStartedShardsHappyCase(nodeGatewayStartedShards); + assertNodeGatewayStartedShardsHappyCase(gatewayShardStarted); } } @@ -816,13 +817,13 @@ public void testShardFetchCorruptedShardsUsingBatchAction() throws Exception { new TransportNodesListGatewayStartedShardsBatch.Request(getDiscoveryNodes(), shardIdShardAttributesMap) ); DiscoveryNode[] discoveryNodes = getDiscoveryNodes(); - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() + GatewayShardStarted gatewayShardStarted = response.getNodesMap() .get(discoveryNodes[0].getId()) .getNodeGatewayStartedShardsBatch() .get(shardId); - assertNotNull(nodeGatewayStartedShards.storeException()); - assertNotNull(nodeGatewayStartedShards.allocationId()); - assertTrue(nodeGatewayStartedShards.primary()); + assertNotNull(gatewayShardStarted.storeException()); + assertNotNull(gatewayShardStarted.allocationId()); + assertTrue(gatewayShardStarted.primary()); } public void testSingleShardStoreFetchUsingBatchAction() throws ExecutionException, InterruptedException { @@ -951,11 +952,11 @@ private void assertNodeStoreFilesMetadataSuccessCase( } private void assertNodeGatewayStartedShardsHappyCase( - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeGatewayStartedShards + GatewayShardStarted gatewayShardStarted ) { - assertNull(nodeGatewayStartedShards.storeException()); - assertNotNull(nodeGatewayStartedShards.allocationId()); - assertTrue(nodeGatewayStartedShards.primary()); + assertNull(gatewayShardStarted.storeException()); + assertNotNull(gatewayShardStarted.allocationId()); + assertTrue(gatewayShardStarted.primary()); } private void prepareIndex(String indexName, int numberOfPrimaryShards) { diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java index 5046873830c01..f76c82478155a 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java @@ -50,6 +50,7 @@ import org.opensearch.cluster.routing.allocation.decider.Decision.Type; import org.opensearch.env.ShardLockObtainFailedException; import org.opensearch.gateway.AsyncShardFetch.FetchResult; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.NodeGatewayShardStarted; import org.opensearch.gateway.TransportNodesListGatewayStartedShards.NodeGatewayStartedShards; import java.util.ArrayList; @@ -125,27 +126,33 @@ public AllocateUnassignedDecision makeAllocationDecision( return decision; } final FetchResult shardState = fetchData(unassignedShard, allocation); - List nodeShardStates = adaptToNodeStartedShardList(shardState); + List nodeShardStates = adaptToNodeStartedShardList(shardState); return getAllocationDecision(unassignedShard, allocation, nodeShardStates, logger); } /** - * Transforms {@link FetchResult} of {@link NodeGatewayStartedShards} to {@link List} of {@link NodeGatewayStartedShards} + * Transforms {@link FetchResult} of {@link NodeGatewayStartedShards} to {@link List} of {@link NodeGatewayShardStarted} * Returns null if {@link FetchResult} does not have any data. */ - private static List adaptToNodeStartedShardList(FetchResult shardsState) { + private static List adaptToNodeStartedShardList(FetchResult shardsState) { if (!shardsState.hasData()) { return null; } - List nodeShardStates = new ArrayList<>(); - shardsState.getData().forEach((node, nodeGatewayStartedShard) -> { nodeShardStates.add(nodeGatewayStartedShard); }); + List nodeShardStates = new ArrayList<>(); + shardsState.getData().forEach((node, nodeGatewayStartedShard) -> { nodeShardStates.add(new NodeGatewayShardStarted( + nodeGatewayStartedShard.allocationId(), + nodeGatewayStartedShard.primary(), + nodeGatewayStartedShard.replicationCheckpoint(), + nodeGatewayStartedShard.storeException(), + node + )); }); return nodeShardStates; } protected AllocateUnassignedDecision getAllocationDecision( ShardRouting unassignedShard, RoutingAllocation allocation, - List shardState, + List shardState, Logger logger ) { final boolean explain = allocation.debugDecision(); @@ -236,7 +243,7 @@ protected AllocateUnassignedDecision getAllocationDecision( nodesToAllocate = buildNodesToAllocate(allocation, nodeShardsResult.orderedAllocationCandidates, unassignedShard, true); if (nodesToAllocate.yesNodeShards.isEmpty() == false) { final DecidedNode decidedNode = nodesToAllocate.yesNodeShards.get(0); - final NodeGatewayStartedShards nodeShardState = decidedNode.nodeShardState; + final NodeGatewayShardStarted nodeShardState = decidedNode.nodeShardState; logger.debug( "[{}][{}]: allocating [{}] to [{}] on forced primary allocation", unassignedShard.index(), @@ -296,11 +303,11 @@ protected AllocateUnassignedDecision getAllocationDecision( */ private static List buildNodeDecisions( NodesToAllocate nodesToAllocate, - List fetchedShardData, + List fetchedShardData, Set inSyncAllocationIds ) { List nodeResults = new ArrayList<>(); - Collection ineligibleShards = new ArrayList<>(); + Collection ineligibleShards = new ArrayList<>(); if (nodesToAllocate != null) { final Set discoNodes = new HashSet<>(); nodeResults.addAll( @@ -334,21 +341,21 @@ private static List buildNodeDecisions( return nodeResults; } - private static ShardStoreInfo shardStoreInfo(NodeGatewayStartedShards nodeShardState, Set inSyncAllocationIds) { + private static ShardStoreInfo shardStoreInfo(NodeGatewayShardStarted nodeShardState, Set inSyncAllocationIds) { final Exception storeErr = nodeShardState.storeException(); final boolean inSync = nodeShardState.allocationId() != null && inSyncAllocationIds.contains(nodeShardState.allocationId()); return new ShardStoreInfo(nodeShardState.allocationId(), inSync, storeErr); } - private static final Comparator NO_STORE_EXCEPTION_FIRST_COMPARATOR = Comparator.comparing( - (NodeGatewayStartedShards state) -> state.storeException() == null + private static final Comparator NO_STORE_EXCEPTION_FIRST_COMPARATOR = Comparator.comparing( + (NodeGatewayShardStarted state) -> state.storeException() == null ).reversed(); - private static final Comparator PRIMARY_FIRST_COMPARATOR = Comparator.comparing( - NodeGatewayStartedShards::primary + private static final Comparator PRIMARY_FIRST_COMPARATOR = Comparator.comparing( + NodeGatewayShardStarted::primary ).reversed(); - private static final Comparator HIGHEST_REPLICATION_CHECKPOINT_FIRST_COMPARATOR = Comparator.comparing( - NodeGatewayStartedShards::replicationCheckpoint, + private static final Comparator HIGHEST_REPLICATION_CHECKPOINT_FIRST_COMPARATOR = Comparator.comparing( + NodeGatewayShardStarted::replicationCheckpoint, Comparator.nullsLast(Comparator.naturalOrder()) ); @@ -362,12 +369,12 @@ protected static NodeShardsResult buildNodeShardsResult( boolean matchAnyShard, Set ignoreNodes, Set inSyncAllocationIds, - List shardState, + List shardState, Logger logger ) { - List nodeShardStates = new ArrayList<>(); + List nodeShardStates = new ArrayList<>(); int numberOfAllocationsFound = 0; - for (NodeGatewayStartedShards nodeShardState : shardState) { + for (NodeGatewayShardStarted nodeShardState : shardState) { DiscoveryNode node = nodeShardState.getNode(); String allocationId = nodeShardState.allocationId(); @@ -432,7 +439,7 @@ protected static NodeShardsResult buildNodeShardsResult( return new NodeShardsResult(nodeShardStates, numberOfAllocationsFound); } - private static Comparator createActiveShardComparator( + private static Comparator createActiveShardComparator( boolean matchAnyShard, Set inSyncAllocationIds ) { @@ -442,11 +449,11 @@ private static Comparator createActiveShardComparator( * 2. Prefer previous primary shard * 3. Prefer shard copy with the highest replication checkpoint. It is NO-OP for doc rep enabled indices. */ - final Comparator comparator; // allocation preference + final Comparator comparator; // allocation preference if (matchAnyShard) { // prefer shards with matching allocation ids - Comparator matchingAllocationsFirst = Comparator.comparing( - (NodeGatewayStartedShards state) -> inSyncAllocationIds.contains(state.allocationId()) + Comparator matchingAllocationsFirst = Comparator.comparing( + (NodeGatewayShardStarted state) -> inSyncAllocationIds.contains(state.allocationId()) ).reversed(); comparator = matchingAllocationsFirst.thenComparing(NO_STORE_EXCEPTION_FIRST_COMPARATOR) .thenComparing(PRIMARY_FIRST_COMPARATOR) @@ -464,14 +471,14 @@ private static Comparator createActiveShardComparator( */ private static NodesToAllocate buildNodesToAllocate( RoutingAllocation allocation, - List nodeShardStates, + List nodeShardStates, ShardRouting shardRouting, boolean forceAllocate ) { List yesNodeShards = new ArrayList<>(); List throttledNodeShards = new ArrayList<>(); List noNodeShards = new ArrayList<>(); - for (NodeGatewayStartedShards nodeShardState : nodeShardStates) { + for (NodeGatewayShardStarted nodeShardState : nodeShardStates) { RoutingNode node = allocation.routingNodes().node(nodeShardState.getNode().getId()); if (node == null) { continue; @@ -502,10 +509,10 @@ private static NodesToAllocate buildNodesToAllocate( * This class encapsulates the result of a call to {@link #buildNodeShardsResult} */ static class NodeShardsResult { - final List orderedAllocationCandidates; + final List orderedAllocationCandidates; final int allocationsFound; - NodeShardsResult(List orderedAllocationCandidates, int allocationsFound) { + NodeShardsResult(List orderedAllocationCandidates, int allocationsFound) { this.orderedAllocationCandidates = orderedAllocationCandidates; this.allocationsFound = allocationsFound; } @@ -531,10 +538,10 @@ protected static class NodesToAllocate { * by the allocator for allocating to the node that holds the shard copy. */ private static class DecidedNode { - final NodeGatewayStartedShards nodeShardState; + final NodeGatewayShardStarted nodeShardState; final Decision decision; - private DecidedNode(NodeGatewayStartedShards nodeShardState, Decision decision) { + private DecidedNode(NodeGatewayShardStarted nodeShardState, Decision decision) { this.nodeShardState = nodeShardState; this.decision = decision; } diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java index d073482405ec2..3f04ac4d2f575 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java @@ -15,7 +15,7 @@ import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.gateway.AsyncShardFetch.FetchResult; -import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.NodeGatewayShardStarted; import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch; import java.util.ArrayList; @@ -54,7 +54,7 @@ protected FetchResult makeAllocationDecision( // process the received data for (ShardRouting unassignedShard : eligibleShards) { - List nodeShardStates = adaptToNodeShardStates( + List nodeShardStates = adaptToNodeShardStates( unassignedShard, shardsState ); @@ -123,27 +123,27 @@ public HashMap makeAllocationDecision( * @param shardsState fetch data result for the whole batch * @return shard state returned from each node */ - private static List adaptToNodeShardStates( + private static List adaptToNodeShardStates( ShardRouting unassignedShard, FetchResult shardsState ) { if (!shardsState.hasData()) { return null; } - List nodeShardStates = new ArrayList<>(); + List nodeShardStates = new ArrayList<>(); Map nodeResponses = shardsState.getData(); // build data for a shard from all the nodes nodeResponses.forEach((node, nodeGatewayStartedShardsBatch) -> { - NodeGatewayStartedShard shardData = nodeGatewayStartedShardsBatch.getNodeGatewayStartedShardsBatch() + TransportNodesGatewayStartedShardHelper.GatewayShardStarted shardData = nodeGatewayStartedShardsBatch.getNodeGatewayStartedShardsBatch() .get(unassignedShard.shardId()); nodeShardStates.add( - new TransportNodesListGatewayStartedShards.NodeGatewayStartedShards( - node, + new NodeGatewayShardStarted( shardData.allocationId(), shardData.primary(), shardData.replicationCheckpoint(), - shardData.storeException() + shardData.storeException(), + node ) ); }); diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java index 403e3e96fa209..01b944d4390d1 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java @@ -12,8 +12,11 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.OpenSearchException; import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.NodeEnvironment; @@ -23,8 +26,10 @@ import org.opensearch.index.shard.ShardStateMetadata; import org.opensearch.index.store.Store; import org.opensearch.indices.IndicesService; +import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import java.io.IOException; +import java.util.Objects; /** * This class has the common code used in {@link TransportNodesListGatewayStartedShards} and @@ -37,7 +42,7 @@ * @opensearch.internal */ public class TransportNodesGatewayStartedShardHelper { - public static TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard getShardInfoOnLocalNode( + public static GatewayShardStarted getShardInfoOnLocalNode( Logger logger, final ShardId shardId, NamedXContentRegistry namedXContentRegistry, @@ -90,7 +95,7 @@ public static TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShar exception ); String allocationId = shardStateMetadata.allocationId != null ? shardStateMetadata.allocationId.getId() : null; - return new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard( + return new GatewayShardStarted( allocationId, shardStateMetadata.primary, null, @@ -102,13 +107,149 @@ public static TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShar logger.debug("{} shard state info found: [{}]", shardId, shardStateMetadata); String allocationId = shardStateMetadata.allocationId != null ? shardStateMetadata.allocationId.getId() : null; final IndexShard shard = indicesService.getShardOrNull(shardId); - return new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard( + return new GatewayShardStarted( allocationId, shardStateMetadata.primary, shard != null ? shard.getLatestReplicationCheckpoint() : null ); } logger.trace("{} no local shard info found", shardId); - return new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard(null, false, null); + return new GatewayShardStarted(null, false, null); + } + + /** + * This class encapsulates the metadata about a started shard that needs to be persisted or sent between nodes. + * This is used in {@link TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch} to construct the response for each node, instead of + * {@link TransportNodesListGatewayStartedShards.NodeGatewayStartedShards} because we don't need to save an extra + * {@link DiscoveryNode} object like in {@link TransportNodesListGatewayStartedShards.NodeGatewayStartedShards} + * which reduces memory footprint of its objects. + * + * @opensearch.internal + */ + public static class GatewayShardStarted { + private final String allocationId; + private final boolean primary; + private final Exception storeException; + private final ReplicationCheckpoint replicationCheckpoint; + + public GatewayShardStarted(StreamInput in) throws IOException { + allocationId = in.readOptionalString(); + primary = in.readBoolean(); + if (in.readBoolean()) { + storeException = in.readException(); + } else { + storeException = null; + } + if (in.readBoolean()) { + replicationCheckpoint = new ReplicationCheckpoint(in); + } else { + replicationCheckpoint = null; + } + } + + public GatewayShardStarted(String allocationId, boolean primary, ReplicationCheckpoint replicationCheckpoint) { + this(allocationId, primary, replicationCheckpoint, null); + } + + public GatewayShardStarted( + String allocationId, + boolean primary, + ReplicationCheckpoint replicationCheckpoint, + Exception storeException + ) { + this.allocationId = allocationId; + this.primary = primary; + this.replicationCheckpoint = replicationCheckpoint; + this.storeException = storeException; + } + + public String allocationId() { + return this.allocationId; + } + + public boolean primary() { + return this.primary; + } + + public ReplicationCheckpoint replicationCheckpoint() { + return this.replicationCheckpoint; + } + + public Exception storeException() { + return this.storeException; + } + + public void writeTo(StreamOutput out) throws IOException { + out.writeOptionalString(allocationId); + out.writeBoolean(primary); + if (storeException != null) { + out.writeBoolean(true); + out.writeException(storeException); + } else { + out.writeBoolean(false); + } + if (replicationCheckpoint != null) { + out.writeBoolean(true); + replicationCheckpoint.writeTo(out); + } else { + out.writeBoolean(false); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + GatewayShardStarted that = (GatewayShardStarted) o; + + return primary == that.primary + && Objects.equals(allocationId, that.allocationId) + && Objects.equals(storeException, that.storeException) + && Objects.equals(replicationCheckpoint, that.replicationCheckpoint); + } + + @Override + public int hashCode() { + int result = (allocationId != null ? allocationId.hashCode() : 0); + result = 31 * result + (primary ? 1 : 0); + result = 31 * result + (storeException != null ? storeException.hashCode() : 0); + result = 31 * result + (replicationCheckpoint != null ? replicationCheckpoint.hashCode() : 0); + return result; + } + + @Override + public String toString() { + StringBuilder buf = new StringBuilder(); + buf.append("NodeGatewayStartedShards[").append("allocationId=").append(allocationId).append(",primary=").append(primary); + if (storeException != null) { + buf.append(",storeException=").append(storeException); + } + if (replicationCheckpoint != null) { + buf.append(",ReplicationCheckpoint=").append(replicationCheckpoint.toString()); + } + buf.append("]"); + return buf.toString(); + } + } + + public static class NodeGatewayShardStarted extends GatewayShardStarted { + + private final DiscoveryNode node; + + public NodeGatewayShardStarted(String allocationId, boolean primary, + ReplicationCheckpoint replicationCheckpoint, Exception storeException, + DiscoveryNode node) { + super(allocationId, primary, replicationCheckpoint, storeException); + this.node = node; + } + + public DiscoveryNode getNode() { + return node; + } } } diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java index 0ba872aab9974..cb4470d0c04da 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java @@ -59,6 +59,7 @@ import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportRequest; import org.opensearch.transport.TransportService; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayShardStarted; import java.io.IOException; import java.util.List; @@ -154,7 +155,7 @@ protected NodesGatewayStartedShards newResponse( @Override protected NodeGatewayStartedShards nodeOperation(NodeRequest request) { try { - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard shardInfo = getShardInfoOnLocalNode( + GatewayShardStarted shardInfo = getShardInfoOnLocalNode( logger, request.getShardId(), namedXContentRegistry, diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java index bc327c1b85748..c6aeca85f984b 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java @@ -28,7 +28,7 @@ import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.NodeEnvironment; import org.opensearch.indices.IndicesService; -import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayShardStarted; import org.opensearch.indices.store.ShardAttributes; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportRequest; @@ -135,7 +135,7 @@ protected NodesGatewayStartedShardsBatch newResponse( */ @Override protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { - Map shardsOnNode = new HashMap<>(); + Map shardsOnNode = new HashMap<>(); for (ShardAttributes shardAttr : request.shardAttributes.values()) { final ShardId shardId = shardAttr.getShardId(); try { @@ -155,7 +155,7 @@ protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { } catch (Exception e) { shardsOnNode.put( shardId, - new NodeGatewayStartedShard(null, false, null, new OpenSearchException("failed to load started shards", e)) + new GatewayShardStarted(null, false, null, new OpenSearchException("failed to load started shards", e)) ); } } @@ -248,125 +248,7 @@ public void writeTo(StreamOutput out) throws IOException { } } - /** - * This class encapsulates the metadata about a started shard that needs to be persisted or sent between nodes. - * This is used in {@link NodeGatewayStartedShardsBatch} to construct the response for each node, instead of - * {@link TransportNodesListGatewayStartedShards.NodeGatewayStartedShards} because we don't need to save an extra - * {@link DiscoveryNode} object like in {@link TransportNodesListGatewayStartedShards.NodeGatewayStartedShards} - * which reduces memory footprint of its objects. - * - * @opensearch.internal - */ - public static class NodeGatewayStartedShard { - private final String allocationId; - private final boolean primary; - private final Exception storeException; - private final ReplicationCheckpoint replicationCheckpoint; - - public NodeGatewayStartedShard(StreamInput in) throws IOException { - allocationId = in.readOptionalString(); - primary = in.readBoolean(); - if (in.readBoolean()) { - storeException = in.readException(); - } else { - storeException = null; - } - if (in.readBoolean()) { - replicationCheckpoint = new ReplicationCheckpoint(in); - } else { - replicationCheckpoint = null; - } - } - - public NodeGatewayStartedShard(String allocationId, boolean primary, ReplicationCheckpoint replicationCheckpoint) { - this(allocationId, primary, replicationCheckpoint, null); - } - - public NodeGatewayStartedShard( - String allocationId, - boolean primary, - ReplicationCheckpoint replicationCheckpoint, - Exception storeException - ) { - this.allocationId = allocationId; - this.primary = primary; - this.replicationCheckpoint = replicationCheckpoint; - this.storeException = storeException; - } - - public String allocationId() { - return this.allocationId; - } - - public boolean primary() { - return this.primary; - } - - public ReplicationCheckpoint replicationCheckpoint() { - return this.replicationCheckpoint; - } - - public Exception storeException() { - return this.storeException; - } - public void writeTo(StreamOutput out) throws IOException { - out.writeOptionalString(allocationId); - out.writeBoolean(primary); - if (storeException != null) { - out.writeBoolean(true); - out.writeException(storeException); - } else { - out.writeBoolean(false); - } - if (replicationCheckpoint != null) { - out.writeBoolean(true); - replicationCheckpoint.writeTo(out); - } else { - out.writeBoolean(false); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - NodeGatewayStartedShard that = (NodeGatewayStartedShard) o; - - return primary == that.primary - && Objects.equals(allocationId, that.allocationId) - && Objects.equals(storeException, that.storeException) - && Objects.equals(replicationCheckpoint, that.replicationCheckpoint); - } - - @Override - public int hashCode() { - int result = (allocationId != null ? allocationId.hashCode() : 0); - result = 31 * result + (primary ? 1 : 0); - result = 31 * result + (storeException != null ? storeException.hashCode() : 0); - result = 31 * result + (replicationCheckpoint != null ? replicationCheckpoint.hashCode() : 0); - return result; - } - - @Override - public String toString() { - StringBuilder buf = new StringBuilder(); - buf.append("NodeGatewayStartedShards[").append("allocationId=").append(allocationId).append(",primary=").append(primary); - if (storeException != null) { - buf.append(",storeException=").append(storeException); - } - if (replicationCheckpoint != null) { - buf.append(",ReplicationCheckpoint=").append(replicationCheckpoint.toString()); - } - buf.append("]"); - return buf.toString(); - } - } /** * This is the response from a single node, this is used in {@link NodesGatewayStartedShardsBatch} for creating @@ -376,15 +258,15 @@ public String toString() { * @opensearch.internal */ public static class NodeGatewayStartedShardsBatch extends BaseNodeResponse { - private final Map nodeGatewayStartedShardsBatch; + private final Map nodeGatewayStartedShardsBatch; - public Map getNodeGatewayStartedShardsBatch() { + public Map getNodeGatewayStartedShardsBatch() { return nodeGatewayStartedShardsBatch; } public NodeGatewayStartedShardsBatch(StreamInput in) throws IOException { super(in); - this.nodeGatewayStartedShardsBatch = in.readMap(ShardId::new, NodeGatewayStartedShard::new); + this.nodeGatewayStartedShardsBatch = in.readMap(ShardId::new, GatewayShardStarted::new); } @Override @@ -393,7 +275,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeMap(nodeGatewayStartedShardsBatch, (o, k) -> k.writeTo(o), (o, v) -> v.writeTo(o)); } - public NodeGatewayStartedShardsBatch(DiscoveryNode node, Map nodeGatewayStartedShardsBatch) { + public NodeGatewayStartedShardsBatch(DiscoveryNode node, Map nodeGatewayStartedShardsBatch) { super(node); this.nodeGatewayStartedShardsBatch = nodeGatewayStartedShardsBatch; } diff --git a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java index a95328b9ddc39..0175961f6a274 100644 --- a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java @@ -289,9 +289,9 @@ public TestBatchAllocator addData( if (data == null) { data = new HashMap<>(); } - Map shardData = Map.of( + Map shardData = Map.of( shardId, - new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard( + new TransportNodesGatewayStartedShardHelper.GatewayShardStarted( allocationId, primary, replicationCheckpoint, @@ -313,10 +313,10 @@ public TestBatchAllocator addShardData( if (data == null) { data = new HashMap<>(); } - Map shardData = new HashMap<>(); + Map shardData = new HashMap<>(); shardData.put( shardId, - new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard( + new TransportNodesGatewayStartedShardHelper.GatewayShardStarted( allocationId, primary, replicationCheckpoint, From e0cff1a2e52991392ee3d0c32e309a1bc39c07d2 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Fri, 15 Mar 2024 16:37:47 +0530 Subject: [PATCH 17/29] spotless apply Signed-off-by: Aman Khare --- .../gateway/RecoveryFromGatewayIT.java | 9 ++------ .../gateway/PrimaryShardAllocator.java | 23 ++++++++++--------- .../gateway/PrimaryShardBatchAllocator.java | 8 +++---- ...ansportNodesGatewayStartedShardHelper.java | 17 +++++++------- ...ransportNodesListGatewayStartedShards.java | 2 +- ...ortNodesListGatewayStartedShardsBatch.java | 4 +--- 6 files changed, 27 insertions(+), 36 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 3128b55c17029..adc233ab6d047 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -793,10 +793,7 @@ public void testShardFetchMultiNodeMultiIndexesUsingBatchAction() { ShardId shardId = clusterSearchShardsGroup.getShardId(); assertEquals(1, clusterSearchShardsGroup.getShards().length); String nodeId = clusterSearchShardsGroup.getShards()[0].currentNodeId(); - GatewayShardStarted gatewayShardStarted = response.getNodesMap() - .get(nodeId) - .getNodeGatewayStartedShardsBatch() - .get(shardId); + GatewayShardStarted gatewayShardStarted = response.getNodesMap().get(nodeId).getNodeGatewayStartedShardsBatch().get(shardId); assertNodeGatewayStartedShardsHappyCase(gatewayShardStarted); } } @@ -951,9 +948,7 @@ private void assertNodeStoreFilesMetadataSuccessCase( assertNotNull(storeFileMetadata.peerRecoveryRetentionLeases()); } - private void assertNodeGatewayStartedShardsHappyCase( - GatewayShardStarted gatewayShardStarted - ) { + private void assertNodeGatewayStartedShardsHappyCase(GatewayShardStarted gatewayShardStarted) { assertNull(gatewayShardStarted.storeException()); assertNotNull(gatewayShardStarted.allocationId()); assertTrue(gatewayShardStarted.primary()); diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java index f76c82478155a..e16e84c95b4b2 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java @@ -139,13 +139,17 @@ private static List adaptToNodeStartedShardList(FetchRe return null; } List nodeShardStates = new ArrayList<>(); - shardsState.getData().forEach((node, nodeGatewayStartedShard) -> { nodeShardStates.add(new NodeGatewayShardStarted( - nodeGatewayStartedShard.allocationId(), - nodeGatewayStartedShard.primary(), - nodeGatewayStartedShard.replicationCheckpoint(), - nodeGatewayStartedShard.storeException(), - node - )); }); + shardsState.getData().forEach((node, nodeGatewayStartedShard) -> { + nodeShardStates.add( + new NodeGatewayShardStarted( + nodeGatewayStartedShard.allocationId(), + nodeGatewayStartedShard.primary(), + nodeGatewayStartedShard.replicationCheckpoint(), + nodeGatewayStartedShard.storeException(), + node + ) + ); + }); return nodeShardStates; } @@ -439,10 +443,7 @@ protected static NodeShardsResult buildNodeShardsResult( return new NodeShardsResult(nodeShardStates, numberOfAllocationsFound); } - private static Comparator createActiveShardComparator( - boolean matchAnyShard, - Set inSyncAllocationIds - ) { + private static Comparator createActiveShardComparator(boolean matchAnyShard, Set inSyncAllocationIds) { /** * Orders the active shards copies based on below comparators * 1. No store exception i.e. shard copy is readable diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java index 3f04ac4d2f575..08ce4ab9b3f39 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java @@ -99,10 +99,7 @@ public HashMap makeAllocationDecision( // process the received data for (ShardRouting unassignedShard : eligibleShards) { - List nodeShardStates = adaptToNodeShardStates( - unassignedShard, - shardsState - ); + List nodeShardStates = adaptToNodeShardStates(unassignedShard, shardsState); // get allocation decision for this shard shardAllocationDecisions.put(unassignedShard, getAllocationDecision(unassignedShard, allocation, nodeShardStates, logger)); } @@ -135,7 +132,8 @@ private static List adaptToNodeShardStates( // build data for a shard from all the nodes nodeResponses.forEach((node, nodeGatewayStartedShardsBatch) -> { - TransportNodesGatewayStartedShardHelper.GatewayShardStarted shardData = nodeGatewayStartedShardsBatch.getNodeGatewayStartedShardsBatch() + TransportNodesGatewayStartedShardHelper.GatewayShardStarted shardData = nodeGatewayStartedShardsBatch + .getNodeGatewayStartedShardsBatch() .get(unassignedShard.shardId()); nodeShardStates.add( new NodeGatewayShardStarted( diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java index 01b944d4390d1..5168cac6f920b 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java @@ -95,12 +95,7 @@ public static GatewayShardStarted getShardInfoOnLocalNode( exception ); String allocationId = shardStateMetadata.allocationId != null ? shardStateMetadata.allocationId.getId() : null; - return new GatewayShardStarted( - allocationId, - shardStateMetadata.primary, - null, - exception - ); + return new GatewayShardStarted(allocationId, shardStateMetadata.primary, null, exception); } } @@ -241,9 +236,13 @@ public static class NodeGatewayShardStarted extends GatewayShardStarted { private final DiscoveryNode node; - public NodeGatewayShardStarted(String allocationId, boolean primary, - ReplicationCheckpoint replicationCheckpoint, Exception storeException, - DiscoveryNode node) { + public NodeGatewayShardStarted( + String allocationId, + boolean primary, + ReplicationCheckpoint replicationCheckpoint, + Exception storeException, + DiscoveryNode node + ) { super(allocationId, primary, replicationCheckpoint, storeException); this.node = node; } diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java index cb4470d0c04da..f81e6bb46bb64 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java @@ -53,13 +53,13 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.NodeEnvironment; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayShardStarted; import org.opensearch.indices.IndicesService; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.store.ShardAttributes; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportRequest; import org.opensearch.transport.TransportService; -import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayShardStarted; import java.io.IOException; import java.util.List; diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java index c6aeca85f984b..d8e83b955d46d 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java @@ -27,8 +27,8 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.NodeEnvironment; -import org.opensearch.indices.IndicesService; import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayShardStarted; +import org.opensearch.indices.IndicesService; import org.opensearch.indices.store.ShardAttributes; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportRequest; @@ -248,8 +248,6 @@ public void writeTo(StreamOutput out) throws IOException { } } - - /** * This is the response from a single node, this is used in {@link NodesGatewayStartedShardsBatch} for creating * node to its response mapping for this transport request. From 6112f4b14c2e3175fdcdd15afa2d86be8ad64be6 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Fri, 15 Mar 2024 16:49:15 +0530 Subject: [PATCH 18/29] Add java doc on new class Signed-off-by: Aman Khare --- .../gateway/TransportNodesGatewayStartedShardHelper.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java index 5168cac6f920b..d08291de2eac5 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java @@ -232,6 +232,14 @@ public String toString() { } } + /** + * This class extends the {@link GatewayShardStarted} which contains all necessary shard metadata like + * allocationId and replication checkpoint. It also has DiscoveryNode which is needed by + * {@link PrimaryShardAllocator} and {@link PrimaryShardBatchAllocator} to make allocation decision. + * This class removes the dependency of + * {@link TransportNodesListGatewayStartedShards.NodeGatewayStartedShards} to make allocation decisions by + * {@link PrimaryShardAllocator} or {@link PrimaryShardBatchAllocator}. + */ public static class NodeGatewayShardStarted extends GatewayShardStarted { private final DiscoveryNode node; From e798d7aaff720136b3228d47b01590502d1b6b2a Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Fri, 15 Mar 2024 22:57:31 +0530 Subject: [PATCH 19/29] Remove code duplication Signed-off-by: Aman Khare --- .../gateway/RecoveryFromGatewayIT.java | 6 +- .../TransportIndicesShardStoresAction.java | 7 +- .../gateway/PrimaryShardAllocator.java | 8 +- ...ransportNodesListGatewayStartedShards.java | 94 +++++-------------- .../gateway/PrimaryShardAllocatorTests.java | 10 +- .../test/gateway/TestGatewayAllocator.java | 10 +- 6 files changed, 49 insertions(+), 86 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index adc233ab6d047..8404d1f60399e 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -721,11 +721,11 @@ public Settings onNodeStopped(String nodeName) throws Exception { ); assertThat(response.getNodes(), hasSize(1)); - assertThat(response.getNodes().get(0).allocationId(), notNullValue()); + assertThat(response.getNodes().get(0).getGatewayShardStarted().allocationId(), notNullValue()); if (corrupt) { - assertThat(response.getNodes().get(0).storeException(), notNullValue()); + assertThat(response.getNodes().get(0).getGatewayShardStarted().storeException(), notNullValue()); } else { - assertThat(response.getNodes().get(0).storeException(), nullValue()); + assertThat(response.getNodes().get(0).getGatewayShardStarted().storeException(), nullValue()); } // start another node so cluster consistency checks won't time out due to the lack of state diff --git a/server/src/main/java/org/opensearch/action/admin/indices/shards/TransportIndicesShardStoresAction.java b/server/src/main/java/org/opensearch/action/admin/indices/shards/TransportIndicesShardStoresAction.java index 04166c88a00ad..3fbf9ac1bb570 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/shards/TransportIndicesShardStoresAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/shards/TransportIndicesShardStoresAction.java @@ -258,9 +258,9 @@ void finish() { storeStatuses.add( new IndicesShardStoresResponse.StoreStatus( response.getNode(), - response.allocationId(), + response.getGatewayShardStarted().allocationId(), allocationStatus, - response.storeException() + response.getGatewayShardStarted().storeException() ) ); } @@ -308,7 +308,8 @@ private IndicesShardStoresResponse.StoreStatus.AllocationStatus getAllocationSta * A shard exists/existed in a node only if shard state file exists in the node */ private boolean shardExistsInNode(final NodeGatewayStartedShards response) { - return response.storeException() != null || response.allocationId() != null; + return response.getGatewayShardStarted().storeException() != null + || response.getGatewayShardStarted().allocationId() != null; } @Override diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java index e16e84c95b4b2..aebbd6525e017 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java @@ -142,10 +142,10 @@ private static List adaptToNodeStartedShardList(FetchRe shardsState.getData().forEach((node, nodeGatewayStartedShard) -> { nodeShardStates.add( new NodeGatewayShardStarted( - nodeGatewayStartedShard.allocationId(), - nodeGatewayStartedShard.primary(), - nodeGatewayStartedShard.replicationCheckpoint(), - nodeGatewayStartedShard.storeException(), + nodeGatewayStartedShard.getGatewayShardStarted().allocationId(), + nodeGatewayStartedShard.getGatewayShardStarted().primary(), + nodeGatewayStartedShard.getGatewayShardStarted().replicationCheckpoint(), + nodeGatewayStartedShard.getGatewayShardStarted().storeException(), node ) ); diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java index f81e6bb46bb64..f18ae26c0c8c2 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java @@ -167,10 +167,12 @@ protected NodeGatewayStartedShards nodeOperation(NodeRequest request) { ); return new NodeGatewayStartedShards( clusterService.localNode(), - shardInfo.allocationId(), - shardInfo.primary(), - shardInfo.replicationCheckpoint(), - shardInfo.storeException() + new GatewayShardStarted( + shardInfo.allocationId(), + shardInfo.primary(), + shardInfo.replicationCheckpoint(), + shardInfo.storeException() + ) ); } catch (Exception e) { throw new OpenSearchException("failed to load started shards", e); @@ -303,81 +305,51 @@ public String getCustomDataPath() { * @opensearch.internal */ public static class NodeGatewayStartedShards extends BaseNodeResponse { - private final String allocationId; - private final boolean primary; - private final Exception storeException; - private final ReplicationCheckpoint replicationCheckpoint; + private final GatewayShardStarted gatewayShardStarted; public NodeGatewayStartedShards(StreamInput in) throws IOException { super(in); - allocationId = in.readOptionalString(); - primary = in.readBoolean(); + String allocationId = in.readOptionalString(); + boolean primary = in.readBoolean(); + Exception storeException; if (in.readBoolean()) { storeException = in.readException(); } else { storeException = null; } + ReplicationCheckpoint replicationCheckpoint; if (in.getVersion().onOrAfter(Version.V_2_3_0) && in.readBoolean()) { replicationCheckpoint = new ReplicationCheckpoint(in); } else { replicationCheckpoint = null; } + this.gatewayShardStarted = new GatewayShardStarted(allocationId, primary, replicationCheckpoint, storeException); } - public NodeGatewayStartedShards( - DiscoveryNode node, - String allocationId, - boolean primary, - ReplicationCheckpoint replicationCheckpoint - ) { - this(node, allocationId, primary, replicationCheckpoint, null); + public GatewayShardStarted getGatewayShardStarted() { + return gatewayShardStarted; } - public NodeGatewayStartedShards( - DiscoveryNode node, - String allocationId, - boolean primary, - ReplicationCheckpoint replicationCheckpoint, - Exception storeException - ) { + public NodeGatewayStartedShards(DiscoveryNode node, GatewayShardStarted gatewayShardStarted) { super(node); - this.allocationId = allocationId; - this.primary = primary; - this.replicationCheckpoint = replicationCheckpoint; - this.storeException = storeException; - } - - public String allocationId() { - return this.allocationId; - } - - public boolean primary() { - return this.primary; - } - - public ReplicationCheckpoint replicationCheckpoint() { - return this.replicationCheckpoint; - } - - public Exception storeException() { - return this.storeException; + this.gatewayShardStarted = gatewayShardStarted; } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeOptionalString(allocationId); - out.writeBoolean(primary); - if (storeException != null) { + out.writeOptionalString(gatewayShardStarted.allocationId()); + out.writeBoolean(gatewayShardStarted.primary()); + if (gatewayShardStarted.storeException() != null) { out.writeBoolean(true); - out.writeException(storeException); + out.writeException(gatewayShardStarted.storeException()); } else { out.writeBoolean(false); } if (out.getVersion().onOrAfter(Version.V_2_3_0)) { - if (replicationCheckpoint != null) { + if (gatewayShardStarted.replicationCheckpoint() != null) { out.writeBoolean(true); - replicationCheckpoint.writeTo(out); + gatewayShardStarted.replicationCheckpoint().writeTo(out); } else { out.writeBoolean(false); } @@ -395,33 +367,17 @@ public boolean equals(Object o) { NodeGatewayStartedShards that = (NodeGatewayStartedShards) o; - return primary == that.primary - && Objects.equals(allocationId, that.allocationId) - && Objects.equals(storeException, that.storeException) - && Objects.equals(replicationCheckpoint, that.replicationCheckpoint); + return gatewayShardStarted.equals(that.gatewayShardStarted); } @Override public int hashCode() { - int result = (allocationId != null ? allocationId.hashCode() : 0); - result = 31 * result + (primary ? 1 : 0); - result = 31 * result + (storeException != null ? storeException.hashCode() : 0); - result = 31 * result + (replicationCheckpoint != null ? replicationCheckpoint.hashCode() : 0); - return result; + return gatewayShardStarted.hashCode(); } @Override public String toString() { - StringBuilder buf = new StringBuilder(); - buf.append("NodeGatewayStartedShards[").append("allocationId=").append(allocationId).append(",primary=").append(primary); - if (storeException != null) { - buf.append(",storeException=").append(storeException); - } - if (replicationCheckpoint != null) { - buf.append(",ReplicationCheckpoint=").append(replicationCheckpoint.toString()); - } - buf.append("]"); - return buf.toString(); + return gatewayShardStarted.toString(); } } } diff --git a/server/src/test/java/org/opensearch/gateway/PrimaryShardAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/PrimaryShardAllocatorTests.java index dceda6433575c..cf3eed82fc940 100644 --- a/server/src/test/java/org/opensearch/gateway/PrimaryShardAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/PrimaryShardAllocatorTests.java @@ -843,10 +843,12 @@ public TestAllocator addData( node, new TransportNodesListGatewayStartedShards.NodeGatewayStartedShards( node, - allocationId, - primary, - replicationCheckpoint, - storeException + new TransportNodesGatewayStartedShardHelper.GatewayShardStarted( + allocationId, + primary, + replicationCheckpoint, + storeException + ) ) ); return this; diff --git a/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java b/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java index f123b926f5bad..c3897e66479be 100644 --- a/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java +++ b/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java @@ -42,6 +42,7 @@ import org.opensearch.gateway.GatewayAllocator; import org.opensearch.gateway.PrimaryShardAllocator; import org.opensearch.gateway.ReplicaShardAllocator; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper; import org.opensearch.gateway.TransportNodesListGatewayStartedShards.NodeGatewayStartedShards; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata.NodeStoreFilesMetadata; @@ -91,9 +92,12 @@ protected AsyncShardFetch.FetchResult fetchData(ShardR routing -> currentNodes.get(routing.currentNodeId()), routing -> new NodeGatewayStartedShards( currentNodes.get(routing.currentNodeId()), - routing.allocationId().getId(), - routing.primary(), - getReplicationCheckpoint(shardId, routing.currentNodeId()) + new TransportNodesGatewayStartedShardHelper.GatewayShardStarted( + routing.allocationId().getId(), + routing.primary(), + getReplicationCheckpoint(shardId, routing.currentNodeId()), + null + ) ) ) ); From d372c54e0f648e3e07c2f9b2ae42c926a5a3e358 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Mon, 18 Mar 2024 18:24:14 +0530 Subject: [PATCH 20/29] rename class to older style Signed-off-by: Aman Khare --- .../gateway/RecoveryFromGatewayIT.java | 26 ++++---- .../gateway/PrimaryShardAllocator.java | 60 +++++++++---------- .../gateway/PrimaryShardBatchAllocator.java | 12 ++-- ...ansportNodesGatewayStartedShardHelper.java | 24 ++++---- ...ransportNodesListGatewayStartedShards.java | 36 +++++------ ...ortNodesListGatewayStartedShardsBatch.java | 14 ++--- .../gateway/PrimaryShardAllocatorTests.java | 2 +- .../PrimaryShardBatchAllocatorTests.java | 8 +-- .../test/gateway/TestGatewayAllocator.java | 2 +- 9 files changed, 92 insertions(+), 92 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 8404d1f60399e..ba03532a9aa2f 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -56,7 +56,7 @@ import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; import org.opensearch.env.NodeEnvironment; -import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayShardStarted; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayStartedShard; import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; import org.opensearch.index.MergePolicyProvider; @@ -765,11 +765,11 @@ public void testSingleShardFetchUsingBatchAction() { ); final Index index = resolveIndex(indexName); final ShardId shardId = new ShardId(index, 0); - GatewayShardStarted gatewayShardStarted = response.getNodesMap() + GatewayStartedShard gatewayStartedShard = response.getNodesMap() .get(searchShardsResponse.getNodes()[0].getId()) .getNodeGatewayStartedShardsBatch() .get(shardId); - assertNodeGatewayStartedShardsHappyCase(gatewayShardStarted); + assertNodeGatewayStartedShardsHappyCase(gatewayStartedShard); } public void testShardFetchMultiNodeMultiIndexesUsingBatchAction() { @@ -793,8 +793,8 @@ public void testShardFetchMultiNodeMultiIndexesUsingBatchAction() { ShardId shardId = clusterSearchShardsGroup.getShardId(); assertEquals(1, clusterSearchShardsGroup.getShards().length); String nodeId = clusterSearchShardsGroup.getShards()[0].currentNodeId(); - GatewayShardStarted gatewayShardStarted = response.getNodesMap().get(nodeId).getNodeGatewayStartedShardsBatch().get(shardId); - assertNodeGatewayStartedShardsHappyCase(gatewayShardStarted); + GatewayStartedShard gatewayStartedShard = response.getNodesMap().get(nodeId).getNodeGatewayStartedShardsBatch().get(shardId); + assertNodeGatewayStartedShardsHappyCase(gatewayStartedShard); } } @@ -814,13 +814,13 @@ public void testShardFetchCorruptedShardsUsingBatchAction() throws Exception { new TransportNodesListGatewayStartedShardsBatch.Request(getDiscoveryNodes(), shardIdShardAttributesMap) ); DiscoveryNode[] discoveryNodes = getDiscoveryNodes(); - GatewayShardStarted gatewayShardStarted = response.getNodesMap() + GatewayStartedShard gatewayStartedShard = response.getNodesMap() .get(discoveryNodes[0].getId()) .getNodeGatewayStartedShardsBatch() .get(shardId); - assertNotNull(gatewayShardStarted.storeException()); - assertNotNull(gatewayShardStarted.allocationId()); - assertTrue(gatewayShardStarted.primary()); + assertNotNull(gatewayStartedShard.storeException()); + assertNotNull(gatewayStartedShard.allocationId()); + assertTrue(gatewayStartedShard.primary()); } public void testSingleShardStoreFetchUsingBatchAction() throws ExecutionException, InterruptedException { @@ -948,10 +948,10 @@ private void assertNodeStoreFilesMetadataSuccessCase( assertNotNull(storeFileMetadata.peerRecoveryRetentionLeases()); } - private void assertNodeGatewayStartedShardsHappyCase(GatewayShardStarted gatewayShardStarted) { - assertNull(gatewayShardStarted.storeException()); - assertNotNull(gatewayShardStarted.allocationId()); - assertTrue(gatewayShardStarted.primary()); + private void assertNodeGatewayStartedShardsHappyCase(GatewayStartedShard gatewayStartedShard) { + assertNull(gatewayStartedShard.storeException()); + assertNotNull(gatewayStartedShard.allocationId()); + assertTrue(gatewayStartedShard.primary()); } private void prepareIndex(String indexName, int numberOfPrimaryShards) { diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java index aebbd6525e017..f41545cbdf9bf 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java @@ -50,7 +50,7 @@ import org.opensearch.cluster.routing.allocation.decider.Decision.Type; import org.opensearch.env.ShardLockObtainFailedException; import org.opensearch.gateway.AsyncShardFetch.FetchResult; -import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.NodeGatewayShardStarted; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.NodeGatewayStartedShard; import org.opensearch.gateway.TransportNodesListGatewayStartedShards.NodeGatewayStartedShards; import java.util.ArrayList; @@ -126,22 +126,22 @@ public AllocateUnassignedDecision makeAllocationDecision( return decision; } final FetchResult shardState = fetchData(unassignedShard, allocation); - List nodeShardStates = adaptToNodeStartedShardList(shardState); + List nodeShardStates = adaptToNodeStartedShardList(shardState); return getAllocationDecision(unassignedShard, allocation, nodeShardStates, logger); } /** - * Transforms {@link FetchResult} of {@link NodeGatewayStartedShards} to {@link List} of {@link NodeGatewayShardStarted} + * Transforms {@link FetchResult} of {@link NodeGatewayStartedShards} to {@link List} of {@link NodeGatewayStartedShard} * Returns null if {@link FetchResult} does not have any data. */ - private static List adaptToNodeStartedShardList(FetchResult shardsState) { + private static List adaptToNodeStartedShardList(FetchResult shardsState) { if (!shardsState.hasData()) { return null; } - List nodeShardStates = new ArrayList<>(); + List nodeShardStates = new ArrayList<>(); shardsState.getData().forEach((node, nodeGatewayStartedShard) -> { nodeShardStates.add( - new NodeGatewayShardStarted( + new NodeGatewayStartedShard( nodeGatewayStartedShard.getGatewayShardStarted().allocationId(), nodeGatewayStartedShard.getGatewayShardStarted().primary(), nodeGatewayStartedShard.getGatewayShardStarted().replicationCheckpoint(), @@ -156,7 +156,7 @@ private static List adaptToNodeStartedShardList(FetchRe protected AllocateUnassignedDecision getAllocationDecision( ShardRouting unassignedShard, RoutingAllocation allocation, - List shardState, + List shardState, Logger logger ) { final boolean explain = allocation.debugDecision(); @@ -247,7 +247,7 @@ protected AllocateUnassignedDecision getAllocationDecision( nodesToAllocate = buildNodesToAllocate(allocation, nodeShardsResult.orderedAllocationCandidates, unassignedShard, true); if (nodesToAllocate.yesNodeShards.isEmpty() == false) { final DecidedNode decidedNode = nodesToAllocate.yesNodeShards.get(0); - final NodeGatewayShardStarted nodeShardState = decidedNode.nodeShardState; + final NodeGatewayStartedShard nodeShardState = decidedNode.nodeShardState; logger.debug( "[{}][{}]: allocating [{}] to [{}] on forced primary allocation", unassignedShard.index(), @@ -307,11 +307,11 @@ protected AllocateUnassignedDecision getAllocationDecision( */ private static List buildNodeDecisions( NodesToAllocate nodesToAllocate, - List fetchedShardData, + List fetchedShardData, Set inSyncAllocationIds ) { List nodeResults = new ArrayList<>(); - Collection ineligibleShards = new ArrayList<>(); + Collection ineligibleShards = new ArrayList<>(); if (nodesToAllocate != null) { final Set discoNodes = new HashSet<>(); nodeResults.addAll( @@ -345,21 +345,21 @@ private static List buildNodeDecisions( return nodeResults; } - private static ShardStoreInfo shardStoreInfo(NodeGatewayShardStarted nodeShardState, Set inSyncAllocationIds) { + private static ShardStoreInfo shardStoreInfo(NodeGatewayStartedShard nodeShardState, Set inSyncAllocationIds) { final Exception storeErr = nodeShardState.storeException(); final boolean inSync = nodeShardState.allocationId() != null && inSyncAllocationIds.contains(nodeShardState.allocationId()); return new ShardStoreInfo(nodeShardState.allocationId(), inSync, storeErr); } - private static final Comparator NO_STORE_EXCEPTION_FIRST_COMPARATOR = Comparator.comparing( - (NodeGatewayShardStarted state) -> state.storeException() == null + private static final Comparator NO_STORE_EXCEPTION_FIRST_COMPARATOR = Comparator.comparing( + (NodeGatewayStartedShard state) -> state.storeException() == null ).reversed(); - private static final Comparator PRIMARY_FIRST_COMPARATOR = Comparator.comparing( - NodeGatewayShardStarted::primary + private static final Comparator PRIMARY_FIRST_COMPARATOR = Comparator.comparing( + NodeGatewayStartedShard::primary ).reversed(); - private static final Comparator HIGHEST_REPLICATION_CHECKPOINT_FIRST_COMPARATOR = Comparator.comparing( - NodeGatewayShardStarted::replicationCheckpoint, + private static final Comparator HIGHEST_REPLICATION_CHECKPOINT_FIRST_COMPARATOR = Comparator.comparing( + NodeGatewayStartedShard::replicationCheckpoint, Comparator.nullsLast(Comparator.naturalOrder()) ); @@ -373,12 +373,12 @@ protected static NodeShardsResult buildNodeShardsResult( boolean matchAnyShard, Set ignoreNodes, Set inSyncAllocationIds, - List shardState, + List shardState, Logger logger ) { - List nodeShardStates = new ArrayList<>(); + List nodeShardStates = new ArrayList<>(); int numberOfAllocationsFound = 0; - for (NodeGatewayShardStarted nodeShardState : shardState) { + for (NodeGatewayStartedShard nodeShardState : shardState) { DiscoveryNode node = nodeShardState.getNode(); String allocationId = nodeShardState.allocationId(); @@ -443,18 +443,18 @@ protected static NodeShardsResult buildNodeShardsResult( return new NodeShardsResult(nodeShardStates, numberOfAllocationsFound); } - private static Comparator createActiveShardComparator(boolean matchAnyShard, Set inSyncAllocationIds) { + private static Comparator createActiveShardComparator(boolean matchAnyShard, Set inSyncAllocationIds) { /** * Orders the active shards copies based on below comparators * 1. No store exception i.e. shard copy is readable * 2. Prefer previous primary shard * 3. Prefer shard copy with the highest replication checkpoint. It is NO-OP for doc rep enabled indices. */ - final Comparator comparator; // allocation preference + final Comparator comparator; // allocation preference if (matchAnyShard) { // prefer shards with matching allocation ids - Comparator matchingAllocationsFirst = Comparator.comparing( - (NodeGatewayShardStarted state) -> inSyncAllocationIds.contains(state.allocationId()) + Comparator matchingAllocationsFirst = Comparator.comparing( + (NodeGatewayStartedShard state) -> inSyncAllocationIds.contains(state.allocationId()) ).reversed(); comparator = matchingAllocationsFirst.thenComparing(NO_STORE_EXCEPTION_FIRST_COMPARATOR) .thenComparing(PRIMARY_FIRST_COMPARATOR) @@ -472,14 +472,14 @@ private static Comparator createActiveShardComparator(b */ private static NodesToAllocate buildNodesToAllocate( RoutingAllocation allocation, - List nodeShardStates, + List nodeShardStates, ShardRouting shardRouting, boolean forceAllocate ) { List yesNodeShards = new ArrayList<>(); List throttledNodeShards = new ArrayList<>(); List noNodeShards = new ArrayList<>(); - for (NodeGatewayShardStarted nodeShardState : nodeShardStates) { + for (NodeGatewayStartedShard nodeShardState : nodeShardStates) { RoutingNode node = allocation.routingNodes().node(nodeShardState.getNode().getId()); if (node == null) { continue; @@ -510,10 +510,10 @@ private static NodesToAllocate buildNodesToAllocate( * This class encapsulates the result of a call to {@link #buildNodeShardsResult} */ static class NodeShardsResult { - final List orderedAllocationCandidates; + final List orderedAllocationCandidates; final int allocationsFound; - NodeShardsResult(List orderedAllocationCandidates, int allocationsFound) { + NodeShardsResult(List orderedAllocationCandidates, int allocationsFound) { this.orderedAllocationCandidates = orderedAllocationCandidates; this.allocationsFound = allocationsFound; } @@ -539,10 +539,10 @@ protected static class NodesToAllocate { * by the allocator for allocating to the node that holds the shard copy. */ private static class DecidedNode { - final NodeGatewayShardStarted nodeShardState; + final NodeGatewayStartedShard nodeShardState; final Decision decision; - private DecidedNode(NodeGatewayShardStarted nodeShardState, Decision decision) { + private DecidedNode(NodeGatewayStartedShard nodeShardState, Decision decision) { this.nodeShardState = nodeShardState; this.decision = decision; } diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java index 08ce4ab9b3f39..8d222903b6f29 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java @@ -15,7 +15,7 @@ import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.gateway.AsyncShardFetch.FetchResult; -import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.NodeGatewayShardStarted; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.NodeGatewayStartedShard; import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch; import java.util.ArrayList; @@ -99,7 +99,7 @@ public HashMap makeAllocationDecision( // process the received data for (ShardRouting unassignedShard : eligibleShards) { - List nodeShardStates = adaptToNodeShardStates(unassignedShard, shardsState); + List nodeShardStates = adaptToNodeShardStates(unassignedShard, shardsState); // get allocation decision for this shard shardAllocationDecisions.put(unassignedShard, getAllocationDecision(unassignedShard, allocation, nodeShardStates, logger)); } @@ -120,23 +120,23 @@ public HashMap makeAllocationDecision( * @param shardsState fetch data result for the whole batch * @return shard state returned from each node */ - private static List adaptToNodeShardStates( + private static List adaptToNodeShardStates( ShardRouting unassignedShard, FetchResult shardsState ) { if (!shardsState.hasData()) { return null; } - List nodeShardStates = new ArrayList<>(); + List nodeShardStates = new ArrayList<>(); Map nodeResponses = shardsState.getData(); // build data for a shard from all the nodes nodeResponses.forEach((node, nodeGatewayStartedShardsBatch) -> { - TransportNodesGatewayStartedShardHelper.GatewayShardStarted shardData = nodeGatewayStartedShardsBatch + TransportNodesGatewayStartedShardHelper.GatewayStartedShard shardData = nodeGatewayStartedShardsBatch .getNodeGatewayStartedShardsBatch() .get(unassignedShard.shardId()); nodeShardStates.add( - new NodeGatewayShardStarted( + new NodeGatewayStartedShard( shardData.allocationId(), shardData.primary(), shardData.replicationCheckpoint(), diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java index d08291de2eac5..27cce76b1b694 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java @@ -42,7 +42,7 @@ * @opensearch.internal */ public class TransportNodesGatewayStartedShardHelper { - public static GatewayShardStarted getShardInfoOnLocalNode( + public static GatewayStartedShard getShardInfoOnLocalNode( Logger logger, final ShardId shardId, NamedXContentRegistry namedXContentRegistry, @@ -95,21 +95,21 @@ public static GatewayShardStarted getShardInfoOnLocalNode( exception ); String allocationId = shardStateMetadata.allocationId != null ? shardStateMetadata.allocationId.getId() : null; - return new GatewayShardStarted(allocationId, shardStateMetadata.primary, null, exception); + return new GatewayStartedShard(allocationId, shardStateMetadata.primary, null, exception); } } logger.debug("{} shard state info found: [{}]", shardId, shardStateMetadata); String allocationId = shardStateMetadata.allocationId != null ? shardStateMetadata.allocationId.getId() : null; final IndexShard shard = indicesService.getShardOrNull(shardId); - return new GatewayShardStarted( + return new GatewayStartedShard( allocationId, shardStateMetadata.primary, shard != null ? shard.getLatestReplicationCheckpoint() : null ); } logger.trace("{} no local shard info found", shardId); - return new GatewayShardStarted(null, false, null); + return new GatewayStartedShard(null, false, null); } /** @@ -121,13 +121,13 @@ public static GatewayShardStarted getShardInfoOnLocalNode( * * @opensearch.internal */ - public static class GatewayShardStarted { + public static class GatewayStartedShard { private final String allocationId; private final boolean primary; private final Exception storeException; private final ReplicationCheckpoint replicationCheckpoint; - public GatewayShardStarted(StreamInput in) throws IOException { + public GatewayStartedShard(StreamInput in) throws IOException { allocationId = in.readOptionalString(); primary = in.readBoolean(); if (in.readBoolean()) { @@ -142,11 +142,11 @@ public GatewayShardStarted(StreamInput in) throws IOException { } } - public GatewayShardStarted(String allocationId, boolean primary, ReplicationCheckpoint replicationCheckpoint) { + public GatewayStartedShard(String allocationId, boolean primary, ReplicationCheckpoint replicationCheckpoint) { this(allocationId, primary, replicationCheckpoint, null); } - public GatewayShardStarted( + public GatewayStartedShard( String allocationId, boolean primary, ReplicationCheckpoint replicationCheckpoint, @@ -200,7 +200,7 @@ public boolean equals(Object o) { return false; } - GatewayShardStarted that = (GatewayShardStarted) o; + GatewayStartedShard that = (GatewayStartedShard) o; return primary == that.primary && Objects.equals(allocationId, that.allocationId) @@ -233,18 +233,18 @@ public String toString() { } /** - * This class extends the {@link GatewayShardStarted} which contains all necessary shard metadata like + * This class extends the {@link GatewayStartedShard} which contains all necessary shard metadata like * allocationId and replication checkpoint. It also has DiscoveryNode which is needed by * {@link PrimaryShardAllocator} and {@link PrimaryShardBatchAllocator} to make allocation decision. * This class removes the dependency of * {@link TransportNodesListGatewayStartedShards.NodeGatewayStartedShards} to make allocation decisions by * {@link PrimaryShardAllocator} or {@link PrimaryShardBatchAllocator}. */ - public static class NodeGatewayShardStarted extends GatewayShardStarted { + public static class NodeGatewayStartedShard extends GatewayStartedShard { private final DiscoveryNode node; - public NodeGatewayShardStarted( + public NodeGatewayStartedShard( String allocationId, boolean primary, ReplicationCheckpoint replicationCheckpoint, diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java index f18ae26c0c8c2..4b1f611bb88ab 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java @@ -53,7 +53,7 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.NodeEnvironment; -import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayShardStarted; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayStartedShard; import org.opensearch.indices.IndicesService; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.store.ShardAttributes; @@ -155,7 +155,7 @@ protected NodesGatewayStartedShards newResponse( @Override protected NodeGatewayStartedShards nodeOperation(NodeRequest request) { try { - GatewayShardStarted shardInfo = getShardInfoOnLocalNode( + GatewayStartedShard shardInfo = getShardInfoOnLocalNode( logger, request.getShardId(), namedXContentRegistry, @@ -167,7 +167,7 @@ protected NodeGatewayStartedShards nodeOperation(NodeRequest request) { ); return new NodeGatewayStartedShards( clusterService.localNode(), - new GatewayShardStarted( + new GatewayStartedShard( shardInfo.allocationId(), shardInfo.primary(), shardInfo.replicationCheckpoint(), @@ -305,7 +305,7 @@ public String getCustomDataPath() { * @opensearch.internal */ public static class NodeGatewayStartedShards extends BaseNodeResponse { - private final GatewayShardStarted gatewayShardStarted; + private final GatewayStartedShard gatewayStartedShard; public NodeGatewayStartedShards(StreamInput in) throws IOException { super(in); @@ -323,33 +323,33 @@ public NodeGatewayStartedShards(StreamInput in) throws IOException { } else { replicationCheckpoint = null; } - this.gatewayShardStarted = new GatewayShardStarted(allocationId, primary, replicationCheckpoint, storeException); + this.gatewayStartedShard = new GatewayStartedShard(allocationId, primary, replicationCheckpoint, storeException); } - public GatewayShardStarted getGatewayShardStarted() { - return gatewayShardStarted; + public GatewayStartedShard getGatewayShardStarted() { + return gatewayStartedShard; } - public NodeGatewayStartedShards(DiscoveryNode node, GatewayShardStarted gatewayShardStarted) { + public NodeGatewayStartedShards(DiscoveryNode node, GatewayStartedShard gatewayStartedShard) { super(node); - this.gatewayShardStarted = gatewayShardStarted; + this.gatewayStartedShard = gatewayStartedShard; } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeOptionalString(gatewayShardStarted.allocationId()); - out.writeBoolean(gatewayShardStarted.primary()); - if (gatewayShardStarted.storeException() != null) { + out.writeOptionalString(gatewayStartedShard.allocationId()); + out.writeBoolean(gatewayStartedShard.primary()); + if (gatewayStartedShard.storeException() != null) { out.writeBoolean(true); - out.writeException(gatewayShardStarted.storeException()); + out.writeException(gatewayStartedShard.storeException()); } else { out.writeBoolean(false); } if (out.getVersion().onOrAfter(Version.V_2_3_0)) { - if (gatewayShardStarted.replicationCheckpoint() != null) { + if (gatewayStartedShard.replicationCheckpoint() != null) { out.writeBoolean(true); - gatewayShardStarted.replicationCheckpoint().writeTo(out); + gatewayStartedShard.replicationCheckpoint().writeTo(out); } else { out.writeBoolean(false); } @@ -367,17 +367,17 @@ public boolean equals(Object o) { NodeGatewayStartedShards that = (NodeGatewayStartedShards) o; - return gatewayShardStarted.equals(that.gatewayShardStarted); + return gatewayStartedShard.equals(that.gatewayStartedShard); } @Override public int hashCode() { - return gatewayShardStarted.hashCode(); + return gatewayStartedShard.hashCode(); } @Override public String toString() { - return gatewayShardStarted.toString(); + return gatewayStartedShard.toString(); } } } diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java index d8e83b955d46d..dc5d85b17bc32 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java @@ -27,7 +27,7 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.NodeEnvironment; -import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayShardStarted; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayStartedShard; import org.opensearch.indices.IndicesService; import org.opensearch.indices.store.ShardAttributes; import org.opensearch.threadpool.ThreadPool; @@ -135,7 +135,7 @@ protected NodesGatewayStartedShardsBatch newResponse( */ @Override protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { - Map shardsOnNode = new HashMap<>(); + Map shardsOnNode = new HashMap<>(); for (ShardAttributes shardAttr : request.shardAttributes.values()) { final ShardId shardId = shardAttr.getShardId(); try { @@ -155,7 +155,7 @@ protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { } catch (Exception e) { shardsOnNode.put( shardId, - new GatewayShardStarted(null, false, null, new OpenSearchException("failed to load started shards", e)) + new GatewayStartedShard(null, false, null, new OpenSearchException("failed to load started shards", e)) ); } } @@ -256,15 +256,15 @@ public void writeTo(StreamOutput out) throws IOException { * @opensearch.internal */ public static class NodeGatewayStartedShardsBatch extends BaseNodeResponse { - private final Map nodeGatewayStartedShardsBatch; + private final Map nodeGatewayStartedShardsBatch; - public Map getNodeGatewayStartedShardsBatch() { + public Map getNodeGatewayStartedShardsBatch() { return nodeGatewayStartedShardsBatch; } public NodeGatewayStartedShardsBatch(StreamInput in) throws IOException { super(in); - this.nodeGatewayStartedShardsBatch = in.readMap(ShardId::new, GatewayShardStarted::new); + this.nodeGatewayStartedShardsBatch = in.readMap(ShardId::new, GatewayStartedShard::new); } @Override @@ -273,7 +273,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeMap(nodeGatewayStartedShardsBatch, (o, k) -> k.writeTo(o), (o, v) -> v.writeTo(o)); } - public NodeGatewayStartedShardsBatch(DiscoveryNode node, Map nodeGatewayStartedShardsBatch) { + public NodeGatewayStartedShardsBatch(DiscoveryNode node, Map nodeGatewayStartedShardsBatch) { super(node); this.nodeGatewayStartedShardsBatch = nodeGatewayStartedShardsBatch; } diff --git a/server/src/test/java/org/opensearch/gateway/PrimaryShardAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/PrimaryShardAllocatorTests.java index cf3eed82fc940..e849f12143b4d 100644 --- a/server/src/test/java/org/opensearch/gateway/PrimaryShardAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/PrimaryShardAllocatorTests.java @@ -843,7 +843,7 @@ public TestAllocator addData( node, new TransportNodesListGatewayStartedShards.NodeGatewayStartedShards( node, - new TransportNodesGatewayStartedShardHelper.GatewayShardStarted( + new TransportNodesGatewayStartedShardHelper.GatewayStartedShard( allocationId, primary, replicationCheckpoint, diff --git a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java index 0175961f6a274..4796def2b8902 100644 --- a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java @@ -289,9 +289,9 @@ public TestBatchAllocator addData( if (data == null) { data = new HashMap<>(); } - Map shardData = Map.of( + Map shardData = Map.of( shardId, - new TransportNodesGatewayStartedShardHelper.GatewayShardStarted( + new TransportNodesGatewayStartedShardHelper.GatewayStartedShard( allocationId, primary, replicationCheckpoint, @@ -313,10 +313,10 @@ public TestBatchAllocator addShardData( if (data == null) { data = new HashMap<>(); } - Map shardData = new HashMap<>(); + Map shardData = new HashMap<>(); shardData.put( shardId, - new TransportNodesGatewayStartedShardHelper.GatewayShardStarted( + new TransportNodesGatewayStartedShardHelper.GatewayStartedShard( allocationId, primary, replicationCheckpoint, diff --git a/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java b/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java index c3897e66479be..b1695ff00e0cc 100644 --- a/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java +++ b/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java @@ -92,7 +92,7 @@ protected AsyncShardFetch.FetchResult fetchData(ShardR routing -> currentNodes.get(routing.currentNodeId()), routing -> new NodeGatewayStartedShards( currentNodes.get(routing.currentNodeId()), - new TransportNodesGatewayStartedShardHelper.GatewayShardStarted( + new TransportNodesGatewayStartedShardHelper.GatewayStartedShard( routing.allocationId().getId(), routing.primary(), getReplicationCheckpoint(shardId, routing.currentNodeId()), From 4c75c493c796d7b3fbf9d44a6c1dbf8eff401e5c Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Mon, 18 Mar 2024 19:47:23 +0530 Subject: [PATCH 21/29] Move BaseShardResponse methods to respective transport classes Signed-off-by: Aman Khare --- .../gateway/RecoveryFromGatewayIT.java | 1 - .../gateway/AsyncShardBatchFetch.java | 74 +++++++++++-------- .../opensearch/gateway/BaseShardResponse.java | 52 ------------- ...ansportNodesGatewayStartedShardHelper.java | 7 ++ ...sportNodesListShardStoreMetadataBatch.java | 5 ++ .../gateway/ShardBatchCacheTests.java | 30 ++++---- 6 files changed, 72 insertions(+), 97 deletions(-) delete mode 100644 server/src/main/java/org/opensearch/gateway/BaseShardResponse.java diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 9a1cec7bfa596..ba03532a9aa2f 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -948,7 +948,6 @@ private void assertNodeStoreFilesMetadataSuccessCase( assertNotNull(storeFileMetadata.peerRecoveryRetentionLeases()); } - private void assertNodeGatewayStartedShardsHappyCase(GatewayStartedShard gatewayStartedShard) { assertNull(gatewayStartedShard.storeException()); assertNotNull(gatewayStartedShard.allocationId()); diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java index e8484d9040ba0..9b961b71653c2 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java @@ -29,8 +29,6 @@ import java.util.function.Function; import java.util.function.Supplier; -import reactor.util.annotation.NonNull; - /** * Implementation of AsyncShardFetch with batching support. This class is responsible for executing the fetch * part using the base class {@link AsyncShardFetch}. Other functionalities needed for a batch are only written here. @@ -43,7 +41,7 @@ * * @opensearch.internal */ -public abstract class AsyncShardBatchFetch extends AsyncShardFetch { +public abstract class AsyncShardBatchFetch extends AsyncShardFetch { private final Consumer removeShardFromBatch; private final List failedShards; @@ -59,10 +57,12 @@ public abstract class AsyncShardBatchFetch, T> responseGetter, Function> shardsBatchDataGetter, Supplier emptyResponseBuilder, - Consumer handleFailedShard + Consumer failedShardHandler, + Function getResponseException, + Function isEmptyResponse ) { super(logger, type, shardAttributesMap, action, batchId); - this.removeShardFromBatch = handleFailedShard; + this.removeShardFromBatch = failedShardHandler; this.failedShards = new ArrayList<>(); this.cache = new ShardBatchCache<>( logger, @@ -73,7 +73,9 @@ public abstract class AsyncShardBatchFetch Response type of transport action. * @param Data type of shard level response. */ - public static class ShardBatchCache extends AsyncShardFetchCache { + static class ShardBatchCache extends AsyncShardFetchCache { private final Map> cache; private final Map shardIdToArray; private final int batchSize; @@ -131,7 +133,9 @@ public static class ShardBatchCache arrayToShardId; private final Function> shardsBatchDataGetter; private final Supplier emptyResponseBuilder; - private final Consumer handleFailedShard; + private final Consumer failedShardHandler; + private final Function getException; + private final Function isEmpty; private final Logger logger; public ShardBatchCache( @@ -143,10 +147,14 @@ public ShardBatchCache( BiFunction, T> responseGetter, Function> shardsBatchDataGetter, Supplier emptyResponseBuilder, - Consumer handleFailedShard + Consumer failedShardHandler, + Function getResponseException, + Function isEmptyResponse ) { super(Loggers.getLogger(logger, "_" + logKey), type); this.batchSize = shardAttributesMap.size(); + this.getException = getResponseException; + this.isEmpty = isEmptyResponse; cache = new HashMap<>(); shardIdToArray = new HashMap<>(); arrayToShardId = new HashMap<>(); @@ -155,11 +163,10 @@ public ShardBatchCache( this.responseConstructor = responseGetter; this.shardsBatchDataGetter = shardsBatchDataGetter; this.emptyResponseBuilder = emptyResponseBuilder; - this.handleFailedShard = handleFailedShard; + this.failedShardHandler = failedShardHandler; this.logger = logger; } - @NonNull @Override public Map getCache() { return cache; @@ -177,7 +184,7 @@ public void deleteShard(ShardId shardId) { @Override public Map getCacheData(DiscoveryNodes nodes, Set failedNodes) { - refreshReverseIdMap(); + fillReverseIdMap(); return super.getCacheData(nodes, failedNodes); } @@ -185,7 +192,7 @@ public Map getCacheData(DiscoveryNodes nodes, Set fail * Build a reverse map to get shardId from the array index, this will be used to construct the response which * PrimaryShardBatchAllocator or ReplicaShardBatchAllocator are looking for. */ - private void refreshReverseIdMap() { + private void fillReverseIdMap() { arrayToShardId.clear(); for (ShardId shardId : shardIdToArray.keySet()) { arrayToShardId.putIfAbsent(shardIdToArray.get(shardId), shardId); @@ -194,7 +201,7 @@ private void refreshReverseIdMap() { @Override public void initData(DiscoveryNode node) { - cache.put(node.getId(), new NodeEntry<>(node.getId(), shardResponseClass, batchSize)); + cache.put(node.getId(), new NodeEntry<>(node.getId(), shardResponseClass, batchSize, getException, isEmpty)); } /** @@ -223,10 +230,10 @@ private void filterFailedShards(Map batchResponse) { for (Iterator it = batchResponse.keySet().iterator(); it.hasNext();) { ShardId shardId = it.next(); if (batchResponse.get(shardId) != null) { - if (batchResponse.get(shardId).getException() != null) { + if (getException.apply(batchResponse.get(shardId)) != null) { // handle per shard level exceptions, process other shards, only throw out this shard from // the batch - Exception shardException = batchResponse.get(shardId).getException(); + Exception shardException = getException.apply(batchResponse.get(shardId)); // if the request got rejected or timed out, we need to try it again next time... if (retryableException(shardException)) { logger.trace( @@ -234,7 +241,7 @@ private void filterFailedShards(Map batchResponse) { shardId.toString(), shardException.toString() ); - handleFailedShard.accept(shardId); + failedShardHandler.accept(shardId); // remove this failed entry. So, while storing the data, we don't need to re-process it. it.remove(); } @@ -268,28 +275,34 @@ private void fillShardIdKeys(Set shardIds) { for (ShardId shardId : shardIds) { this.shardIdToArray.putIfAbsent(shardId, shardIdIndex++); } - this.shardIdToArray.keySet().removeIf(shardId -> { - if (!shardIds.contains(shardId)) { - deleteShard(shardId); - return true; - } else { - return false; - } - }); } /** * A node entry, holding the state of the fetched data for a specific shard * for a giving node. */ - static class NodeEntry extends BaseNodeEntry { + static class NodeEntry extends BaseNodeEntry { private final V[] shardData; - private final boolean[] emptyShardResponse; + private final boolean[] emptyShardResponse; // we can not rely on null entries of the shardData array, + // those null entries means that we need to ignore those entries. Empty responses on the other hand are + // actually needed in allocation/explain API response. So instead of storing full empty response object + // in cache, it's better to just store a boolean and create that object on the fly just before + // decision-making. + private final Function getException; + private final Function isEmpty; - NodeEntry(String nodeId, Class clazz, int batchSize) { + NodeEntry( + String nodeId, + Class clazz, + int batchSize, + Function getResponseException, + Function isEmptyResponse + ) { super(nodeId); this.shardData = (V[]) Array.newInstance(clazz, batchSize); this.emptyShardResponse = new boolean[batchSize]; + this.getException = getResponseException; + this.isEmpty = isEmptyResponse; } void doneFetching(Map shardDataFromNode, Map shardIdKey) { @@ -299,6 +312,7 @@ void doneFetching(Map shardDataFromNode, Map shard void clearShard(Integer shardIdIndex) { this.shardData[shardIdIndex] = null; + emptyShardResponse[shardIdIndex] = false; } V[] getData() { @@ -312,10 +326,10 @@ boolean[] getEmptyShardResponse() { private void fillShardData(Map shardDataFromNode, Map shardIdKey) { for (ShardId shardId : shardDataFromNode.keySet()) { if (shardDataFromNode.get(shardId) != null) { - if (shardDataFromNode.get(shardId).isEmpty()) { + if (isEmpty.apply(shardDataFromNode.get(shardId))) { this.emptyShardResponse[shardIdKey.get(shardId)] = true; this.shardData[shardIdKey.get(shardId)] = null; - } else if (shardDataFromNode.get(shardId).getException() == null) { + } else if (getException.apply(shardDataFromNode.get(shardId)) == null) { this.shardData[shardIdKey.get(shardId)] = shardDataFromNode.get(shardId); } // if exception is not null, we got unhandled failure for the shard which needs to be ignored diff --git a/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java b/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java deleted file mode 100644 index 876d9632b5ed8..0000000000000 --- a/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -package org.opensearch.gateway; - -import org.opensearch.core.common.io.stream.StreamInput; -import org.opensearch.core.common.io.stream.StreamOutput; - -import java.io.IOException; - -/** - * Base response class for shard response. Provides necessary information about shard level response. Based on these - * functionalities, receiver decides if it needs to store the response or ignore it or retry the fetch. - * - * @opensearch.internal - */ -public abstract class BaseShardResponse { - - private Exception storeException; - - public BaseShardResponse(Exception storeException) { - this.storeException = storeException; - } - - public abstract boolean isEmpty(); - - public Exception getException() { - return storeException; - } - - public BaseShardResponse(StreamInput in) throws IOException { - if (in.readBoolean()) { - storeException = in.readException(); - } else { - storeException = null; - } - } - - public void writeTo(StreamOutput out) throws IOException { - if (storeException != null) { - out.writeBoolean(true); - out.writeException(storeException); - } else { - out.writeBoolean(false); - } - } -} diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java index 345221aa3976b..1f16f784515d5 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java @@ -233,6 +233,13 @@ public String toString() { buf.append("]"); return buf.toString(); } + + public static Boolean isEmpty(GatewayStartedShard response) { + return response.allocationId() == null + && response.primary() == false + && response.storeException() == null + && response.replicationCheckpoint() == null; + } } /** diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java index 3f151fe1c5ca0..3d37ab026835a 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java @@ -276,6 +276,11 @@ public void writeTo(StreamOutput out) throws IOException { } } + boolean isEmpty(NodeStoreFilesMetadata response) { + return response.storeFilesMetadata() == null + || response.storeFilesMetadata().isEmpty() && response.getStoreFileFetchException() == null; + } + public Exception getStoreFileFetchException() { return storeFileFetchException; } diff --git a/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java b/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java index 6922bafd5ef68..647b85a9a1a34 100644 --- a/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java +++ b/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java @@ -16,7 +16,7 @@ import org.opensearch.cluster.routing.TestShardRouting; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.core.index.shard.ShardId; -import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayStartedShard; import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch; import org.opensearch.indices.store.ShardAttributes; @@ -31,7 +31,7 @@ public class ShardBatchCacheTests extends OpenSearchAllocationTestCase { private final DiscoveryNode node2 = newNode("node2"); // Needs to be enabled once ShardsBatchGatewayAllocator is pushed // private final Map batchInfo = new HashMap<>(); - private AsyncShardBatchFetch.ShardBatchCache shardCache; + private AsyncShardBatchFetch.ShardBatchCache shardCache; private List shardsInBatch = new ArrayList<>(); private static final int NUMBER_OF_SHARDS_DEFAULT = 10; @@ -50,11 +50,13 @@ public void setupShardBatchCache(String batchId, int numberOfShards) { "batch_shards_started", shardAttributesMap, "BatchID=[" + batchId + "]", - NodeGatewayStartedShard.class, + GatewayStartedShard.class, NodeGatewayStartedShardsBatch::new, NodeGatewayStartedShardsBatch::getNodeGatewayStartedShardsBatch, - () -> new NodeGatewayStartedShard(null, false, null, null), - this::removeShard + () -> new GatewayStartedShard(null, false, null, null), + this::removeShard, + GatewayStartedShard::storeException, + GatewayStartedShard::isEmpty ); } @@ -129,7 +131,7 @@ public void testPutData() { assertEquals("alloc-1", fetchData.get(node1).getNodeGatewayStartedShardsBatch().get(shard).allocationId()); assertEquals(10, fetchData.get(node2).getNodeGatewayStartedShardsBatch().size()); - assertTrue(fetchData.get(node2).getNodeGatewayStartedShardsBatch().get(shard).isEmpty()); + assertTrue(GatewayStartedShard.isEmpty(fetchData.get(node2).getNodeGatewayStartedShardsBatch().get(shard))); // test GetData after fetch assertEquals(10, shardCache.getData(node1).getNodeGatewayStartedShardsBatch().size()); @@ -165,19 +167,19 @@ public void testFilterFailedShards() { assertTrue(fetchData.get(node2).getNodeGatewayStartedShardsBatch().isEmpty()); } - private Map getPrimaryResponse(List shards, ResponseType responseType) { + private Map getPrimaryResponse(List shards, ResponseType responseType) { int allocationId = 1; - Map shardData = new HashMap<>(); + Map shardData = new HashMap<>(); for (ShardId shard : shards) { switch (responseType) { case NULL: shardData.put(shard, null); break; case EMPTY: - shardData.put(shard, new NodeGatewayStartedShard(null, false, null, null)); + shardData.put(shard, new GatewayStartedShard(null, false, null, null)); break; case VALID: - shardData.put(shard, new NodeGatewayStartedShard("alloc-" + allocationId++, false, null, null)); + shardData.put(shard, new GatewayStartedShard("alloc-" + allocationId++, false, null, null)); break; default: throw new AssertionError("unknown response type"); @@ -186,17 +188,17 @@ private Map getPrimaryResponse(List s return shardData; } - private Map getFailedPrimaryResponse(List shards, int failedShardsCount) { + private Map getFailedPrimaryResponse(List shards, int failedShardsCount) { int allocationId = 1; - Map shardData = new HashMap<>(); + Map shardData = new HashMap<>(); for (ShardId shard : shards) { if (failedShardsCount-- > 0) { shardData.put( shard, - new NodeGatewayStartedShard("alloc-" + allocationId++, false, null, new OpenSearchRejectedExecutionException()) + new GatewayStartedShard("alloc-" + allocationId++, false, null, new OpenSearchRejectedExecutionException()) ); } else { - shardData.put(shard, new NodeGatewayStartedShard("alloc-" + allocationId++, false, null, null)); + shardData.put(shard, new GatewayStartedShard("alloc-" + allocationId++, false, null, null)); } } return shardData; From a893cb84ebd33265917a3a706c63d34e3e5ea4b2 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Tue, 19 Mar 2024 18:20:09 +0530 Subject: [PATCH 22/29] Remove PSBA and test files, modify exception handling Signed-off-by: Aman Khare --- .../gateway/RecoveryFromGatewayIT.java | 14 +- .../gateway/AsyncShardBatchFetch.java | 4 +- .../gateway/PrimaryShardBatchAllocator.java | 150 -------- ...ansportNodesGatewayStartedShardHelper.java | 4 + ...ortNodesListGatewayStartedShardsBatch.java | 71 +++- .../PrimaryShardBatchAllocatorTests.java | 340 ------------------ .../gateway/ShardBatchCacheTests.java | 37 +- 7 files changed, 102 insertions(+), 518 deletions(-) delete mode 100644 server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java delete mode 100644 server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index ba03532a9aa2f..26714f9a65329 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -56,7 +56,7 @@ import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; import org.opensearch.env.NodeEnvironment; -import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayStartedShard; +import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.GatewayStartedShard; import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; import org.opensearch.index.MergePolicyProvider; @@ -818,9 +818,9 @@ public void testShardFetchCorruptedShardsUsingBatchAction() throws Exception { .get(discoveryNodes[0].getId()) .getNodeGatewayStartedShardsBatch() .get(shardId); - assertNotNull(gatewayStartedShard.storeException()); - assertNotNull(gatewayStartedShard.allocationId()); - assertTrue(gatewayStartedShard.primary()); + assertNotNull(gatewayStartedShard.get().storeException()); + assertNotNull(gatewayStartedShard.get().allocationId()); + assertTrue(gatewayStartedShard.get().primary()); } public void testSingleShardStoreFetchUsingBatchAction() throws ExecutionException, InterruptedException { @@ -949,9 +949,9 @@ private void assertNodeStoreFilesMetadataSuccessCase( } private void assertNodeGatewayStartedShardsHappyCase(GatewayStartedShard gatewayStartedShard) { - assertNull(gatewayStartedShard.storeException()); - assertNotNull(gatewayStartedShard.allocationId()); - assertTrue(gatewayStartedShard.primary()); + assertNull(gatewayStartedShard.get().storeException()); + assertNotNull(gatewayStartedShard.get().allocationId()); + assertTrue(gatewayStartedShard.get().primary()); } private void prepareIndex(String indexName, int numberOfPrimaryShards) { diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java index 9b961b71653c2..18133f9b4bd1a 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java @@ -194,8 +194,8 @@ public Map getCacheData(DiscoveryNodes nodes, Set fail */ private void fillReverseIdMap() { arrayToShardId.clear(); - for (ShardId shardId : shardIdToArray.keySet()) { - arrayToShardId.putIfAbsent(shardIdToArray.get(shardId), shardId); + for (Map.Entry indexMapping : shardIdToArray.entrySet()) { + arrayToShardId.putIfAbsent(indexMapping.getValue(), indexMapping.getKey()); } } diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java deleted file mode 100644 index 8d222903b6f29..0000000000000 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java +++ /dev/null @@ -1,150 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -package org.opensearch.gateway; - -import org.apache.logging.log4j.Logger; -import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.cluster.routing.RoutingNodes; -import org.opensearch.cluster.routing.ShardRouting; -import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; -import org.opensearch.cluster.routing.allocation.RoutingAllocation; -import org.opensearch.gateway.AsyncShardFetch.FetchResult; -import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.NodeGatewayStartedShard; -import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * PrimaryShardBatchAllocator is similar to {@link org.opensearch.gateway.PrimaryShardAllocator} only difference is - * that it can allocate multiple unassigned primary shards wherein PrimaryShardAllocator can only allocate single - * unassigned shard. - * The primary shard batch allocator allocates multiple unassigned primary shards to nodes that hold - * valid copies of the unassigned primaries. It does this by iterating over all unassigned - * primary shards in the routing table and fetching shard metadata from each node in the cluster - * that holds a copy of the shard. The shard metadata from each node is compared against the - * set of valid allocation IDs and for all valid shard copies (if any), the primary shard batch allocator - * executes the allocation deciders to chose a copy to assign the primary shard to. - *

- * Note that the PrimaryShardBatchAllocator does *not* allocate primaries on index creation - * (see {@link org.opensearch.cluster.routing.allocation.allocator.BalancedShardsAllocator}), - * nor does it allocate primaries when a primary shard failed and there is a valid replica - * copy that can immediately be promoted to primary, as this takes place in {@link RoutingNodes#failShard}. - * - * @opensearch.internal - */ -public abstract class PrimaryShardBatchAllocator extends PrimaryShardAllocator { - - abstract protected FetchResult fetchData( - List eligibleShards, - List inEligibleShards, - RoutingAllocation allocation - ); - - protected FetchResult fetchData( - ShardRouting shard, - RoutingAllocation allocation - ) { - logger.error("fetchData for single shard called via batch allocator, shard id {}", shard.shardId()); - throw new IllegalStateException("PrimaryShardBatchAllocator should only be used for a batch of shards"); - } - - @Override - public AllocateUnassignedDecision makeAllocationDecision(ShardRouting unassignedShard, RoutingAllocation allocation, Logger logger) { - return makeAllocationDecision(Collections.singletonList(unassignedShard), allocation, logger).get(unassignedShard); - } - - /** - * Build allocation decisions for all the shards present in the batch identified by batchId. - * - * @param shards set of shards given for allocation - * @param allocation current allocation of all the shards - * @param logger logger used for logging - * @return shard to allocation decision map - */ - @Override - public HashMap makeAllocationDecision( - List shards, - RoutingAllocation allocation, - Logger logger - ) { - HashMap shardAllocationDecisions = new HashMap<>(); - List eligibleShards = new ArrayList<>(); - List inEligibleShards = new ArrayList<>(); - // identify ineligible shards - for (ShardRouting shard : shards) { - AllocateUnassignedDecision decision = getInEligibleShardDecision(shard, allocation); - if (decision != null) { - inEligibleShards.add(shard); - shardAllocationDecisions.put(shard, decision); - } else { - eligibleShards.add(shard); - } - } - // Do not call fetchData if there are no eligible shards - if (eligibleShards.isEmpty()) { - return shardAllocationDecisions; - } - // only fetch data for eligible shards - final FetchResult shardsState = fetchData(eligibleShards, inEligibleShards, allocation); - - // process the received data - for (ShardRouting unassignedShard : eligibleShards) { - List nodeShardStates = adaptToNodeShardStates(unassignedShard, shardsState); - // get allocation decision for this shard - shardAllocationDecisions.put(unassignedShard, getAllocationDecision(unassignedShard, allocation, nodeShardStates, logger)); - } - return shardAllocationDecisions; - } - - /** - * Transforms {@link FetchResult} of {@link NodeGatewayStartedShardsBatch} to {@link List} of {@link TransportNodesListGatewayStartedShards.NodeGatewayStartedShards}. - *

- * Returns null if {@link FetchResult} does not have any data. - *

- * shardsState contain the Data, there key is DiscoveryNode but value is Map of ShardId - * and NodeGatewayStartedShardsBatch so to get one shard level data (from all the nodes), we'll traverse the map - * and construct the nodeShardState along the way before making any allocation decision. As metadata for a - * particular shard is needed from all the discovery nodes. - * - * @param unassignedShard unassigned shard - * @param shardsState fetch data result for the whole batch - * @return shard state returned from each node - */ - private static List adaptToNodeShardStates( - ShardRouting unassignedShard, - FetchResult shardsState - ) { - if (!shardsState.hasData()) { - return null; - } - List nodeShardStates = new ArrayList<>(); - Map nodeResponses = shardsState.getData(); - - // build data for a shard from all the nodes - nodeResponses.forEach((node, nodeGatewayStartedShardsBatch) -> { - TransportNodesGatewayStartedShardHelper.GatewayStartedShard shardData = nodeGatewayStartedShardsBatch - .getNodeGatewayStartedShardsBatch() - .get(unassignedShard.shardId()); - nodeShardStates.add( - new NodeGatewayStartedShard( - shardData.allocationId(), - shardData.primary(), - shardData.replicationCheckpoint(), - shardData.storeException(), - node - ) - ); - }); - return nodeShardStates; - } -} diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java index 1f16f784515d5..914119185ddd8 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java @@ -240,6 +240,10 @@ public static Boolean isEmpty(GatewayStartedShard response) { && response.storeException() == null && response.replicationCheckpoint() == null; } + + public Boolean isEmpty() { + return allocationId == null && primary == false && storeException == null && replicationCheckpoint == null; + } } /** diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java index 70e7df785155f..f187b97b82319 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java @@ -26,7 +26,6 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.NodeEnvironment; -import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayStartedShard; import org.opensearch.indices.IndicesService; import org.opensearch.indices.store.ShardAttributes; import org.opensearch.threadpool.ThreadPool; @@ -141,15 +140,18 @@ protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { try { shardsOnNode.put( shardId, - getShardInfoOnLocalNode( - logger, - shardId, - namedXContentRegistry, - nodeEnv, - indicesService, - shardAttr.getValue().getCustomDataPath(), - settings, - clusterService + new GatewayStartedShard( + getShardInfoOnLocalNode( + logger, + shardId, + namedXContentRegistry, + nodeEnv, + indicesService, + shardAttr.getValue().getCustomDataPath(), + settings, + clusterService + ), + null ) ); } catch (Exception e) { @@ -158,7 +160,10 @@ protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { shardsOnNode.put(shardId, null); } else { // return actual exception as it is for unknown exceptions - shardsOnNode.put(shardId, new GatewayStartedShard(null, false, null, e)); + shardsOnNode.put( + shardId, + new GatewayStartedShard(new TransportNodesGatewayStartedShardHelper.GatewayStartedShard(null, false, null, null), e) + ); } } } @@ -251,6 +256,50 @@ public void writeTo(StreamOutput out) throws IOException { } } + public static class GatewayStartedShard { + private final TransportNodesGatewayStartedShardHelper.GatewayStartedShard gatewayStartedShard; + private final Exception transportError; + + public GatewayStartedShard( + TransportNodesGatewayStartedShardHelper.GatewayStartedShard gatewayStartedShard, + Exception transportError + ) { + this.gatewayStartedShard = gatewayStartedShard; + this.transportError = transportError; + } + + public GatewayStartedShard(StreamInput in) throws IOException { + this.gatewayStartedShard = new TransportNodesGatewayStartedShardHelper.GatewayStartedShard(in); + if (in.readBoolean()) { + this.transportError = in.readException(); + } else { + this.transportError = null; + } + } + + public void writeTo(StreamOutput out) throws IOException { + gatewayStartedShard.writeTo(out); + if (transportError != null) { + out.writeBoolean(true); + out.writeException(transportError); + } else { + out.writeBoolean(false); + } + } + + public static boolean isEmpty(GatewayStartedShard gatewayStartedShard) { + return gatewayStartedShard.get().isEmpty() && gatewayStartedShard.getTransportError() == null; + } + + public Exception getTransportError() { + return transportError; + } + + public TransportNodesGatewayStartedShardHelper.GatewayStartedShard get() { + return gatewayStartedShard; + } + } + /** * This is the response from a single node, this is used in {@link NodesGatewayStartedShardsBatch} for creating * node to its response mapping for this transport request. diff --git a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java deleted file mode 100644 index 4796def2b8902..0000000000000 --- a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java +++ /dev/null @@ -1,340 +0,0 @@ -/* - * 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.lucene.codecs.Codec; -import org.opensearch.Version; -import org.opensearch.cluster.ClusterName; -import org.opensearch.cluster.ClusterState; -import org.opensearch.cluster.OpenSearchAllocationTestCase; -import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.cluster.metadata.Metadata; -import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.cluster.routing.RoutingNodes; -import org.opensearch.cluster.routing.RoutingTable; -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.AllocationDecision; -import org.opensearch.cluster.routing.allocation.RoutingAllocation; -import org.opensearch.cluster.routing.allocation.decider.AllocationDeciders; -import org.opensearch.common.Nullable; -import org.opensearch.common.settings.Settings; -import org.opensearch.common.util.set.Sets; -import org.opensearch.core.index.shard.ShardId; -import org.opensearch.env.Environment; -import org.opensearch.index.IndexSettings; -import org.opensearch.index.codec.CodecService; -import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; -import org.opensearch.test.IndexSettingsModule; -import org.junit.Before; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import static org.opensearch.cluster.routing.UnassignedInfo.Reason.CLUSTER_RECOVERED; - -public class PrimaryShardBatchAllocatorTests extends OpenSearchAllocationTestCase { - - private final ShardId shardId = new ShardId("test", "_na_", 0); - private static Set shardsInBatch; - private final DiscoveryNode node1 = newNode("node1"); - private final DiscoveryNode node2 = newNode("node2"); - private final DiscoveryNode node3 = newNode("node3"); - private TestBatchAllocator batchAllocator; - - public static void setUpShards(int numberOfShards) { - shardsInBatch = new HashSet<>(); - for (int shardNumber = 0; shardNumber < numberOfShards; shardNumber++) { - ShardId shardId = new ShardId("test", "_na_", shardNumber); - shardsInBatch.add(shardId); - } - } - - @Before - public void buildTestAllocator() { - this.batchAllocator = new TestBatchAllocator(); - } - - private void allocateAllUnassigned(final RoutingAllocation allocation) { - final RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); - while (iterator.hasNext()) { - batchAllocator.allocateUnassigned(iterator.next(), allocation, iterator); - } - } - - private void allocateAllUnassignedBatch(final RoutingAllocation allocation) { - final RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); - List shardsToBatch = new ArrayList<>(); - while (iterator.hasNext()) { - shardsToBatch.add(iterator.next()); - } - batchAllocator.allocateUnassignedBatch(shardsToBatch, allocation); - } - - public void testMakeAllocationDecisionDataFetching() { - final RoutingAllocation allocation = routingAllocationWithOnePrimary(noAllocationDeciders(), CLUSTER_RECOVERED, "allocId1"); - - List shards = new ArrayList<>(); - allocateAllUnassignedBatch(allocation); - ShardRouting shard = allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).primaryShard(); - shards.add(shard); - HashMap allDecisions = batchAllocator.makeAllocationDecision(shards, allocation, logger); - // verify we get decisions for all the shards - assertEquals(shards.size(), allDecisions.size()); - assertEquals(shards, new ArrayList<>(allDecisions.keySet())); - assertEquals(AllocationDecision.AWAITING_INFO, allDecisions.get(shard).getAllocationDecision()); - } - - public void testMakeAllocationDecisionForReplicaShard() { - final RoutingAllocation allocation = routingAllocationWithOnePrimary(noAllocationDeciders(), CLUSTER_RECOVERED, "allocId1"); - - List replicaShards = allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).replicaShards(); - List shards = new ArrayList<>(replicaShards); - HashMap allDecisions = batchAllocator.makeAllocationDecision(shards, allocation, logger); - // verify we get decisions for all the shards - assertEquals(shards.size(), allDecisions.size()); - assertEquals(shards, new ArrayList<>(allDecisions.keySet())); - assertFalse(allDecisions.get(replicaShards.get(0)).isDecisionTaken()); - } - - public void testMakeAllocationDecisionDataFetched() { - final RoutingAllocation allocation = routingAllocationWithOnePrimary(noAllocationDeciders(), CLUSTER_RECOVERED, "allocId1"); - - List shards = new ArrayList<>(); - ShardRouting shard = allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).primaryShard(); - shards.add(shard); - batchAllocator.addData(node1, "allocId1", true, new ReplicationCheckpoint(shardId, 20, 101, 1, Codec.getDefault().getName())); - HashMap allDecisions = batchAllocator.makeAllocationDecision(shards, allocation, logger); - // verify we get decisions for all the shards - assertEquals(shards.size(), allDecisions.size()); - assertEquals(shards, new ArrayList<>(allDecisions.keySet())); - assertEquals(AllocationDecision.YES, allDecisions.get(shard).getAllocationDecision()); - } - - public void testMakeAllocationDecisionDataFetchedMultipleShards() { - setUpShards(2); - final RoutingAllocation allocation = routingAllocationWithMultiplePrimaries( - noAllocationDeciders(), - CLUSTER_RECOVERED, - 2, - 0, - "allocId-0", - "allocId-1" - ); - List shards = new ArrayList<>(); - for (ShardId shardId : shardsInBatch) { - ShardRouting shard = allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).primaryShard(); - allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).primaryShard().recoverySource(); - shards.add(shard); - batchAllocator.addShardData( - node1, - "allocId-" + shardId.id(), - shardId, - true, - new ReplicationCheckpoint(shardId, 20, 101, 1, Codec.getDefault().getName()), - null - ); - } - HashMap allDecisions = batchAllocator.makeAllocationDecision(shards, allocation, logger); - // verify we get decisions for all the shards - assertEquals(shards.size(), allDecisions.size()); - assertEquals(new HashSet<>(shards), allDecisions.keySet()); - for (ShardRouting shard : shards) { - assertEquals(AllocationDecision.YES, allDecisions.get(shard).getAllocationDecision()); - } - } - - private RoutingAllocation routingAllocationWithOnePrimary( - AllocationDeciders deciders, - UnassignedInfo.Reason reason, - String... activeAllocationIds - ) { - Metadata metadata = Metadata.builder() - .put( - IndexMetadata.builder(shardId.getIndexName()) - .settings(settings(Version.CURRENT)) - .numberOfShards(1) - .numberOfReplicas(1) - .putInSyncAllocationIds(shardId.id(), Sets.newHashSet(activeAllocationIds)) - ) - .build(); - RoutingTable.Builder routingTableBuilder = RoutingTable.builder(); - switch (reason) { - - case INDEX_CREATED: - routingTableBuilder.addAsNew(metadata.index(shardId.getIndex())); - break; - case CLUSTER_RECOVERED: - routingTableBuilder.addAsRecovery(metadata.index(shardId.getIndex())); - break; - case INDEX_REOPENED: - routingTableBuilder.addAsFromCloseToOpen(metadata.index(shardId.getIndex())); - break; - default: - throw new IllegalArgumentException("can't do " + reason + " for you. teach me"); - } - ClusterState state = ClusterState.builder(org.opensearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) - .metadata(metadata) - .routingTable(routingTableBuilder.build()) - .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) - .build(); - return new RoutingAllocation(deciders, new RoutingNodes(state, false), state, null, null, System.nanoTime()); - } - - private RoutingAllocation routingAllocationWithMultiplePrimaries( - AllocationDeciders deciders, - UnassignedInfo.Reason reason, - int numberOfShards, - int replicas, - String... activeAllocationIds - ) { - Iterator shardIterator = shardsInBatch.iterator(); - Metadata metadata = Metadata.builder() - .put( - IndexMetadata.builder(shardId.getIndexName()) - .settings(settings(Version.CURRENT)) - .numberOfShards(numberOfShards) - .numberOfReplicas(replicas) - .putInSyncAllocationIds(shardIterator.next().id(), Sets.newHashSet(activeAllocationIds[0])) - .putInSyncAllocationIds(shardIterator.next().id(), Sets.newHashSet(activeAllocationIds[1])) - ) - .build(); - - RoutingTable.Builder routingTableBuilder = RoutingTable.builder(); - for (ShardId shardIdFromBatch : shardsInBatch) { - switch (reason) { - case INDEX_CREATED: - routingTableBuilder.addAsNew(metadata.index(shardIdFromBatch.getIndex())); - break; - case CLUSTER_RECOVERED: - routingTableBuilder.addAsRecovery(metadata.index(shardIdFromBatch.getIndex())); - break; - case INDEX_REOPENED: - routingTableBuilder.addAsFromCloseToOpen(metadata.index(shardIdFromBatch.getIndex())); - break; - default: - throw new IllegalArgumentException("can't do " + reason + " for you. teach me"); - } - } - ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) - .metadata(metadata) - .routingTable(routingTableBuilder.build()) - .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) - .build(); - return new RoutingAllocation(deciders, new RoutingNodes(state, false), state, null, null, System.nanoTime()); - } - - class TestBatchAllocator extends PrimaryShardBatchAllocator { - - private Map data; - - public TestBatchAllocator clear() { - data = null; - return this; - } - - public TestBatchAllocator addData( - DiscoveryNode node, - String allocationId, - boolean primary, - ReplicationCheckpoint replicationCheckpoint - ) { - return addData(node, allocationId, primary, replicationCheckpoint, null); - } - - public TestBatchAllocator addData(DiscoveryNode node, String allocationId, boolean primary) { - Settings nodeSettings = Settings.builder().put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()).build(); - IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test", nodeSettings); - return addData( - node, - allocationId, - primary, - ReplicationCheckpoint.empty(shardId, new CodecService(null, indexSettings, null).codec("default").getName()), - null - ); - } - - public TestBatchAllocator addData(DiscoveryNode node, String allocationId, boolean primary, @Nullable Exception storeException) { - Settings nodeSettings = Settings.builder().put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()).build(); - IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test", nodeSettings); - return addData( - node, - allocationId, - primary, - ReplicationCheckpoint.empty(shardId, new CodecService(null, indexSettings, null).codec("default").getName()), - storeException - ); - } - - public TestBatchAllocator addData( - DiscoveryNode node, - String allocationId, - boolean primary, - ReplicationCheckpoint replicationCheckpoint, - @Nullable Exception storeException - ) { - if (data == null) { - data = new HashMap<>(); - } - Map shardData = Map.of( - shardId, - new TransportNodesGatewayStartedShardHelper.GatewayStartedShard( - allocationId, - primary, - replicationCheckpoint, - storeException - ) - ); - data.put(node, new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch(node, shardData)); - return this; - } - - public TestBatchAllocator addShardData( - DiscoveryNode node, - String allocationId, - ShardId shardId, - boolean primary, - ReplicationCheckpoint replicationCheckpoint, - @Nullable Exception storeException - ) { - if (data == null) { - data = new HashMap<>(); - } - Map shardData = new HashMap<>(); - shardData.put( - shardId, - new TransportNodesGatewayStartedShardHelper.GatewayStartedShard( - allocationId, - primary, - replicationCheckpoint, - storeException - ) - ); - if (data.get(node) != null) shardData.putAll(data.get(node).getNodeGatewayStartedShardsBatch()); - data.put(node, new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch(node, shardData)); - return this; - } - - @Override - protected AsyncShardFetch.FetchResult fetchData( - List shardsEligibleForFetch, - List inEligibleShards, - RoutingAllocation allocation - ) { - return new AsyncShardFetch.FetchResult<>(data, Collections.>emptyMap()); - } - } -} diff --git a/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java b/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java index 647b85a9a1a34..085793162b3c4 100644 --- a/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java +++ b/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java @@ -16,7 +16,7 @@ import org.opensearch.cluster.routing.TestShardRouting; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.core.index.shard.ShardId; -import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayStartedShard; +import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.GatewayStartedShard; import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch; import org.opensearch.indices.store.ShardAttributes; @@ -53,9 +53,9 @@ public void setupShardBatchCache(String batchId, int numberOfShards) { GatewayStartedShard.class, NodeGatewayStartedShardsBatch::new, NodeGatewayStartedShardsBatch::getNodeGatewayStartedShardsBatch, - () -> new GatewayStartedShard(null, false, null, null), + () -> new GatewayStartedShard(new TransportNodesGatewayStartedShardHelper.GatewayStartedShard(null, false, null, null), null), this::removeShard, - GatewayStartedShard::storeException, + GatewayStartedShard::getTransportError, GatewayStartedShard::isEmpty ); } @@ -128,7 +128,7 @@ public void testPutData() { ); assertEquals(2, fetchData.size()); assertEquals(10, fetchData.get(node1).getNodeGatewayStartedShardsBatch().size()); - assertEquals("alloc-1", fetchData.get(node1).getNodeGatewayStartedShardsBatch().get(shard).allocationId()); + assertEquals("alloc-1", fetchData.get(node1).getNodeGatewayStartedShardsBatch().get(shard).get().allocationId()); assertEquals(10, fetchData.get(node2).getNodeGatewayStartedShardsBatch().size()); assertTrue(GatewayStartedShard.isEmpty(fetchData.get(node2).getNodeGatewayStartedShardsBatch().get(shard))); @@ -176,10 +176,22 @@ private Map getPrimaryResponse(List shard shardData.put(shard, null); break; case EMPTY: - shardData.put(shard, new GatewayStartedShard(null, false, null, null)); + shardData.put( + shard, + new GatewayStartedShard( + new TransportNodesGatewayStartedShardHelper.GatewayStartedShard(null, false, null, null), + null + ) + ); break; case VALID: - shardData.put(shard, new GatewayStartedShard("alloc-" + allocationId++, false, null, null)); + shardData.put( + shard, + new GatewayStartedShard( + new TransportNodesGatewayStartedShardHelper.GatewayStartedShard("alloc-" + allocationId++, false, null, null), + null + ) + ); break; default: throw new AssertionError("unknown response type"); @@ -195,10 +207,19 @@ private Map getFailedPrimaryResponse(List if (failedShardsCount-- > 0) { shardData.put( shard, - new GatewayStartedShard("alloc-" + allocationId++, false, null, new OpenSearchRejectedExecutionException()) + new GatewayStartedShard( + new TransportNodesGatewayStartedShardHelper.GatewayStartedShard("alloc-" + allocationId++, false, null, null), + new OpenSearchRejectedExecutionException() + ) ); } else { - shardData.put(shard, new GatewayStartedShard("alloc-" + allocationId++, false, null, null)); + shardData.put( + shard, + new GatewayStartedShard( + new TransportNodesGatewayStartedShardHelper.GatewayStartedShard("alloc-" + allocationId++, false, null, null), + null + ) + ); } } return shardData; From e07ddc87ba66ce907b6763e1ee523ce8076e1669 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Tue, 19 Mar 2024 18:40:34 +0530 Subject: [PATCH 23/29] spotless apply Signed-off-by: Aman Khare --- .../org/opensearch/gateway/PrimaryShardBatchAllocator.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java index 795a415c84ac5..960bbaa1e0ede 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java @@ -16,8 +16,8 @@ import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.gateway.AsyncShardFetch.FetchResult; import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.NodeGatewayStartedShard; -import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch; import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.GatewayStartedShard; +import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch; import java.util.ArrayList; import java.util.Collections; @@ -133,9 +133,7 @@ private static List adaptToNodeShardStates( // build data for a shard from all the nodes nodeResponses.forEach((node, nodeGatewayStartedShardsBatch) -> { - GatewayStartedShard shardData = nodeGatewayStartedShardsBatch - .getNodeGatewayStartedShardsBatch() - .get(unassignedShard.shardId()); + GatewayStartedShard shardData = nodeGatewayStartedShardsBatch.getNodeGatewayStartedShardsBatch().get(unassignedShard.shardId()); nodeShardStates.add( new NodeGatewayStartedShard( shardData.get().allocationId(), From f8093b148919096edee9f227228fd9cdd6dbd2a4 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Tue, 19 Mar 2024 18:56:38 +0530 Subject: [PATCH 24/29] java doc Signed-off-by: Aman Khare --- .../TransportNodesListGatewayStartedShardsBatch.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java index f187b97b82319..513b46d0429c1 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java @@ -256,6 +256,13 @@ public void writeTo(StreamOutput out) throws IOException { } } + /** + * Primary shard response from node. It contains the metadata in + * {@link TransportNodesGatewayStartedShardHelper.GatewayStartedShard} and any exception thrown from code will be + * stored in transportError. This exception is stored specifically to disambiguate the store related exceptions + * present in {@link TransportNodesGatewayStartedShardHelper.GatewayStartedShard} as those exceptions may be used + * during decision-making. + */ public static class GatewayStartedShard { private final TransportNodesGatewayStartedShardHelper.GatewayStartedShard gatewayStartedShard; private final Exception transportError; From ba6cbb4ed4e59dc729a7176fed236c87a9805d5d Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Tue, 2 Apr 2024 20:47:05 +0530 Subject: [PATCH 25/29] Use single exception parameter, modify exception handling in batch mode Signed-off-by: Aman Khare --- .../gateway/RecoveryFromGatewayIT.java | 14 +- .../gateway/AsyncShardBatchFetch.java | 169 ++++-------------- .../opensearch/gateway/AsyncShardFetch.java | 5 +- .../gateway/PrimaryShardBatchAllocator.java | 10 +- ...ansportNodesGatewayStartedShardHelper.java | 7 +- ...ortNodesListGatewayStartedShardsBatch.java | 80 ++------- ...sportNodesListShardStoreMetadataBatch.java | 2 +- .../gateway/AsyncShardFetchTests.java | 18 +- .../PrimaryShardBatchAllocatorTests.java | 28 +-- .../gateway/ShardBatchCacheTests.java | 41 +---- 10 files changed, 85 insertions(+), 289 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 26714f9a65329..ba03532a9aa2f 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -56,7 +56,7 @@ import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; import org.opensearch.env.NodeEnvironment; -import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.GatewayStartedShard; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayStartedShard; import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; import org.opensearch.index.MergePolicyProvider; @@ -818,9 +818,9 @@ public void testShardFetchCorruptedShardsUsingBatchAction() throws Exception { .get(discoveryNodes[0].getId()) .getNodeGatewayStartedShardsBatch() .get(shardId); - assertNotNull(gatewayStartedShard.get().storeException()); - assertNotNull(gatewayStartedShard.get().allocationId()); - assertTrue(gatewayStartedShard.get().primary()); + assertNotNull(gatewayStartedShard.storeException()); + assertNotNull(gatewayStartedShard.allocationId()); + assertTrue(gatewayStartedShard.primary()); } public void testSingleShardStoreFetchUsingBatchAction() throws ExecutionException, InterruptedException { @@ -949,9 +949,9 @@ private void assertNodeStoreFilesMetadataSuccessCase( } private void assertNodeGatewayStartedShardsHappyCase(GatewayStartedShard gatewayStartedShard) { - assertNull(gatewayStartedShard.get().storeException()); - assertNotNull(gatewayStartedShard.get().allocationId()); - assertTrue(gatewayStartedShard.get().primary()); + assertNull(gatewayStartedShard.storeException()); + assertNotNull(gatewayStartedShard.allocationId()); + assertTrue(gatewayStartedShard.primary()); } private void prepareIndex(String indexName, int numberOfPrimaryShards) { diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java index 18133f9b4bd1a..dbc1ee8d40832 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java @@ -12,22 +12,16 @@ import org.opensearch.action.support.nodes.BaseNodeResponse; import org.opensearch.action.support.nodes.BaseNodesResponse; import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.common.logging.Loggers; import org.opensearch.core.index.shard.ShardId; import org.opensearch.indices.store.ShardAttributes; import java.lang.reflect.Array; -import java.util.ArrayList; import java.util.HashMap; -import java.util.Iterator; -import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.BiFunction; -import java.util.function.Consumer; import java.util.function.Function; -import java.util.function.Supplier; /** * Implementation of AsyncShardFetch with batching support. This class is responsible for executing the fetch @@ -43,9 +37,6 @@ */ public abstract class AsyncShardBatchFetch extends AsyncShardFetch { - private final Consumer removeShardFromBatch; - private final List failedShards; - @SuppressWarnings("unchecked") AsyncShardBatchFetch( Logger logger, @@ -56,57 +47,29 @@ public abstract class AsyncShardBatchFetch extend Class clazz, BiFunction, T> responseGetter, Function> shardsBatchDataGetter, - Supplier emptyResponseBuilder, - Consumer failedShardHandler, - Function getResponseException, + V emptyResponse, Function isEmptyResponse ) { - super(logger, type, shardAttributesMap, action, batchId); - this.removeShardFromBatch = failedShardHandler; - this.failedShards = new ArrayList<>(); - this.cache = new ShardBatchCache<>( + super( logger, type, shardAttributesMap, - "BatchID=[" + batchId + "]", - clazz, - responseGetter, - shardsBatchDataGetter, - emptyResponseBuilder, - this::cleanUpFailedShards, - getResponseException, - isEmptyResponse + action, + batchId, + new ShardBatchCache<>( + logger, + type, + shardAttributesMap, + "BatchID=[" + batchId + "]", + clazz, + responseGetter, + shardsBatchDataGetter, + emptyResponse, + isEmptyResponse + ) ); } - public synchronized FetchResult fetchData(DiscoveryNodes nodes, Map> ignoreNodes) { - FetchResult result = super.fetchData(nodes, ignoreNodes); - if (result.hasData()) { - // trigger reroute for failed shards only when all nodes have completed fetching - if (failedShards.isEmpty() == false) { - // trigger a reroute if there are any shards failed, to make sure they're picked up in next run - logger.trace("triggering another reroute for failed shards in {}", reroutingKey); - reroute("shards-failed", "shards failed in " + reroutingKey); - failedShards.clear(); - } - } - return result; - } - - /** - * Remove the shard from shardAttributesMap so it's not sent in next asyncFetch. - * Call removeShardFromBatch method to remove the shardId from the batch object created in - * ShardsBatchGatewayAllocator. - * Add shardId to failedShards, so it can be used to trigger another reroute as part of upcoming fetchData call. - * - * @param shardId shardId to be cleaned up from batch and cache. - */ - private void cleanUpFailedShards(ShardId shardId) { - shardAttributesMap.remove(shardId); - removeShardFromBatch.accept(shardId); - failedShards.add(shardId); - } - /** * Remove a shard from the cache maintaining a full batch of shards. This is needed to clear the shard once it's * assigned or failed. @@ -119,7 +82,10 @@ public void clearShard(ShardId shardId) { } /** - * Cache implementation of transport actions returning batch of shards related data in the response. It'll + * Cache implementation of transport actions returning batch of shards related data in the response. + * Store node level responses of transport actions like {@link TransportNodesListGatewayStartedShardsBatch} or + * {@link org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch} with memory efficient caching + * approach. * * @param Response type of transport action. * @param Data type of shard level response. @@ -130,11 +96,8 @@ static class ShardBatchCache extends AsyncShardFe private final int batchSize; private final Class shardResponseClass; private final BiFunction, T> responseConstructor; - private final Map arrayToShardId; private final Function> shardsBatchDataGetter; - private final Supplier emptyResponseBuilder; - private final Consumer failedShardHandler; - private final Function getException; + private final V emptyResponse; private final Function isEmpty; private final Logger logger; @@ -146,24 +109,19 @@ public ShardBatchCache( Class clazz, BiFunction, T> responseGetter, Function> shardsBatchDataGetter, - Supplier emptyResponseBuilder, - Consumer failedShardHandler, - Function getResponseException, + V emptyResponse, Function isEmptyResponse ) { super(Loggers.getLogger(logger, "_" + logKey), type); this.batchSize = shardAttributesMap.size(); - this.getException = getResponseException; this.isEmpty = isEmptyResponse; cache = new HashMap<>(); shardIdToArray = new HashMap<>(); - arrayToShardId = new HashMap<>(); fillShardIdKeys(shardAttributesMap.keySet()); this.shardResponseClass = clazz; this.responseConstructor = responseGetter; this.shardsBatchDataGetter = shardsBatchDataGetter; - this.emptyResponseBuilder = emptyResponseBuilder; - this.failedShardHandler = failedShardHandler; + this.emptyResponse = emptyResponse; this.logger = logger; } @@ -182,26 +140,9 @@ public void deleteShard(ShardId shardId) { } } - @Override - public Map getCacheData(DiscoveryNodes nodes, Set failedNodes) { - fillReverseIdMap(); - return super.getCacheData(nodes, failedNodes); - } - - /** - * Build a reverse map to get shardId from the array index, this will be used to construct the response which - * PrimaryShardBatchAllocator or ReplicaShardBatchAllocator are looking for. - */ - private void fillReverseIdMap() { - arrayToShardId.clear(); - for (Map.Entry indexMapping : shardIdToArray.entrySet()) { - arrayToShardId.putIfAbsent(indexMapping.getValue(), indexMapping.getKey()); - } - } - @Override public void initData(DiscoveryNode node) { - cache.put(node.getId(), new NodeEntry<>(node.getId(), shardResponseClass, batchSize, getException, isEmpty)); + cache.put(node.getId(), new NodeEntry<>(node.getId(), shardResponseClass, batchSize, isEmpty)); } /** @@ -216,40 +157,9 @@ public void initData(DiscoveryNode node) { public void putData(DiscoveryNode node, T response) { NodeEntry nodeEntry = cache.get(node.getId()); Map batchResponse = shardsBatchDataGetter.apply(response); - filterFailedShards(batchResponse); nodeEntry.doneFetching(batchResponse, shardIdToArray); } - /** - * Return the shard for which we got unhandled exceptions. - * - * @param batchResponse response from one node for the batch. - */ - private void filterFailedShards(Map batchResponse) { - logger.trace("filtering failed shards"); - for (Iterator it = batchResponse.keySet().iterator(); it.hasNext();) { - ShardId shardId = it.next(); - if (batchResponse.get(shardId) != null) { - if (getException.apply(batchResponse.get(shardId)) != null) { - // handle per shard level exceptions, process other shards, only throw out this shard from - // the batch - Exception shardException = getException.apply(batchResponse.get(shardId)); - // if the request got rejected or timed out, we need to try it again next time... - if (retryableException(shardException)) { - logger.trace( - "got unhandled retryable exception for shard {} {}", - shardId.toString(), - shardException.toString() - ); - failedShardHandler.accept(shardId); - // remove this failed entry. So, while storing the data, we don't need to re-process it. - it.remove(); - } - } - } - } - } - @Override public T getData(DiscoveryNode node) { return this.responseConstructor.apply(node, getBatchData(cache.get(node.getId()))); @@ -259,12 +169,14 @@ private HashMap getBatchData(NodeEntry nodeEntry) { V[] nodeShardEntries = nodeEntry.getData(); boolean[] emptyResponses = nodeEntry.getEmptyShardResponse(); HashMap shardData = new HashMap<>(); - for (Integer shardIdIndex : shardIdToArray.values()) { - if (emptyResponses[shardIdIndex]) { - shardData.put(arrayToShardId.get(shardIdIndex), emptyResponseBuilder.get()); - } else if (nodeShardEntries[shardIdIndex] != null) { + for (Map.Entry shardIdIndex : shardIdToArray.entrySet()) { + ShardId shardId = shardIdIndex.getKey(); + Integer arrIndex = shardIdIndex.getValue(); + if (emptyResponses[arrIndex]) { + shardData.put(shardId, emptyResponse); + } else if (nodeShardEntries[arrIndex] != null) { // ignore null responses here - shardData.put(arrayToShardId.get(shardIdIndex), nodeShardEntries[shardIdIndex]); + shardData.put(shardId, nodeShardEntries[arrIndex]); } } return shardData; @@ -288,20 +200,12 @@ static class NodeEntry extends BaseNodeEntry { // actually needed in allocation/explain API response. So instead of storing full empty response object // in cache, it's better to just store a boolean and create that object on the fly just before // decision-making. - private final Function getException; private final Function isEmpty; - NodeEntry( - String nodeId, - Class clazz, - int batchSize, - Function getResponseException, - Function isEmptyResponse - ) { + NodeEntry(String nodeId, Class clazz, int batchSize, Function isEmptyResponse) { super(nodeId); this.shardData = (V[]) Array.newInstance(clazz, batchSize); this.emptyShardResponse = new boolean[batchSize]; - this.getException = getResponseException; this.isEmpty = isEmptyResponse; } @@ -324,15 +228,14 @@ boolean[] getEmptyShardResponse() { } private void fillShardData(Map shardDataFromNode, Map shardIdKey) { - for (ShardId shardId : shardDataFromNode.keySet()) { - if (shardDataFromNode.get(shardId) != null) { - if (isEmpty.apply(shardDataFromNode.get(shardId))) { + for (Map.Entry shardData : shardDataFromNode.entrySet()) { + if (shardData.getValue() != null) { + ShardId shardId = shardData.getKey(); + if (isEmpty.apply(shardData.getValue())) { this.emptyShardResponse[shardIdKey.get(shardId)] = true; this.shardData[shardIdKey.get(shardId)] = null; - } else if (getException.apply(shardDataFromNode.get(shardId)) == null) { - this.shardData[shardIdKey.get(shardId)] = shardDataFromNode.get(shardId); } - // if exception is not null, we got unhandled failure for the shard which needs to be ignored + this.shardData[shardIdKey.get(shardId)] = shardData.getValue(); } } } diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java index 3aa431c327a9d..29ec41beedd21 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java @@ -120,14 +120,15 @@ protected AsyncShardFetch( String type, Map shardAttributesMap, Lister, T> action, - String batchId + String batchId, + AsyncShardFetchCache cache ) { this.logger = logger; this.type = type; this.shardAttributesMap = shardAttributesMap; this.action = (Lister, T>) action; this.reroutingKey = "BatchID=[" + batchId + "]"; - cache = new ShardCache<>(logger, reroutingKey, type); + this.cache = cache; } @Override diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java index 960bbaa1e0ede..1979f33484d49 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java @@ -15,8 +15,8 @@ import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.gateway.AsyncShardFetch.FetchResult; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayStartedShard; import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.NodeGatewayStartedShard; -import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.GatewayStartedShard; import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch; import java.util.ArrayList; @@ -136,10 +136,10 @@ private static List adaptToNodeShardStates( GatewayStartedShard shardData = nodeGatewayStartedShardsBatch.getNodeGatewayStartedShardsBatch().get(unassignedShard.shardId()); nodeShardStates.add( new NodeGatewayStartedShard( - shardData.get().allocationId(), - shardData.get().primary(), - shardData.get().replicationCheckpoint(), - shardData.get().storeException(), + shardData.allocationId(), + shardData.primary(), + shardData.replicationCheckpoint(), + shardData.storeException(), node ) ); diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java index c341fd9e981ba..71c2c2cd2a444 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java @@ -233,8 +233,11 @@ public String toString() { return buf.toString(); } - public Boolean isEmpty() { - return allocationId == null && primary == false && storeException == null && replicationCheckpoint == null; + public static Boolean isEmpty(GatewayStartedShard gatewayStartedShard) { + return gatewayStartedShard.allocationId() == null + && gatewayStartedShard.primary() == false + && gatewayStartedShard.storeException() == null + && gatewayStartedShard.replicationCheckpoint() == null; } } diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java index 513b46d0429c1..9886e8472422c 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java @@ -38,6 +38,7 @@ import java.util.Map; import java.util.Objects; +import static org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayStartedShard; import static org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.INDEX_NOT_FOUND; import static org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.getShardInfoOnLocalNode; @@ -135,23 +136,22 @@ protected NodesGatewayStartedShardsBatch newResponse( @Override protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { Map shardsOnNode = new HashMap<>(); + // NOTE : If we ever change this for loop to run in parallel threads, we should re-visit the exception + // handling in AsyncShardBatchFetch class. for (Map.Entry shardAttr : request.shardAttributes.entrySet()) { final ShardId shardId = shardAttr.getKey(); try { shardsOnNode.put( shardId, - new GatewayStartedShard( - getShardInfoOnLocalNode( - logger, - shardId, - namedXContentRegistry, - nodeEnv, - indicesService, - shardAttr.getValue().getCustomDataPath(), - settings, - clusterService - ), - null + getShardInfoOnLocalNode( + logger, + shardId, + namedXContentRegistry, + nodeEnv, + indicesService, + shardAttr.getValue().getCustomDataPath(), + settings, + clusterService ) ); } catch (Exception e) { @@ -160,10 +160,7 @@ protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { shardsOnNode.put(shardId, null); } else { // return actual exception as it is for unknown exceptions - shardsOnNode.put( - shardId, - new GatewayStartedShard(new TransportNodesGatewayStartedShardHelper.GatewayStartedShard(null, false, null, null), e) - ); + shardsOnNode.put(shardId, new GatewayStartedShard(null, false, null, e)); } } } @@ -256,57 +253,6 @@ public void writeTo(StreamOutput out) throws IOException { } } - /** - * Primary shard response from node. It contains the metadata in - * {@link TransportNodesGatewayStartedShardHelper.GatewayStartedShard} and any exception thrown from code will be - * stored in transportError. This exception is stored specifically to disambiguate the store related exceptions - * present in {@link TransportNodesGatewayStartedShardHelper.GatewayStartedShard} as those exceptions may be used - * during decision-making. - */ - public static class GatewayStartedShard { - private final TransportNodesGatewayStartedShardHelper.GatewayStartedShard gatewayStartedShard; - private final Exception transportError; - - public GatewayStartedShard( - TransportNodesGatewayStartedShardHelper.GatewayStartedShard gatewayStartedShard, - Exception transportError - ) { - this.gatewayStartedShard = gatewayStartedShard; - this.transportError = transportError; - } - - public GatewayStartedShard(StreamInput in) throws IOException { - this.gatewayStartedShard = new TransportNodesGatewayStartedShardHelper.GatewayStartedShard(in); - if (in.readBoolean()) { - this.transportError = in.readException(); - } else { - this.transportError = null; - } - } - - public void writeTo(StreamOutput out) throws IOException { - gatewayStartedShard.writeTo(out); - if (transportError != null) { - out.writeBoolean(true); - out.writeException(transportError); - } else { - out.writeBoolean(false); - } - } - - public static boolean isEmpty(GatewayStartedShard gatewayStartedShard) { - return gatewayStartedShard.get().isEmpty() && gatewayStartedShard.getTransportError() == null; - } - - public Exception getTransportError() { - return transportError; - } - - public TransportNodesGatewayStartedShardHelper.GatewayStartedShard get() { - return gatewayStartedShard; - } - } - /** * This is the response from a single node, this is used in {@link NodesGatewayStartedShardsBatch} for creating * node to its response mapping for this transport request. diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java index 3d37ab026835a..85d5bff4677ef 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java @@ -155,7 +155,7 @@ private Map listStoreMetadata(NodeRequest reque shardStoreMetadataMap.put(shardId, new NodeStoreFilesMetadata(storeFilesMetadata, null)); } catch (Exception e) { // should return null in case of known exceptions being returned from listShardMetadataInternal method. - if (e.getMessage().contains(INDEX_NOT_FOUND)) { + if (e.getMessage().contains(INDEX_NOT_FOUND) || e instanceof IOException) { shardStoreMetadataMap.put(shardId, null); } else { // return actual exception as it is for unknown exceptions diff --git a/server/src/test/java/org/opensearch/gateway/AsyncShardFetchTests.java b/server/src/test/java/org/opensearch/gateway/AsyncShardFetchTests.java index 3502cc8996fa2..db97c3ece94ba 100644 --- a/server/src/test/java/org/opensearch/gateway/AsyncShardFetchTests.java +++ b/server/src/test/java/org/opensearch/gateway/AsyncShardFetchTests.java @@ -39,7 +39,6 @@ import org.opensearch.cluster.node.DiscoveryNodeRole; import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.core.index.shard.ShardId; -import org.opensearch.indices.store.ShardAttributes; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; @@ -47,7 +46,6 @@ import org.junit.Before; import java.util.Collections; -import java.util.HashMap; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; @@ -86,16 +84,7 @@ public class AsyncShardFetchTests extends OpenSearchTestCase { public void setUp() throws Exception { super.setUp(); this.threadPool = new TestThreadPool(getTestName()); - if (randomBoolean()) { - this.test = new TestFetch(threadPool); - } else { - HashMap shardToCustomDataPath = new HashMap<>(); - ShardId shardId0 = new ShardId("index1", "index_uuid1", 0); - ShardId shardId1 = new ShardId("index2", "index_uuid2", 0); - shardToCustomDataPath.put(shardId0, new ShardAttributes("")); - shardToCustomDataPath.put(shardId1, new ShardAttributes("")); - this.test = new TestFetch(threadPool, shardToCustomDataPath); - } + this.test = new TestFetch(threadPool); } @After @@ -414,11 +403,6 @@ static class Entry { this.threadPool = threadPool; } - TestFetch(ThreadPool threadPool, Map shardAttributesMap) { - super(LogManager.getLogger(TestFetch.class), "test", shardAttributesMap, null, "test-batch"); - this.threadPool = threadPool; - } - public void addSimulation(String nodeId, Response response) { simulations.put(nodeId, new Entry(response, null)); } diff --git a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java index ca66998bd448f..522ad2a64ea5d 100644 --- a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java @@ -29,6 +29,7 @@ import org.opensearch.common.util.set.Sets; import org.opensearch.core.index.shard.ShardId; import org.opensearch.env.Environment; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayStartedShard; import org.opensearch.index.IndexSettings; import org.opensearch.index.codec.CodecService; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; @@ -289,17 +290,9 @@ public TestBatchAllocator addData( if (data == null) { data = new HashMap<>(); } - Map shardData = Map.of( + Map shardData = Map.of( shardId, - new TransportNodesListGatewayStartedShardsBatch.GatewayStartedShard( - new TransportNodesGatewayStartedShardHelper.GatewayStartedShard( - allocationId, - primary, - replicationCheckpoint, - storeException - ), - null - ) + new GatewayStartedShard(allocationId, primary, replicationCheckpoint, storeException) ); data.put(node, new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch(node, shardData)); return this; @@ -316,19 +309,8 @@ public TestBatchAllocator addShardData( if (data == null) { data = new HashMap<>(); } - Map shardData = new HashMap<>(); - shardData.put( - shardId, - new TransportNodesListGatewayStartedShardsBatch.GatewayStartedShard( - new TransportNodesGatewayStartedShardHelper.GatewayStartedShard( - allocationId, - primary, - replicationCheckpoint, - storeException - ), - null - ) - ); + Map shardData = new HashMap<>(); + shardData.put(shardId, new GatewayStartedShard(allocationId, primary, replicationCheckpoint, storeException)); if (data.get(node) != null) shardData.putAll(data.get(node).getNodeGatewayStartedShardsBatch()); data.put(node, new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch(node, shardData)); return this; diff --git a/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java b/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java index 085793162b3c4..623103a0848cd 100644 --- a/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java +++ b/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java @@ -16,7 +16,7 @@ import org.opensearch.cluster.routing.TestShardRouting; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.core.index.shard.ShardId; -import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.GatewayStartedShard; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayStartedShard; import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch; import org.opensearch.indices.store.ShardAttributes; @@ -53,9 +53,7 @@ public void setupShardBatchCache(String batchId, int numberOfShards) { GatewayStartedShard.class, NodeGatewayStartedShardsBatch::new, NodeGatewayStartedShardsBatch::getNodeGatewayStartedShardsBatch, - () -> new GatewayStartedShard(new TransportNodesGatewayStartedShardHelper.GatewayStartedShard(null, false, null, null), null), - this::removeShard, - GatewayStartedShard::getTransportError, + new GatewayStartedShard(null, false, null, null), GatewayStartedShard::isEmpty ); } @@ -128,7 +126,7 @@ public void testPutData() { ); assertEquals(2, fetchData.size()); assertEquals(10, fetchData.get(node1).getNodeGatewayStartedShardsBatch().size()); - assertEquals("alloc-1", fetchData.get(node1).getNodeGatewayStartedShardsBatch().get(shard).get().allocationId()); + assertEquals("alloc-1", fetchData.get(node1).getNodeGatewayStartedShardsBatch().get(shard).allocationId()); assertEquals(10, fetchData.get(node2).getNodeGatewayStartedShardsBatch().size()); assertTrue(GatewayStartedShard.isEmpty(fetchData.get(node2).getNodeGatewayStartedShardsBatch().get(shard))); @@ -150,7 +148,7 @@ public void testNullResponses() { assertTrue(fetchData.get(node1).getNodeGatewayStartedShardsBatch().isEmpty()); } - public void testFilterFailedShards() { + public void testShardsDataWithException() { setupShardBatchCache(BATCH_ID, NUMBER_OF_SHARDS_DEFAULT); this.shardCache.initData(node1); this.shardCache.initData(node2); @@ -163,7 +161,7 @@ public void testFilterFailedShards() { // assertEquals(5, batchInfo.size()); assertEquals(2, fetchData.size()); - assertEquals(5, fetchData.get(node1).getNodeGatewayStartedShardsBatch().size()); + assertEquals(10, fetchData.get(node1).getNodeGatewayStartedShardsBatch().size()); assertTrue(fetchData.get(node2).getNodeGatewayStartedShardsBatch().isEmpty()); } @@ -176,22 +174,10 @@ private Map getPrimaryResponse(List shard shardData.put(shard, null); break; case EMPTY: - shardData.put( - shard, - new GatewayStartedShard( - new TransportNodesGatewayStartedShardHelper.GatewayStartedShard(null, false, null, null), - null - ) - ); + shardData.put(shard, new GatewayStartedShard(null, false, null, null)); break; case VALID: - shardData.put( - shard, - new GatewayStartedShard( - new TransportNodesGatewayStartedShardHelper.GatewayStartedShard("alloc-" + allocationId++, false, null, null), - null - ) - ); + shardData.put(shard, new GatewayStartedShard("alloc-" + allocationId++, false, null, null)); break; default: throw new AssertionError("unknown response type"); @@ -207,19 +193,10 @@ private Map getFailedPrimaryResponse(List if (failedShardsCount-- > 0) { shardData.put( shard, - new GatewayStartedShard( - new TransportNodesGatewayStartedShardHelper.GatewayStartedShard("alloc-" + allocationId++, false, null, null), - new OpenSearchRejectedExecutionException() - ) + new GatewayStartedShard("alloc-" + allocationId++, false, null, new OpenSearchRejectedExecutionException()) ); } else { - shardData.put( - shard, - new GatewayStartedShard( - new TransportNodesGatewayStartedShardHelper.GatewayStartedShard("alloc-" + allocationId++, false, null, null), - null - ) - ); + shardData.put(shard, new GatewayStartedShard("alloc-" + allocationId++, false, null, null)); } } return shardData; From 424625649150ea6b43e1b9177b0ccbb55f42c397 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Thu, 4 Apr 2024 10:04:20 +0530 Subject: [PATCH 26/29] Add factory class to create new response objects Signed-off-by: Aman Khare --- .../gateway/AsyncShardBatchFetch.java | 26 ++++------ .../gateway/ShardBatchResponseFactory.java | 51 +++++++++++++++++++ .../gateway/ShardBatchCacheTests.java | 5 +- 3 files changed, 63 insertions(+), 19 deletions(-) create mode 100644 server/src/main/java/org/opensearch/gateway/ShardBatchResponseFactory.java diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java index dbc1ee8d40832..41cf0da434c4c 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java @@ -20,7 +20,6 @@ import java.util.HashMap; import java.util.Map; import java.util.Set; -import java.util.function.BiFunction; import java.util.function.Function; /** @@ -45,10 +44,9 @@ public abstract class AsyncShardBatchFetch extend AsyncShardFetch.Lister, T> action, String batchId, Class clazz, - BiFunction, T> responseGetter, - Function> shardsBatchDataGetter, V emptyResponse, - Function isEmptyResponse + Function isEmptyResponse, + ShardBatchResponseFactory responseFactory ) { super( logger, @@ -62,10 +60,9 @@ public abstract class AsyncShardBatchFetch extend shardAttributesMap, "BatchID=[" + batchId + "]", clazz, - responseGetter, - shardsBatchDataGetter, emptyResponse, - isEmptyResponse + isEmptyResponse, + responseFactory ) ); } @@ -95,8 +92,7 @@ static class ShardBatchCache extends AsyncShardFe private final Map shardIdToArray; private final int batchSize; private final Class shardResponseClass; - private final BiFunction, T> responseConstructor; - private final Function> shardsBatchDataGetter; + private final ShardBatchResponseFactory responseFactory; private final V emptyResponse; private final Function isEmpty; private final Logger logger; @@ -107,10 +103,9 @@ public ShardBatchCache( Map shardAttributesMap, String logKey, Class clazz, - BiFunction, T> responseGetter, - Function> shardsBatchDataGetter, V emptyResponse, - Function isEmptyResponse + Function isEmptyResponse, + ShardBatchResponseFactory responseFactory ) { super(Loggers.getLogger(logger, "_" + logKey), type); this.batchSize = shardAttributesMap.size(); @@ -119,10 +114,9 @@ public ShardBatchCache( shardIdToArray = new HashMap<>(); fillShardIdKeys(shardAttributesMap.keySet()); this.shardResponseClass = clazz; - this.responseConstructor = responseGetter; - this.shardsBatchDataGetter = shardsBatchDataGetter; this.emptyResponse = emptyResponse; this.logger = logger; + this.responseFactory = responseFactory; } @Override @@ -156,13 +150,13 @@ public void initData(DiscoveryNode node) { @Override public void putData(DiscoveryNode node, T response) { NodeEntry nodeEntry = cache.get(node.getId()); - Map batchResponse = shardsBatchDataGetter.apply(response); + Map batchResponse = responseFactory.getShardBatchData(response); nodeEntry.doneFetching(batchResponse, shardIdToArray); } @Override public T getData(DiscoveryNode node) { - return this.responseConstructor.apply(node, getBatchData(cache.get(node.getId()))); + return this.responseFactory.getNewResponse(node, getBatchData(cache.get(node.getId()))); } private HashMap getBatchData(NodeEntry nodeEntry) { diff --git a/server/src/main/java/org/opensearch/gateway/ShardBatchResponseFactory.java b/server/src/main/java/org/opensearch/gateway/ShardBatchResponseFactory.java new file mode 100644 index 0000000000000..4b85ef995f1e1 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/ShardBatchResponseFactory.java @@ -0,0 +1,51 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway; + +import org.opensearch.action.support.nodes.BaseNodeResponse; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayStartedShard; +import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch; + +import java.util.Map; + +/** + * A factory class to create new responses of batch transport actions like + * {@link TransportNodesListGatewayStartedShardsBatch} or {@link org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch} + * + * @param Node level response returned by batch transport actions. + * @param Shard level metadata returned by batch transport actions. + */ +public class ShardBatchResponseFactory { + private final boolean primary; + + public ShardBatchResponseFactory(boolean primary) { + this.primary = primary; + } + + public T getNewResponse(DiscoveryNode node, Map shardData) { + if (primary) { + return (T) new NodeGatewayStartedShardsBatch(node, (Map) shardData); + } else { + return (T) new NodeStoreFilesMetadataBatch(node, (Map) shardData); + } + } + + public Map getShardBatchData(T response) { + if (primary) { + return (Map) ((NodeGatewayStartedShardsBatch) response).getNodeGatewayStartedShardsBatch(); + } else { + return (Map) ((NodeStoreFilesMetadataBatch) response).getNodeStoreFilesMetadataBatch(); + } + } + +} diff --git a/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java b/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java index 623103a0848cd..e6f6f43a03fe8 100644 --- a/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java +++ b/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java @@ -51,10 +51,9 @@ public void setupShardBatchCache(String batchId, int numberOfShards) { shardAttributesMap, "BatchID=[" + batchId + "]", GatewayStartedShard.class, - NodeGatewayStartedShardsBatch::new, - NodeGatewayStartedShardsBatch::getNodeGatewayStartedShardsBatch, new GatewayStartedShard(null, false, null, null), - GatewayStartedShard::isEmpty + GatewayStartedShard::isEmpty, + new ShardBatchResponseFactory<>(true) ); } From d61477f64d367681666ef2e2e309f518656811d2 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Mon, 8 Apr 2024 14:00:52 +0530 Subject: [PATCH 27/29] Save non empty response in else condition Signed-off-by: Aman Khare --- .../main/java/org/opensearch/gateway/AsyncShardBatchFetch.java | 3 ++- .../src/main/java/org/opensearch/gateway/AsyncShardFetch.java | 2 +- .../gateway/TransportNodesListGatewayStartedShardsBatch.java | 2 +- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java index 41cf0da434c4c..8a292e0925686 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java @@ -228,8 +228,9 @@ private void fillShardData(Map shardDataFromNode, Map, N protected final String type; protected final Map shardAttributesMap; private final Lister, T> action; - protected AsyncShardFetchCache cache; + protected final AsyncShardFetchCache cache; private final AtomicLong round = new AtomicLong(); private boolean closed; final String reroutingKey; diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java index 9886e8472422c..89362988b4d85 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java @@ -156,7 +156,7 @@ protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { ); } catch (Exception e) { // should return null in case of known exceptions being returned from getShardInfoOnLocalNode method. - if (e instanceof IllegalStateException || e.getMessage().contains(INDEX_NOT_FOUND)) { + if (e instanceof IllegalStateException || e.getMessage().contains(INDEX_NOT_FOUND) || e instanceof IOException) { shardsOnNode.put(shardId, null); } else { // return actual exception as it is for unknown exceptions From 65229fce55e24f250d93204ecab836d40c75c827 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Tue, 9 Apr 2024 13:46:08 +0530 Subject: [PATCH 28/29] Cleanup java doc and review comments changes Signed-off-by: Aman Khare --- .../gateway/AsyncShardBatchFetch.java | 44 ++++++++++--------- ...ansportNodesGatewayStartedShardHelper.java | 2 +- .../gateway/ShardBatchCacheTests.java | 8 ++-- 3 files changed, 29 insertions(+), 25 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java index 8a292e0925686..ca769235a8053 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java @@ -20,14 +20,16 @@ import java.util.HashMap; import java.util.Map; import java.util.Set; -import java.util.function.Function; +import java.util.function.Predicate; + +import reactor.util.annotation.NonNull; /** * Implementation of AsyncShardFetch with batching support. This class is responsible for executing the fetch * part using the base class {@link AsyncShardFetch}. Other functionalities needed for a batch are only written here. - * Cleanup of failed shards is necessary in a batch and based on that a reroute should be triggered to take care of - * those in the next run. This separation also takes care of the extra generic type V which is only needed for batch - * transport actions like {@link TransportNodesListGatewayStartedShardsBatch}. + * This separation also takes care of the extra generic type V which is only needed for batch + * transport actions like {@link TransportNodesListGatewayStartedShardsBatch} and + * {@link org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch}. * * @param Response type of the transport action. * @param Data type of shard level response. @@ -44,8 +46,8 @@ public abstract class AsyncShardBatchFetch extend AsyncShardFetch.Lister, T> action, String batchId, Class clazz, - V emptyResponse, - Function isEmptyResponse, + V emptyShardResponse, + Predicate emptyShardResponsePredicate, ShardBatchResponseFactory responseFactory ) { super( @@ -60,8 +62,8 @@ public abstract class AsyncShardBatchFetch extend shardAttributesMap, "BatchID=[" + batchId + "]", clazz, - emptyResponse, - isEmptyResponse, + emptyShardResponse, + emptyShardResponsePredicate, responseFactory ) ); @@ -82,7 +84,8 @@ public void clearShard(ShardId shardId) { * Cache implementation of transport actions returning batch of shards related data in the response. * Store node level responses of transport actions like {@link TransportNodesListGatewayStartedShardsBatch} or * {@link org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch} with memory efficient caching - * approach. + * approach. This cache class is not thread safe, all of its methods are being called from + * {@link AsyncShardFetch} class which has synchronized blocks present to handle multiple threads. * * @param Response type of transport action. * @param Data type of shard level response. @@ -94,7 +97,7 @@ static class ShardBatchCache extends AsyncShardFe private final Class shardResponseClass; private final ShardBatchResponseFactory responseFactory; private final V emptyResponse; - private final Function isEmpty; + private final Predicate emptyShardResponsePredicate; private final Logger logger; public ShardBatchCache( @@ -104,12 +107,12 @@ public ShardBatchCache( String logKey, Class clazz, V emptyResponse, - Function isEmptyResponse, + Predicate emptyShardResponsePredicate, ShardBatchResponseFactory responseFactory ) { super(Loggers.getLogger(logger, "_" + logKey), type); this.batchSize = shardAttributesMap.size(); - this.isEmpty = isEmptyResponse; + this.emptyShardResponsePredicate = emptyShardResponsePredicate; cache = new HashMap<>(); shardIdToArray = new HashMap<>(); fillShardIdKeys(shardAttributesMap.keySet()); @@ -120,6 +123,7 @@ public ShardBatchCache( } @Override + @NonNull public Map getCache() { return cache; } @@ -136,7 +140,7 @@ public void deleteShard(ShardId shardId) { @Override public void initData(DiscoveryNode node) { - cache.put(node.getId(), new NodeEntry<>(node.getId(), shardResponseClass, batchSize, isEmpty)); + cache.put(node.getId(), new NodeEntry<>(node.getId(), shardResponseClass, batchSize, emptyShardResponsePredicate)); } /** @@ -163,9 +167,9 @@ private HashMap getBatchData(NodeEntry nodeEntry) { V[] nodeShardEntries = nodeEntry.getData(); boolean[] emptyResponses = nodeEntry.getEmptyShardResponse(); HashMap shardData = new HashMap<>(); - for (Map.Entry shardIdIndex : shardIdToArray.entrySet()) { - ShardId shardId = shardIdIndex.getKey(); - Integer arrIndex = shardIdIndex.getValue(); + for (Map.Entry shardIdEntry : shardIdToArray.entrySet()) { + ShardId shardId = shardIdEntry.getKey(); + Integer arrIndex = shardIdEntry.getValue(); if (emptyResponses[arrIndex]) { shardData.put(shardId, emptyResponse); } else if (nodeShardEntries[arrIndex] != null) { @@ -194,13 +198,13 @@ static class NodeEntry extends BaseNodeEntry { // actually needed in allocation/explain API response. So instead of storing full empty response object // in cache, it's better to just store a boolean and create that object on the fly just before // decision-making. - private final Function isEmpty; + private final Predicate emptyShardResponsePredicate; - NodeEntry(String nodeId, Class clazz, int batchSize, Function isEmptyResponse) { + NodeEntry(String nodeId, Class clazz, int batchSize, Predicate emptyShardResponsePredicate) { super(nodeId); this.shardData = (V[]) Array.newInstance(clazz, batchSize); this.emptyShardResponse = new boolean[batchSize]; - this.isEmpty = isEmptyResponse; + this.emptyShardResponsePredicate = emptyShardResponsePredicate; } void doneFetching(Map shardDataFromNode, Map shardIdKey) { @@ -225,7 +229,7 @@ private void fillShardData(Map shardDataFromNode, Map shardData : shardDataFromNode.entrySet()) { if (shardData.getValue() != null) { ShardId shardId = shardData.getKey(); - if (isEmpty.apply(shardData.getValue())) { + if (emptyShardResponsePredicate.test(shardData.getValue())) { this.emptyShardResponse[shardIdKey.get(shardId)] = true; this.shardData[shardIdKey.get(shardId)] = null; } else { diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java index 71c2c2cd2a444..2ddae1d8410c9 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java @@ -233,7 +233,7 @@ public String toString() { return buf.toString(); } - public static Boolean isEmpty(GatewayStartedShard gatewayStartedShard) { + public static boolean isEmpty(GatewayStartedShard gatewayStartedShard) { return gatewayStartedShard.allocationId() == null && gatewayStartedShard.primary() == false && gatewayStartedShard.storeException() == null diff --git a/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java b/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java index e6f6f43a03fe8..1b42a31a4fd84 100644 --- a/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java +++ b/server/src/test/java/org/opensearch/gateway/ShardBatchCacheTests.java @@ -119,6 +119,10 @@ public void testPutData() { this.shardCache.putData(node1, new NodeGatewayStartedShardsBatch(node1, getPrimaryResponse(shardsInBatch, ResponseType.VALID))); this.shardCache.putData(node2, new NodeGatewayStartedShardsBatch(node1, getPrimaryResponse(shardsInBatch, ResponseType.EMPTY))); + // assert that fetching is done as both node's responses are stored in cache + assertFalse(this.shardCache.getCache().get(node1.getId()).isFetching()); + assertFalse(this.shardCache.getCache().get(node2.getId()).isFetching()); + Map fetchData = shardCache.getCacheData( DiscoveryNodes.builder().add(node1).add(node2).build(), null @@ -201,10 +205,6 @@ private Map getFailedPrimaryResponse(List return shardData; } - public void removeShard(ShardId shardId) { - // batchInfo.remove(shardId); - } - private void fillShards(Map shardAttributesMap, int numberOfShards) { shardsInBatch = BatchTestUtil.setUpShards(numberOfShards); for (ShardId shardId : shardsInBatch) { From 08763ed497b2353aeff304f5e88bae0f7f63c6e0 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Wed, 10 Apr 2024 11:52:41 +0530 Subject: [PATCH 29/29] Mark clearShard method as synchronized Signed-off-by: Aman Khare --- .../main/java/org/opensearch/gateway/AsyncShardBatchFetch.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java index ca769235a8053..4f39a39cea678 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardBatchFetch.java @@ -75,7 +75,7 @@ public abstract class AsyncShardBatchFetch extend * * @param shardId shardId to be removed from the batch. */ - public void clearShard(ShardId shardId) { + public synchronized void clearShard(ShardId shardId) { this.shardAttributesMap.remove(shardId); this.cache.deleteShard(shardId); }