diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerSelectionCriteria.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerSelectionCriteria.java index 3965a1593238..380b019f89a3 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerSelectionCriteria.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerSelectionCriteria.java @@ -24,7 +24,6 @@ import org.apache.hadoop.hdds.scm.container.ContainerManager; import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException; import org.apache.hadoop.hdds.scm.container.ContainerReplica; -import org.apache.hadoop.hdds.scm.container.replication.LegacyReplicationManager; import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager; import org.apache.hadoop.hdds.scm.node.NodeManager; import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; @@ -143,18 +142,6 @@ private Comparator orderContainersByUsedBytes() { return this::isContainerMoreUsed; } - /** - * Checks whether a Container has the ReplicationType - * {@link HddsProtos.ReplicationType#EC} and the Legacy Replication Manger is enabled. - * @param container container to check - * @return true if the ReplicationType is EC and "hdds.scm.replication - * .enable.legacy" is true, else false - */ - private boolean isECContainerAndLegacyRMEnabled(ContainerInfo container) { - return container.getReplicationType().equals(HddsProtos.ReplicationType.EC) - && replicationManager.getConfig().isLegacyEnabled(); - } - /** * Gets containers that are suitable for moving based on the following * required criteria: @@ -163,7 +150,6 @@ private boolean isECContainerAndLegacyRMEnabled(ContainerInfo container) { * 3. Container size should be closer to 5GB. * 4. Container must not be in the configured exclude containers list. * 5. Container should be closed. - * 6. If the {@link LegacyReplicationManager} is enabled, then the container should not be an EC container. * @param node DatanodeDetails for which to find candidate containers. * @return true if the container should be excluded, else false */ @@ -179,7 +165,7 @@ public boolean shouldBeExcluded(ContainerID containerID, } return excludeContainers.contains(containerID) || excludeContainersDueToFailure.contains(containerID) || containerToSourceMap.containsKey(containerID) || - !isContainerClosed(container, node) || isECContainerAndLegacyRMEnabled(container) || + !isContainerClosed(container, node) || isContainerReplicatingOrDeleting(containerID) || !findSourceStrategy.canSizeLeaveSource(node, container.getUsedBytes()) || breaksMaxSizeToMoveLimit(container.containerID(), diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java index f1eee8c6755b..54c0f4d81e7b 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java @@ -988,18 +988,8 @@ private boolean moveContainer(DatanodeDetails source, CompletableFuture future; try { ContainerInfo containerInfo = containerManager.getContainer(containerID); + future = moveManager.move(containerID, source, moveSelection.getTargetNode()); - /* - If LegacyReplicationManager is enabled, ReplicationManager will - redirect to it. Otherwise, use MoveManager. - */ - if (replicationManager.getConfig().isLegacyEnabled()) { - future = replicationManager - .move(containerID, source, moveSelection.getTargetNode()); - } else { - future = moveManager.move(containerID, source, - moveSelection.getTargetNode()); - } metrics.incrementNumContainerMovesScheduledInLatestIteration(1); future = future.whenComplete((result, ex) -> { @@ -1038,7 +1028,7 @@ private boolean moveContainer(DatanodeDetails source, selectionCriteria.addToExcludeDueToFailContainers(moveSelection.getContainerID()); metrics.incrementNumContainerMovesFailedInLatestIteration(1); return false; - } catch (NodeNotFoundException | TimeoutException e) { + } catch (NodeNotFoundException e) { LOG.warn("Container move failed for container {}", containerID, e); metrics.incrementNumContainerMovesFailedInLatestIteration(1); return false; diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ContainerReplicaCount.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ContainerReplicaCount.java index 80ef8ab4c0d5..8d864fbc4e0b 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ContainerReplicaCount.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ContainerReplicaCount.java @@ -68,7 +68,7 @@ default boolean isHealthy() { || containerState == HddsProtos.LifeCycleState.QUASI_CLOSED) && getReplicas().stream() .filter(r -> r.getDatanodeDetails().getPersistedOpState() == IN_SERVICE) - .allMatch(r -> LegacyReplicationManager.compareState( + .allMatch(r -> ReplicationManager.compareState( containerState, r.getState())); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/LegacyRatisContainerReplicaCount.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/LegacyRatisContainerReplicaCount.java deleted file mode 100644 index f491e2bd6f52..000000000000 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/LegacyRatisContainerReplicaCount.java +++ /dev/null @@ -1,142 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package org.apache.hadoop.hdds.scm.container.replication; - -import org.apache.hadoop.hdds.protocol.DatanodeDetails; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState; -import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto; -import org.apache.hadoop.hdds.scm.container.ContainerInfo; -import org.apache.hadoop.hdds.scm.container.ContainerReplica; -import org.apache.hadoop.hdds.scm.node.NodeManager; -import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; - -import java.util.List; -import java.util.Set; - -import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_SERVICE; - -/** - * When HDDS-6447 was done to improve the LegacyReplicationManager, work on - * the new replication manager had already started. When this class was added, - * the LegacyReplicationManager needed separate handling for healthy and - * unhealthy container replicas, but the new replication manager did not yet - * have this functionality. This class is used by the - * LegacyReplicationManager to allow {@link RatisContainerReplicaCount} to - * function for both use cases. When the new replication manager is finished - * and LegacyReplicationManager is removed, this class should be deleted and - * all necessary functionality consolidated to - * {@link RatisContainerReplicaCount} - */ -public class LegacyRatisContainerReplicaCount extends - RatisContainerReplicaCount { - public LegacyRatisContainerReplicaCount(ContainerInfo container, - Set replicas, - int inFlightAdd, - int inFlightDelete, int replicationFactor, - int minHealthyForMaintenance) { - super(container, replicas, inFlightAdd, inFlightDelete, replicationFactor, - minHealthyForMaintenance); - } - - public LegacyRatisContainerReplicaCount(ContainerInfo container, - Set replicas, List pendingOps, - int minHealthyForMaintenance, boolean considerUnhealthy) { - super(container, replicas, pendingOps, minHealthyForMaintenance, - considerUnhealthy); - } - - @Override - protected int healthyReplicaCountAdapter() { - return -getMisMatchedReplicaCount(); - } - - /** - * For LegacyReplicationManager, unhealthy replicas are all replicas that - * don't match the container's state. For a CLOSED container with replicas - * {CLOSED, CLOSING, UNHEALTHY, OPEN}, unhealthy replica count is 3. 2 - * mismatches (CLOSING, OPEN) + 1 UNHEALTHY = 3. - */ - @Override - public int getUnhealthyReplicaCountAdapter() { - return getMisMatchedReplicaCount(); - } - - /** - * Checks if all replicas (except UNHEALTHY) on in-service nodes are in the - * same health state as the container. This is similar to what - * {@link ContainerReplicaCount#isHealthy()} does. The difference is in how - * both methods treat UNHEALTHY replicas. - *

- * This method is the interface between the decommissioning flow and - * Replication Manager. Callers can use it to check whether replicas of a - * container are in the same state as the container before a datanode is - * taken offline. - *

- * Note that this method's purpose is to only compare the replica state with - * the container state. It does not check if the container has sufficient - * number of replicas - that is the job of {@link ContainerReplicaCount - * #isSufficientlyReplicatedForOffline(DatanodeDetails, NodeManager)}. - * @return true if the container is healthy enough, which is determined by - * various checks - */ - @Override - public boolean isHealthyEnoughForOffline() { - long countInService = getReplicas().stream() - .filter(r -> r.getDatanodeDetails().getPersistedOpState() == IN_SERVICE) - .count(); - if (countInService == 0) { - /* - Having no in-service nodes is unexpected and SCM shouldn't allow this - to happen in the first place. Return false here just to be safe. - */ - return false; - } - - LifeCycleState containerState = getContainer().getState(); - return (containerState == LifeCycleState.CLOSED - || containerState == LifeCycleState.QUASI_CLOSED) - && getReplicas().stream() - .filter(r -> r.getDatanodeDetails().getPersistedOpState() == IN_SERVICE) - .filter(r -> r.getState() != - ContainerReplicaProto.State.UNHEALTHY) - .allMatch(r -> ReplicationManager.compareState( - containerState, r.getState())); - } - - /** - * For Legacy Replication Manager and Ratis Containers, this method checks - * if the container is sufficiently replicated. It also checks whether - * there are any UNHEALTHY replicas that need to be replicated. - * @param datanode Not used in this implementation - * @param nodeManager An instance of NodeManager, used to check the health - * status of a node - * @return true if the container is sufficiently replicated and there are - * no UNHEALTHY replicas that need to be replicated, false otherwise - */ - @Override - public boolean isSufficientlyReplicatedForOffline(DatanodeDetails datanode, - NodeManager nodeManager) { - return super.isSufficientlyReplicated() && - super.getVulnerableUnhealthyReplicas(dn -> { - try { - return nodeManager.getNodeStatus(dn); - } catch (NodeNotFoundException e) { - return null; - } - }).isEmpty(); - } -} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/LegacyReplicationManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/LegacyReplicationManager.java deleted file mode 100644 index 04862e0d3171..000000000000 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/LegacyReplicationManager.java +++ /dev/null @@ -1,2581 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hdds.scm.container.replication; - -import com.google.common.collect.ImmutableList; -import com.google.protobuf.Message; -import org.apache.hadoop.hdds.conf.Config; -import org.apache.hadoop.hdds.conf.ConfigGroup; -import org.apache.hadoop.hdds.conf.ConfigType; -import org.apache.hadoop.hdds.conf.ConfigurationSource; -import org.apache.hadoop.hdds.conf.StorageUnit; -import org.apache.hadoop.hdds.protocol.DatanodeDetails; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState; -import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto.State; -import org.apache.hadoop.hdds.scm.ContainerPlacementStatus; -import org.apache.hadoop.hdds.scm.PlacementPolicy; -import org.apache.hadoop.hdds.scm.ScmConfigKeys; -import org.apache.hadoop.hdds.scm.container.ContainerID; -import org.apache.hadoop.hdds.scm.container.ContainerInfo; -import org.apache.hadoop.hdds.scm.container.ContainerManager; -import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException; -import org.apache.hadoop.hdds.scm.container.ContainerReplica; -import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport; -import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport.HealthState; -import org.apache.hadoop.hdds.scm.container.balancer.MoveManager; -import org.apache.hadoop.hdds.scm.container.common.helpers.MoveDataNodePair; -import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager.ReplicationManagerConfiguration; -import org.apache.hadoop.hdds.scm.events.SCMEvents; -import org.apache.hadoop.hdds.scm.exceptions.SCMException; -import org.apache.hadoop.hdds.scm.ha.SCMContext; -import org.apache.hadoop.hdds.scm.ha.SCMHAInvocationHandler; -import org.apache.hadoop.hdds.scm.ha.SCMHAManager; -import org.apache.hadoop.hdds.scm.ha.SCMRatisServer; -import org.apache.hadoop.hdds.scm.metadata.DBTransactionBuffer; -import org.apache.hadoop.hdds.scm.metadata.Replicate; -import org.apache.hadoop.hdds.scm.node.NodeManager; -import org.apache.hadoop.hdds.scm.node.NodeStatus; -import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; -import org.apache.hadoop.hdds.scm.server.StorageContainerManager; -import org.apache.hadoop.hdds.server.events.EventPublisher; -import org.apache.hadoop.hdds.utils.db.Table; -import org.apache.hadoop.hdds.utils.db.TableIterator; -import org.apache.hadoop.ozone.ClientVersion; -import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException; -import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand; -import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode; -import org.apache.hadoop.ozone.protocol.commands.DeleteContainerCommand; -import org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand; -import org.apache.hadoop.ozone.protocol.commands.SCMCommand; -import org.apache.ratis.protocol.exceptions.NotLeaderException; -import org.apache.ratis.util.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.lang.reflect.Proxy; -import java.time.Clock; -import java.time.Duration; -import java.time.Instant; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashSet; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Consumer; -import java.util.function.Predicate; -import java.util.stream.Collectors; - -import static org.apache.hadoop.hdds.conf.ConfigTag.OZONE; -import static org.apache.hadoop.hdds.conf.ConfigTag.SCM; -import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_SERVICE; -import static org.apache.hadoop.hdds.protocol.proto.SCMRatisProtocol.RequestType.MOVE; -import static org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp.PendingOpType.ADD; -import static org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp.PendingOpType.DELETE; - -/** - * Legacy Replication Manager (RM) is a legacy , which is used to process - * non-EC container, and hopefully to be replaced int the future. - */ -public class LegacyReplicationManager { - - public static final Logger LOG = - LoggerFactory.getLogger(LegacyReplicationManager.class); - - static class InflightMap { - private final Map> map - = new ConcurrentHashMap<>(); - private final InflightType type; - private final int sizeLimit; - private final AtomicInteger inflightCount = new AtomicInteger(); - - InflightMap(InflightType type, int sizeLimit) { - this.type = type; - this.sizeLimit = sizeLimit > 0 ? sizeLimit : Integer.MAX_VALUE; - } - - boolean isReplication() { - return type == InflightType.REPLICATION; - } - - private List get(ContainerID id) { - return map.get(id); - } - - boolean containsKey(ContainerID id) { - return map.containsKey(id); - } - - int inflightActionCount(ContainerID id) { - return Optional.ofNullable(map.get(id)).map(List::size).orElse(0); - } - - int containerCount() { - return map.size(); - } - - boolean isFull() { - return inflightCount.get() >= sizeLimit; - } - - void clear() { - map.clear(); - } - - void iterate(ContainerID id, Predicate processor) { - for (; ;) { - final List actions = get(id); - if (actions == null) { - return; - } - synchronized (actions) { - if (get(id) != actions) { - continue; //actions is changed, retry - } - for (Iterator i = actions.iterator(); i.hasNext();) { - final boolean remove = processor.test(i.next()); - if (remove) { - i.remove(); - inflightCount.decrementAndGet(); - } - } - map.computeIfPresent(id, - (k, v) -> v == actions && v.isEmpty() ? null : v); - return; - } - } - } - - boolean add(ContainerID id, InflightAction a) { - final int previous = inflightCount.getAndUpdate( - n -> n < sizeLimit ? n + 1 : n); - if (previous >= sizeLimit) { - return false; - } - for (; ;) { - final List actions = map.computeIfAbsent(id, - key -> new LinkedList<>()); - synchronized (actions) { - if (get(id) != actions) { - continue; //actions is changed, retry - } - final boolean added = actions.add(a); - if (!added) { - inflightCount.decrementAndGet(); - } - return added; - } - } - } - - List getDatanodeDetails(ContainerID id) { - for (; ;) { - final List actions = get(id); - if (actions == null) { - return Collections.emptyList(); - } - synchronized (actions) { - if (get(id) != actions) { - continue; //actions is changed, retry - } - return actions.stream() - .map(InflightAction::getDatanode) - .collect(Collectors.toList()); - } - } - } - } - - /** - * Reference to the ContainerManager. - */ - private final ContainerManager containerManager; - - /** - * PlacementPolicy which is used to identify where a container - * should be replicated. - */ - private final PlacementPolicy containerPlacement; - - /** - * EventPublisher to fire Replicate and Delete container events. - */ - private final EventPublisher eventPublisher; - - /** - * SCMContext from StorageContainerManager. - */ - private final SCMContext scmContext; - - /** - * Used to lookup the health of a nodes or the nodes operational state. - */ - private final NodeManager nodeManager; - - /** - * This is used for tracking container replication commands which are issued - * by ReplicationManager and not yet complete. - */ - private final InflightMap inflightReplication; - - /** - * This is used for tracking container deletion commands which are issued - * by ReplicationManager and not yet complete. - */ - private final InflightMap inflightDeletion; - - /** - * This is used for tracking container move commands - * which are not yet complete. - */ - private final Map> inflightMoveFuture; - - /** - * ReplicationManager specific configuration. - */ - private final ReplicationManagerConfiguration rmConf; - - /** - * Minimum number of replica in a healthy state for maintenance. - */ - private int minHealthyForMaintenance; - - private final Clock clock; - - /** - * Current container size as a bound for choosing datanodes with - * enough space for a replica. - */ - private long currentContainerSize; - - /** - * Replication progress related metrics. - */ - private ReplicationManagerMetrics metrics; - - /** - * scheduler move option. - */ - private final MoveScheduler moveScheduler; - - - /** - * Constructs ReplicationManager instance with the given configuration. - * - * @param conf OzoneConfiguration - * @param containerManager ContainerManager - * @param containerPlacement PlacementPolicy - * @param eventPublisher EventPublisher - */ - @SuppressWarnings("parameternumber") - public LegacyReplicationManager(final ConfigurationSource conf, - final ContainerManager containerManager, - final PlacementPolicy containerPlacement, - final EventPublisher eventPublisher, - final SCMContext scmContext, - final NodeManager nodeManager, - final SCMHAManager scmhaManager, - final Clock clock, - final Table moveTable) - throws IOException { - this.containerManager = containerManager; - this.containerPlacement = containerPlacement; - this.eventPublisher = eventPublisher; - this.scmContext = scmContext; - this.nodeManager = nodeManager; - this.rmConf = conf.getObject(ReplicationManagerConfiguration.class); - LegacyReplicationManagerConfiguration legacyConf = conf - .getObject(LegacyReplicationManagerConfiguration.class); - this.inflightReplication = new InflightMap(InflightType.REPLICATION, - legacyConf.getContainerInflightReplicationLimit()); - this.inflightDeletion = new InflightMap(InflightType.DELETION, - legacyConf.getContainerInflightDeletionLimit()); - this.inflightMoveFuture = new ConcurrentHashMap<>(); - this.minHealthyForMaintenance = rmConf.getMaintenanceReplicaMinimum(); - this.clock = clock; - - this.currentContainerSize = (long) conf.getStorageSize( - ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE, - ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, - StorageUnit.BYTES); - this.metrics = null; - - moveScheduler = new MoveSchedulerImpl.Builder() - .setDBTransactionBuffer(scmhaManager.getDBTransactionBuffer()) - .setRatisServer(scmhaManager.getRatisServer()) - .setMoveTable(moveTable).build(); - } - - - protected synchronized void clearInflightActions() { - inflightReplication.clear(); - inflightDeletion.clear(); - } - - protected synchronized void setMetrics(ReplicationManagerMetrics metrics) { - this.metrics = metrics; - } - - /** - * Process the given container. - * - * @param container ContainerInfo - */ - @SuppressWarnings("checkstyle:methodlength") - protected void processContainer(ContainerInfo container, - ReplicationManagerReport report) { - final ContainerID id = container.containerID(); - try { - // synchronize on the containerInfo object to solve container - // race conditions with ICR/FCR handlers - synchronized (container) { - final Set replicas = containerManager - .getContainerReplicas(id); - final LifeCycleState state = container.getState(); - - /* - * We don't take any action if the container is in OPEN state and - * the container is healthy. If the container is not healthy, i.e. - * the replicas are not in OPEN state, send CLOSE_CONTAINER command. - */ - if (state == LifeCycleState.OPEN) { - if (!isOpenContainerHealthy(container, replicas)) { - report.incrementAndSample( - HealthState.OPEN_UNHEALTHY, container.containerID()); - eventPublisher.fireEvent(SCMEvents.CLOSE_CONTAINER, id); - } - return; - } - - /* - * If the container is in CLOSING state, the replicas can either - * be in OPEN or in CLOSING state. In both of this cases - * we have to resend close container command to the datanodes. - */ - if (state == LifeCycleState.CLOSING) { - setHealthStateForClosing(replicas, container, report); - boolean foundHealthy = false; - for (ContainerReplica replica: replicas) { - if (replica.getState() != State.UNHEALTHY) { - foundHealthy = true; - sendCloseCommand( - container, replica.getDatanodeDetails(), false); - } - } - - /* - * Empty containers in CLOSING state should be CLOSED. - * - * These are containers that are allocated in SCM but never got - * created on Datanodes. Since these containers don't have any - * replica associated with them, they are stuck in CLOSING state - * forever as there is no replicas to CLOSE. - */ - if (replicas.isEmpty() && (container.getNumberOfKeys() == 0)) { - closeEmptyContainer(container); - return; - } - - if (!foundHealthy) { - /* If we get here, then this container has replicas and all are - UNHEALTHY. Move it from CLOSING to QUASI_CLOSED so RM can then try - to maintain replication factor number of replicas. - */ - containerManager.updateContainerState(container.containerID(), - HddsProtos.LifeCycleEvent.QUASI_CLOSE); - LOG.debug("Moved container {} from CLOSING to QUASI_CLOSED " + - "because it has only UNHEALTHY replicas: {}.", container, - replicas); - } - - return; - } - - /* - * If the container is in QUASI_CLOSED state, check and close the - * container if possible. - */ - if (state == LifeCycleState.QUASI_CLOSED) { - if (canForceCloseContainer(container, replicas)) { - forceCloseContainer(container, replicas); - return; - } else { - report.incrementAndSample(HealthState.QUASI_CLOSED_STUCK, - container.containerID()); - } - } - - if (container.getReplicationType() == HddsProtos.ReplicationType.EC) { - // TODO We do not support replicating EC containers as yet, so at this - // point, after handing the closing etc states, we just return. - // EC Support will be added later. - return; - } - - /* - * Before processing the container we have to reconcile the - * inflightReplication and inflightDeletion actions. - * - * We remove the entry from inflightReplication and inflightDeletion - * list, if the operation is completed or if it has timed out. - */ - updateInflightAction(container, inflightReplication, - action -> replicas.stream().anyMatch( - r -> r.getDatanodeDetails().equals(action.getDatanode())), - () -> metrics.incrReplicaCreateTimeoutTotal(), - action -> updateCompletedReplicationMetrics(container, action)); - - updateInflightAction(container, inflightDeletion, - action -> replicas.stream().noneMatch( - r -> r.getDatanodeDetails().equals(action.getDatanode())), - () -> metrics.incrReplicaDeleteTimeoutTotal(), - action -> updateCompletedDeletionMetrics(container, action)); - - /* - * If container is under deleting and all it's replicas are deleted, - * then make the container as CLEANED, - * or resend the delete replica command if needed. - */ - if (state == LifeCycleState.DELETING) { - handleContainerUnderDelete(container, replicas); - return; - } - - /** - * We don't need to take any action for a DELETE container - eventually - * it will be removed from SCM. - */ - if (state == LifeCycleState.DELETED) { - return; - } - - RatisContainerReplicaCount replicaSet = - getContainerReplicaCount(container, replicas); - ContainerPlacementStatus placementStatus = getPlacementStatus( - replicas, container.getReplicationConfig().getRequiredNodes()); - - /* - * We don't have to take any action if the container is healthy. - * - * According to ReplicationMonitor container is considered healthy if - * the container is either in QUASI_CLOSED or in CLOSED state and has - * exact number of replicas in the same state. - */ - if (isContainerEmpty(container, replicas)) { - report.incrementAndSample( - HealthState.EMPTY, container.containerID()); - /* - * If container is empty, schedule task to delete the container. - */ - deleteContainerReplicas(container, replicas); - return; - } - - // If the container is empty and has no replicas, it is possible it was - // a container which stuck in the closing state which never got any - // replicas created on the datanodes. In this case, we don't have enough - // information to delete the container, so we just log it as EMPTY, - // leaving it as CLOSED and return true, otherwise, it will end up - // marked as missing by the under replication handling. - if (replicas.isEmpty() - && container.getState() == LifeCycleState.CLOSED - && container.getNumberOfKeys() == 0) { - LOG.debug("Container {} appears empty and is closed, but cannot be " + - "deleted because it has no replicas. Marking as EMPTY.", - container); - report.incrementAndSample(HealthState.EMPTY, container.containerID()); - return; - } - - /* - * Check if the container is under replicated and take appropriate - * action. - */ - boolean sufficientlyReplicated = replicaSet.isSufficientlyReplicated(); - boolean placementSatisfied = placementStatus.isPolicySatisfied(); - ContainerID containerID = container.containerID(); - if (!placementStatus.isPolicySatisfied()) { - report.incrementAndSample(HealthState.MIS_REPLICATED, containerID); - } - if (!replicaSet.isHealthy()) { - report.incrementAndSample(HealthState.UNHEALTHY, containerID); - } - if (!sufficientlyReplicated || !placementSatisfied) { - // Replicate container if needed. - if (!inflightReplication.isFull() || !inflightDeletion.isFull()) { - if (replicaSet.isUnrecoverable()) { - // There are no healthy or unhealthy replicas. - report.incrementAndSample(HealthState.MISSING, containerID); - report.incrementAndSample(HealthState.UNDER_REPLICATED, - containerID); - } else { - if (replicaSet.getHealthyReplicaCount() == 0 && - replicaSet.getUnhealthyReplicaCount() != 0) { - handleAllReplicasUnhealthy(container, replicaSet, - placementStatus, report); - } else { - handleUnderReplicatedHealthy(container, - replicaSet, placementStatus, report); - } - } - } - return; - } - - /* - * A QUASI_CLOSED container may have some UNHEALTHY replicas with the - * same Sequence ID as the container. RM should try to maintain one - * copy of such replicas when there are no healthy replicas that - * match the container's Sequence ID. - */ - List vulnerableUnhealthy = - replicaSet.getVulnerableUnhealthyReplicas(dn -> { - try { - return nodeManager.getNodeStatus(dn); - } catch (NodeNotFoundException e) { - LOG.warn("Exception for datanode {} while getting vulnerable replicas for container {}, with all " + - "replicas {}.", dn, container, replicas, e); - return null; - } - }); - if (!vulnerableUnhealthy.isEmpty()) { - report.incrementAndSample(HealthState.UNDER_REPLICATED, - container.containerID()); - handleVulnerableUnhealthyReplicas(replicaSet, vulnerableUnhealthy); - return; - } - - /* - * Check if the container is over replicated and take appropriate - * action. - */ - if (replicaSet.getReplicas().size() > - container.getReplicationConfig().getRequiredNodes()) { - if (replicaSet.isHealthy()) { - handleOverReplicatedHealthy(container, replicaSet, report); - } else { - handleOverReplicatedExcessUnhealthy(container, replicaSet, report); - } - return; - } - - /* - * If we get here, the container is not over replicated or under - * replicated, but it may be "unhealthy", which means it has one or - * more replica which are not in the same state as the container itself. - */ - if (!replicaSet.isHealthy()) { - handleContainerWithUnhealthyReplica(container, replicaSet); - } - } - } catch (ContainerNotFoundException ex) { - LOG.warn("Missing container {}.", id); - } catch (Exception ex) { - LOG.warn("Process container {} error: ", id, ex); - } - } - - /** - * Sends a replicate command for each replica specified in - * vulnerableUnhealthy. - * @param replicaCount RatisContainerReplicaCount for this container - * @param vulnerableUnhealthy List of UNHEALTHY replicas that need to be - * replicated - */ - private void handleVulnerableUnhealthyReplicas( - RatisContainerReplicaCount replicaCount, - List vulnerableUnhealthy) { - ContainerInfo container = replicaCount.getContainer(); - LOG.debug("Handling vulnerable UNHEALTHY replicas {} for container {}.", - vulnerableUnhealthy, container); - int pendingAdds = getInflightAdd(container.containerID()); - if (pendingAdds >= vulnerableUnhealthy.size()) { - LOG.debug("There are {} pending adds for container {}, while " + - "the number of UNHEALTHY replicas is {}.", pendingAdds, - container.containerID(), vulnerableUnhealthy.size()); - return; - } - - /* - Since we're replicating UNHEALTHY replicas, it's possible that - replication keeps on failing. Shuffling gives other replicas a chance to be - replicated since there's a limit on inflight adds. - */ - Collections.shuffle(vulnerableUnhealthy); - replicateEachSource(container, vulnerableUnhealthy, - replicaCount.getReplicas()); - } - - private void updateCompletedReplicationMetrics(ContainerInfo container, - InflightAction action) { - metrics.incrReplicasCreatedTotal(); - metrics.incrReplicationBytesCompletedTotal(container.getUsedBytes()); - metrics.addReplicationTime(clock.millis() - action.getTime()); - } - - private void updateCompletedDeletionMetrics(ContainerInfo container, - InflightAction action) { - metrics.incrReplicasDeletedTotal(); - metrics.incrDeletionBytesCompletedTotal(container.getUsedBytes()); - metrics.addDeletionTime(clock.millis() - action.getTime()); - } - - /** - * Reconciles the InflightActions for a given container. - * - * @param container Container to update - * @param inflightActions inflightReplication (or) inflightDeletion - * @param filter filter to check if the operation is completed - * @param timeoutCounter update timeout metrics - * @param completedCounter update completed metrics - */ - private void updateInflightAction(final ContainerInfo container, - final InflightMap inflightActions, - final Predicate filter, - final Runnable timeoutCounter, - final Consumer completedCounter) { - final ContainerID id = container.containerID(); - final long deadline = clock.millis() - rmConf.getEventTimeout(); - inflightActions.iterate(id, a -> updateInflightAction( - container, a, filter, timeoutCounter, completedCounter, - deadline, inflightActions.isReplication())); - } - - private boolean updateInflightAction(final ContainerInfo container, - final InflightAction a, - final Predicate filter, - final Runnable timeoutCounter, - final Consumer completedCounter, - final long deadline, - final boolean isReplication) { - boolean remove = false; - try { - final NodeStatus status = nodeManager.getNodeStatus(a.getDatanode()); - final boolean isUnhealthy = status.getHealth() != NodeState.HEALTHY; - final boolean isCompleted = filter.test(a); - final boolean isTimeout = a.getTime() < deadline; - final boolean isNotInService = status.getOperationalState() != - NodeOperationalState.IN_SERVICE; - if (isCompleted || isUnhealthy || isTimeout || isNotInService) { - if (isTimeout) { - timeoutCounter.run(); - } else if (isCompleted) { - completedCounter.accept(a); - } - - updateMoveIfNeeded(isUnhealthy, isCompleted, isTimeout, - isNotInService, container, a.getDatanode(), isReplication); - remove = true; - } - } catch (NodeNotFoundException | ContainerNotFoundException e) { - // Should not happen, but if it does, just remove the action as the - // node somehow does not exist; - remove = true; - } catch (Exception e) { - LOG.error("Got exception while updating.", e); - } - return remove; - } - - /** - * update inflight move if needed. - * - * @param isUnhealthy is the datanode unhealthy - * @param isCompleted is the action completed - * @param isTimeout is the action timeout - * @param container Container to update - * @param dn datanode which is removed from the inflightActions - * @param isInflightReplication is inflightReplication? - */ - private void updateMoveIfNeeded(final boolean isUnhealthy, - final boolean isCompleted, final boolean isTimeout, - final boolean isNotInService, - final ContainerInfo container, final DatanodeDetails dn, - final boolean isInflightReplication) - throws SCMException { - // make sure inflightMove contains the container - ContainerID id = container.containerID(); - - // make sure the datanode , which is removed from inflightActions, - // is source or target datanode. - MoveDataNodePair kv = moveScheduler.getMoveDataNodePair(id); - if (kv == null) { - return; - } - final boolean isSource = kv.getSrc().equals(dn); - final boolean isTarget = kv.getTgt().equals(dn); - if (!isSource && !isTarget) { - return; - } - - /* - * there are some case: - ********************************************************** - * * InflightReplication * InflightDeletion * - ********************************************************** - *source removed* unexpected * expected * - ********************************************************** - *target removed* expected * unexpected * - ********************************************************** - * unexpected action may happen somehow. to make it deterministic, - * if unexpected action happens, we just fail the completableFuture. - */ - - if (isSource && isInflightReplication) { - //if RM is reinitialize, inflightMove will be restored, - //but inflightMoveFuture not. so there will be a case that - //container is in inflightMove, but not in inflightMoveFuture. - compleleteMoveFutureWithResult(id, - MoveManager.MoveResult.FAIL_UNEXPECTED_ERROR); - LOG.info("Move failed because replication for container {} " + - "unexpectedly happened at the source {}, not the target {}.", - container, kv.getSrc().getUuidString(), kv.getTgt().getUuidString()); - moveScheduler.completeMove(id.getProtobuf()); - return; - } - - if (isTarget && !isInflightReplication) { - compleleteMoveFutureWithResult(id, - MoveManager.MoveResult.FAIL_UNEXPECTED_ERROR); - LOG.info("Move failed because deletion for container {} unexpectedly " + - "happened at the target {}, not the source {}.", container, - kv.getTgt().getUuidString(), kv.getSrc().getUuidString()); - moveScheduler.completeMove(id.getProtobuf()); - return; - } - - if (!(isInflightReplication && isCompleted)) { - if (isInflightReplication) { - if (isUnhealthy) { - compleleteMoveFutureWithResult(id, - MoveManager.MoveResult.REPLICATION_FAIL_NODE_UNHEALTHY); - } else if (isNotInService) { - compleleteMoveFutureWithResult(id, - MoveManager.MoveResult.REPLICATION_FAIL_NODE_NOT_IN_SERVICE); - } else { - compleleteMoveFutureWithResult(id, - MoveManager.MoveResult.REPLICATION_FAIL_TIME_OUT); - } - } else { - if (isUnhealthy) { - compleleteMoveFutureWithResult(id, - MoveManager.MoveResult.DELETION_FAIL_NODE_UNHEALTHY); - } else if (isTimeout) { - compleleteMoveFutureWithResult(id, - MoveManager.MoveResult.DELETION_FAIL_TIME_OUT); - } else if (isNotInService) { - compleleteMoveFutureWithResult(id, - MoveManager.MoveResult.DELETION_FAIL_NODE_NOT_IN_SERVICE); - } else { - compleleteMoveFutureWithResult(id, MoveManager.MoveResult.COMPLETED); - } - } - moveScheduler.completeMove(id.getProtobuf()); - } else { - deleteSrcDnForMove(container, - containerManager.getContainerReplicas(id)); - } - } - - /** - * add a move action for a given container. - * - * @param cid Container to move - * @param src source datanode - * @param tgt target datanode - */ - public CompletableFuture move(ContainerID cid, - DatanodeDetails src, DatanodeDetails tgt) - throws ContainerNotFoundException, NodeNotFoundException { - return move(cid, new MoveDataNodePair(src, tgt)); - } - - /** - * add a move action for a given container. - * - * @param cid Container to move - * @param mp MoveDataNodePair which contains source and target datanodes - */ - private CompletableFuture move(ContainerID cid, - MoveDataNodePair mp) throws ContainerNotFoundException, - NodeNotFoundException { - CompletableFuture ret = new CompletableFuture<>(); - - if (!scmContext.isLeader()) { - ret.complete(MoveManager.MoveResult.FAIL_LEADER_NOT_READY); - return ret; - } - - /* - * make sure the flowing conditions are met: - * 1 the given two datanodes are in healthy state - * 2 the given container exists on the given source datanode - * 3 the given container does not exist on the given target datanode - * 4 the given container is in closed state - * 5 the giver container is not taking any inflight action - * 6 the given two datanodes are in IN_SERVICE state - * 7 {Existing replicas + Target_Dn - Source_Dn} satisfies - * the placement policy - * - * move is a combination of two steps : replication and deletion. - * if the conditions above are all met, then we take a conservative - * strategy here : replication can always be executed, but the execution - * of deletion always depends on placement policy - */ - - DatanodeDetails srcDn = mp.getSrc(); - DatanodeDetails targetDn = mp.getTgt(); - NodeStatus currentNodeStat = nodeManager.getNodeStatus(srcDn); - NodeState healthStat = currentNodeStat.getHealth(); - NodeOperationalState operationalState = - currentNodeStat.getOperationalState(); - if (healthStat != NodeState.HEALTHY) { - ret.complete(MoveManager.MoveResult.REPLICATION_FAIL_NODE_UNHEALTHY); - LOG.info("Failing move for container {} because source {} is {}", cid, - srcDn.getUuidString(), healthStat.toString()); - return ret; - } - if (operationalState != NodeOperationalState.IN_SERVICE) { - ret.complete(MoveManager.MoveResult.REPLICATION_FAIL_NODE_NOT_IN_SERVICE); - LOG.info("Failing move for container {} because source {} is {}", cid, - srcDn.getUuidString(), operationalState.toString()); - return ret; - } - - currentNodeStat = nodeManager.getNodeStatus(targetDn); - healthStat = currentNodeStat.getHealth(); - operationalState = currentNodeStat.getOperationalState(); - if (healthStat != NodeState.HEALTHY) { - ret.complete(MoveManager.MoveResult.REPLICATION_FAIL_NODE_UNHEALTHY); - LOG.info("Failing move for container {} because target {} is {}", cid, - targetDn.getUuidString(), healthStat.toString()); - return ret; - } - if (operationalState != NodeOperationalState.IN_SERVICE) { - ret.complete(MoveManager.MoveResult.REPLICATION_FAIL_NODE_NOT_IN_SERVICE); - LOG.info("Failing move for container {} because target {} is {}", cid, - targetDn.getUuidString(), operationalState.toString()); - return ret; - } - - // we need to synchronize on ContainerInfo, since it is - // shared by ICR/FCR handler and this.processContainer - // TODO: use a Read lock after introducing a RW lock into ContainerInfo - ContainerInfo cif = containerManager.getContainer(cid); - synchronized (cif) { - final Set currentReplicas = containerManager - .getContainerReplicas(cid); - final Set replicas = currentReplicas.stream() - .map(ContainerReplica::getDatanodeDetails) - .collect(Collectors.toSet()); - if (replicas.contains(targetDn)) { - ret.complete(MoveManager.MoveResult.REPLICATION_FAIL_EXIST_IN_TARGET); - return ret; - } - if (!replicas.contains(srcDn)) { - ret.complete( - MoveManager.MoveResult.REPLICATION_FAIL_NOT_EXIST_IN_SOURCE); - return ret; - } - - /* - * the reason why the given container should not be taking any inflight - * action is that: if the given container is being replicated or deleted, - * the num of its replica is not deterministic, so move operation issued - * by balancer may cause a nondeterministic result, so we should drop - * this option for this time. - * */ - - if (inflightReplication.containsKey(cid)) { - ret.complete( - MoveManager.MoveResult.REPLICATION_FAIL_INFLIGHT_REPLICATION); - return ret; - } - if (inflightDeletion.containsKey(cid)) { - ret.complete(MoveManager.MoveResult.REPLICATION_FAIL_INFLIGHT_DELETION); - return ret; - } - - /* - * here, no need to see whether cid is in inflightMove, because - * these three map are all synchronized on ContainerInfo, if cid - * is in infligtMove , it must now being replicated or deleted, - * so it must be in inflightReplication or in infligthDeletion. - * thus, if we can not find cid in both of them , this cid must - * not be in inflightMove. - */ - - LifeCycleState currentContainerStat = cif.getState(); - if (currentContainerStat != LifeCycleState.CLOSED) { - ret.complete( - MoveManager.MoveResult.REPLICATION_FAIL_CONTAINER_NOT_CLOSED); - return ret; - } - - // check whether {Existing replicas + Target_Dn - Source_Dn} - // satisfies current placement policy - if (!isPolicySatisfiedAfterMove(cif, srcDn, targetDn, - new ArrayList<>(currentReplicas))) { - ret.complete(MoveManager.MoveResult.REPLICATION_NOT_HEALTHY_AFTER_MOVE); - return ret; - } - - try { - moveScheduler.startMove(cid.getProtobuf(), - mp.getProtobufMessage(ClientVersion.CURRENT_VERSION)); - } catch (IOException e) { - LOG.warn("Exception while starting move for container {}", cid, e); - ret.complete(MoveManager.MoveResult.FAIL_UNEXPECTED_ERROR); - return ret; - } - - inflightMoveFuture.putIfAbsent(cid, ret); - sendReplicateCommand(cif, targetDn, Collections.singletonList(srcDn)); - } - LOG.info("receive a move request about container {} , from {} to {}", - cid, srcDn.getUuid(), targetDn.getUuid()); - return ret; - } - - /** - * Returns whether {Existing replicas + Target_Dn - Source_Dn} - * satisfies current placement policy. - * @param cif Container Info of moved container - * @param srcDn DatanodeDetails of source data node - * @param targetDn DatanodeDetails of target data node - * @param replicas container replicas - * @return whether the placement policy is satisfied after move - */ - private boolean isPolicySatisfiedAfterMove(ContainerInfo cif, - DatanodeDetails srcDn, DatanodeDetails targetDn, - final List replicas) { - Set movedReplicas = new HashSet<>(replicas); - movedReplicas.removeIf(r -> r.getDatanodeDetails().equals(srcDn)); - movedReplicas.add(ContainerReplica.newBuilder() - .setDatanodeDetails(targetDn) - .setContainerID(cif.containerID()) - .setContainerState(State.CLOSED).build()); - ContainerPlacementStatus placementStatus = getPlacementStatus( - movedReplicas, cif.getReplicationConfig().getRequiredNodes()); - return placementStatus.isPolicySatisfied(); - } - - /** - * Returns the number replica which are pending creation for the given - * container ID. - * @param id The ContainerID for which to check the pending replica - * @return The number of inflight additions or zero if none - */ - private int getInflightAdd(final ContainerID id) { - return inflightReplication.inflightActionCount(id); - } - - /** - * Returns the number replica which are pending delete for the given - * container ID. - * @param id The ContainerID for which to check the pending replica - * @return The number of inflight deletes or zero if none - */ - private int getInflightDel(final ContainerID id) { - return inflightDeletion.inflightActionCount(id); - } - - /** - * Returns true if the container is empty and CLOSED. - * A container is deemed empty if its keyCount (num of blocks) is 0. The - * usedBytes counter is not checked here because usedBytes is not a - * accurate representation of the committed blocks. There could be orphaned - * chunks in the container which contribute to the usedBytes. - * - * @param container Container to check - * @param replicas Set of ContainerReplicas - * @return true if the container is empty, false otherwise - */ - private boolean isContainerEmpty(final ContainerInfo container, - final Set replicas) { - return container.getState() == LifeCycleState.CLOSED && - !replicas.isEmpty() && - replicas.stream().allMatch( - r -> r.getState() == State.CLOSED && r.isEmpty()); - } - - /** - * Given a ContainerID, lookup the ContainerInfo and then return a - * ContainerReplicaCount object for the container. - * @param containerID The ID of the container - * @return ContainerReplicaCount for the given container - * @throws ContainerNotFoundException - */ - public ContainerReplicaCount getContainerReplicaCount(ContainerID containerID) - throws ContainerNotFoundException { - ContainerInfo container = containerManager.getContainer(containerID); - return getContainerReplicaCount(container); - } - - /** - * Given a container, obtain the set of known replica for it, and return a - * ContainerReplicaCount object. This object will contain the set of replica - * as well as all information required to determine if the container is over - * or under replicated, including the delta of replica required to repair the - * over or under replication. - * - * @param container The container to create a ContainerReplicaCount for - * @return ContainerReplicaCount representing the replicated state of the - * container. - * @throws ContainerNotFoundException - */ - public ContainerReplicaCount getContainerReplicaCount(ContainerInfo container) - throws ContainerNotFoundException { - // TODO: using a RW lock for only read - synchronized (container) { - final Set replica = containerManager - .getContainerReplicas(container.containerID()); - return getReplicaCountOptionallyConsiderUnhealthy(container, replica); - } - } - - /** - * Given a container and its set of replicas, create and return a - * ContainerReplicaCount representing the container. - * - * @param container The container for which to construct a - * ContainerReplicaCount - * @param replica The set of existing replica for this container - * @return ContainerReplicaCount representing the current state of the - * container - */ - private RatisContainerReplicaCount getContainerReplicaCount( - ContainerInfo container, Set replica) { - return new LegacyRatisContainerReplicaCount( - container, - replica, - getInflightAdd(container.containerID()), - getInflightDel(container.containerID()), - container.getReplicationConfig().getRequiredNodes(), - minHealthyForMaintenance); - } - - private RatisContainerReplicaCount getReplicaCountOptionallyConsiderUnhealthy( - ContainerInfo container, Set replicas) { - LegacyRatisContainerReplicaCount withUnhealthy = - new LegacyRatisContainerReplicaCount(container, replicas, - getPendingOps(container.containerID()), minHealthyForMaintenance, - true); - if (withUnhealthy.getHealthyReplicaCount() == 0 && - withUnhealthy.getUnhealthyReplicaCount() > 0) { - // if the container has only UNHEALTHY replicas, return the correct - // RatisContainerReplicaCount object which can handle UNHEALTHY replicas - return withUnhealthy; - } - - return new LegacyRatisContainerReplicaCount( - container, - replicas, - getInflightAdd(container.containerID()), - getInflightDel(container.containerID()), - container.getReplicationConfig().getRequiredNodes(), - minHealthyForMaintenance); - } - - /** - * Returns true if more than 50% of the container replicas with unique - * originNodeId are in QUASI_CLOSED state. - * - * @param container Container to check - * @param replicas Set of ContainerReplicas - * @return true if we can force close the container, false otherwise - */ - private boolean canForceCloseContainer(final ContainerInfo container, - final Set replicas) { - Preconditions.assertTrue(container.getState() == - LifeCycleState.QUASI_CLOSED); - final int replicationFactor = - container.getReplicationConfig().getRequiredNodes(); - final long uniqueQuasiClosedReplicaCount = replicas.stream() - .filter(r -> r.getState() == State.QUASI_CLOSED) - .map(ContainerReplica::getOriginDatanodeId) - .distinct() - .count(); - return uniqueQuasiClosedReplicaCount > (replicationFactor / 2); - } - - /** - * Delete the container and its replicas. - * - * @param container ContainerInfo - * @param replicas Set of ContainerReplicas - */ - private void deleteContainerReplicas(final ContainerInfo container, - final Set replicas) throws IOException, - InvalidStateTransitionException { - Preconditions.assertTrue(container.getState() == - LifeCycleState.CLOSED); - - replicas.stream().forEach(rp -> { - Preconditions.assertTrue(rp.getState() == State.CLOSED); - Preconditions.assertTrue(rp.isEmpty()); - sendDeleteCommand(container, rp.getDatanodeDetails(), false); - }); - containerManager.updateContainerState(container.containerID(), - HddsProtos.LifeCycleEvent.DELETE); - LOG.debug("Deleting empty container replicas for {},", container); - } - - /** - * Handle the container which is under delete. - * - * @param container ContainerInfo - * @param replicas Set of ContainerReplicas - */ - private void handleContainerUnderDelete(final ContainerInfo container, - final Set replicas) throws IOException, - InvalidStateTransitionException { - if (replicas.size() == 0) { - containerManager.updateContainerState(container.containerID(), - HddsProtos.LifeCycleEvent.CLEANUP); - LOG.debug("Container {} state changes to DELETED", container); - } else { - // Check whether to resend the delete replica command - final List deletionInFlight - = inflightDeletion.getDatanodeDetails(container.containerID()); - Set filteredReplicas = replicas.stream().filter( - r -> !deletionInFlight.contains(r.getDatanodeDetails())) - .collect(Collectors.toSet()); - // Resend the delete command - if (filteredReplicas.size() > 0) { - filteredReplicas.stream().forEach(rp -> { - sendDeleteCommand(container, rp.getDatanodeDetails(), false); - }); - LOG.debug("Resend delete Container command for {}", container); - } - } - } - - /** - * Force close the container replica(s) with highest sequence Id. - * - *

- * Note: We should force close the container only if >50% (quorum) - * of replicas with unique originNodeId are in QUASI_CLOSED state. - *

- * - * @param container ContainerInfo - * @param replicas Set of ContainerReplicas - */ - private void forceCloseContainer(final ContainerInfo container, - final Set replicas) { - Preconditions.assertTrue(container.getState() == - LifeCycleState.QUASI_CLOSED); - - final List quasiClosedReplicas = replicas.stream() - .filter(r -> r.getState() == State.QUASI_CLOSED) - .collect(Collectors.toList()); - - final Long sequenceId = quasiClosedReplicas.stream() - .map(ContainerReplica::getSequenceId) - .max(Long::compare) - .orElse(-1L); - - LOG.info("Force closing container {} with BCSID {}," + - " which is in QUASI_CLOSED state.", - container.containerID(), sequenceId); - - quasiClosedReplicas.stream() - .filter(r -> sequenceId != -1L) - .filter(replica -> replica.getSequenceId().equals(sequenceId)) - .forEach(replica -> sendCloseCommand( - container, replica.getDatanodeDetails(), true)); - } - - /** - * If the given container is under replicated, identify a new set of - * datanode(s) to replicate the container using PlacementPolicy - * and send replicate container command to the identified datanode(s). - * - * @param container ContainerInfo - * @param replicaSet An instance of ContainerReplicaCount, containing the - * current replica count and inflight adds and deletes - */ - private void handleUnderReplicatedHealthy(final ContainerInfo container, - final RatisContainerReplicaCount replicaSet, - final ContainerPlacementStatus placementStatus, - ReplicationManagerReport report) { - LOG.debug("Handling under-replicated container: {}", container); - if (replicaSet.isSufficientlyReplicated() - && placementStatus.isPolicySatisfied()) { - LOG.info("The container {} with replicas {} is sufficiently " + - "replicated and is not mis-replicated", - container.getContainerID(), replicaSet); - return; - } - - List allReplicas = replicaSet.getReplicas(); - int numCloseCommandsSent = closeReplicasIfPossible(container, allReplicas); - int replicasNeeded = - replicaSet.additionalReplicaNeeded() - numCloseCommandsSent; - - if (replicasNeeded > 0) { - report.incrementAndSample(HealthState.UNDER_REPLICATED, - container.containerID()); - } - - State matchingReplicaState = State.CLOSED; - if (container.getState() == LifeCycleState.QUASI_CLOSED) { - // If we are replicating quasi closed replicas, they should have the - // same origin node ID and therefore the same BCSID. If they have - // different origin node IDs, then we have 2/3 containers and it should - // have been closed before replicating. - matchingReplicaState = State.QUASI_CLOSED; - } - List replicationSources = getReplicationSources(container, - replicaSet.getReplicas(), matchingReplicaState); - // This method will handle topology even if replicasNeeded <= 0. - try { - replicateAnyWithTopology(container, replicationSources, - placementStatus, replicasNeeded, replicaSet.getReplicas()); - } catch (SCMException e) { - if (e.getResult() - .equals(SCMException.ResultCodes.FAILED_TO_FIND_SUITABLE_NODE) && - replicasNeeded > 0) { - /* - If we reach here, the container is under replicated but placement - policy could not find any target Datanodes to host new replicas. - We can try unblocking under replication handling by removing any - unhealthy replicas. This will free up those datanodes, so they can host - healthy replicas. - */ - deleteUnhealthyReplicaIfNeeded(container, replicaSet); - } - } - } - - /** - * Finds and deletes an unhealthy replica (UNHEALTHY or QUASI_CLOSED) under - * certain conditions. - */ - private void deleteUnhealthyReplicaIfNeeded(ContainerInfo container, - RatisContainerReplicaCount replicaCount) { - LOG.info("Finding an unhealthy replica to delete for container {} with " + - "replicas {} to unblock under replication handling.", container, - replicaCount.getReplicas()); - - Set replicas = new HashSet<>(replicaCount.getReplicas()); - ContainerReplica replica = ReplicationManagerUtil - .selectUnhealthyReplicaForDelete(container, replicas, - getInflightDel(container.containerID()), - (dnd) -> { - try { - return nodeManager.getNodeStatus(dnd); - } catch (NodeNotFoundException e) { - LOG.warn("Exception while finding an unhealthy replica to " + - "delete for container {}.", container, e); - return null; - } - }); - - if (replica == null) { - LOG.info( - "Could not find any unhealthy replica to delete when unblocking " + - "under replication handling for container {} with replicas {}.", - container, replicas); - } else { - sendDeleteCommand(container, replica.getDatanodeDetails(), true); - } - } - - /** - * If the given container is over replicated, identify the datanode(s) - * to delete the container and send delete container command to the - * identified datanode(s). This method ignores unhealthy containers. - * - * @param container ContainerInfo - * @param replicaSet An instance of ContainerReplicaCount, containing the - * current replica count and inflight adds and deletes - */ - private void handleOverReplicatedHealthy(final ContainerInfo container, - final RatisContainerReplicaCount replicaSet, - ReplicationManagerReport report) { - - final ContainerID id = container.containerID(); - final int replicationFactor = - container.getReplicationConfig().getRequiredNodes(); - int excess = replicaSet.additionalReplicaNeeded() * -1; - if (excess > 0) { - LOG.info("Container {} is over replicated. Expected replica count" + - " is {}, but found {}.", id, replicationFactor, - replicationFactor + excess); - - report.incrementAndSample(HealthState.OVER_REPLICATED, - container.containerID()); - - // The list of replicas that we can potentially delete to fix the over - // replicated state. This method is only concerned with healthy replicas. - final List deleteCandidates = - getHealthyDeletionCandidates(container, replicaSet.getReplicas()); - - if (container.getState() == LifeCycleState.CLOSED) { - // Container is closed, so all healthy replicas are equal. - // We can choose which ones to delete based on topology. - // TODO Legacy RM implementation can only handle topology when all - // container replicas are closed and equal. - deleteExcessWithTopology(excess, container, deleteCandidates); - } else { - // Container is not yet closed. Choose which healthy replicas to - // delete so that we do not lose any origin node IDs. - deleteExcessWithNonUniqueOriginNodeIDs(container, - replicaSet.getReplicas(), - deleteCandidates, excess); - } - } - } - - /** - * Processes replicas of the container when all replicas are unhealthy (in - * a state that does not match the container state). - * - * Unhealthy replicas will first be checked to see if they can be closed. - * If there are more unhealthy replicas than required, some may be deleted. - * If there are fewer unhealthy replicas than required, some may be - * replicated. - */ - private void handleAllReplicasUnhealthy(ContainerInfo container, - RatisContainerReplicaCount replicaSet, - ContainerPlacementStatus placementStatus, - ReplicationManagerReport report) { - - List replicas = replicaSet.getReplicas(); - - RatisContainerReplicaCount unhealthyReplicaSet = - new LegacyRatisContainerReplicaCount(container, - new HashSet<>(replicaSet.getReplicas()), - getPendingOps(container.containerID()), - minHealthyForMaintenance, - true); - - if (unhealthyReplicaSet.isUnderReplicated()) { - handleUnderReplicatedAllUnhealthy(container, replicas, - placementStatus, unhealthyReplicaSet.additionalReplicaNeeded(), - report); - } else if (unhealthyReplicaSet.isOverReplicated()) { - handleOverReplicatedAllUnhealthy(container, replicas, - unhealthyReplicaSet.getExcessRedundancy(true), report); - } else { - // We have the correct number of unhealthy replicas. See if any of them - // can be closed. - closeReplicasIfPossible(container, replicas); - } - } - - /** - * Transform the Legacy inflight operation in the pendingOps format. - * @param containerID The contaiuner to get the pending ops for. - * @return A list of pendingOp, or an empty list if none exist. - */ - private List getPendingOps(ContainerID containerID) { - List pendingOps = new ArrayList<>(); - List inflightActions = inflightReplication.get(containerID); - if (inflightActions != null) { - for (InflightAction a : inflightActions) { - pendingOps.add(new ContainerReplicaOp( - ADD, a.getDatanode(), 0, Long.MAX_VALUE)); - } - } - inflightActions = inflightDeletion.get(containerID); - if (inflightActions != null) { - for (InflightAction a : inflightActions) { - pendingOps.add(new ContainerReplicaOp( - DELETE, a.getDatanode(), 0, Long.MAX_VALUE)); - } - } - return pendingOps; - } - - /** - * Handles a container which has the correct number of healthy replicas, - * but an excess of unhealthy replicas. - * - * If the container is closed, the unhealthy replicas can be deleted. If the - * container is not yet closed, the unhealthy replicas with non-unique - * origin node IDs can be deleted. - */ - private void handleOverReplicatedExcessUnhealthy( - final ContainerInfo container, - final RatisContainerReplicaCount replicaSet, - ReplicationManagerReport report) { - // Note - ReplicationManager would reach here only if the - // following conditions are met: - // 1. Container is in either CLOSED or QUASI-CLOSED state - // 2. We have adequate healthy replicas with extra unhealthy - // replicas. - - List replicas = replicaSet.getReplicas(); - List unhealthyReplicas = - getUnhealthyDeletionCandidates(container, replicas); - - // Only unhealthy replicas which cannot be closed will remain eligible - // for deletion, since this method is deleting unhealthy containers only. - closeReplicasIfPossible(container, unhealthyReplicas); - if (!unhealthyReplicas.isEmpty()) { - int excessReplicaCount = replicas.size() - - container.getReplicationConfig().getRequiredNodes(); - boolean excessDeleted = false; - if (container.getState() == LifeCycleState.CLOSED) { - // The container is already closed. The unhealthy replicas are extras - // and unnecessary. - deleteExcess(container, unhealthyReplicas, excessReplicaCount); - excessDeleted = true; - } else { - // Container is not yet closed. - // We only need to save the unhealthy replicas if they - // represent unique origin node IDs. If recovering these replicas is - // possible in the future they could be used to close the container. - excessDeleted = deleteExcessWithNonUniqueOriginNodeIDs(container, - replicaSet.getReplicas(), unhealthyReplicas, excessReplicaCount); - } - - if (excessDeleted) { - LOG.info("Container {} has {} excess unhealthy replicas. Excess " + - "unhealthy replicas will be deleted.", - container.getContainerID(), unhealthyReplicas.size()); - report.incrementAndSample(HealthState.OVER_REPLICATED, - container.containerID()); - } - } - } - - /** - * This method handles container with unhealthy replica by over-replicating - * the healthy replica. Once the container becomes over-replicated, - * we delete the unhealthy replica in the next cycle of replication manager - * in handleOverReplicatedExcessUnhealthy method. - */ - private void handleContainerWithUnhealthyReplica( - final ContainerInfo container, - final RatisContainerReplicaCount replicaSet) { - /* - * When there is an Unhealthy or Quasi Closed replica with incorrect - * sequence id for a Closed container, it should be deleted and one of - * the healthy replica has to be re-replicated. - * - * We first do the re-replication and over replicate the container, - * in the next cycle of replication manager the excess unhealthy replica - * is deleted. - */ - - if (container.getState() == LifeCycleState.CLOSED) { - final List replicas = replicaSet.getReplicas(); - final List replicationSources = getReplicationSources( - container, replicaSet.getReplicas(), State.CLOSED); - if (replicationSources.isEmpty()) { - LOG.warn("No healthy CLOSED replica for replication."); - return; - } - final ContainerPlacementStatus placementStatus = getPlacementStatus( - new HashSet<>(replicationSources), - container.getReplicationConfig().getRequiredNodes()); - try { - replicateAnyWithTopology(container, replicationSources, - placementStatus, replicas.size() - replicationSources.size(), - replicas); - } catch (SCMException e) { - LOG.warn("Could not fix container {} with replicas {}.", container, - replicas, e); - } - } - } - - /** - * Returns the replicas from {@code replicas} that: - * - Do not have in flight deletions - * - Exist on healthy datanodes - * - Have a replica state matching one of {@code validReplicaStates}. If - * this parameter is empty, any replica state is valid. - */ - private List getReplicationSources(ContainerInfo container, - List replicas, State... validReplicaStates) { - final List deletionInFlight - = inflightDeletion.getDatanodeDetails(container.containerID()); - final Set validReplicaStateSet = Arrays.stream(validReplicaStates) - .collect(Collectors.toSet()); - return replicas.stream() - // Exclude stale and dead nodes. This is particularly important for - // maintenance nodes, as the replicas will remain present in the - // container manager, even when they go dead. - .filter(r -> getNodeStatus(r.getDatanodeDetails()).isHealthy() - && !deletionInFlight.contains(r.getDatanodeDetails()) - && (validReplicaStateSet.isEmpty() || - validReplicaStateSet.contains(r.getState()))) - .collect(Collectors.toList()); - } - - private List getHealthyDeletionCandidates( - ContainerInfo container, List replicas) { - return getDeletionCandidates(container, replicas, true); - } - - private List getUnhealthyDeletionCandidates( - ContainerInfo container, List replicas) { - return getDeletionCandidates(container, replicas, false); - } - - /** - * A replica is eligible for deletion if its datanode is healthy and - * IN_SERVICE. - */ - private List getDeletionCandidates(ContainerInfo container, - List replicas, boolean healthy) { - return replicas.stream() - .filter(r -> getNodeStatus(r.getDatanodeDetails()).isHealthy() - && compareState(container.getState(), r.getState()) == healthy - && r.getDatanodeDetails().getPersistedOpState() == IN_SERVICE) - .collect(Collectors.toList()); - } - - /** - * if the container is in inflightMove, handle move. - * This function assumes replication has been completed - * - * @param cif ContainerInfo - * @param replicaSet An Set of replicas, which may have excess replicas - */ - private void deleteSrcDnForMove(final ContainerInfo cif, - final Set replicaSet) - throws SCMException { - final ContainerID cid = cif.containerID(); - MoveDataNodePair movePair = moveScheduler.getMoveDataNodePair(cid); - if (movePair == null) { - return; - } - final DatanodeDetails srcDn = movePair.getSrc(); - ContainerReplicaCount replicaCount = - getContainerReplicaCount(cif, replicaSet); - - if (!replicaSet.stream() - .anyMatch(r -> r.getDatanodeDetails().equals(srcDn))) { - // if the target is present but source disappears somehow, - // we can consider move is successful. - compleleteMoveFutureWithResult(cid, MoveManager.MoveResult.COMPLETED); - moveScheduler.completeMove(cid.getProtobuf()); - return; - } - - int replicationFactor = - cif.getReplicationConfig().getRequiredNodes(); - ContainerPlacementStatus currentCPS = - getPlacementStatus(replicaSet, replicationFactor); - Set newReplicaSet = new HashSet<>(replicaSet); - newReplicaSet.removeIf(r -> r.getDatanodeDetails().equals(srcDn)); - ContainerPlacementStatus newCPS = - getPlacementStatus(newReplicaSet, replicationFactor); - - if (replicaCount.isOverReplicated() && - isPlacementStatusActuallyEqual(currentCPS, newCPS)) { - sendDeleteCommand(cif, srcDn, true); - } else { - // if source and target datanode are both in the replicaset, - // but we can not delete source datanode for now (e.g., - // there is only 3 replicas or not policy-statisfied , etc.), - // we just complete the future without sending a delete command. - LOG.info("can not remove source replica after successfully " + - "replicated to target datanode"); - compleleteMoveFutureWithResult(cid, - MoveManager.MoveResult.DELETE_FAIL_POLICY); - moveScheduler.completeMove(cid.getProtobuf()); - } - } - - /** - * whether the given two ContainerPlacementStatus are actually equal. - * - * @param cps1 ContainerPlacementStatus - * @param cps2 ContainerPlacementStatus - */ - private boolean isPlacementStatusActuallyEqual( - ContainerPlacementStatus cps1, - ContainerPlacementStatus cps2) { - return (!cps1.isPolicySatisfied() && - cps1.actualPlacementCount() == cps2.actualPlacementCount()) || - cps1.isPolicySatisfied() && cps2.isPolicySatisfied(); - } - - /** - * Given a set of ContainerReplica, transform it to a list of DatanodeDetails - * and then check if the list meets the container placement policy. - * @param replicas List of containerReplica - * @param replicationFactor Expected Replication Factor of the containe - * @return ContainerPlacementStatus indicating if the policy is met or not - */ - private ContainerPlacementStatus getPlacementStatus( - Set replicas, int replicationFactor) { - List replicaDns = replicas.stream() - .map(ContainerReplica::getDatanodeDetails) - .collect(Collectors.toList()); - return containerPlacement.validateContainerPlacement( - replicaDns, replicationFactor); - } - - /** - * Sends close container command for the given container to the given - * datanode. - * - * @param container Container to be closed - * @param datanode The datanode on which the container - * has to be closed - * @param force Should be set to true if we want to close a - * QUASI_CLOSED container - */ - private void sendCloseCommand(final ContainerInfo container, - final DatanodeDetails datanode, - final boolean force) { - - ContainerID containerID = container.containerID(); - LOG.info("Sending close container command for container {}" + - " to datanode {}.", containerID, datanode); - CloseContainerCommand closeContainerCommand = - new CloseContainerCommand(container.getContainerID(), - container.getPipelineID(), force); - try { - closeContainerCommand.setTerm(scmContext.getTermOfLeader()); - } catch (NotLeaderException nle) { - LOG.warn("Skip sending close container command," - + " since current SCM is not leader.", nle); - return; - } - closeContainerCommand.setEncodedToken(getContainerToken(containerID)); - eventPublisher.fireEvent(SCMEvents.DATANODE_COMMAND, - new CommandForDatanode<>(datanode.getUuid(), closeContainerCommand)); - } - - private String getContainerToken(ContainerID containerID) { - if (scmContext.getScm() instanceof StorageContainerManager) { - StorageContainerManager scm = - (StorageContainerManager) scmContext.getScm(); - return scm.getContainerTokenGenerator().generateEncodedToken(containerID); - } - return ""; // unit test - } - - private boolean addInflight(InflightType type, ContainerID id, - InflightAction action) { - final boolean added = getInflightMap(type).add(id, action); - if (!added) { - metrics.incrInflightSkipped(type); - } - return added; - } - - /** - * Sends replicate container command for the given container to the given - * datanode. - * - * @param container Container to be replicated - * @param target The destination datanode to replicate - * @param sources List of source nodes from where we can replicate - */ - private void sendReplicateCommand(final ContainerInfo container, - final DatanodeDetails target, - final List sources) { - - final ContainerID id = container.containerID(); - final long containerID = id.getId(); - final ReplicateContainerCommand replicateCommand = - ReplicateContainerCommand.fromSources(containerID, sources); - LOG.debug("Trying to send {} to {}", replicateCommand, target); - - final boolean sent = sendAndTrackDatanodeCommand(target, replicateCommand, - action -> addInflight(InflightType.REPLICATION, id, action)); - - if (sent) { - LOG.info("Sent {} to {}", replicateCommand, target); - metrics.incrReplicationCmdsSentTotal(); - metrics.incrReplicationBytesTotal(container.getUsedBytes()); - } - } - - /** - * Sends delete container command for the given container to the given - * datanode. - * - * @param container Container to be deleted - * @param datanode The datanode on which the replica should be deleted - * @param force Should be set to true to delete an OPEN replica - */ - private void sendDeleteCommand(final ContainerInfo container, - final DatanodeDetails datanode, - final boolean force) { - - LOG.info("Sending delete container command for container {}" + - " to datanode {}", container.containerID(), datanode); - - final ContainerID id = container.containerID(); - final DeleteContainerCommand deleteCommand = - new DeleteContainerCommand(id.getId(), force); - final boolean sent = sendAndTrackDatanodeCommand(datanode, deleteCommand, - action -> addInflight(InflightType.DELETION, id, action)); - - if (sent) { - metrics.incrDeletionCmdsSentTotal(); - metrics.incrDeletionBytesTotal(container.getUsedBytes()); - } - } - - /** - * Creates CommandForDatanode with the given SCMCommand and fires - * DATANODE_COMMAND event to event queue. - * - * Tracks the command using the given tracker. - * - * @param datanode Datanode to which the command has to be sent - * @param command SCMCommand to be sent - * @param tracker Tracker which tracks the inflight actions - * @param Type of SCMCommand - */ - private boolean sendAndTrackDatanodeCommand( - final DatanodeDetails datanode, - final SCMCommand command, - final Predicate tracker) { - try { - command.setTerm(scmContext.getTermOfLeader()); - } catch (NotLeaderException nle) { - LOG.warn("Skip sending datanode command," - + " since current SCM is not leader.", nle); - return false; - } - final boolean allowed = tracker.test( - new InflightAction(datanode, clock.millis())); - if (!allowed) { - return false; - } - final CommandForDatanode datanodeCommand = - new CommandForDatanode<>(datanode.getUuid(), command); - eventPublisher.fireEvent(SCMEvents.DATANODE_COMMAND, datanodeCommand); - return true; - } - - /** - * Wrap the call to nodeManager.getNodeStatus, catching any - * NodeNotFoundException and instead throwing an IllegalStateException. - * @param dn The datanodeDetails to obtain the NodeStatus for - * @return NodeStatus corresponding to the given Datanode. - */ - private NodeStatus getNodeStatus(DatanodeDetails dn) { - try { - return nodeManager.getNodeStatus(dn); - } catch (NodeNotFoundException e) { - throw new IllegalStateException("Unable to find NodeStatus for " + dn, e); - } - } - - /** - * Compares the container state with the replica state. - * - * @param containerState ContainerState - * @param replicaState ReplicaState - * @return true if the state matches, false otherwise - */ - public static boolean compareState(final LifeCycleState containerState, - final State replicaState) { - switch (containerState) { - case OPEN: - return replicaState == State.OPEN; - case CLOSING: - return replicaState == State.CLOSING; - case QUASI_CLOSED: - return replicaState == State.QUASI_CLOSED; - case CLOSED: - return replicaState == State.CLOSED; - case DELETING: - return false; - case DELETED: - return false; - default: - return false; - } - } - - /** - * An open container is healthy if all its replicas are in the same state as - * the container. - * @param container The container to check - * @param replicas The replicas belonging to the container - * @return True if the container is healthy, false otherwise - */ - private boolean isOpenContainerHealthy( - ContainerInfo container, Set replicas) { - LifeCycleState state = container.getState(); - return replicas.stream() - .allMatch(r -> compareState(state, r.getState())); - } - - private void setHealthStateForClosing(Set replicas, - ContainerInfo container, - ReplicationManagerReport report) { - if (replicas.size() == 0) { - report.incrementAndSample(HealthState.MISSING, container.containerID()); - report.incrementAndSample(HealthState.UNDER_REPLICATED, - container.containerID()); - report.incrementAndSample(HealthState.MIS_REPLICATED, - container.containerID()); - } - } - - public boolean isContainerReplicatingOrDeleting(ContainerID containerID) { - return inflightReplication.containsKey(containerID) || - inflightDeletion.containsKey(containerID); - } - - /** - * Configuration used by the Replication Manager. - */ - @ConfigGroup(prefix = "hdds.scm.replication") - public static class LegacyReplicationManagerConfiguration { - - @Config(key = "container.inflight.replication.limit", - type = ConfigType.INT, - defaultValue = "0", // 0 means unlimited. - tags = {SCM, OZONE}, - description = "This property is used to limit" + - " the maximum number of inflight replication." - ) - private int containerInflightReplicationLimit = 0; - - @Config(key = "container.inflight.deletion.limit", - type = ConfigType.INT, - defaultValue = "0", // 0 means unlimited. - tags = {SCM, OZONE}, - description = "This property is used to limit" + - " the maximum number of inflight deletion." - ) - private int containerInflightDeletionLimit = 0; - - public void setContainerInflightReplicationLimit(int replicationLimit) { - this.containerInflightReplicationLimit = replicationLimit; - } - - public void setContainerInflightDeletionLimit(int deletionLimit) { - this.containerInflightDeletionLimit = deletionLimit; - } - - public int getContainerInflightReplicationLimit() { - return containerInflightReplicationLimit; - } - - public int getContainerInflightDeletionLimit() { - return containerInflightDeletionLimit; - } - - } - - protected void notifyStatusChanged() { - //now, as the current scm is leader and it`s state is up-to-date, - //we need to take some action about replicated inflight move options. - onLeaderReadyAndOutOfSafeMode(); - } - - private InflightMap getInflightMap(InflightType type) { - switch (type) { - case REPLICATION: return inflightReplication; - case DELETION: return inflightDeletion; - default: throw new IllegalStateException("Unexpected type " + type); - } - } - - int getInflightCount(InflightType type) { - return getInflightMap(type).containerCount(); - } - - DatanodeDetails getFirstDatanode(InflightType type, ContainerID id) { - return getInflightMap(type).get(id).get(0).getDatanode(); - } - - public Map> - getInflightMove() { - return inflightMoveFuture; - } - - /** - * make move option HA aware. - */ - public interface MoveScheduler { - /** - * completeMove a move action for a given container. - * - * @param contianerIDProto Container to which the move option is finished - */ - @Replicate - void completeMove(HddsProtos.ContainerID contianerIDProto) - throws SCMException; - - /** - * start a move action for a given container. - * - * @param contianerIDProto Container to move - * @param mp encapsulates the source and target datanode infos - */ - @Replicate - void startMove(HddsProtos.ContainerID contianerIDProto, - HddsProtos.MoveDataNodePairProto mp) - throws IOException; - - /** - * get the MoveDataNodePair of the giver container. - * - * @param cid Container to move - * @return null if cid is not found in MoveScheduler, - * or the corresponding MoveDataNodePair - */ - MoveDataNodePair getMoveDataNodePair(ContainerID cid); - - /** - * Reinitialize the MoveScheduler with DB if become leader. - */ - void reinitialize(Table moveTable) throws IOException; - - /** - * get all the inflight move info. - */ - Map getInflightMove(); - } - - /** - * @return the moveScheduler of RM - */ - public MoveScheduler getMoveScheduler() { - return moveScheduler; - } - - /** - * Ratis based MoveScheduler, db operations are stored in - * DBTransactionBuffer until a snapshot is taken. - */ - public static final class MoveSchedulerImpl implements MoveScheduler { - private Table moveTable; - private final DBTransactionBuffer transactionBuffer; - /** - * This is used for tracking container move commands - * which are not yet complete. - */ - private final Map inflightMove; - - private MoveSchedulerImpl(Table moveTable, - DBTransactionBuffer transactionBuffer) throws IOException { - this.moveTable = moveTable; - this.transactionBuffer = transactionBuffer; - this.inflightMove = new ConcurrentHashMap<>(); - initialize(); - } - - @Override - public void completeMove(HddsProtos.ContainerID contianerIDProto) { - ContainerID cid = null; - try { - cid = ContainerID.getFromProtobuf(contianerIDProto); - transactionBuffer.removeFromBuffer(moveTable, cid); - } catch (IOException e) { - LOG.warn("Exception while completing move {}", cid); - } - inflightMove.remove(cid); - } - - @Override - public void startMove(HddsProtos.ContainerID contianerIDProto, - HddsProtos.MoveDataNodePairProto mdnpp) - throws IOException { - ContainerID cid = null; - MoveDataNodePair mp = null; - try { - cid = ContainerID.getFromProtobuf(contianerIDProto); - mp = MoveDataNodePair.getFromProtobuf(mdnpp); - if (!inflightMove.containsKey(cid)) { - transactionBuffer.addToBuffer(moveTable, cid, mp); - inflightMove.putIfAbsent(cid, mp); - } - } catch (IOException e) { - LOG.warn("Exception while completing move {}", cid); - } - } - - @Override - public MoveDataNodePair getMoveDataNodePair(ContainerID cid) { - return inflightMove.get(cid); - } - - @Override - public void reinitialize(Table mt) throws IOException { - moveTable = mt; - inflightMove.clear(); - initialize(); - } - - private void initialize() throws IOException { - try (TableIterator> iterator = - moveTable.iterator()) { - - while (iterator.hasNext()) { - Table.KeyValue kv = iterator.next(); - final ContainerID cid = kv.getKey(); - final MoveDataNodePair mp = kv.getValue(); - Preconditions.assertNotNull(cid, - "moved container id should not be null"); - Preconditions.assertNotNull(mp, - "MoveDataNodePair container id should not be null"); - inflightMove.put(cid, mp); - } - } - } - - @Override - public Map getInflightMove() { - return inflightMove; - } - - /** - * Builder for Ratis based MoveSchedule. - */ - public static class Builder { - private Table moveTable; - private DBTransactionBuffer transactionBuffer; - private SCMRatisServer ratisServer; - - public Builder setRatisServer(final SCMRatisServer scmRatisServer) { - ratisServer = scmRatisServer; - return this; - } - - public Builder setMoveTable( - final Table mt) { - moveTable = mt; - return this; - } - - public Builder setDBTransactionBuffer(DBTransactionBuffer trxBuffer) { - transactionBuffer = trxBuffer; - return this; - } - - public MoveScheduler build() throws IOException { - Preconditions.assertNotNull(moveTable, "moveTable is null"); - Preconditions.assertNotNull(transactionBuffer, - "transactionBuffer is null"); - - final MoveScheduler impl = - new MoveSchedulerImpl(moveTable, transactionBuffer); - - final SCMHAInvocationHandler invocationHandler - = new SCMHAInvocationHandler(MOVE, impl, ratisServer); - - return (MoveScheduler) Proxy.newProxyInstance( - SCMHAInvocationHandler.class.getClassLoader(), - new Class[]{MoveScheduler.class}, - invocationHandler); - } - } - } - - /** - * when scm become LeaderReady and out of safe mode, some actions - * should be taken. for now , it is only used for handle replicated - * infligtht move. - */ - private void onLeaderReadyAndOutOfSafeMode() { - List needToRemove = new LinkedList<>(); - moveScheduler.getInflightMove().forEach((k, v) -> { - Set replicas; - ContainerInfo cif; - try { - replicas = containerManager.getContainerReplicas(k); - cif = containerManager.getContainer(k); - } catch (ContainerNotFoundException e) { - needToRemove.add(k.getProtobuf()); - LOG.error("can not find container {} " + - "while processing replicated move", k); - return; - } - boolean isSrcExist = replicas.stream() - .anyMatch(r -> r.getDatanodeDetails().equals(v.getSrc())); - boolean isTgtExist = replicas.stream() - .anyMatch(r -> r.getDatanodeDetails().equals(v.getTgt())); - - if (isSrcExist) { - if (isTgtExist) { - //the former scm leader may or may not send the deletion command - //before reelection.here, we just try to send the command again. - try { - deleteSrcDnForMove(cif, replicas); - } catch (Exception ex) { - LOG.error("Exception while cleaning up excess replicas.", ex); - } - } else { - // resenting replication command is ok , no matter whether there is an - // on-going replication - sendReplicateCommand(cif, v.getTgt(), - Collections.singletonList(v.getSrc())); - } - } else { - // if container does not exist in src datanode, no matter it exists - // in target datanode, we can not take more actions to this option, - // so just remove it through ratis - needToRemove.add(k.getProtobuf()); - } - }); - - for (HddsProtos.ContainerID containerID : needToRemove) { - try { - moveScheduler.completeMove(containerID); - } catch (Exception ex) { - LOG.error("Exception while moving container.", ex); - } - } - } - - /** - * complete the CompletableFuture of the container in the given Map with - * the given MoveManager.MoveResult. - */ - private void compleleteMoveFutureWithResult(ContainerID cid, - MoveManager.MoveResult mr) { - if (inflightMoveFuture.containsKey(cid)) { - inflightMoveFuture.get(cid).complete(mr); - inflightMoveFuture.remove(cid); - } - } - - private int closeReplicasIfPossible(ContainerInfo container, - List replicas) { - // This method should not be used on open containers. - if (container.getState() == LifeCycleState.OPEN) { - return 0; - } - - int numCloseCmdsSent = 0; - Iterator iterator = replicas.iterator(); - while (iterator.hasNext()) { - final ContainerReplica replica = iterator.next(); - final State state = replica.getState(); - if (state == State.OPEN || state == State.CLOSING) { - sendCloseCommand(container, replica.getDatanodeDetails(), false); - numCloseCmdsSent++; - iterator.remove(); - } else if (state == State.QUASI_CLOSED && - container.getState() == LifeCycleState.CLOSED) { - // Send force close command if the BCSID matches - if (container.getSequenceId() == replica.getSequenceId()) { - sendCloseCommand(container, replica.getDatanodeDetails(), true); - numCloseCmdsSent++; - iterator.remove(); - } - } - } - - return numCloseCmdsSent; - } - - /* HELPER METHODS FOR UNHEALTHY OVER AND UNDER REPLICATED CONTAINERS */ - - /** - * Process a container with more replicas than required where all replicas - * are unhealthy. - * - * First try to close any replicas that are unhealthy due to pending - * closure. Replicas that can be closed will become healthy and will not be - * processed by this method. - * If the container is closed, delete replicas with lower BCSIDs first. - * If the container is not yet closed, delete replicas with origin node IDs - * already represented by other replicas. - */ - private void handleOverReplicatedAllUnhealthy(ContainerInfo container, - List replicas, int excess, - ReplicationManagerReport report) { - List deleteCandidates = - getUnhealthyDeletionCandidates(container, replicas); - - // Only unhealthy replicas which cannot be closed will remain eligible - // for deletion, since this method is deleting unhealthy containers only. - closeReplicasIfPossible(container, deleteCandidates); - if (deleteCandidates.isEmpty()) { - return; - } - - if (excess > 0) { - boolean excessDeleted = false; - if (container.getState() == LifeCycleState.CLOSED) { - // Prefer to delete unhealthy replicas with lower BCS IDs. - // If the replica became unhealthy after the container was closed but - // before the replica could be closed, it may have a smaller BCSID. - deleteExcessLowestBcsIDs(container, deleteCandidates, excess); - excessDeleted = true; - } else { - // Container is not yet closed. - // We only need to save the unhealthy replicas if they - // represent unique origin node IDs. If recovering these replicas is - // possible in the future they could be used to close the container. - // If all excess replicas are unique then it is possible none of them - // are deleted. - excessDeleted = deleteExcessWithNonUniqueOriginNodeIDs(container, - replicas, deleteCandidates, excess); - } - - if (excessDeleted) { - report.incrementAndSample(HealthState.OVER_REPLICATED, - container.containerID()); - int replicationFactor = container.getReplicationFactor().getNumber(); - LOG.info("Container {} has all unhealthy replicas and is over " + - "replicated. Expected replica count" + - " is {}, but found {}.", container.getContainerID(), - replicationFactor, replicationFactor + excess); - } - } - } - - /** - * Processes container replicas when all replicas are unhealthy and there - * are fewer than the required number of replicas. - * - * If any of these replicas unhealthy because they are pending closure and - * they can be closed, close them to create a healthy replica that can be - * replicated. - * If none of the replicas can be closed, use one of the unhealthy replicas - * to restore replica count while satisfying topology requirements. - */ - private void handleUnderReplicatedAllUnhealthy(ContainerInfo container, - List replicas, ContainerPlacementStatus placementStatus, - int additionalReplicasNeeded, ReplicationManagerReport report) { - - report.incrementAndSample(HealthState.UNDER_REPLICATED, - container.containerID()); - int numCloseCmdsSent = closeReplicasIfPossible(container, replicas); - // Only replicate unhealthy containers if none of the unhealthy replicas - // could be closed. If we sent a close command to an unhealthy replica, - // we should wait for that to complete and replicate it when it becomes - // healthy on a future iteration. - if (numCloseCmdsSent == 0) { - LOG.info("Container {} is under replicated missing {} replicas with all" + - " replicas unhealthy. Copying unhealthy replicas.", - container.getContainerID(), additionalReplicasNeeded); - // TODO Datanodes currently shuffle sources, so we cannot prioritize - // some replicas based on BCSID or origin node ID. - try { - replicateAnyWithTopology(container, - getReplicationSources(container, replicas), placementStatus, - additionalReplicasNeeded, replicas); - } catch (SCMException e) { - LOG.warn("Could not fix container {} with replicas {}.", container, - replicas, e); - } - } - } - - /* HELPER METHODS FOR ALL OVER AND UNDER REPLICATED CONTAINERS */ - - /** - * Deletes the first {@code excess} replicas from {@code deleteCandidates}. - * Replicas whose datanode operation state is not IN_SERVICE will be skipped. - */ - private void deleteExcess(ContainerInfo container, - List deleteCandidates, int excess) { - // Replica which are maintenance or decommissioned are not eligible to - // be removed, as they do not count toward over-replication and they - // also may not be available - deleteCandidates.removeIf(r -> - r.getDatanodeDetails().getPersistedOpState() != - NodeOperationalState.IN_SERVICE); - - deleteCandidates.stream().limit(excess).forEach(r -> - sendDeleteCommand(container, r.getDatanodeDetails(), true)); - } - - /** - * remove execess replicas if needed, replicationFactor and placement policy - * will be take into consideration. - * - * @param excess the excess number after subtracting replicationFactor - * @param container ContainerInfo - * @param eligibleReplicas An list of replicas, which may have excess replicas - */ - private void deleteExcessWithTopology(int excess, - final ContainerInfo container, - final List eligibleReplicas) { - // After removing all unhealthy replicas, if the container is still over - // replicated then we need to check if it is already mis-replicated. - // If it is, we do no harm by removing excess replicas. However, if it is - // not mis-replicated, then we can only remove replicas if they don't - // make the container become mis-replicated. - if (excess > 0) { - Set eligibleSet = new HashSet<>(eligibleReplicas); - final int replicationFactor = - container.getReplicationConfig().getRequiredNodes(); - ContainerPlacementStatus ps = - getPlacementStatus(eligibleSet, replicationFactor); - - for (ContainerReplica r : eligibleReplicas) { - if (excess <= 0) { - break; - } - // First remove the replica we are working on from the set, and then - // check if the set is now mis-replicated. - eligibleSet.remove(r); - ContainerPlacementStatus nowPS = - getPlacementStatus(eligibleSet, replicationFactor); - if (isPlacementStatusActuallyEqual(ps, nowPS)) { - // Remove the replica if the container was already unsatisfied - // and losing this replica keep actual placement count unchanged. - // OR if losing this replica still keep satisfied - sendDeleteCommand(container, r.getDatanodeDetails(), true); - excess -= 1; - continue; - } - // If we decided not to remove this replica, put it back into the set - eligibleSet.add(r); - } - if (excess > 0) { - LOG.info("The container {} is over replicated with {} excess " + - "replica. The excess replicas cannot be removed without " + - "violating the placement policy", container, excess); - } - } - } - - /** - * @param container The container to operate on. - * @param allReplicas All replicas, providing all unique origin node IDs to - * this method. - * @param deleteCandidates The subset of allReplicas that are eligible for - * deletion. - * @param excess The maximum number of replicas to delete. If all origin - * node IDs are unique, no replicas may be deleted. - * @return True if replicas could be deleted. False otherwise. - */ - private boolean deleteExcessWithNonUniqueOriginNodeIDs( - ContainerInfo container, - List allReplicas, - List deleteCandidates, int excess) { - // Remove delete candidates whose origin node ID is not already covered - // by an existing replica. - // TODO topology handling must be improved to make an optimal - // choice as to which replica to keep. - Set allReplicasSet = new HashSet<>(allReplicas); - List nonUniqueDeleteCandidates = - ReplicationManagerUtil.findNonUniqueDeleteCandidates(allReplicasSet, - deleteCandidates, (dnd) -> { - try { - return nodeManager.getNodeStatus(dnd); - } catch (NodeNotFoundException e) { - LOG.warn( - "Exception while finding excess unhealthy replicas to " + - "delete for container {} with replicas {}.", container, - allReplicas, e); - return null; - } - }); - - if (LOG.isDebugEnabled() && nonUniqueDeleteCandidates.size() < excess) { - LOG.debug("Unable to delete {} excess replicas of container {}. Only {}" + - " replicas can be deleted to preserve unique origin node IDs for " + - "this unclosed container.", excess, container.getContainerID(), - nonUniqueDeleteCandidates.size()); - } - - boolean deleteCandidatesPresent = !nonUniqueDeleteCandidates.isEmpty(); - if (deleteCandidatesPresent) { - deleteExcess(container, nonUniqueDeleteCandidates, excess); - } - return deleteCandidatesPresent; - } - - /** - * Delete {@code excess} replicas from {@code deleteCandidates}, deleting - * those with lowest BCSIDs first. - */ - private void deleteExcessLowestBcsIDs(ContainerInfo container, - List deleteCandidates, int excess) { - // Sort containers with lowest BCSID first. These will be the first ones - // deleted. - deleteCandidates.sort( - Comparator.comparingLong(ContainerReplica::getSequenceId)); - deleteExcess(container, deleteCandidates, excess); - } - - /** - * Choose {@code additionalReplicasNeeded} datanodes to make copies of some - * of the container replicas to restore replication factor or satisfy - * topology requirements. - */ - private void replicateAnyWithTopology(ContainerInfo container, - List sourceReplicas, - ContainerPlacementStatus placementStatus, int additionalReplicasNeeded, - List allReplicas) - throws SCMException { - try { - final ContainerID id = container.containerID(); - - final List sourceDNs = sourceReplicas.stream() - .map(ContainerReplica::getDatanodeDetails) - .collect(Collectors.toList()); - final List replicationInFlight - = inflightReplication.getDatanodeDetails(id); - if (sourceDNs.size() > 0) { - final int replicationFactor = container - .getReplicationConfig().getRequiredNodes(); - // Want to check if the container is mis-replicated after considering - // inflight add and delete. - // Create a new list from source (healthy replicas minus pending delete) - List targetReplicas = new ArrayList<>(sourceDNs); - // Then add any pending additions - targetReplicas.addAll(replicationInFlight); - final ContainerPlacementStatus inFlightplacementStatus = - containerPlacement.validateContainerPlacement( - targetReplicas, replicationFactor); - final int misRepDelta = inFlightplacementStatus.misReplicationCount(); - final int replicasNeeded = - Math.max(additionalReplicasNeeded, misRepDelta); - if (replicasNeeded <= 0) { - LOG.debug("Container {} meets replication requirement with " + - "inflight replicas", id); - return; - } - - final List excludeList = allReplicas.stream() - .map(ContainerReplica::getDatanodeDetails) - .collect(Collectors.toList()); - excludeList.addAll(replicationInFlight); - final List selectedDatanodes = - ReplicationManagerUtil.getTargetDatanodes(containerPlacement, - replicasNeeded, null, excludeList, currentContainerSize, - container); - - if (additionalReplicasNeeded > 0) { - LOG.info("Container {} is under replicated. Expected replica count" + - " is {}, but found {}.", id, replicationFactor, - replicationFactor - additionalReplicasNeeded); - } - int newMisRepDelta = misRepDelta; - if (misRepDelta > 0) { - LOG.info("Container: {}. {}", - id, placementStatus.misReplicatedReason()); - // Check if the new target nodes (original plus newly selected nodes) - // makes the placement policy valid. - targetReplicas.addAll(selectedDatanodes); - newMisRepDelta = containerPlacement.validateContainerPlacement( - targetReplicas, replicationFactor).misReplicationCount(); - } - if (additionalReplicasNeeded > 0 || newMisRepDelta < misRepDelta) { - // Only create new replicas if we are missing a replicas or - // the number of pending mis-replication has improved. No point in - // creating new replicas for mis-replicated containers unless it - // improves things. - for (DatanodeDetails datanode : selectedDatanodes) { - sendReplicateCommand(container, datanode, sourceDNs); - } - } else { - LOG.warn("Container {} is mis-replicated, requiring {} additional " + - "replicas. After selecting new nodes, mis-replication has" + - "not improved. No additional replicas will be scheduled", - id, misRepDelta); - } - } else { - LOG.warn("Cannot replicate container {}, no healthy datanodes with " + - "replica found.", - container.containerID()); - } - } catch (IllegalStateException ex) { - LOG.warn("Exception while replicating container {}.", - container.getContainerID(), ex); - } - } - - /** - * Replicates each of the ContainerReplica specified in sources to new - * Datanodes. Will not consider Datanodes hosting existing replicas and - * Datanodes pending adds as targets. Note that this method simply skips - * the replica if there's an exception. - * @param container Container whose replicas are specified as sources - * @param sources List containing replicas, each will be replicated - * @param allReplicas all existing replicas of this container - */ - private void replicateEachSource(ContainerInfo container, - List sources, List allReplicas) { - final List excludeList = allReplicas.stream() - .map(ContainerReplica::getDatanodeDetails) - .collect(Collectors.toList()); - - for (ContainerReplica replica : sources) { - // also exclude any DNs pending to receive a replica of this container - final List replicationInFlight - = inflightReplication.getDatanodeDetails(container.containerID()); - for (DatanodeDetails dn : replicationInFlight) { - if (!excludeList.contains(dn)) { - excludeList.add(dn); - } - } - - try { - final List target = - ReplicationManagerUtil.getTargetDatanodes(containerPlacement, - 1, null, excludeList, currentContainerSize, - container); - sendReplicateCommand(container, target.iterator().next(), - ImmutableList.of(replica.getDatanodeDetails())); - } catch (SCMException e) { - LOG.warn("Exception while trying to replicate {} of container {}.", - replica, container, e); - } - } - } - - private void closeEmptyContainer(ContainerInfo containerInfo) { - /* - * We should wait for sometime before moving the container to CLOSED state. - * This will give enough time for Datanodes to report the container, - * in cases where the container creation was successful on Datanodes. - * - * Should we have a separate configuration for this wait time? - * For now, we are using ReplicationManagerThread Interval * 5 as the wait - * time. - */ - - final Duration waitTime = rmConf.getInterval().multipliedBy(5); - final Instant closingTime = containerInfo.getStateEnterTime(); - - try { - if (clock.instant().isAfter(closingTime.plus(waitTime))) { - containerManager.updateContainerState(containerInfo.containerID(), - HddsProtos.LifeCycleEvent.CLOSE); - } - } catch (IOException | InvalidStateTransitionException e) { - LOG.error("Failed to CLOSE the container {}", - containerInfo.containerID(), e); - } - } -} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisContainerReplicaCount.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisContainerReplicaCount.java index 4e14798ccdcc..cb2d906edba3 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisContainerReplicaCount.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisContainerReplicaCount.java @@ -223,11 +223,7 @@ private int getAvailableReplicas() { /** * The new replication manager now does not consider replicas with - * UNHEALTHY state when counting sufficient replication. This method is - * overridden to ensure LegacyReplicationManager works as intended in - * HDDS-6447. - * See {@link LegacyRatisContainerReplicaCount}, which overrides this - * method, for details. + * UNHEALTHY state when counting sufficient replication. */ protected int healthyReplicaCountAdapter() { return 0; diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java index b43caabd8d86..d183c876e956 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java @@ -42,7 +42,6 @@ import org.apache.hadoop.hdds.scm.container.ContainerReplica; import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport; -import org.apache.hadoop.hdds.scm.container.balancer.MoveManager; import org.apache.hadoop.hdds.scm.container.replication.health.ECMisReplicationCheckHandler; import org.apache.hadoop.hdds.scm.container.replication.health.MismatchedReplicasHandler; import org.apache.hadoop.hdds.scm.container.replication.health.ClosedWithUnhealthyReplicasHandler; @@ -85,10 +84,8 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -155,13 +152,6 @@ public class ReplicationManager implements SCMService { */ private ReplicationManagerMetrics metrics; - - /** - * Legacy RM will hopefully be removed after completing refactor - * for now, it is used to process non-EC container. - */ - private final LegacyReplicationManager legacyReplicationManager; - /** * Set of nodes which have been excluded for replication commands due to the * number of commands queued on a datanode. This can be used when generating @@ -214,7 +204,6 @@ public class ReplicationManager implements SCMService { * @param scmContext The SCMContext instance * @param nodeManager The nodeManager instance * @param clock Clock object used to get the current time - * @param legacyReplicationManager The legacy ReplicationManager instance * @param replicaPendingOps The pendingOps instance */ @SuppressWarnings("parameternumber") @@ -226,7 +215,6 @@ public ReplicationManager(final ConfigurationSource conf, final SCMContext scmContext, final NodeManager nodeManager, final Clock clock, - final LegacyReplicationManager legacyReplicationManager, final ContainerReplicaPendingOps replicaPendingOps) throws IOException { this.containerManager = containerManager; @@ -243,7 +231,6 @@ public ReplicationManager(final ConfigurationSource conf, HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT_DEFAULT, TimeUnit.MILLISECONDS); this.containerReplicaPendingOps = replicaPendingOps; - this.legacyReplicationManager = legacyReplicationManager; this.ecReplicationCheckHandler = new ECReplicationCheckHandler(); this.ecMisReplicationCheckHandler = new ECMisReplicationCheckHandler(ecContainerPlacement); @@ -296,9 +283,6 @@ public synchronized void start() { LOG.info("Starting Replication Monitor Thread."); running = true; metrics = ReplicationManagerMetrics.create(this); - if (rmConf.isLegacyEnabled()) { - legacyReplicationManager.setMetrics(metrics); - } containerReplicaPendingOps.setReplicationMetrics(metrics); startSubServices(); } else { @@ -330,9 +314,6 @@ public synchronized void stop() { underReplicatedProcessorThread.interrupt(); overReplicatedProcessorThread.interrupt(); running = false; - if (rmConf.isLegacyEnabled()) { - legacyReplicationManager.clearInflightActions(); - } metrics.unRegister(); replicationMonitor.interrupt(); } else { @@ -385,10 +366,6 @@ public synchronized void processAll() { break; } report.increment(c.getState()); - if (rmConf.isLegacyEnabled() && !isEC(c.getReplicationConfig())) { - legacyReplicationManager.processContainer(c, report); - continue; - } try { processContainer(c, newRepQueue, report); // TODO - send any commands contained in the health result @@ -955,12 +932,8 @@ public ContainerReplicaCount getContainerReplicaCount(ContainerID containerID) throws ContainerNotFoundException { ContainerInfo container = containerManager.getContainer(containerID); final boolean isEC = isEC(container.getReplicationConfig()); + return getContainerReplicaCount(container, isEC); - if (!isEC && rmConf.isLegacyEnabled()) { - return legacyReplicationManager.getContainerReplicaCount(container); - } else { - return getContainerReplicaCount(container, isEC); - } } /** @@ -1076,28 +1049,6 @@ ReplicationQueue getQueue() { @ConfigGroup(prefix = "hdds.scm.replication") public static class ReplicationManagerConfiguration extends ReconfigurableConfig { - /** - * True if LegacyReplicationManager should be used for RATIS containers. - */ - @Config(key = "enable.legacy", - type = ConfigType.BOOLEAN, - defaultValue = "false", - tags = {SCM, OZONE}, - description = - "If true, LegacyReplicationManager will handle RATIS containers " + - "while ReplicationManager will handle EC containers. If false, " + - "ReplicationManager will handle both RATIS and EC." - ) - private boolean enableLegacy; - - public boolean isLegacyEnabled() { - return enableLegacy; - } - - public void setEnableLegacy(boolean enableLegacy) { - this.enableLegacy = enableLegacy; - } - /** * The frequency in which ReplicationMonitor thread should run. */ @@ -1414,11 +1365,6 @@ public void notifyStatusChanged() { containerReplicaPendingOps.clear(); serviceStatus = ServiceStatus.RUNNING; } - if (rmConf.isLegacyEnabled()) { - //now, as the current scm is leader and it`s state is up-to-date, - //we need to take some action about replicated inflight move options. - legacyReplicationManager.notifyStatusChanged(); - } } else { serviceStatus = ServiceStatus.PAUSING; } @@ -1456,46 +1402,8 @@ public Clock getClock() { return clock; } - /** - * following functions will be refactored in a separate jira. - */ - public CompletableFuture move( - ContainerID cid, DatanodeDetails src, DatanodeDetails tgt) - throws NodeNotFoundException, ContainerNotFoundException, - TimeoutException { - CompletableFuture ret = - new CompletableFuture<>(); - if (!isRunning()) { - ret.complete(MoveManager.MoveResult.FAIL_UNEXPECTED_ERROR); - LOG.warn("Failing move because Replication Monitor thread's " + - "running state is {}", isRunning()); - return ret; - } - - return legacyReplicationManager.move(cid, src, tgt); - } - - public Map> - getInflightMove() { - return legacyReplicationManager.getInflightMove(); - } - - public LegacyReplicationManager.MoveScheduler getMoveScheduler() { - return legacyReplicationManager.getMoveScheduler(); - } - - @VisibleForTesting - public LegacyReplicationManager getLegacyReplicationManager() { - return legacyReplicationManager; - } - public boolean isContainerReplicatingOrDeleting(ContainerID containerID) { - if (rmConf.isLegacyEnabled()) { - return legacyReplicationManager - .isContainerReplicatingOrDeleting(containerID); - } else { - return !getPendingReplicationOps(containerID).isEmpty(); - } + return !getPendingReplicationOps(containerID).isEmpty(); } private ContainerReplicaCount getContainerReplicaCount( diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManagerMetrics.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManagerMetrics.java index eb75db9bd504..dc48ae222796 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManagerMetrics.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManagerMetrics.java @@ -163,8 +163,6 @@ public final class ReplicationManagerMetrics implements MetricsSource { private final ReplicationManager replicationManager; - private final boolean legacyReplicationManager; - //EC Metrics @Metric("Number of EC Replication commands sent.") private MutableCounterLong ecReplicationCmdsSentTotal; @@ -231,7 +229,6 @@ public final class ReplicationManagerMetrics implements MetricsSource { public ReplicationManagerMetrics(ReplicationManager manager) { this.registry = new MetricsRegistry(METRICS_SOURCE_NAME); this.replicationManager = manager; - legacyReplicationManager = replicationManager.getConfig().isLegacyEnabled(); } public static ReplicationManagerMetrics create(ReplicationManager manager) { @@ -254,24 +251,10 @@ public void getMetrics(MetricsCollector collector, boolean all) { .addGauge(INFLIGHT_EC_REPLICATION, getEcReplication()) .addGauge(INFLIGHT_EC_DELETION, getEcDeletion()); - if (legacyReplicationManager) { - // For non-legacy RM, we don't need to expose these metrics as the timeout - // metrics below replace them. - builder - .addGauge(INFLIGHT_REPLICATION_SKIPPED, - getInflightReplicationSkipped()) - .addGauge(INFLIGHT_DELETION_SKIPPED, getInflightDeletionSkipped()) - // If not using Legacy RM, move manager should expose its own metrics - // and therefore we don't need IN_FLIGHT_MOVE here. - .addGauge(INFLIGHT_MOVE, getInflightMove()); - } - if (!legacyReplicationManager) { - builder - .addGauge(UNDER_REPLICATED_QUEUE, - replicationManager.getQueue().underReplicatedQueueSize()) + builder.addGauge(UNDER_REPLICATED_QUEUE, + replicationManager.getQueue().underReplicatedQueueSize()) .addGauge(OVER_REPLICATED_QUEUE, replicationManager.getQueue().overReplicatedQueueSize()); - } ReplicationManagerReport report = replicationManager.getContainerReport(); for (Map.Entry e : @@ -289,15 +272,6 @@ public void getMetrics(MetricsCollector collector, boolean all) { deletionCmdsSentTotal.snapshot(builder, all); replicasDeletedTotal.snapshot(builder, all); replicaDeleteTimeoutTotal.snapshot(builder, all); - if (legacyReplicationManager) { - // As things stand, the new RM does not track bytes sent / completed - replicationBytesTotal.snapshot(builder, all); - replicationBytesCompletedTotal.snapshot(builder, all); - deletionBytesTotal.snapshot(builder, all); - deletionBytesCompletedTotal.snapshot(builder, all); - replicationTime.snapshot(builder, all); - deletionTime.snapshot(builder, all); - } ecReplicationCmdsSentTotal.snapshot(builder, all); ecDeletionCmdsSentTotal.snapshot(builder, all); ecReplicasCreatedTotal.snapshot(builder, all); @@ -385,14 +359,9 @@ public void incrInflightSkipped(InflightType type) { } public long getInflightReplication() { - if (legacyReplicationManager) { - return replicationManager.getLegacyReplicationManager() - .getInflightCount(InflightType.REPLICATION); - } else { - return replicationManager.getContainerReplicaPendingOps() + return replicationManager.getContainerReplicaPendingOps() .getPendingOpCount(ContainerReplicaOp.PendingOpType.ADD, ReplicationType.RATIS); - } } public long getInflightReplicationSkipped() { @@ -400,24 +369,15 @@ public long getInflightReplicationSkipped() { } public long getInflightDeletion() { - if (legacyReplicationManager) { - return replicationManager.getLegacyReplicationManager() - .getInflightCount(InflightType.DELETION); - } else { - return replicationManager.getContainerReplicaPendingOps() - .getPendingOpCount(ContainerReplicaOp.PendingOpType.DELETE, + return replicationManager.getContainerReplicaPendingOps() + .getPendingOpCount(ContainerReplicaOp.PendingOpType.DELETE, ReplicationType.RATIS); - } } public long getInflightDeletionSkipped() { return this.inflightDeletionSkippedTotal.value(); } - public long getInflightMove() { - return replicationManager.getInflightMove().size(); - } - public long getReplicationCmdsSentTotal() { return this.replicationCmdsSentTotal.value(); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMHAManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMHAManagerImpl.java index 5d0ea444ef8a..048f5fc4f533 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMHAManagerImpl.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMHAManagerImpl.java @@ -463,8 +463,6 @@ public void startServices() throws IOException { scm.getContainerManager().reinitialize(metadataStore.getContainerTable()); scm.getScmBlockManager().getDeletedBlockLog().reinitialize( metadataStore.getDeletedBlocksTXTable()); - scm.getReplicationManager().getMoveScheduler() - .reinitialize(metadataStore.getMoveTable()); scm.getStatefulServiceStateManager().reinitialize( metadataStore.getStatefulServiceConfigTable()); if (OzoneSecurityUtil.isSecurityEnabled(conf)) { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DatanodeAdminMonitorImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DatanodeAdminMonitorImpl.java index 23bf41dc83e8..41c410a9032d 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DatanodeAdminMonitorImpl.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DatanodeAdminMonitorImpl.java @@ -441,20 +441,10 @@ private boolean checkContainersReplicatedOnNode(TrackedNode dn) continue; } - // If we get here, the container is closed or quasi-closed and all the replicas match that - // state, except for any which are unhealthy. As the container is closed, we can check - // if it is sufficiently replicated using replicationManager, but this only works if the - // legacy RM is not enabled. - boolean legacyEnabled = conf.getBoolean("hdds.scm.replication.enable" + - ".legacy", false); - boolean replicatedOK; - if (legacyEnabled) { - replicatedOK = replicaSet.isSufficientlyReplicatedForOffline(dn.getDatanodeDetails(), nodeManager); - } else { - ReplicationManagerReport report = new ReplicationManagerReport(); - replicationManager.checkContainerStatus(replicaSet.getContainer(), report); - replicatedOK = report.getStat(ReplicationManagerReport.HealthState.UNDER_REPLICATED) == 0; - } + ReplicationManagerReport report = new ReplicationManagerReport(); + replicationManager.checkContainerStatus(replicaSet.getContainer(), report); + boolean replicatedOK = report.getStat(ReplicationManagerReport.HealthState.UNDER_REPLICATED) == 0; + if (replicatedOK) { sufficientlyReplicated++; } else { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java index 9da0c686c3b7..d117e891c4b9 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java @@ -53,7 +53,6 @@ import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMPerformanceMetrics; import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaPendingOps; import org.apache.hadoop.hdds.scm.container.replication.DatanodeCommandCountUpdatedHandler; -import org.apache.hadoop.hdds.scm.container.replication.LegacyReplicationManager; import org.apache.hadoop.hdds.scm.ha.SCMServiceException; import org.apache.hadoop.hdds.scm.ha.BackgroundSCMService; import org.apache.hadoop.hdds.scm.ha.HASecurityUtils; @@ -825,10 +824,6 @@ private void initializeSystemManagers(OzoneConfiguration conf, if (configurator.getReplicationManager() != null) { replicationManager = configurator.getReplicationManager(); } else { - LegacyReplicationManager legacyRM = new LegacyReplicationManager( - conf, containerManager, containerPlacementPolicy, eventQueue, - scmContext, scmNodeManager, scmHAManager, systemClock, - getScmMetadataStore().getMoveTable()); replicationManager = new ReplicationManager( conf, containerManager, @@ -838,7 +833,6 @@ private void initializeSystemManagers(OzoneConfiguration conf, scmContext, scmNodeManager, systemClock, - legacyRM, containerReplicaPendingOps); reconfigurationHandler.register(replicationManager.getConfig()); } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java index cf213b963cde..d453fb6ca81f 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java @@ -169,14 +169,6 @@ public int getNodeCount() { return cluster.getNodeCount(); } - public void enableLegacyReplicationManager() { - mockedReplicaManager.conf.setEnableLegacy(true); - } - - public void disableLegacyReplicationManager() { - mockedReplicaManager.conf.setEnableLegacy(false); - } - public @Nonnull MoveManager getMoveManager() { return moveManager; } @@ -262,9 +254,6 @@ private static final class MockedReplicationManager { private MockedReplicationManager() { manager = mock(ReplicationManager.class); conf = new ReplicationManager.ReplicationManagerConfiguration(); - // Disable LegacyReplicationManager. This means balancer should select RATIS as well as - // EC containers for balancing. Also, MoveManager will be used. - conf.setEnableLegacy(false); } private static @Nonnull MockedReplicationManager doMock() @@ -279,13 +268,6 @@ private MockedReplicationManager() { .when(mockedManager.manager.isContainerReplicatingOrDeleting(Mockito.any(ContainerID.class))) .thenReturn(false); - Mockito - .when(mockedManager.manager.move( - Mockito.any(ContainerID.class), - Mockito.any(DatanodeDetails.class), - Mockito.any(DatanodeDetails.class))) - .thenReturn(CompletableFuture.completedFuture(MoveManager.MoveResult.COMPLETED)); - Mockito .when(mockedManager.manager.getClock()) .thenReturn(Clock.system(ZoneId.systemDefault())); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerDatanodeNodeLimit.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerDatanodeNodeLimit.java index 8441b8023dcf..2e44c3b4a5de 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerDatanodeNodeLimit.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerDatanodeNodeLimit.java @@ -225,17 +225,13 @@ public void testCalculationOfUtilization(@Nonnull MockedSCM mockedSCM) { @ParameterizedTest(name = "MockedSCM #{index}: {0}") @MethodSource("createMockedSCMs") public void testBalancerWithMoveManager(@Nonnull MockedSCM mockedSCM) - throws IOException, NodeNotFoundException, TimeoutException { + throws IOException, NodeNotFoundException { ContainerBalancerConfiguration config = new ContainerBalancerConfigBuilder(mockedSCM.getNodeCount()).build(); - mockedSCM.disableLegacyReplicationManager(); mockedSCM.startBalancerTask(config); verify(mockedSCM.getMoveManager(), atLeastOnce()). move(any(ContainerID.class), any(DatanodeDetails.class), any(DatanodeDetails.class)); - - verify(mockedSCM.getReplicationManager(), times(0)) - .move(any(ContainerID.class), any(DatanodeDetails.class), any(DatanodeDetails.class)); } @ParameterizedTest(name = "MockedSCM #{index}: {0}") @@ -437,29 +433,19 @@ public void targetDatanodeShouldBeInServiceHealthy(@Nonnull MockedSCM mockedSCM) @ParameterizedTest(name = "MockedSCM #{index}: {0}") @MethodSource("createMockedSCMs") public void selectedContainerShouldNotAlreadyHaveBeenSelected(@Nonnull MockedSCM mockedSCM) - throws NodeNotFoundException, ContainerNotFoundException, TimeoutException, ContainerReplicaNotFoundException { + throws NodeNotFoundException, ContainerNotFoundException, ContainerReplicaNotFoundException { ContainerBalancerConfiguration config = new ContainerBalancerConfigBuilder(mockedSCM.getNodeCount()).build(); - mockedSCM.enableLegacyReplicationManager(); - - ContainerBalancerTask task = mockedSCM.startBalancerTask(config); - int numContainers = task.getContainerToTargetMap().size(); - /* Assuming move is called exactly once for each unique container, number of calls to move should equal number of unique containers. If number of calls to move is more than number of unique containers, at least one container has been re-selected. It's expected that number of calls to move should equal number of unique, selected containers (from containerToTargetMap). */ - verify(mockedSCM.getReplicationManager(), times(numContainers)) - .move(any(ContainerID.class), any(DatanodeDetails.class), any(DatanodeDetails.class)); - - // Try the same test by disabling LegacyReplicationManager so that MoveManager is used. - mockedSCM.disableLegacyReplicationManager(); ContainerBalancerTask nextTask = mockedSCM.startBalancerTask(config); - numContainers = nextTask.getContainerToTargetMap().size(); + int numContainers = nextTask.getContainerToTargetMap().size(); verify(mockedSCM.getMoveManager(), times(numContainers)) - .move(any(ContainerID.class), any(DatanodeDetails.class), any(DatanodeDetails.class)); + .move(any(ContainerID.class), any(DatanodeDetails.class), any(DatanodeDetails.class)); } @ParameterizedTest(name = "MockedSCM #{index}: {0}") @@ -478,30 +464,23 @@ public void balancerShouldNotSelectConfiguredExcludeContainers(@Nonnull MockedSC @ParameterizedTest(name = "MockedSCM #{index}: {0}") @MethodSource("createMockedSCMs") - public void checkIterationResult(@Nonnull MockedSCM mockedSCM) - throws NodeNotFoundException, ContainerNotFoundException, TimeoutException { + public void checkIterationResult(@Nonnull MockedSCM mockedSCM) { ContainerBalancerConfiguration config = new ContainerBalancerConfigBuilder(mockedSCM.getNodeCount()).build(); config.setMaxSizeEnteringTarget(10 * STORAGE_UNIT); config.setMaxSizeToMovePerIteration(100 * STORAGE_UNIT); - mockedSCM.enableLegacyReplicationManager(); - ContainerBalancerTask task = mockedSCM.startBalancerTask(config); // According to the setup and configurations, this iteration's result should be ITERATION_COMPLETED. assertEquals(ContainerBalancerTask.IterationResult.ITERATION_COMPLETED, task.getIterationResult()); // Now, limit maxSizeToMovePerIteration but fail all container moves. // The result should still be ITERATION_COMPLETED. - when(mockedSCM.getReplicationManager() - .move(any(ContainerID.class), any(DatanodeDetails.class), any(DatanodeDetails.class))) - .thenReturn(CompletableFuture.completedFuture(MoveManager.MoveResult.REPLICATION_FAIL_NODE_UNHEALTHY)); config.setMaxSizeToMovePerIteration(10 * STORAGE_UNIT); task = mockedSCM.startBalancerTask(config); assertEquals(ContainerBalancerTask.IterationResult.ITERATION_COMPLETED, task.getIterationResult()); //Try the same but use MoveManager for container move instead of legacy RM. - mockedSCM.disableLegacyReplicationManager(); task = mockedSCM.startBalancerTask(config); assertEquals(ContainerBalancerTask.IterationResult.ITERATION_COMPLETED, task.getIterationResult()); } @@ -521,30 +500,15 @@ public void checkIterationResultTimeout(@Nonnull MockedSCM mockedSCM) CompletableFuture completedFuture = CompletableFuture.completedFuture(MoveManager.MoveResult.COMPLETED); - when(mockedSCM.getReplicationManager() - .move(any(ContainerID.class), any(DatanodeDetails.class), any(DatanodeDetails.class))) - .thenReturn(completedFuture) - .thenAnswer(invocation -> genCompletableFuture(150)); - - mockedSCM.enableLegacyReplicationManager(); - ContainerBalancerTask task = mockedSCM.startBalancerTask(config); - - // According to the setup and configurations, this iteration's result should be ITERATION_COMPLETED. - assertEquals(ContainerBalancerTask.IterationResult.ITERATION_COMPLETED, task.getIterationResult()); - assertEquals(1, task.getMetrics().getNumContainerMovesCompletedInLatestIteration()); - assertThat(task.getMetrics().getNumContainerMovesTimeoutInLatestIteration()).isGreaterThanOrEqualTo(1); - /* - Test the same but use MoveManager instead of LegacyReplicationManager. The first move being 10ms falls within the timeout duration of 500ms. It should be successful. The rest should fail. */ - mockedSCM.disableLegacyReplicationManager(); when(mockedSCM.getMoveManager() .move(any(ContainerID.class), any(DatanodeDetails.class), any(DatanodeDetails.class))) .thenReturn(completedFuture) .thenAnswer(invocation -> genCompletableFuture(150)); - task = mockedSCM.startBalancerTask(config); + ContainerBalancerTask task = mockedSCM.startBalancerTask(config); assertEquals(ContainerBalancerTask.IterationResult.ITERATION_COMPLETED, task.getIterationResult()); assertEquals(1, task.getMetrics().getNumContainerMovesCompletedInLatestIteration()); assertThat(task.getMetrics().getNumContainerMovesTimeoutInLatestIteration()).isGreaterThanOrEqualTo(1); @@ -563,24 +527,12 @@ public void checkIterationResultTimeoutFromReplicationManager(@Nonnull MockedSCM CompletableFuture.supplyAsync(() -> MoveManager.MoveResult.REPLICATION_FAIL_TIME_OUT); CompletableFuture future2 = CompletableFuture.supplyAsync(() -> MoveManager.MoveResult.DELETION_FAIL_TIME_OUT); - when(mockedSCM.getReplicationManager() - .move(any(ContainerID.class), any(DatanodeDetails.class), any(DatanodeDetails.class))) - .thenReturn(future, future2); - - mockedSCM.enableLegacyReplicationManager(); - - ContainerBalancerTask task = mockedSCM.startBalancerTask(config); - assertThat(task.getMetrics().getNumContainerMovesTimeoutInLatestIteration()).isGreaterThan(0); - assertEquals(0, task.getMetrics().getNumContainerMovesCompletedInLatestIteration()); - // Try the same test with MoveManager instead of LegacyReplicationManager. when(mockedSCM.getMoveManager() .move(any(ContainerID.class), any(DatanodeDetails.class), any(DatanodeDetails.class))) .thenReturn(future).thenAnswer(invocation -> future2); - mockedSCM.disableLegacyReplicationManager(); - - task = mockedSCM.startBalancerTask(config); + ContainerBalancerTask task = mockedSCM.startBalancerTask(config); assertThat(task.getMetrics().getNumContainerMovesTimeoutInLatestIteration()).isGreaterThan(0); assertEquals(0, task.getMetrics().getNumContainerMovesCompletedInLatestIteration()); } @@ -598,20 +550,8 @@ public void checkIterationResultException(@Nonnull MockedSCM mockedSCM) CompletableFuture future = new CompletableFuture<>(); future.completeExceptionally(new RuntimeException("Runtime Exception")); - when(mockedSCM.getReplicationManager() - .move(any(ContainerID.class), any(DatanodeDetails.class), any(DatanodeDetails.class))) - .thenReturn(genCompletableFutureWithException(1)) - .thenThrow(new ContainerNotFoundException("Test Container not found")) - .thenReturn(future); - - mockedSCM.enableLegacyReplicationManager(); - - ContainerBalancerTask task = mockedSCM.startBalancerTask(config); - assertEquals(ContainerBalancerTask.IterationResult.ITERATION_COMPLETED, task.getIterationResult()); int expectedMovesFailed = (nodeCount > 6) ? 3 : 1; - assertThat(task.getMetrics().getNumContainerMovesFailed()).isGreaterThanOrEqualTo(expectedMovesFailed); - // Try the same test but with MoveManager instead of ReplicationManager. when(mockedSCM.getMoveManager() .move(any(ContainerID.class), any(DatanodeDetails.class), any(DatanodeDetails.class))) @@ -619,8 +559,7 @@ public void checkIterationResultException(@Nonnull MockedSCM mockedSCM) .thenThrow(new ContainerNotFoundException("Test Container not found")) .thenReturn(future); - mockedSCM.disableLegacyReplicationManager(); - task = mockedSCM.startBalancerTask(config); + ContainerBalancerTask task = mockedSCM.startBalancerTask(config); assertEquals(ContainerBalancerTask.IterationResult.ITERATION_COMPLETED, task.getIterationResult()); assertThat(task.getMetrics().getNumContainerMovesFailed()).isGreaterThanOrEqualTo(expectedMovesFailed); } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerTask.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerTask.java index e689e8d1144c..ff1ff4f32c32 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerTask.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerTask.java @@ -75,7 +75,6 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertNotSame; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyString; @@ -137,12 +136,7 @@ public void setup(TestInfo testInfo) throws IOException, NodeNotFoundException, .thenReturn(CompletableFuture.completedFuture( MoveManager.MoveResult.COMPLETED)); - /* - Disable LegacyReplicationManager. This means balancer should select RATIS - as well as EC containers for balancing. Also, MoveManager will be used. - */ when(replicationManager.getConfig()).thenReturn(rmConf); - rmConf.setEnableLegacy(false); // these configs will usually be specified in each test balancerConfiguration = conf.getObject(ContainerBalancerConfiguration.class); @@ -176,12 +170,6 @@ public void setup(TestInfo testInfo) throws IOException, NodeNotFoundException, .isContainerReplicatingOrDeleting(any(ContainerID.class))) .thenReturn(false); - when(replicationManager.move(any(ContainerID.class), - any(DatanodeDetails.class), - any(DatanodeDetails.class))) - .thenReturn(CompletableFuture. - completedFuture(MoveManager.MoveResult.COMPLETED)); - when(replicationManager.getClock()) .thenReturn(Clock.system(ZoneId.systemDefault())); @@ -368,40 +356,6 @@ public void testDelayedStart() throws InterruptedException, TimeoutException { assertFalse(balancingThread.isAlive()); } - /** - * The expectation is that only RATIS containers should be selected for - * balancing when LegacyReplicationManager is enabled. This is because - * LegacyReplicationManager does not support moving EC containers. - */ - @Test - public void balancerShouldExcludeECContainersWhenLegacyRmIsEnabled() - throws IllegalContainerBalancerStateException, IOException, - InvalidContainerBalancerConfigurationException, TimeoutException { - // Enable LegacyReplicationManager - rmConf.setEnableLegacy(true); - balancerConfiguration.setThreshold(10); - balancerConfiguration.setIterations(1); - balancerConfiguration.setMaxSizeEnteringTarget(10 * STORAGE_UNIT); - balancerConfiguration.setMaxSizeToMovePerIteration(100 * STORAGE_UNIT); - balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100); - - startBalancer(balancerConfiguration); - - /* - Get all containers that were selected by balancer and assert none of - them is an EC container. - */ - Map containerToSource = - containerBalancerTask.getContainerToSourceMap(); - assertFalse(containerToSource.isEmpty()); - for (Map.Entry entry : - containerToSource.entrySet()) { - ContainerInfo containerInfo = cidToInfoMap.get(entry.getKey()); - assertNotSame(HddsProtos.ReplicationType.EC, - containerInfo.getReplicationType()); - } - } - /** * Tests if balancer is adding the polled source datanode back to potentialSources queue * if a move has failed due to a container related failure, like REPLICATION_FAIL_NOT_EXIST_IN_SOURCE. diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestLegacyReplicationManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestLegacyReplicationManager.java deleted file mode 100644 index 7f36279ba08f..000000000000 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestLegacyReplicationManager.java +++ /dev/null @@ -1,3487 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hdds.scm.container.replication; - -import org.apache.hadoop.hdds.HddsConfigKeys; -import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.hdds.protocol.DatanodeDetails; -import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.ContainerReplicaProto.State; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMCommandProto; -import org.apache.hadoop.hdds.scm.container.CloseContainerEventHandler; -import org.apache.hadoop.hdds.scm.container.ContainerID; -import org.apache.hadoop.hdds.scm.container.ContainerInfo; -import org.apache.hadoop.hdds.scm.container.ContainerManager; -import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException; -import org.apache.hadoop.hdds.scm.container.ContainerReplica; -import org.apache.hadoop.hdds.scm.container.ContainerStateManager; -import org.apache.hadoop.hdds.scm.container.ContainerStateManagerImpl; -import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport; -import org.apache.hadoop.hdds.scm.container.SimpleMockNodeManager; -import org.apache.hadoop.hdds.scm.container.balancer.MoveManager; -import org.apache.hadoop.hdds.scm.container.replication.LegacyReplicationManager.LegacyReplicationManagerConfiguration; -import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager.ReplicationManagerConfiguration; -import org.apache.hadoop.hdds.scm.PlacementPolicy; -import org.apache.hadoop.hdds.scm.container.common.helpers.MoveDataNodePair; -import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementStatusDefault; -import org.apache.hadoop.hdds.scm.events.SCMEvents; -import org.apache.hadoop.hdds.scm.exceptions.SCMException; -import org.apache.hadoop.hdds.scm.ha.SCMHAManagerStub; -import org.apache.hadoop.hdds.scm.ha.SCMContext; -import org.apache.hadoop.hdds.scm.ha.SCMHAManager; -import org.apache.hadoop.hdds.scm.ha.SCMServiceManager; -import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition; -import org.apache.hadoop.hdds.scm.metadata.SCMDBTransactionBufferImpl; -import org.apache.hadoop.hdds.scm.pipeline.PipelineID; -import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; -import org.apache.hadoop.hdds.utils.db.DBStore; -import org.apache.hadoop.hdds.scm.node.NodeStatus; -import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; -import org.apache.hadoop.hdds.server.events.EventHandler; -import org.apache.hadoop.hdds.server.events.EventPublisher; -import org.apache.hadoop.hdds.server.events.EventQueue; -import org.apache.hadoop.hdds.utils.db.DBStoreBuilder; -import org.apache.hadoop.hdds.utils.db.LongCodec; -import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException; -import org.apache.hadoop.ozone.container.common.SCMTestUtils; -import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand; -import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode; -import org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand; -import org.apache.hadoop.ozone.protocol.commands.SCMCommand; -import org.apache.hadoop.ozone.protocol.commands.DeleteContainerCommand; -import org.apache.ozone.test.GenericTestUtils; -import org.apache.ozone.test.TestClock; -import org.apache.ozone.test.tag.Unhealthy; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Nested; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.EnumSource; -import org.junit.jupiter.params.provider.ValueSource; - -import java.io.File; -import java.io.IOException; -import java.time.Clock; -import java.time.Instant; -import java.time.ZoneId; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.Callable; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; -import java.util.stream.IntStream; - -import static org.apache.hadoop.hdds.protocol.MockDatanodeDetails.createDatanodeDetails; -import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONED; -import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONING; -import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_MAINTENANCE; -import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_SERVICE; -import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.HEALTHY; -import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.STALE; -import static org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.ContainerReplicaProto.State.CLOSED; -import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto.State.QUASI_CLOSED; -import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto.State.UNHEALTHY; -import static org.apache.hadoop.hdds.scm.HddsTestUtils.CONTAINER_NUM_KEYS_DEFAULT; -import static org.apache.hadoop.hdds.scm.HddsTestUtils.CONTAINER_USED_BYTES_DEFAULT; -import static org.apache.hadoop.hdds.scm.HddsTestUtils.getContainer; -import static org.apache.hadoop.hdds.scm.HddsTestUtils.getReplicaBuilder; -import static org.apache.hadoop.hdds.scm.HddsTestUtils.getReplicas; -import static org.apache.hadoop.hdds.protocol.MockDatanodeDetails.randomDatanodeDetails; -import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.Assertions.assertInstanceOf; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertSame; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.anyLong; -import static org.mockito.Mockito.anyInt; -import static org.mockito.Mockito.argThat; -import static org.mockito.Mockito.anyList; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.any; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.when; - -/** - * Test cases to verify the functionality of ReplicationManager. - */ -public class TestLegacyReplicationManager { - - private ReplicationManager replicationManager; - private ContainerStateManager containerStateManager; - private PlacementPolicy ratisContainerPlacementPolicy; - private PlacementPolicy ecContainerPlacementPolicy; - private EventQueue eventQueue; - private DatanodeCommandHandler datanodeCommandHandler; - private SimpleMockNodeManager nodeManager; - private ContainerManager containerManager; - private GenericTestUtils.LogCapturer scmLogs; - private SCMServiceManager serviceManager; - private TestClock clock; - private DBStore dbStore; - private ContainerReplicaPendingOps containerReplicaPendingOps; - - @TempDir - private File tempDir; - - int getInflightCount(InflightType type) { - return replicationManager.getLegacyReplicationManager() - .getInflightCount(type); - } - - @BeforeEach - void setup(@TempDir File testDir) throws IOException, InterruptedException, - NodeNotFoundException, InvalidStateTransitionException { - OzoneConfiguration conf = SCMTestUtils.getConf(testDir); - conf.setTimeDuration( - HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT, - 0, TimeUnit.SECONDS); - - scmLogs = GenericTestUtils.LogCapturer. - captureLogs(LegacyReplicationManager.LOG); - containerManager = mock(ContainerManager.class); - nodeManager = new SimpleMockNodeManager(); - eventQueue = new EventQueue(); - SCMHAManager scmhaManager = SCMHAManagerStub.getInstance(true); - dbStore = DBStoreBuilder.createDBStore(conf, SCMDBDefinition.get()); - PipelineManager pipelineManager = mock(PipelineManager.class); - when(pipelineManager.containsPipeline(any(PipelineID.class))) - .thenReturn(true); - containerStateManager = ContainerStateManagerImpl.newBuilder() - .setConfiguration(conf) - .setPipelineManager(pipelineManager) - .setRatisServer(scmhaManager.getRatisServer()) - .setContainerStore(SCMDBDefinition.CONTAINERS.getTable(dbStore)) - .setSCMDBTransactionBuffer(scmhaManager.getDBTransactionBuffer()) - .setContainerReplicaPendingOps(new ContainerReplicaPendingOps( - Clock.system(ZoneId.systemDefault()))) - .build(); - serviceManager = new SCMServiceManager(); - - datanodeCommandHandler = new DatanodeCommandHandler(); - eventQueue.addHandler(SCMEvents.DATANODE_COMMAND, datanodeCommandHandler); - - when(containerManager.getContainers()) - .thenAnswer(invocation -> { - Set ids = containerStateManager.getContainerIDs(); - List containers = new ArrayList<>(); - for (ContainerID id : ids) { - containers.add(containerStateManager.getContainer( - id)); - } - return containers; - }); - - when(containerManager.getContainer(any(ContainerID.class))) - .thenAnswer(invocation -> containerStateManager - .getContainer(((ContainerID)invocation - .getArguments()[0]))); - - when(containerManager.getContainerReplicas( - any(ContainerID.class))) - .thenAnswer(invocation -> containerStateManager - .getContainerReplicas(((ContainerID)invocation - .getArguments()[0]))); - - ratisContainerPlacementPolicy = mock(PlacementPolicy.class); - ecContainerPlacementPolicy = mock(PlacementPolicy.class); - - when(ratisContainerPlacementPolicy.chooseDatanodes( - any(), any(), anyInt(), - anyLong(), anyLong())) - .thenAnswer(invocation -> { - int count = (int) invocation.getArguments()[2]; - return IntStream.range(0, count) - .mapToObj(i -> randomDatanodeDetails()) - .collect(Collectors.toList()); - }); - - when(ratisContainerPlacementPolicy.validateContainerPlacement( - any(), - anyInt() - )).thenAnswer(invocation -> - new ContainerPlacementStatusDefault(2, 2, 3)); - clock = new TestClock(Instant.now(), ZoneId.of("UTC")); - containerReplicaPendingOps = new ContainerReplicaPendingOps(clock); - createReplicationManager(newRMConfig()); - } - - void createReplicationManager(int replicationLimit, int deletionLimit) - throws Exception { - replicationManager.stop(); - dbStore.close(); - final LegacyReplicationManagerConfiguration conf - = new LegacyReplicationManagerConfiguration(); - conf.setContainerInflightReplicationLimit(replicationLimit); - conf.setContainerInflightDeletionLimit(deletionLimit); - createReplicationManager(conf); - } - - void createReplicationManager( - LegacyReplicationManagerConfiguration conf) - throws Exception { - createReplicationManager(newRMConfig(), conf); - } - - private void createReplicationManager(ReplicationManagerConfiguration rmConf) - throws InterruptedException, IOException { - createReplicationManager(rmConf, null); - } - - private void createReplicationManager(ReplicationManagerConfiguration rmConf, - LegacyReplicationManagerConfiguration lrmConf) - throws InterruptedException, IOException { - OzoneConfiguration config = SCMTestUtils.getConf(tempDir); - config.setTimeDuration( - HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT, - 0, TimeUnit.SECONDS); - Optional.ofNullable(rmConf).ifPresent(config::setFromObject); - Optional.ofNullable(lrmConf).ifPresent(config::setFromObject); - - SCMHAManager scmHAManager = SCMHAManagerStub - .getInstance(true, new SCMDBTransactionBufferImpl()); - dbStore = DBStoreBuilder.createDBStore(config, SCMDBDefinition.get()); - - LegacyReplicationManager legacyRM = new LegacyReplicationManager( - config, containerManager, ratisContainerPlacementPolicy, eventQueue, - SCMContext.emptyContext(), nodeManager, scmHAManager, clock, - SCMDBDefinition.MOVE.getTable(dbStore)); - - replicationManager = new ReplicationManager( - config, - containerManager, - ratisContainerPlacementPolicy, - ecContainerPlacementPolicy, - eventQueue, - SCMContext.emptyContext(), - nodeManager, - clock, - legacyRM, - containerReplicaPendingOps); - - serviceManager.register(replicationManager); - serviceManager.notifyStatusChanged(); - scmLogs.clearOutput(); - Thread.sleep(100L); - } - - @AfterEach - public void teardown() throws Exception { - containerStateManager.close(); - replicationManager.stop(); - if (dbStore != null) { - dbStore.close(); - } - } - - @Nested - class Misc { - /** - * Checks if restarting of replication manager works. - */ - @Test - public void testReplicationManagerRestart() throws InterruptedException { - assertTrue(replicationManager.isRunning()); - replicationManager.stop(); - // Stop is a non-blocking call, it might take sometime for the - // ReplicationManager to shutdown - Thread.sleep(500); - assertFalse(replicationManager.isRunning()); - replicationManager.start(); - assertTrue(replicationManager.isRunning()); - } - - @Test - public void testGetContainerReplicaCount() - throws IOException, TimeoutException { - ContainerInfo container = createContainer(LifeCycleState.QUASI_CLOSED); - addReplica(container, NodeStatus.inServiceHealthy(), UNHEALTHY); - addReplica(container, NodeStatus.inServiceHealthy(), UNHEALTHY); - ContainerReplica decommissioningReplica = - addReplica(container, new NodeStatus(DECOMMISSIONING, HEALTHY), - UNHEALTHY); - - ContainerReplicaCount replicaCount = - replicationManager.getLegacyReplicationManager() - .getContainerReplicaCount(container); - - assertInstanceOf(LegacyRatisContainerReplicaCount.class, replicaCount); - assertFalse(replicaCount.isSufficientlyReplicated()); - assertFalse(replicaCount.isSufficientlyReplicatedForOffline( - decommissioningReplica.getDatanodeDetails(), nodeManager)); - - addReplica(container, NodeStatus.inServiceHealthy(), UNHEALTHY); - replicaCount = replicationManager.getLegacyReplicationManager() - .getContainerReplicaCount(container); - assertTrue(replicaCount.isSufficientlyReplicated()); - assertTrue(replicaCount.isSufficientlyReplicatedForOffline( - decommissioningReplica.getDatanodeDetails(), nodeManager)); - assertTrue(replicaCount.isHealthyEnoughForOffline()); - } - } - - /** - * Tests replication manager with healthy open and closed containers. No - * quasi closed or unhealthy containers are involved. - */ - @Nested - class StableReplicas { - /** - * Open containers are not handled by ReplicationManager. - * This test-case makes sure that ReplicationManages doesn't take - * any action on OPEN containers. - */ - @Test - public void testOpenContainer() throws IOException, TimeoutException { - final ContainerInfo container = getContainer(LifeCycleState.OPEN); - containerStateManager.addContainer(container.getProtobuf()); - replicationManager.processAll(); - eventQueue.processAll(1000); - ReplicationManagerReport report = replicationManager.getContainerReport(); - assertEquals(1, report.getStat(LifeCycleState.OPEN)); - assertEquals(0, datanodeCommandHandler.getInvocation()); - } - - /** - * 1 open replica - * 2 closing replicas - * Expectation: Close command is sent to the open replicas. - */ - @Test - public void testClosingContainer() throws IOException, TimeoutException { - final ContainerInfo container = getContainer(LifeCycleState.CLOSING); - final ContainerID id = container.containerID(); - - containerStateManager.addContainer(container.getProtobuf()); - - // Two replicas in CLOSING state - final Set replicas = getReplicas(id, State.CLOSING, - randomDatanodeDetails(), - randomDatanodeDetails()); - - // One replica in OPEN state - final DatanodeDetails datanode = randomDatanodeDetails(); - replicas.addAll(getReplicas(id, State.OPEN, datanode)); - - for (ContainerReplica replica : replicas) { - containerStateManager.updateContainerReplica(id, replica); - } - - final int currentCloseCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.closeContainerCommand); - - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(currentCloseCommandCount + 3, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.closeContainerCommand)); - - // Update the OPEN to CLOSING - for (ContainerReplica replica: getReplicas(id, State.CLOSING, datanode)) { - containerStateManager.updateContainerReplica(id, replica); - } - - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(currentCloseCommandCount + 6, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.closeContainerCommand)); - ReplicationManagerReport report = replicationManager.getContainerReport(); - assertEquals(1, report.getStat(LifeCycleState.CLOSING)); - } - - /** - * Create closing container with 1 replica. - * Expectation: Missing containers 0. - * Remove the only replica. - * Expectation: Missing containers 1. - */ - @Test - public void testClosingMissingContainer() - throws IOException, InvalidStateTransitionException { - final ContainerInfo container = getContainer(LifeCycleState.CLOSING); - final ContainerID id = container.containerID(); - - containerStateManager.addContainer(container.getProtobuf()); - - // One replica in OPEN state - final Set replicas = getReplicas(id, State.OPEN, - randomDatanodeDetails()); - - for (ContainerReplica replica : replicas) { - containerStateManager.updateContainerReplica(id, replica); - } - - final int currentCloseCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.closeContainerCommand); - - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(currentCloseCommandCount + 1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.closeContainerCommand)); - - ReplicationManagerReport report = replicationManager.getContainerReport(); - assertEquals(1, report.getStat(LifeCycleState.CLOSING)); - assertEquals(0, report.getStat( - ReplicationManagerReport.HealthState.MISSING)); - - for (ContainerReplica replica : replicas) { - containerStateManager.removeContainerReplica(id, replica); - } - - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(currentCloseCommandCount + 1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.closeContainerCommand)); - - report = replicationManager.getContainerReport(); - assertEquals(1, report.getStat(LifeCycleState.CLOSING)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.MISSING)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.UNDER_REPLICATED)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.MIS_REPLICATED)); - - verify(containerManager, times(0)).updateContainerState(container.containerID(), - LifeCycleEvent.QUASI_CLOSE); - } - - @Test - public void testReplicateCommandTimeout() - throws IOException, TimeoutException { - long timeout = new ReplicationManagerConfiguration().getEventTimeout(); - - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - assertReplicaScheduled(1); - - // Already a pending replica, so nothing scheduled - assertReplicaScheduled(0); - - // Advance the clock past the timeout, and there should be a replica - // scheduled - clock.fastForward(timeout + 1000); - assertReplicaScheduled(1); - assertEquals(1, replicationManager.getMetrics() - .getReplicaCreateTimeoutTotal()); - } - - @Test - public void testDeleteCommandTimeout() - throws IOException, TimeoutException { - long timeout = new ReplicationManagerConfiguration().getEventTimeout(); - - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - assertDeleteScheduled(1); - - // Already a pending replica, so nothing scheduled - assertReplicaScheduled(0); - - // Advance the clock past the timeout, and there should be a replica - // scheduled - clock.fastForward(timeout + 1000); - assertDeleteScheduled(1); - assertEquals(1, replicationManager.getMetrics() - .getReplicaDeleteTimeoutTotal()); - } - - /** - * A closed empty container with all the replicas also closed and empty - * should be deleted. - * A container/ replica should be deemed empty when it has 0 keyCount even - * if the usedBytes is not 0 (usedBytes should not be used to determine if - * the container or replica is empty). - */ - @Test - public void testDeleteEmptyContainer() throws Exception { - runTestDeleteEmptyContainer(3); - } - - Void runTestDeleteEmptyContainer(int expectedDelete) throws Exception { - // Create container with usedBytes = 1000 and keyCount = 0 - final ContainerInfo container = createContainer( - LifeCycleState.CLOSED, 1000, 0); - addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - // Create a replica with usedBytes != 0 and keyCount = 0 - addReplica(container, - new NodeStatus(IN_SERVICE, HEALTHY), CLOSED, 100, 0); - - assertDeleteScheduled(expectedDelete); - return null; - } - - @Test - public void testEmptyContainerWithNoReplicas() throws Exception { - final ContainerInfo container = createContainer( - LifeCycleState.CLOSED, 0, 0); - // No replicas - replicationManager.processAll(); - eventQueue.processAll(1000); - ReplicationManagerReport report = replicationManager.getContainerReport(); - assertEquals(1, - report.getStat(ReplicationManagerReport.HealthState.EMPTY)); - assertEquals(LifeCycleState.CLOSED, container.getState()); - } - - @Test - public void testDeletionLimit() throws Exception { - runTestLimit(0, 2, 0, 1, - () -> runTestDeleteEmptyContainer(2)); - } - - /** - * A closed empty container with a non-empty replica should not be deleted. - */ - @Test - public void testDeleteEmptyContainerNonEmptyReplica() throws Exception { - final ContainerInfo container = createContainer(LifeCycleState.CLOSED, 0, - 0); - addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - // Create the 3rd replica with non-zero key count and used bytes - addReplica(container, - new NodeStatus(IN_SERVICE, HEALTHY), CLOSED, 100, 1); - assertDeleteScheduled(0); - } - - /** - * ReplicationManager should replicate zero replica when all copies - * are missing. - */ - @Test - public void testContainerWithMissingReplicas() - throws IOException, TimeoutException { - createContainer(LifeCycleState.CLOSED); - assertReplicaScheduled(0); - assertUnderReplicatedCount(1); - assertMissingCount(1); - } - - /** - * 3 healthy closed replicas. - * Expectation: No action. - * - * ReplicationManager should not take any action if the container is - * CLOSED and healthy. - */ - @Test - public void testHealthyClosedContainer() - throws IOException, TimeoutException { - final ContainerInfo container = getContainer(LifeCycleState.CLOSED); - final ContainerID id = container.containerID(); - final Set replicas = getReplicas(id, State.CLOSED, - randomDatanodeDetails(), - randomDatanodeDetails(), - randomDatanodeDetails()); - - containerStateManager.addContainer(container.getProtobuf()); - for (ContainerReplica replica : replicas) { - containerStateManager.updateContainerReplica(id, replica); - } - - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(0, datanodeCommandHandler.getInvocation()); - - ReplicationManagerReport report = replicationManager.getContainerReport(); - assertEquals(1, report.getStat(LifeCycleState.CLOSED)); - for (ReplicationManagerReport.HealthState s : - ReplicationManagerReport.HealthState.values()) { - assertEquals(0, report.getStat(s)); - } - } - } - - /** - * Tests replication manager with unhealthy and quasi-closed container - * replicas. - */ - @Nested - class UnstableReplicas { - - /** - * A CLOSING container which has only UNHEALTHY replicas should be moved - * to QUASI_CLOSED state so that RM can then maintain replication factor - * number of replicas. - */ - @Test - public void testClosingContainerWithOnlyUnhealthyReplicas() - throws IOException, InvalidStateTransitionException { - final ContainerInfo container = getContainer(LifeCycleState.CLOSING); - final ContainerID id = container.containerID(); - containerStateManager.addContainer(container.getProtobuf()); - - // all replicas are UNHEALTHY - final Set replicas = getReplicas(id, UNHEALTHY, - randomDatanodeDetails(), randomDatanodeDetails(), - randomDatanodeDetails()); - for (ContainerReplica replica : replicas) { - containerStateManager.updateContainerReplica(id, replica); - } - - replicationManager.processAll(); - verify(containerManager, times(1)) - .updateContainerState(container.containerID(), - LifeCycleEvent.QUASI_CLOSE); - - containerStateManager.updateContainerState( - container.containerID().getProtobuf(), LifeCycleEvent.QUASI_CLOSE); - - replicationManager.processAll(); - assertEquals(1, - replicationManager.getContainerReport().getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - } - - /** - * Close command should be sent to the healthy replicas. The container - * should not be moved to quasi-closed immediately. - */ - @Test - public void testClosingContainerWithSomeUnhealthyReplicas() - throws IOException, InvalidStateTransitionException { - final ContainerInfo container = getContainer(LifeCycleState.CLOSING); - final ContainerID id = container.containerID(); - containerStateManager.addContainer(container.getProtobuf()); - - // 2 UNHEALTHY, 1 OPEN - final Set replicas = getReplicas(id, UNHEALTHY, - randomDatanodeDetails(), randomDatanodeDetails()); - final DatanodeDetails datanode = randomDatanodeDetails(); - replicas.addAll(getReplicas(id, State.OPEN, datanode)); - for (ContainerReplica replica : replicas) { - containerStateManager.updateContainerReplica(id, replica); - } - - final int currentCloseCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.closeContainerCommand); - replicationManager.processAll(); - eventQueue.processAll(1000); - - verify(containerManager, times(0)) - .updateContainerState(container.containerID(), - LifeCycleEvent.QUASI_CLOSE); - assertEquals(currentCloseCommandCount + 1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.closeContainerCommand)); - assertEquals(1, datanodeCommandHandler.getReceivedCommands().size()); - SCMCommand command = - datanodeCommandHandler.getReceivedCommands().iterator().next() - .getCommand(); - assertSame(SCMCommandProto.Type.closeContainerCommand, - command.getType()); - CloseContainerCommand closeCommand = (CloseContainerCommand) command; - assertFalse(closeCommand.isForce()); - } - - /** - * 2 open replicas - * 1 quasi-closed replica - * Expectation: close command is sent to the open replicas. - */ - @Test - public void testQuasiClosedContainerWithTwoOpenReplica() - throws IOException, TimeoutException { - final ContainerInfo container = getContainer(LifeCycleState.QUASI_CLOSED); - final ContainerID id = container.containerID(); - final UUID originNodeId = UUID.randomUUID(); - final ContainerReplica replicaOne = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaTwo = getReplicas( - id, State.OPEN, 1000L, originNodeId, randomDatanodeDetails()); - final DatanodeDetails datanodeDetails = randomDatanodeDetails(); - final ContainerReplica replicaThree = getReplicas( - id, State.OPEN, 1000L, datanodeDetails.getUuid(), datanodeDetails); - - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, replicaOne); - containerStateManager.updateContainerReplica(id, replicaTwo); - containerStateManager.updateContainerReplica( - id, replicaThree); - - // First iteration - - final int currentCloseCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.closeContainerCommand); - // Two of the replicas are in OPEN state - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(currentCloseCommandCount + 2, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.closeContainerCommand)); - assertTrue(datanodeCommandHandler.received( - SCMCommandProto.Type.closeContainerCommand, - replicaTwo.getDatanodeDetails())); - assertTrue(datanodeCommandHandler.received( - SCMCommandProto.Type.closeContainerCommand, - replicaThree.getDatanodeDetails())); - ReplicationManagerReport report = replicationManager.getContainerReport(); - assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - } - - /** - * 3 quasi closed replicas with the same origin node ID. - * Expectation: No action taken. - * - * When the container is in QUASI_CLOSED state and all the replicas are - * also in QUASI_CLOSED state and doesn't have a quorum to force close - * the container, ReplicationManager will not do anything. - */ - @Test - public void testHealthyQuasiClosedContainer() - throws IOException, TimeoutException { - final ContainerInfo container = getContainer(LifeCycleState.QUASI_CLOSED); - final ContainerID id = container.containerID(); - final UUID originNodeId = UUID.randomUUID(); - final ContainerReplica replicaOne = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaTwo = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaThree = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, replicaOne); - containerStateManager.updateContainerReplica(id, replicaTwo); - containerStateManager.updateContainerReplica( - id, replicaThree); - - // All the QUASI_CLOSED replicas have same originNodeId, so the - // container will not be closed. ReplicationManager should take no action. - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(0, datanodeCommandHandler.getInvocation()); - ReplicationManagerReport report = replicationManager.getContainerReport(); - assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - } - - /** - * 2 quasi-closed replicas. - * 1 unhealthy replica. - * All replicas have same origin node ID. - * Expectation: - * Round 1: Quasi closed replica is replicated. - * Round 2: Unhealthy replica is deleted. - * - * When a container is QUASI_CLOSED and we don't have quorum to force close - * the container, the container should have all the replicas in QUASI_CLOSED - * state, else ReplicationManager will take action. - * - * In this test case we make one of the replica unhealthy, replication - * manager will send delete container command to the datanode which has the - * unhealthy replica. - */ - @Test - public void testQuasiClosedContainerWithUnhealthyReplica() - throws IOException, TimeoutException { - final ContainerInfo container = getContainer(LifeCycleState.QUASI_CLOSED); - container.setUsedBytes(100); - final ContainerID id = container.containerID(); - final UUID originNodeId = UUID.randomUUID(); - final ContainerReplica replicaOne = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaTwo = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaThree = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, replicaOne); - containerStateManager.updateContainerReplica(id, replicaTwo); - containerStateManager.updateContainerReplica(id, replicaThree); - - int currentReplicateCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.replicateContainerCommand); - - // All the QUASI_CLOSED replicas have same originNodeId, so the - // container will not be closed. ReplicationManager should take no action. - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(0, datanodeCommandHandler.getInvocation()); - - // Make the first replica unhealthy - final ContainerReplica unhealthyReplica = getReplicas( - id, UNHEALTHY, 1000L, originNodeId, - replicaOne.getDatanodeDetails()); - containerStateManager.updateContainerReplica( - id, unhealthyReplica); - - long currentBytesToReplicate = replicationManager.getMetrics() - .getReplicationBytesTotal(); - replicationManager.processAll(); - eventQueue.processAll(1000); - // Under replication handler should first re-replicate one of the quasi - // closed containers. - // The unhealthy container should not have been deleted in the first pass. - assertDeleteScheduled(0); - currentReplicateCommandCount += 1; - currentBytesToReplicate += 100L; - assertEquals(currentReplicateCommandCount, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - assertEquals(currentReplicateCommandCount, - replicationManager.getMetrics().getReplicationCmdsSentTotal()); - assertEquals(currentBytesToReplicate, - replicationManager.getMetrics().getReplicationBytesTotal()); - assertEquals(1, getInflightCount(InflightType.REPLICATION)); - assertEquals(1, replicationManager.getMetrics() - .getInflightReplication()); - - // The quasi closed container cannot be closed, but it should have been - // restored to full replication on the previous run. - // The unhealthy replica should remain until the next iteration. - ReplicationManagerReport report = replicationManager.getContainerReport(); - assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - assertEquals(0, report.getStat( - ReplicationManagerReport.HealthState.UNDER_REPLICATED)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.UNHEALTHY)); - - // Create the replica so replication manager sees it on the next run. - List replicateCommands = datanodeCommandHandler - .getReceivedCommands().stream() - .filter(c -> c.getCommand().getType() - .equals(SCMCommandProto.Type.replicateContainerCommand)) - .collect(Collectors.toList()); - for (CommandForDatanode replicateCommand: replicateCommands) { - DatanodeDetails newNode = createDatanodeDetails( - replicateCommand.getDatanodeId()); - ContainerReplica newReplica = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, newNode); - containerStateManager.updateContainerReplica(id, newReplica); - } - - // On the next run, the unhealthy container should be scheduled for - // deletion, since the quasi closed container is now sufficiently - // replicated. - // This method runs an iteration of replication manager. - assertDeleteScheduled(1); - assertExactDeleteTargets(unhealthyReplica.getDatanodeDetails()); - // Replication should have finished on the previous iteration, leaving - // these numbers unchanged. - assertEquals(currentReplicateCommandCount, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - assertEquals(currentReplicateCommandCount, - replicationManager.getMetrics().getReplicationCmdsSentTotal()); - assertEquals(currentBytesToReplicate, - replicationManager.getMetrics().getReplicationBytesTotal()); - assertEquals(0, getInflightCount(InflightType.REPLICATION)); - assertEquals(0, replicationManager.getMetrics().getInflightReplication()); - - // Now we will delete the unhealthy replica. - containerStateManager.removeContainerReplica(id, unhealthyReplica); - - // There should be no work left on the following runs. - replicationManager.processAll(); - eventQueue.processAll(1000); - // The two commands shown are the previous delete and replicate commands. - assertEquals(2, datanodeCommandHandler.getInvocation()); - } - - - /** - * Container is quasi closed. - * 3 quasi-closed replicas with the same origin node ID. - * 1 unhealthy replica with unique origin node ID. - * - * Expectation: - * No action taken. 3 healthy replicas are present. The unhealthy replica - * should not be deleted since it has a unique origin node ID. The - * container cannot be closed because there are not enough healthy unique - * origin node IDs. - */ - @Test - public void testQuasiClosedContainerWithUniqueUnhealthyReplica() - throws IOException, TimeoutException { - final ContainerInfo container = getContainer(LifeCycleState.QUASI_CLOSED); - container.setUsedBytes(100); - final ContainerID id = container.containerID(); - final UUID originNodeId = UUID.randomUUID(); - final ContainerReplica replicaOne = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaTwo = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaThree = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replica4 = getReplicas( - id, UNHEALTHY, 1000L, randomDatanodeDetails().getUuid(), - randomDatanodeDetails()); - - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, replicaOne); - containerStateManager.updateContainerReplica(id, replicaTwo); - containerStateManager.updateContainerReplica(id, replicaThree); - containerStateManager.updateContainerReplica(id, replica4); - - replicationManager.processAll(); - eventQueue.processAll(1000); - - assertEquals(0, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - - assertEquals(0, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.deleteContainerCommand)); - - // now, add a copy of the UNHEALTHY replica on a decommissioning node, - // and another on a dead node. The expectation is still that no replica - // should be deleted as both these nodes are likely going away soon. - final ContainerReplica replica5 = getReplicas( - id, UNHEALTHY, 1000L, replica4.getOriginDatanodeId(), - randomDatanodeDetails()); - nodeManager.register(replica5.getDatanodeDetails(), - new NodeStatus(DECOMMISSIONING, HEALTHY)); - DatanodeDetails deadNode = randomDatanodeDetails(); - nodeManager.register(deadNode, NodeStatus.inServiceDead()); - final ContainerReplica replica6 = getReplicas( - id, UNHEALTHY, 1000L, replica4.getOriginDatanodeId(), - deadNode); - containerStateManager.updateContainerReplica(container.containerID(), - replica5); - containerStateManager.updateContainerReplica(container.containerID(), - replica6); - - replicationManager.processAll(); - eventQueue.processAll(1000); - - assertEquals(0, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - - assertEquals(0, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.deleteContainerCommand)); - - ReplicationManagerReport report = replicationManager.getContainerReport(); - assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - assertEquals(0, report.getStat( - ReplicationManagerReport.HealthState.UNDER_REPLICATED)); - // Even though we have extra replicas, we are deliberately keeping them - // since they are unique. This does not count as over-replication. - assertEquals(0, report.getStat( - ReplicationManagerReport.HealthState.OVER_REPLICATED)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.UNHEALTHY)); - } - - /** - * Situation: QUASI_CLOSED container with 3 QUASI_CLOSED replicas - * whose Sequence ID is smaller than the container's. There are 2 - * UNHEALTHY replicas with Sequence ID same as the container's. One of - * them is on a decommissioning node. - * - * Expectation: Replication command should be sent for the UNHEALTHY - * replica on the decommissioning node. - */ - @Test - public void testQuasiClosedHavingUnhealthyReplicaWithGreatestBCSID() - throws IOException { - final ContainerInfo container = getContainer(LifeCycleState.QUASI_CLOSED); - container.setUsedBytes(100); - final ContainerID id = container.containerID(); - final UUID originNodeId = UUID.randomUUID(); - long sequenceID = container.getSequenceId(); - final ContainerReplica replica1 = - getReplicas(id, QUASI_CLOSED, sequenceID - 1, originNodeId, - randomDatanodeDetails()); - final ContainerReplica replica2 = getReplicas( - id, QUASI_CLOSED, sequenceID - 1, originNodeId, - randomDatanodeDetails()); - final ContainerReplica replica3 = getReplicas( - id, QUASI_CLOSED, sequenceID - 1, originNodeId, - randomDatanodeDetails()); - DatanodeDetails decommissioning = - MockDatanodeDetails.randomDatanodeDetails(); - decommissioning.setPersistedOpState(DECOMMISSIONING); - nodeManager.register(decommissioning, - new NodeStatus(DECOMMISSIONING, HEALTHY)); - final ContainerReplica replica4 = getReplicas( - id, UNHEALTHY, sequenceID, decommissioning.getUuid(), - decommissioning); - final ContainerReplica replica5 = getReplicas( - id, UNHEALTHY, sequenceID, randomDatanodeDetails().getUuid(), - randomDatanodeDetails()); - - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, replica1); - containerStateManager.updateContainerReplica(id, replica2); - containerStateManager.updateContainerReplica(id, replica3); - containerStateManager.updateContainerReplica(id, replica4); - containerStateManager.updateContainerReplica(id, replica5); - - replicationManager.processAll(); - eventQueue.processAll(1000); - - // 1 replicate command should have been sent - assertEquals(1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - assertEquals(1, - replicationManager.getContainerReport().getStat( - ReplicationManagerReport.HealthState.UNDER_REPLICATED)); - - // the following code asserts that replicate was sent for the UNHEALTHY - // replica on the decommissioning node - CommandForDatanode command = - datanodeCommandHandler.getReceivedCommands().iterator().next(); - assertEquals(SCMCommandProto.Type.replicateContainerCommand, - command.getCommand().getType()); - ReplicateContainerCommand replicateCommand = - (ReplicateContainerCommand) command.getCommand(); - assertEquals(1, replicateCommand.getSourceDatanodes().size()); - assertEquals(replica4.getDatanodeDetails(), - replicateCommand.getSourceDatanodes().iterator().next()); - - assertEquals(0, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.deleteContainerCommand)); - - // If we don't complete the pending add by the next iteration, it's - // expected that another replicate command is not sent. - replicationManager.processAll(); - eventQueue.processAll(100); - - // that 1 command is the one RM sent in the last iteration - assertEquals(1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - assertEquals(0, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.deleteContainerCommand)); - - // Now, we will complete the add. Expectation is that no new commands - // should be sent in the next iteration. - ContainerReplica newReplica = - getReplicas(container.containerID(), UNHEALTHY, - container.getSequenceId(), replica4.getOriginDatanodeId(), - MockDatanodeDetails.createDatanodeDetails( - command.getDatanodeId())); - containerStateManager.updateContainerReplica(container.containerID(), - newReplica); - - replicationManager.processAll(); - eventQueue.processAll(100); - - // that 1 command is the one RM sent in the last iteration - assertEquals(1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - assertEquals(0, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.deleteContainerCommand)); - } - - /** - * Situation: QUASI_CLOSED container with 2 QUASI_CLOSED replicas - * whose Sequence ID is smaller than the container's. There's 1 - * UNHEALTHY replica with Sequence ID same as the container's and on a - * decommissioning node. - *

- * Expectation: First, one of the QUASI_CLOSED should be replicated to - * get 3 of them. In the next iteration, the UNHEALTHY replica should be - * replicated. This also verifies that HDDS-9321 does not introduce a - * regression in the under replication flow. - */ - @Test - public void testUnderRepQuasiClosedHavingUnhealthyWithGreatestBCSID() - throws IOException { - final ContainerInfo container = getContainer(LifeCycleState.QUASI_CLOSED); - container.setUsedBytes(100); - final ContainerID id = container.containerID(); - final UUID originNodeId = UUID.randomUUID(); - long sequenceID = container.getSequenceId(); - final ContainerReplica replica1 = - getReplicas(id, QUASI_CLOSED, sequenceID - 1, originNodeId, - randomDatanodeDetails()); - final ContainerReplica replica2 = getReplicas( - id, QUASI_CLOSED, sequenceID - 1, originNodeId, - randomDatanodeDetails()); - DatanodeDetails decommissioning = - MockDatanodeDetails.randomDatanodeDetails(); - decommissioning.setPersistedOpState(DECOMMISSIONING); - final ContainerReplica unhealthyReplica = getReplicas( - id, UNHEALTHY, sequenceID, decommissioning.getUuid(), - decommissioning); - - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, replica1); - containerStateManager.updateContainerReplica(id, replica2); - containerStateManager.updateContainerReplica(id, unhealthyReplica); - - replicationManager.processAll(); - eventQueue.processAll(1000); - - assertEquals(1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - assertEquals(1, - replicationManager.getContainerReport().getStat( - ReplicationManagerReport.HealthState.UNDER_REPLICATED)); - - // the following code asserts that the replicate command was sent for - // either of the QUASI_CLOSED replicas - CommandForDatanode command = - datanodeCommandHandler.getReceivedCommands().iterator().next(); - assertEquals(SCMCommandProto.Type.replicateContainerCommand, - command.getCommand().getType()); - ReplicateContainerCommand replicateCommand = - (ReplicateContainerCommand) command.getCommand(); - List sourceDatanodes = - replicateCommand.getSourceDatanodes(); - assertEquals(2, sourceDatanodes.size()); - assertThat(sourceDatanodes).doesNotContain(unhealthyReplica.getDatanodeDetails()); - assertEquals(0, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.deleteContainerCommand)); - - // now, add a QUASI_CLOSED replica, which is a copy of replica1 - ContainerReplica newReplica = - getReplicas(container.containerID(), QUASI_CLOSED, - container.getSequenceId() - 1, replica1.getOriginDatanodeId(), - MockDatanodeDetails.createDatanodeDetails( - command.getDatanodeId())); - containerStateManager.updateContainerReplica(container.containerID(), - newReplica); - datanodeCommandHandler.clearState(); - replicationManager.processAll(); - eventQueue.processAll(1000); - - assertEquals(1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - assertEquals(1, - replicationManager.getContainerReport().getStat( - ReplicationManagerReport.HealthState.UNDER_REPLICATED)); - assertEquals(1, - datanodeCommandHandler.getReceivedCommands().size()); - command = datanodeCommandHandler.getReceivedCommands().iterator().next(); - assertEquals(SCMCommandProto.Type.replicateContainerCommand, - command.getCommand().getType()); - replicateCommand = (ReplicateContainerCommand) command.getCommand(); - assertEquals(1, replicateCommand.getSourceDatanodes().size()); - assertEquals(unhealthyReplica.getDatanodeDetails(), - replicateCommand.getSourceDatanodes().iterator().next()); - assertEquals(0, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.deleteContainerCommand)); - } - - /** - * 2 closed replica - * 1 quasi-closed replicas - * SCM state is closed. - * Expectation: The replicate container command should only contain the - * closed replicas as sources. - */ - @Test - public void testOnlyMatchingClosedReplicasReplicated() - throws IOException { - final ContainerInfo container = getContainer(LifeCycleState.CLOSED); - final ContainerID id = container.containerID(); - final UUID originNodeId = UUID.randomUUID(); - final ContainerReplica quasiReplica = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica closedReplica1 = getReplicas( - id, State.CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final DatanodeDetails datanodeDetails = randomDatanodeDetails(); - final ContainerReplica closedReplica2 = getReplicas( - id, State.CLOSED, 1000L, datanodeDetails.getUuid(), datanodeDetails); - - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, quasiReplica); - containerStateManager.updateContainerReplica(id, closedReplica1); - containerStateManager.updateContainerReplica(id, closedReplica2); - - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - - Optional cmdOptional = - datanodeCommandHandler.getReceivedCommands().stream().findFirst(); - assertTrue(cmdOptional.isPresent()); - SCMCommand scmCmd = cmdOptional.get().getCommand(); - assertInstanceOf(ReplicateContainerCommand.class, scmCmd); - ReplicateContainerCommand repCmd = (ReplicateContainerCommand) scmCmd; - - // Only the closed replicas should have been used as sources. - List repSources = repCmd.getSourceDatanodes(); - assertEquals(2, repSources.size()); - assertThat(repSources).containsAll( - Arrays.asList(closedReplica1.getDatanodeDetails(), - closedReplica2.getDatanodeDetails())); - assertThat(repSources).doesNotContain(quasiReplica.getDatanodeDetails()); - } - - /** - * 2 quasi-closed replicas - * 1 unhealthy replica - * SCM state is quasi-closed. - * Expectation: The replicate container command should only contain the - * quasi-closed replicas as sources. - */ - @Test - public void testOnlyMatchingQuasiClosedReplicasReplicated() - throws IOException { - final ContainerInfo container = getContainer(LifeCycleState.QUASI_CLOSED); - final ContainerID id = container.containerID(); - final UUID originNodeId = UUID.randomUUID(); - final ContainerReplica quasiReplica1 = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica quasiReplica2 = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final DatanodeDetails datanodeDetails = randomDatanodeDetails(); - final ContainerReplica unhealthyReplica = getReplicas( - id, UNHEALTHY, 1000L, datanodeDetails.getUuid(), datanodeDetails); - - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, quasiReplica1); - containerStateManager.updateContainerReplica(id, quasiReplica2); - containerStateManager.updateContainerReplica(id, unhealthyReplica); - - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - - Optional cmdOptional = - datanodeCommandHandler.getReceivedCommands().stream().findFirst(); - assertTrue(cmdOptional.isPresent()); - SCMCommand scmCmd = cmdOptional.get().getCommand(); - assertInstanceOf(ReplicateContainerCommand.class, scmCmd); - ReplicateContainerCommand repCmd = (ReplicateContainerCommand) scmCmd; - - // Only the quasi closed replicas should have been used as a sources. - List repSources = repCmd.getSourceDatanodes(); - assertEquals(2, repSources.size()); - assertThat(repSources).containsAll( - Arrays.asList(quasiReplica1.getDatanodeDetails(), - quasiReplica2.getDatanodeDetails())); - assertThat(repSources).doesNotContain(unhealthyReplica.getDatanodeDetails()); - } - - /** - * Container is closed. - * 2 quasi-closed replicas. - * 1 unhealthy replica. - * All replicas have unique origin node IDs. - * Quasi closed replicas BCS IDs match closed container's BCS ID. - * - * Expectation: - * Iteration 1: Quasi closed replicas are closed since their BCS IDs - * match the closed container state. - * Iteration 2: The now closed replicas are replicated. - * Iteration 3: The unhealthy replica is deleted. - */ - @Test - public void testCloseableContainerWithUniqueUnhealthyReplica() - throws Exception { - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - ContainerReplica quasi1 = addReplicaToDn(container, - randomDatanodeDetails(), QUASI_CLOSED, container.getSequenceId()); - ContainerReplica quasi2 = addReplicaToDn(container, - randomDatanodeDetails(), QUASI_CLOSED, container.getSequenceId()); - ContainerReplica unhealthyReplica = addReplicaToDn(container, - randomDatanodeDetails(), - UNHEALTHY, - 900L); - - // First RM iteration. - // The quasi containers should be closed since their BCSIDs match the - // closed container's state. - assertDeleteScheduled(0); - // All the containers are unhealthy, so it will not be counted as under - // replicated. - assertUnderReplicatedCount(0); - assertEquals(2, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.closeContainerCommand)); - - // Update RM with the results of the close commands. - ContainerReplica closedRep1 = getReplicas( - container.containerID(), CLOSED, - container.getSequenceId(), quasi1.getDatanodeDetails()) - .stream().findFirst().get(); - ContainerReplica closedRep2 = getReplicas( - container.containerID(), CLOSED, - container.getSequenceId(), quasi2.getDatanodeDetails()) - .stream().findFirst().get(); - - containerStateManager.updateContainerReplica(container.containerID(), - closedRep1); - containerStateManager.updateContainerReplica(container.containerID(), - closedRep2); - - // Second RM iteration - // Now that we have healthy replicas, they should be replicated. - assertDeleteScheduled(0); - assertUnderReplicatedCount(1); - assertEquals(1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - - - // Process the replicate command and report the replica back to SCM. - List replicateCommands = datanodeCommandHandler - .getReceivedCommands().stream() - .filter(c -> c.getCommand().getType() - .equals(SCMCommandProto.Type.replicateContainerCommand)) - .collect(Collectors.toList()); - - // Report the new replica to SCM. - for (CommandForDatanode replicateCommand: replicateCommands) { - DatanodeDetails newNode = createDatanodeDetails( - replicateCommand.getDatanodeId()); - ContainerReplica newReplica = getReplicas( - container.containerID(), CLOSED, - container.getSequenceId(), newNode.getUuid(), newNode); - containerStateManager.updateContainerReplica(container.containerID(), - newReplica); - } - - // Third RM iteration - // The unhealthy replica can be deleted since we have 3 healthy copies - // of a closed container. - assertDeleteScheduled(1); - assertUnderReplicatedCount(0); - assertEquals(1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.deleteContainerCommand)); - assertExactDeleteTargets(unhealthyReplica.getDatanodeDetails()); - } - - /** - * In small clusters, handling an under replicated container can get - * blocked because DNs are occupied by unhealthy replicas. This - * would make the placement policy throw an exception because it could - * not find any target datanodes for new replicas. - * - * Situation: - * Consider a CLOSED container with replicas 1 CLOSED, 1 QUASI_CLOSED with - * same seq id as the container, and 1 QUASI_CLOSED with smaller seq id. - * Placement policy is mocked to simulate no other target DNs are available. - * - * Expectation: - * 1st iteration: QUASI_CLOSED with same seq id should get closed, and the - * one with smaller seq id should get deleted to free up a DN. - * 2nd iteration: Any CLOSED replica should be replicated. - * 3rd iteration: Container should be OK now. - */ - @Test - public void testUnderReplicationBlockedByUnhealthyReplicas() - throws IOException, TimeoutException { - /* - In the first iteration, throw an SCMException to simulate that placement - policy could not find any targets. In the second iteration, return a list - of required targets. - */ - when(ratisContainerPlacementPolicy.chooseDatanodes(any(), any(), anyInt(), anyLong(), anyLong())) - .thenAnswer(invocation -> { - throw new SCMException( - SCMException.ResultCodes.FAILED_TO_FIND_SUITABLE_NODE); - }) - .thenAnswer(invocation -> { - int nodesRequired = invocation.getArgument(2); - List nodes = new ArrayList<>(nodesRequired); - while (nodesRequired != 0) { - nodes.add(MockDatanodeDetails.randomDatanodeDetails()); - nodesRequired--; - } - return nodes; - }); - - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - addReplicaToDn(container, randomDatanodeDetails(), CLOSED, - container.getSequenceId()); - ContainerReplica quasiToDelete = addReplicaToDn(container, - randomDatanodeDetails(), QUASI_CLOSED, container.getSequenceId() - 1); - ContainerReplica quasi2 = addReplicaToDn(container, - randomDatanodeDetails(), QUASI_CLOSED, container.getSequenceId()); - - // First RM iteration. - // this container is under replicated by 2 replicas. - // quasi2 should be closed since its BCSID matches the container's. - // delete command should be sent for quasiToDelete to unblock under rep - // handling. - assertDeleteScheduled(1); - assertTrue(datanodeCommandHandler.received( - SCMCommandProto.Type.closeContainerCommand, - quasi2.getDatanodeDetails())); - assertTrue(datanodeCommandHandler.received( - SCMCommandProto.Type.deleteContainerCommand, - quasiToDelete.getDatanodeDetails())); - assertUnderReplicatedCount(1); - - // Update RM with the results of the close and delete commands - ContainerReplica quasiToClosed = getReplicaBuilder( - container.containerID(), CLOSED, quasi2.getBytesUsed(), - quasi2.getKeyCount(), container.getSequenceId(), - quasi2.getOriginDatanodeId(), quasi2.getDatanodeDetails()).build(); - containerStateManager.updateContainerReplica(container.containerID(), - quasiToClosed); - containerStateManager.removeContainerReplica( - container.containerID(), quasiToDelete); - - // Second RM iteration - // Now that we have a free DN, a closed replica should be replicated - assertReplicaScheduled(1); - assertUnderReplicatedCount(1); - - // Process the replicate command and report the replica back to SCM. - List replicateCommands = datanodeCommandHandler - .getReceivedCommands().stream() - .filter(c -> c.getCommand().getType() - .equals(SCMCommandProto.Type.replicateContainerCommand)) - .collect(Collectors.toList()); - assertEquals(1, replicateCommands.size()); - // Report the new replica to SCM. - for (CommandForDatanode replicateCommand: replicateCommands) { - DatanodeDetails newNode = createDatanodeDetails( - replicateCommand.getDatanodeId()); - ContainerReplica newReplica = getReplicas( - container.containerID(), CLOSED, - container.getSequenceId(), newNode.getUuid(), newNode); - containerStateManager.updateContainerReplica(container.containerID(), - newReplica); - } - - // Third RM iteration - assertReplicaScheduled(0); - assertUnderReplicatedCount(0); - assertOverReplicatedCount(0); - } - - /** - * Test for when a quasi_closed container's under replication cannot be - * solved because there are UNHEALTHY replicas occupying datanodes. - */ - @Test - public void testUnderRepQuasiClosedContainerBlockedByUnhealthyReplicas() - throws IOException, TimeoutException { - when(ratisContainerPlacementPolicy.chooseDatanodes(anyList(), any(), anyInt(), anyLong(), anyLong())) - .thenAnswer(invocation -> { - List excluded = invocation.getArgument(0); - if (excluded.size() == 3) { - throw new SCMException( - SCMException.ResultCodes.FAILED_TO_FIND_SUITABLE_NODE); - } else { - int nodesRequired = invocation.getArgument(2); - List nodes = new ArrayList<>(nodesRequired); - while (nodesRequired != 0) { - DatanodeDetails dn = - MockDatanodeDetails.randomDatanodeDetails(); - nodeManager.register(dn, NodeStatus.inServiceHealthy()); - nodes.add(dn); - nodesRequired--; - } - return nodes; - } - }); - - final ContainerInfo container = - createContainer(LifeCycleState.QUASI_CLOSED); - ContainerReplica quasi1 = addReplicaToDn(container, - randomDatanodeDetails(), QUASI_CLOSED, container.getSequenceId()); - DatanodeDetails nodeForQuasi2 = - MockDatanodeDetails.randomDatanodeDetails(); - nodeManager.register(nodeForQuasi2, NodeStatus.inServiceHealthy()); - ContainerReplica quasi2 = getReplicaBuilder(container.containerID(), - QUASI_CLOSED, container.getUsedBytes(), container.getNumberOfKeys(), - container.getSequenceId(), quasi1.getOriginDatanodeId(), - nodeForQuasi2).build(); - containerStateManager - .updateContainerReplica(container.containerID(), quasi2); - ContainerReplica unhealthy = addReplicaToDn(container, - randomDatanodeDetails(), UNHEALTHY, container.getSequenceId()); - - // First RM iteration. - // this container is under replicated by 1 replica. - // delete command should be sent for unhealthy to unblock under rep - // handling. - assertDeleteScheduled(1); - assertTrue(datanodeCommandHandler.received( - SCMCommandProto.Type.deleteContainerCommand, - unhealthy.getDatanodeDetails())); - - List commands = datanodeCommandHandler - .getReceivedCommands(); - assertEquals(1, commands.size()); - DeleteContainerCommand deleteContainerCommand = - (DeleteContainerCommand) commands.get(0).getCommand(); - assertTrue(deleteContainerCommand.isForce()); - - assertUnderReplicatedCount(1); - // Update RM with the result of delete command - containerStateManager.removeContainerReplica( - container.containerID(), unhealthy); - - // Second RM iteration - // Now that we have a free DN, a quasi_closed replica should be replicated - assertReplicaScheduled(1); - assertUnderReplicatedCount(1); - - // Process the replicate command and report the replica back to SCM. - List replicateCommands = - datanodeCommandHandler.getReceivedCommands().stream() - .filter(command -> command.getCommand().getType() - .equals(SCMCommandProto.Type.replicateContainerCommand)) - .collect(Collectors.toList()); - assertEquals(1, replicateCommands.size()); - ReplicateContainerCommand command = (ReplicateContainerCommand) - replicateCommands.iterator().next().getCommand(); - List sources = command.getSourceDatanodes(); - assertThat(sources).contains(quasi1.getDatanodeDetails(), quasi2.getDatanodeDetails()); - ContainerReplica replica3 = - getReplicas(container.containerID(), QUASI_CLOSED, - container.getSequenceId(), quasi1.getOriginDatanodeId(), - MockDatanodeDetails.randomDatanodeDetails()); - containerStateManager.updateContainerReplica(container.containerID(), - replica3); - - // Third RM iteration - assertReplicaScheduled(0); - assertUnderReplicatedCount(0); - assertOverReplicatedCount(0); - } - - /** - * $numReplicas unhealthy replicas. - * Expectation: The remaining replicas are scheduled. - */ - @ParameterizedTest - @ValueSource(ints = {1, 2}) - public void testUnderReplicatedWithOnlyUnhealthyReplicas(int numReplicas) - throws Exception { - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - for (int i = 0; i < numReplicas; i++) { - addReplica(container, NodeStatus.inServiceHealthy(), UNHEALTHY); - } - int numReplicasNeeded = HddsProtos.ReplicationFactor.THREE_VALUE - - numReplicas; - assertReplicaScheduled(numReplicasNeeded); - assertUnderReplicatedCount(1); - } - - @ParameterizedTest - @EnumSource(value = LifeCycleState.class, - names = {"CLOSED", "QUASI_CLOSED"}) - public void testUnderReplicatedWithOnlyUnhealthyReplicasDecommission( - LifeCycleState state) - throws Exception { - final ContainerInfo container = createContainer(state); - for (int i = 0; i < 2; i++) { - addReplica(container, NodeStatus.inServiceHealthy(), UNHEALTHY); - } - addReplica(container, new NodeStatus(DECOMMISSIONING, HEALTHY), - UNHEALTHY); - assertReplicaScheduled(1); - assertUnderReplicatedCount(1); - // Run again, and there should be a pending add scheduled, so nothing - // else should get scheduled. - assertReplicaScheduled(0); - } - - @ParameterizedTest - @EnumSource(value = LifeCycleState.class, - names = {"CLOSED", "QUASI_CLOSED"}) - public void testOverReplicatedWithOnlyUnhealthyReplicas( - LifeCycleState state) throws Exception { - final ContainerInfo container = createContainer(state); - for (int i = 0; i < 4; i++) { - addReplica(container, NodeStatus.inServiceHealthy(), UNHEALTHY); - } - assertDeleteScheduled(1); - assertUnderReplicatedCount(0); - assertOverReplicatedCount(1); - - // Run again, and there should be a pending delete scheduled, so nothing - // else should get scheduled. - assertDeleteScheduled(0); - } - - /** - * 1 unhealthy replica. - * 4 closed replicas. - * Expectation: - * Iteration 1: The unhealthy replica should be deleted. - * Iteration 2: One of the closed replicas should be deleted. - */ - @Test - public void testOverReplicatedClosedAndUnhealthy() throws Exception { - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - ContainerReplica unhealthy = addReplica(container, - NodeStatus.inServiceHealthy(), UNHEALTHY); - addReplica(container, NodeStatus.inServiceHealthy(), CLOSED); - addReplica(container, NodeStatus.inServiceHealthy(), CLOSED); - addReplica(container, NodeStatus.inServiceHealthy(), CLOSED); - addReplica(container, NodeStatus.inServiceHealthy(), CLOSED); - - // This method does one run of replication manager. - assertReplicaScheduled(0); - assertUnderReplicatedCount(0); - boolean unhealthyDeleted = false; - boolean closedDeleted = false; - - for (CommandForDatanode command : - datanodeCommandHandler.getReceivedCommands()) { - if (command.getCommand().getType() == - SCMCommandProto.Type.deleteContainerCommand) { - if (command.getDatanodeId() == - unhealthy.getDatanodeDetails().getUuid()) { - unhealthyDeleted = true; - } else { - closedDeleted = true; - } - } - } - - assertTrue(unhealthyDeleted); - assertFalse(closedDeleted); - - containerStateManager.removeContainerReplica( - container.containerID(), unhealthy); - - // Do a second run. - assertReplicaScheduled(0); - assertUnderReplicatedCount(0); - for (CommandForDatanode command : - datanodeCommandHandler.getReceivedCommands()) { - if (command.getCommand().getType() == - SCMCommandProto.Type.deleteContainerCommand) { - if (command.getDatanodeId() == - unhealthy.getDatanodeDetails().getUuid()) { - unhealthyDeleted = true; - } else { - closedDeleted = true; - } - } - } - - assertTrue(unhealthyDeleted); - assertTrue(closedDeleted); - } - - /** - * 4 unhealthy replicas. - * Expectation: One unhealthy replica should be deleted. - */ - @Test - public void testOverReplicatedUnhealthy() throws Exception { - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - Set unhealthyContainerDNIDs = new HashSet<>(); - - final int numReplicas = 4; - for (int i = 0; i < numReplicas; i++) { - ContainerReplica replica = addReplica(container, - NodeStatus.inServiceHealthy(), UNHEALTHY); - unhealthyContainerDNIDs.add(replica.getDatanodeDetails().getUuid()); - } - - // No replications should be scheduled. - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(0, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - assertUnderReplicatedCount(0); - - // One replica should be deleted. - assertEquals(1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.deleteContainerCommand)); - assertTrue( - datanodeCommandHandler.getReceivedCommands().stream() - .anyMatch(c -> c.getCommand().getType() == - SCMCommandProto.Type.deleteContainerCommand && - unhealthyContainerDNIDs.contains(c.getDatanodeId()))); - } - - /** - * 4 quasi-closed replicas. - * All have same origin node ID. - * Expectation: One of the replicas is deleted. - */ - @Test - public void testOverReplicatedQuasiClosedContainer() - throws IOException, TimeoutException { - final ContainerInfo container = getContainer(LifeCycleState.QUASI_CLOSED); - container.setUsedBytes(101); - final ContainerID id = container.containerID(); - final UUID originNodeId = UUID.randomUUID(); - final ContainerReplica replicaOne = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaTwo = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaThree = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaFour = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, replicaOne); - containerStateManager.updateContainerReplica(id, replicaTwo); - containerStateManager.updateContainerReplica( - id, replicaThree); - containerStateManager.updateContainerReplica(id, replicaFour); - - final int currentDeleteCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand); - - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(currentDeleteCommandCount + 1, - datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand)); - assertEquals(currentDeleteCommandCount + 1, - replicationManager.getMetrics().getDeletionCmdsSentTotal()); - assertEquals(1, getInflightCount(InflightType.DELETION)); - assertEquals(1, replicationManager.getMetrics() - .getInflightDeletion()); - - ReplicationManagerReport report = replicationManager.getContainerReport(); - assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.OVER_REPLICATED)); - - // Now we remove the replica according to inflight - DatanodeDetails targetDn = - replicationManager.getLegacyReplicationManager() - .getFirstDatanode(InflightType.DELETION, id); - if (targetDn.equals(replicaOne.getDatanodeDetails())) { - containerStateManager.removeContainerReplica( - id, replicaOne); - } else if (targetDn.equals(replicaTwo.getDatanodeDetails())) { - containerStateManager.removeContainerReplica( - id, replicaTwo); - } else if (targetDn.equals(replicaThree.getDatanodeDetails())) { - containerStateManager.removeContainerReplica( - id, replicaThree); - } else if (targetDn.equals(replicaFour.getDatanodeDetails())) { - containerStateManager.removeContainerReplica( - id, replicaFour); - } - - final long currentDeleteCommandCompleted = replicationManager.getMetrics() - .getReplicasDeletedTotal(); - final long deleteBytesCompleted = - replicationManager.getMetrics().getDeletionBytesCompletedTotal(); - - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(0, getInflightCount(InflightType.DELETION)); - assertEquals(0, replicationManager.getMetrics() - .getInflightDeletion()); - assertEquals(currentDeleteCommandCompleted + 1, - replicationManager.getMetrics().getReplicasDeletedTotal()); - assertEquals(deleteBytesCompleted + 101, - replicationManager.getMetrics().getDeletionBytesCompletedTotal()); - - report = replicationManager.getContainerReport(); - assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - assertEquals(0, report.getStat( - ReplicationManagerReport.HealthState.OVER_REPLICATED)); - } - - /** - * 2 open replicas. - * 1 unhealthy replica. - * Expectation: Container is closed. - * - * ReplicationManager should close the unhealthy OPEN container. - */ - @Test - public void testUnhealthyOpenContainer() - throws IOException, TimeoutException { - final ContainerInfo container = getContainer(LifeCycleState.OPEN); - final ContainerID id = container.containerID(); - final Set replicas = getReplicas(id, State.OPEN, - randomDatanodeDetails(), - randomDatanodeDetails()); - replicas.addAll(getReplicas(id, UNHEALTHY, randomDatanodeDetails())); - - containerStateManager.addContainer(container.getProtobuf()); - for (ContainerReplica replica : replicas) { - containerStateManager.updateContainerReplica(id, replica); - } - - final CloseContainerEventHandler closeContainerHandler = mock(CloseContainerEventHandler.class); - eventQueue.addHandler(SCMEvents.CLOSE_CONTAINER, closeContainerHandler); - - replicationManager.processAll(); - eventQueue.processAll(1000); - verify(closeContainerHandler, times(1)).onMessage(id, eventQueue); - - ReplicationManagerReport report = replicationManager.getContainerReport(); - assertEquals(1, report.getStat(LifeCycleState.OPEN)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.OPEN_UNHEALTHY)); - } - - /** - * 1 unhealthy replica. - * 2 open replicas. - * Expectation: Close command should be sent to open replicas only. - * - * ReplicationManager should skip send close command to unhealthy replica. - */ - @Test - public void testCloseUnhealthyReplica() - throws IOException, TimeoutException { - final ContainerInfo container = getContainer(LifeCycleState.CLOSING); - final ContainerID id = container.containerID(); - final Set replicas = getReplicas(id, UNHEALTHY, - randomDatanodeDetails()); - replicas.addAll(getReplicas(id, State.OPEN, randomDatanodeDetails())); - replicas.addAll(getReplicas(id, State.OPEN, randomDatanodeDetails())); - - containerStateManager.addContainer(container.getProtobuf()); - for (ContainerReplica replica : replicas) { - containerStateManager.updateContainerReplica(id, replica); - } - - replicationManager.processAll(); - // Wait for EventQueue to call the event handler - eventQueue.processAll(1000); - assertEquals(2, - datanodeCommandHandler.getInvocation()); - } - - /** - * 1 unhealthy replica. - * 3 quasi closed replicas. - * All have same origin node ID. - * Expectation: Unhealthy replica is deleted. - * - * When a QUASI_CLOSED container is over replicated, ReplicationManager - * deletes the excess replicas. While choosing the replica for deletion - * ReplicationManager should prioritize deleting the unhealthy replica over - * QUASI_CLOSED replica if the unhealthy replica does not have a unique - * origin node ID. - */ - @Test - public void testQuasiClosedContainerWithExtraUnhealthyReplica() - throws IOException, TimeoutException { - final ContainerInfo container = getContainer(LifeCycleState.QUASI_CLOSED); - final ContainerID id = container.containerID(); - final UUID originNodeId = UUID.randomUUID(); - final ContainerReplica unhealthyReplica = getReplicas( - id, UNHEALTHY, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaTwo = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaThree = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaFour = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, unhealthyReplica); - containerStateManager.updateContainerReplica(id, replicaTwo); - containerStateManager.updateContainerReplica(id, replicaThree); - containerStateManager.updateContainerReplica(id, replicaFour); - - assertDeleteScheduled(1); - assertTrue( - datanodeCommandHandler.getReceivedCommands().stream() - .anyMatch(c -> c.getCommand().getType() == - SCMCommandProto.Type.deleteContainerCommand && - c.getDatanodeId().equals( - unhealthyReplica.getDatanodeDetails().getUuid()))); - - ReplicationManagerReport report = replicationManager.getContainerReport(); - assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - // Container should have been considered over replicated including the - // unhealthy replica. - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.OVER_REPLICATED)); - - final long currentDeleteCommandCompleted = replicationManager.getMetrics() - .getReplicasDeletedTotal(); - // Now we remove the replica to simulate deletion complete - containerStateManager.removeContainerReplica(id, unhealthyReplica); - - // On the next run, the over replicated status should be reconciled and - // the delete completed. - replicationManager.processAll(); - eventQueue.processAll(1000); - - assertEquals(currentDeleteCommandCompleted + 1, - replicationManager.getMetrics().getReplicasDeletedTotal()); - assertEquals(0, getInflightCount(InflightType.DELETION)); - assertEquals(0, replicationManager.getMetrics() - .getInflightDeletion()); - - report = replicationManager.getContainerReport(); - assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - assertEquals(0, report.getStat( - ReplicationManagerReport.HealthState.OVER_REPLICATED)); - } - - /** - * 2 quasi-closed replicas. - * Expectation: Replicate one of the replicas. - */ - @Test - public void testUnderReplicatedQuasiClosedContainer() - throws IOException, TimeoutException { - final ContainerInfo container = getContainer(LifeCycleState.QUASI_CLOSED); - container.setUsedBytes(100); - final ContainerID id = container.containerID(); - final UUID originNodeId = UUID.randomUUID(); - final ContainerReplica replicaOne = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaTwo = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, replicaOne); - containerStateManager.updateContainerReplica(id, replicaTwo); - - final int currentReplicateCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.replicateContainerCommand); - final long currentBytesToReplicate = replicationManager.getMetrics() - .getReplicationBytesTotal(); - - // On the first iteration, one of the quasi closed replicas should be - // replicated. - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(currentReplicateCommandCount + 1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - assertEquals(currentReplicateCommandCount + 1, - replicationManager.getMetrics().getReplicationCmdsSentTotal()); - assertEquals(currentBytesToReplicate + 100, - replicationManager.getMetrics().getReplicationBytesTotal()); - assertEquals(1, getInflightCount(InflightType.REPLICATION)); - assertEquals(1, replicationManager.getMetrics() - .getInflightReplication()); - - ReplicationManagerReport report = replicationManager.getContainerReport(); - assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.UNDER_REPLICATED)); - - final long currentReplicateCommandCompleted = replicationManager - .getMetrics().getReplicasCreatedTotal(); - final long currentReplicateBytesCompleted = replicationManager - .getMetrics().getReplicationBytesCompletedTotal(); - - // Now we add the replicated new replica - DatanodeDetails targetDn = - replicationManager.getLegacyReplicationManager() - .getFirstDatanode(InflightType.REPLICATION, id); - final ContainerReplica replicatedReplicaThree = getReplicas( - id, State.QUASI_CLOSED, 1000L, originNodeId, targetDn); - containerStateManager.updateContainerReplica( - id, replicatedReplicaThree); - - // On the next run, no additional replications should be scheduled. - replicationManager.processAll(); - eventQueue.processAll(1000); - - assertEquals(currentReplicateCommandCompleted + 1, - replicationManager.getMetrics().getReplicasCreatedTotal()); - assertEquals(currentReplicateBytesCompleted + 100, - replicationManager.getMetrics().getReplicationBytesCompletedTotal()); - assertEquals(0, getInflightCount(InflightType.REPLICATION)); - assertEquals(0, replicationManager.getMetrics() - .getInflightReplication()); - - report = replicationManager.getContainerReport(); - assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - assertEquals(0, report.getStat( - ReplicationManagerReport.HealthState.UNDER_REPLICATED)); - } - - /** - * 1 quasi-closed replica. - * 1 unhealthy replica. - * All have same origin node ID. - * Expectation: - * - * In the first iteration of ReplicationManager, it should re-replicate - * the quasi closed replicas so that there are 3 of them. - * - * In the second iteration, ReplicationManager should delete the unhealthy - * replica since its origin node ID is not unique. - */ - @Test - public void testUnderReplicatedQuasiClosedContainerWithUnhealthyReplica() - throws IOException, InterruptedException, - TimeoutException { - final ContainerInfo container = getContainer(LifeCycleState.QUASI_CLOSED); - container.setUsedBytes(99); - final ContainerID id = container.containerID(); - final UUID originNodeId = UUID.randomUUID(); - final ContainerReplica replicaOne = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, - randomDatanodeDetails()); - final ContainerReplica unhealthyReplica = getReplicas( - id, UNHEALTHY, 1000L, originNodeId, - randomDatanodeDetails()); - - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, replicaOne); - containerStateManager.updateContainerReplica(id, unhealthyReplica); - - final int currentReplicateCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.replicateContainerCommand); - final int currentDeleteCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand); - final long currentBytesToDelete = replicationManager.getMetrics() - .getDeletionBytesTotal(); - - // Run first iteraiton - - replicationManager.processAll(); - GenericTestUtils.waitFor( - () -> (currentReplicateCommandCount + 2) == datanodeCommandHandler - .getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand), - 50, 5000); - - ReplicationManagerReport report = replicationManager.getContainerReport(); - assertEquals(1, - report.getStat(LifeCycleState.QUASI_CLOSED)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.UNDER_REPLICATED)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.UNHEALTHY)); - - List replicateCommands = datanodeCommandHandler - .getReceivedCommands().stream() - .filter(c -> c.getCommand().getType() - .equals(SCMCommandProto.Type.replicateContainerCommand)) - .collect(Collectors.toList()); - - assertEquals(2, replicateCommands.size()); - - // Report the two new replicas to SCM. - for (CommandForDatanode replicateCommand: replicateCommands) { - DatanodeDetails newNode = createDatanodeDetails( - replicateCommand.getDatanodeId()); - ContainerReplica newReplica = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, newNode); - containerStateManager.updateContainerReplica(id, newReplica); - } - - // Run second iteration. - // Now that the quasi closed replica is sufficiently replicated, SCM - // should delete the unhealthy replica on the next iteration. - - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(currentDeleteCommandCount + 1, - datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand)); - assertTrue(datanodeCommandHandler.received( - SCMCommandProto.Type.deleteContainerCommand, - unhealthyReplica.getDatanodeDetails())); - assertEquals(currentDeleteCommandCount + 1, - replicationManager.getMetrics().getDeletionCmdsSentTotal()); - assertEquals(currentBytesToDelete + 99, - replicationManager.getMetrics().getDeletionBytesTotal()); - assertEquals(1, - getInflightCount(InflightType.DELETION)); - assertEquals(1, replicationManager.getMetrics() - .getInflightDeletion()); - - containerStateManager.removeContainerReplica(id, unhealthyReplica); - - report = replicationManager.getContainerReport(); - assertEquals(1, - report.getStat(LifeCycleState.QUASI_CLOSED)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - assertEquals(0, report.getStat( - ReplicationManagerReport.HealthState.UNDER_REPLICATED)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.UNHEALTHY)); - } - - /** - * If a QUASI_CLOSED container and a QUASI_CLOSED replica have the same - * sequence ID, it's not guaranteed that this is the latest sequence ID. - * Such a replica should not be closed. - */ - @Test - public void testQuasiClosedContainerAndReplicaWithSameSequenceID() - throws IOException, TimeoutException { - /* - Create an under replicated QUASI_CLOSED container with 2 QUASI_CLOSED - replicas. All have the container's sequence ID. - */ - final ContainerInfo container = getContainer(LifeCycleState.QUASI_CLOSED); - container.setUsedBytes(100); - final ContainerID id = container.containerID(); - final UUID originNodeId = UUID.randomUUID(); - final ContainerReplica replicaOne = getReplicas( - id, QUASI_CLOSED, container.getSequenceId(), originNodeId, - randomDatanodeDetails()); - final ContainerReplica replicaTwo = getReplicas( - id, QUASI_CLOSED, container.getSequenceId(), originNodeId, - randomDatanodeDetails()); - - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, replicaOne); - containerStateManager.updateContainerReplica(id, replicaTwo); - - final int currentCloseCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.closeContainerCommand); - final int currentReplicateCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.replicateContainerCommand); - final long currentBytesToReplicate = replicationManager.getMetrics() - .getReplicationBytesTotal(); - - /* - One of the quasi closed replicas should be replicated and no close - commands should be sent. - */ - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(currentCloseCommandCount, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.closeContainerCommand)); - - // One replication command should be sent - assertEquals(currentReplicateCommandCount + 1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - assertEquals(currentReplicateCommandCount + 1, - replicationManager.getMetrics().getReplicationCmdsSentTotal()); - assertEquals( - currentBytesToReplicate + container.getUsedBytes(), - replicationManager.getMetrics().getReplicationBytesTotal()); - assertEquals(1, getInflightCount(InflightType.REPLICATION)); - assertEquals(1, replicationManager.getMetrics() - .getInflightReplication()); - - ReplicationManagerReport report = replicationManager.getContainerReport(); - assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.UNDER_REPLICATED)); - } - - /** - * 3 quasi-closed replicas. - * All unique origin IDs. - * Expectation: Container is closed. - * - * When a container is QUASI_CLOSED and it has >50% of its replica - * in QUASI_CLOSED state with unique origin node id, - * ReplicationManager should force close the replica(s) with - * highest BCSID. - */ - @Test - public void testQuasiClosedToClosed() throws IOException, TimeoutException { - final ContainerInfo container = getContainer(LifeCycleState.QUASI_CLOSED); - final ContainerID id = container.containerID(); - final Set replicas = getReplicas(id, QUASI_CLOSED, - randomDatanodeDetails(), - randomDatanodeDetails(), - randomDatanodeDetails()); - containerStateManager.addContainer(container.getProtobuf()); - for (ContainerReplica replica : replicas) { - containerStateManager.updateContainerReplica(id, replica); - } - - final int currentCloseCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.closeContainerCommand); - - replicationManager.processAll(); - eventQueue.processAll(1000); - - // All the replicas have same BCSID, so all of them will be closed. - assertEquals(currentCloseCommandCount + 3, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.closeContainerCommand)); - - ReplicationManagerReport report = replicationManager.getContainerReport(); - assertEquals(1, - report.getStat(LifeCycleState.QUASI_CLOSED)); - assertEquals(0, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - } - - /** - * 2 quasi-closed replicas. - * 2 unique origin IDs with different BCSIDs. - * Expectation: Container is closed, then replicated on the next iteration. - */ - @Test - public void testCloseableIsClosedBeforeReplication() throws IOException { - final ContainerInfo container = getContainer(LifeCycleState.QUASI_CLOSED); - final ContainerID id = container.containerID(); - final ContainerReplica replicaOne = getReplicas( - id, QUASI_CLOSED, 1000L, UUID.randomUUID(), randomDatanodeDetails()); - final ContainerReplica replicaTwo = getReplicas( - id, QUASI_CLOSED, 900L, UUID.randomUUID(), randomDatanodeDetails()); - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, replicaOne); - containerStateManager.updateContainerReplica(id, replicaTwo); - - // Since we have 2/3 replicas, the container should be closed on this - // iteration. The replica with the higher BCSID can be closed. - int currentCloseCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.closeContainerCommand); - - replicationManager.processAll(); - eventQueue.processAll(1000); - - assertEquals(currentCloseCommandCount + 1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.closeContainerCommand)); - - // The highest BCSID replica should have been closed. - Optional cmd = - datanodeCommandHandler.getReceivedCommands().stream().findFirst(); - assertTrue(cmd.isPresent()); - assertEquals(replicaOne.getDatanodeDetails().getUuid(), - cmd.get().getDatanodeId()); - - ReplicationManagerReport report = replicationManager.getContainerReport(); - // Container will register as closed after the closed replica is - // reported in the next iteration. - assertEquals(0, - report.getStat(LifeCycleState.CLOSED)); - // Legacy replication manager does not count over/under replicated - // status until the container is eligible to be replicated. - assertEquals(0, report.getStat( - ReplicationManagerReport.HealthState.UNDER_REPLICATED)); - assertEquals(1, - report.getStat(LifeCycleState.QUASI_CLOSED)); - assertEquals(0, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - - // Move the higher BCSID replica to closed state. - final ContainerReplica closedReplicaOne = getReplicas( - id, CLOSED, 1000L, replicaOne.getOriginDatanodeId(), - replicaOne.getDatanodeDetails()); - containerStateManager.updateContainerReplica(id, closedReplicaOne); - - // On the next iteration, the container should be replicated. - currentCloseCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.closeContainerCommand); - int currentReplicateCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.replicateContainerCommand); - - replicationManager.processAll(); - eventQueue.processAll(1000); - // No more close commands should be sent. - assertEquals(currentCloseCommandCount, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.closeContainerCommand)); - // Two replicate commands should be triggered for the now closed - // container. - assertEquals(currentReplicateCommandCount + 2, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - assertEquals(currentReplicateCommandCount + 2, - replicationManager.getMetrics().getReplicationCmdsSentTotal()); - assertEquals(1, getInflightCount(InflightType.REPLICATION)); - assertEquals(1, replicationManager.getMetrics() - .getInflightReplication()); - - // Once the replicas are closed, moving the container to CLOSED state - // in SCM is done on the container report, not in the replication manager. - } - } - - /** - * Tests replication manager handling of decommissioning and maintainence - * mode datanodes. - */ - @Nested - class DecomAndMaintenance { - /** - * ReplicationManager should replicate an additional replica if there are - * decommissioned replicas. - */ - @Test - public void testUnderReplicatedDueToOutOfService() - throws IOException, TimeoutException { - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(DECOMMISSIONING, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(DECOMMISSIONING, HEALTHY), CLOSED); - assertReplicaScheduled(2); - assertUnderReplicatedCount(1); - } - - /** - * ReplicationManager should replicate an additional replica when all copies - * are decommissioning. - */ - @Test - public void testUnderReplicatedDueToAllDecommission() - throws IOException, TimeoutException { - runTestUnderReplicatedDueToAllDecommission(3); - } - - Void runTestUnderReplicatedDueToAllDecommission(int expectedReplication) - throws IOException, TimeoutException { - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - addReplica(container, new NodeStatus(DECOMMISSIONING, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(DECOMMISSIONING, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(DECOMMISSIONING, HEALTHY), CLOSED); - assertReplicaScheduled(expectedReplication); - assertUnderReplicatedCount(1); - return null; - } - - @Test - public void testReplicationLimit() throws Exception { - runTestLimit(1, 0, 2, 0, - () -> runTestUnderReplicatedDueToAllDecommission(1)); - } - - /** - * ReplicationManager should not take any action when the container is - * correctly replicated with decommissioned replicas still present. - */ - @Test - public void testCorrectlyReplicatedWithDecommission() - throws IOException, TimeoutException { - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(DECOMMISSIONING, HEALTHY), CLOSED); - assertReplicaScheduled(0); - assertUnderReplicatedCount(0); - } - - /** - * ReplicationManager should replicate an additional replica when min rep - * is not met for maintenance. - */ - @Test - public void testUnderReplicatedDueToMaintenance() - throws IOException, TimeoutException { - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED); - assertReplicaScheduled(1); - assertUnderReplicatedCount(1); - } - - /** - * ReplicationManager should not replicate an additional replica when if - * min replica for maintenance is 1 and another replica is available. - */ - @Test - public void testNotUnderReplicatedDueToMaintenanceMinRepOne() - throws Exception { - replicationManager.stop(); - ReplicationManagerConfiguration newConf = newRMConfig(); - newConf.setMaintenanceReplicaMinimum(1); - dbStore.close(); - createReplicationManager(newConf); - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED); - assertReplicaScheduled(0); - assertUnderReplicatedCount(0); - } - - /** - * ReplicationManager should replicate an additional replica when all copies - * are going off line and min rep is 1. - */ - @Test - public void testUnderReplicatedDueToMaintenanceMinRepOne() - throws Exception { - replicationManager.stop(); - ReplicationManagerConfiguration newConf = newRMConfig(); - newConf.setMaintenanceReplicaMinimum(1); - dbStore.close(); - createReplicationManager(newConf); - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED); - assertReplicaScheduled(1); - assertUnderReplicatedCount(1); - } - - /** - * ReplicationManager should replicate additional replica when all copies - * are going into maintenance. - */ - @Test - public void testUnderReplicatedDueToAllMaintenance() - throws IOException, TimeoutException { - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED); - assertReplicaScheduled(2); - assertUnderReplicatedCount(1); - } - - /** - * ReplicationManager should not replicate additional replica sufficient - * replica are available. - */ - @Test - public void testCorrectlyReplicatedWithMaintenance() - throws IOException, TimeoutException { - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED); - assertReplicaScheduled(0); - assertUnderReplicatedCount(0); - } - - /** - * ReplicationManager should replicate additional replica when all copies - * are decommissioning or maintenance. - */ - @Test - public void testUnderReplicatedWithDecommissionAndMaintenance() - throws IOException, TimeoutException { - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - addReplica(container, new NodeStatus(DECOMMISSIONED, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(DECOMMISSIONED, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED); - assertReplicaScheduled(2); - assertUnderReplicatedCount(1); - } - - /** - * When a CLOSED container is over replicated, ReplicationManager - * deletes the excess replicas. While choosing the replica for deletion - * ReplicationManager should not attempt to remove a DECOMMISSION or - * MAINTENANCE replica. - */ - @Test - public void testOverReplicatedClosedContainerWithDecomAndMaint() - throws IOException, TimeoutException { - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - addReplica(container, NodeStatus.inServiceHealthy(), CLOSED); - addReplica(container, new NodeStatus(DECOMMISSIONED, HEALTHY), CLOSED); - addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED); - addReplica(container, NodeStatus.inServiceHealthy(), CLOSED); - addReplica(container, NodeStatus.inServiceHealthy(), CLOSED); - addReplica(container, NodeStatus.inServiceHealthy(), CLOSED); - addReplica(container, NodeStatus.inServiceHealthy(), CLOSED); - - final int currentDeleteCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand); - - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(currentDeleteCommandCount + 2, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.deleteContainerCommand)); - assertEquals(currentDeleteCommandCount + 2, - replicationManager.getMetrics().getDeletionCmdsSentTotal()); - assertEquals(1, getInflightCount(InflightType.DELETION)); - assertEquals(1, replicationManager.getMetrics() - .getInflightDeletion()); - // Get the DECOM and Maint replica and ensure none of them are scheduled - // for removal - Set decom = - containerStateManager.getContainerReplicas( - container.containerID()) - .stream() - .filter(r -> - r.getDatanodeDetails().getPersistedOpState() != IN_SERVICE) - .collect(Collectors.toSet()); - for (ContainerReplica r : decom) { - assertFalse(datanodeCommandHandler.received( - SCMCommandProto.Type.deleteContainerCommand, - r.getDatanodeDetails())); - } - assertOverReplicatedCount(1); - } - - /** - * Replication Manager should not attempt to replicate from an unhealthy - * (stale or dead) node. To test this, setup a scenario where a replia needs - * to be created, but mark all nodes stale. That way, no new replica will be - * scheduled. - */ - @Test - public void testUnderReplicatedNotHealthySource() - throws IOException, TimeoutException { - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - addReplica(container, NodeStatus.inServiceStale(), CLOSED); - addReplica(container, new NodeStatus(DECOMMISSIONED, STALE), CLOSED); - addReplica(container, new NodeStatus(DECOMMISSIONED, STALE), CLOSED); - // There should be replica scheduled, but as all nodes are stale, nothing - // gets scheduled. - assertReplicaScheduled(0); - assertUnderReplicatedCount(1); - } - } - - private static ReplicationManagerConfiguration newRMConfig() { - ReplicationManagerConfiguration conf = - new ReplicationManagerConfiguration(); - conf.setEnableLegacy(true); - return conf; - } - - /** - * Tests replication manager move command. - */ - @Nested - class Move { - /** - * if all the prerequisites are satisfied, move should work as expected. - */ - @Test - public void testMove() throws IOException, NodeNotFoundException, - InterruptedException, ExecutionException, TimeoutException { - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - ContainerID id = container.containerID(); - ContainerReplica dn1 = addReplica(container, - new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - addReplica(container, - new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - addReplica(container, - new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - DatanodeDetails dn3 = addNode(new NodeStatus(IN_SERVICE, HEALTHY)); - CompletableFuture cf = - replicationManager.move(id, dn1.getDatanodeDetails(), dn3); - assertThat(scmLogs.getOutput()).contains( - "receive a move request about container"); - Thread.sleep(100L); - assertTrue(datanodeCommandHandler.received( - SCMCommandProto.Type.replicateContainerCommand, dn3)); - assertEquals(1, datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - - //replicate container to dn3 - addReplicaToDn(container, dn3, CLOSED); - replicationManager.processAll(); - eventQueue.processAll(1000); - - assertTrue(datanodeCommandHandler.received( - SCMCommandProto.Type.deleteContainerCommand, - dn1.getDatanodeDetails())); - assertEquals(1, datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.deleteContainerCommand)); - containerStateManager.removeContainerReplica(id, dn1); - - replicationManager.processAll(); - eventQueue.processAll(1000); - - assertTrue( - cf.isDone() && cf.get() == MoveManager.MoveResult.COMPLETED); - } - - /** - * if crash happened and restarted, move option should work as expected. - */ - @Test - public void testMoveCrashAndRestart() throws IOException, - NodeNotFoundException, InterruptedException, TimeoutException { - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - ContainerID id = container.containerID(); - ContainerReplica dn1 = addReplica(container, - new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - addReplica(container, - new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - addReplica(container, - new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - DatanodeDetails dn3 = addNode(new NodeStatus(IN_SERVICE, HEALTHY)); - replicationManager.move(id, dn1.getDatanodeDetails(), dn3); - assertThat(scmLogs.getOutput()).contains( - "receive a move request about container"); - Thread.sleep(100L); - assertTrue(datanodeCommandHandler.received( - SCMCommandProto.Type.replicateContainerCommand, dn3)); - assertEquals(1, datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - - //crash happens, restart scm. - //clear current inflight actions and reload inflightMove from DBStore. - resetReplicationManager(); - replicationManager.getMoveScheduler() - .reinitialize(SCMDBDefinition.MOVE.getTable(dbStore)); - assertTrue(replicationManager.getMoveScheduler() - .getInflightMove().containsKey(id)); - MoveDataNodePair kv = replicationManager.getMoveScheduler() - .getInflightMove().get(id); - assertEquals(kv.getSrc(), dn1.getDatanodeDetails()); - assertEquals(kv.getTgt(), dn3); - serviceManager.notifyStatusChanged(); - - Thread.sleep(100L); - // now, the container is not over-replicated, - // so no deleteContainerCommand will be sent - assertFalse(datanodeCommandHandler.received( - SCMCommandProto.Type.deleteContainerCommand, - dn1.getDatanodeDetails())); - //replica does not exist in target datanode, so a - // replicateContainerCommand will be sent again at - // notifyStatusChanged#onLeaderReadyAndOutOfSafeMode - assertEquals(2, datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - - - //replicate container to dn3, now, over-replicated - addReplicaToDn(container, dn3, CLOSED); - replicationManager.processAll(); - eventQueue.processAll(1000); - - //deleteContainerCommand is sent, but the src replica is not deleted now - assertEquals(1, datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.deleteContainerCommand)); - - //crash happens, restart scm. - //clear current inflight actions and reload inflightMove from DBStore. - resetReplicationManager(); - replicationManager.getMoveScheduler() - .reinitialize(SCMDBDefinition.MOVE.getTable(dbStore)); - assertTrue(replicationManager.getMoveScheduler() - .getInflightMove().containsKey(id)); - kv = replicationManager.getMoveScheduler() - .getInflightMove().get(id); - assertEquals(kv.getSrc(), dn1.getDatanodeDetails()); - assertEquals(kv.getTgt(), dn3); - serviceManager.notifyStatusChanged(); - - //after restart and the container is over-replicated now, - //deleteContainerCommand will be sent again - assertEquals(2, datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.deleteContainerCommand)); - containerStateManager.removeContainerReplica(id, dn1); - - //replica in src datanode is deleted now - containerStateManager.removeContainerReplica(id, dn1); - replicationManager.processAll(); - eventQueue.processAll(1000); - - //since the move is complete,so after scm crash and restart - //inflightMove should not contain the container again - resetReplicationManager(); - replicationManager.getMoveScheduler() - .reinitialize(SCMDBDefinition.MOVE.getTable(dbStore)); - assertThat(replicationManager.getMoveScheduler() - .getInflightMove()).doesNotContainKey(id); - - //completeableFuture is not stored in DB, so after scm crash and - //restart ,completeableFuture is missing - } - - /** - * make sure RM does not delete replica if placement policy is not - * satisfied. - */ - @Test - public void testMoveNotDeleteSrcIfPolicyNotSatisfied() - throws IOException, NodeNotFoundException, - InterruptedException, ExecutionException, TimeoutException { - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - ContainerID id = container.containerID(); - ContainerReplica dn1 = addReplica(container, - new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - ContainerReplica dn2 = addReplica(container, - new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - addReplica(container, - new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - DatanodeDetails dn4 = addNode(new NodeStatus(IN_SERVICE, HEALTHY)); - CompletableFuture cf = - replicationManager.move(id, dn1.getDatanodeDetails(), dn4); - assertThat(scmLogs.getOutput()).contains( - "receive a move request about container"); - Thread.sleep(100L); - assertTrue(datanodeCommandHandler.received( - SCMCommandProto.Type.replicateContainerCommand, dn4)); - assertEquals(1, datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - - //replicate container to dn4 - addReplicaToDn(container, dn4, CLOSED); - //now, replication succeeds, but replica in dn2 lost, - //and there are only tree replicas totally, so rm should - //not delete the replica on dn1 - containerStateManager.removeContainerReplica(id, dn2); - replicationManager.processAll(); - eventQueue.processAll(1000); - - assertFalse( - datanodeCommandHandler.received( - SCMCommandProto.Type.deleteContainerCommand, - dn1.getDatanodeDetails())); - - assertTrue(cf.isDone() && - cf.get() == MoveManager.MoveResult.DELETE_FAIL_POLICY); - } - - - /** - * test src and target datanode become unhealthy when moving. - */ - @Test - public void testDnBecameUnhealthyWhenMoving() throws IOException, - NodeNotFoundException, InterruptedException, ExecutionException, - TimeoutException { - final ContainerInfo container = createContainer(LifeCycleState.CLOSED); - ContainerID id = container.containerID(); - ContainerReplica dn1 = addReplica(container, - new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - addReplica(container, - new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - addReplica(container, - new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - DatanodeDetails dn3 = addNode(new NodeStatus(IN_SERVICE, HEALTHY)); - CompletableFuture cf = - replicationManager.move(id, dn1.getDatanodeDetails(), dn3); - assertThat(scmLogs.getOutput()).contains( - "receive a move request about container"); - - nodeManager.setNodeStatus(dn3, new NodeStatus(IN_SERVICE, STALE)); - replicationManager.processAll(); - eventQueue.processAll(1000); - - assertTrue(cf.isDone() && cf.get() == - MoveManager.MoveResult.REPLICATION_FAIL_NODE_UNHEALTHY); - - nodeManager.setNodeStatus(dn3, new NodeStatus(IN_SERVICE, HEALTHY)); - cf = replicationManager.move(id, dn1.getDatanodeDetails(), dn3); - addReplicaToDn(container, dn3, CLOSED); - replicationManager.processAll(); - eventQueue.processAll(1000); - nodeManager.setNodeStatus(dn1.getDatanodeDetails(), - new NodeStatus(IN_SERVICE, STALE)); - replicationManager.processAll(); - eventQueue.processAll(1000); - - assertTrue(cf.isDone() && cf.get() == - MoveManager.MoveResult.DELETION_FAIL_NODE_UNHEALTHY); - } - - /** - * before Replication Manager generates a completablefuture for a move - * option, some Prerequisites should be satisfied. - */ - @Test - public void testMovePrerequisites() throws IOException, - NodeNotFoundException, InterruptedException, ExecutionException, - InvalidStateTransitionException, TimeoutException { - //all conditions is met - final ContainerInfo container = createContainer(LifeCycleState.OPEN); - ContainerID id = container.containerID(); - ContainerReplica dn1 = addReplica(container, - new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - ContainerReplica dn2 = addReplica(container, - new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - DatanodeDetails dn3 = addNode(new NodeStatus(IN_SERVICE, HEALTHY)); - ContainerReplica dn4 = addReplica(container, - new NodeStatus(IN_SERVICE, HEALTHY), CLOSED); - - CompletableFuture cf; - //the above move is executed successfully, so there may be some item in - //inflightReplication or inflightDeletion. here we stop replication - // manager to clear these states, which may impact the tests below. - //we don't need a running replicationManamger now - replicationManager.stop(); - Thread.sleep(100L); - cf = replicationManager.move(id, dn1.getDatanodeDetails(), dn3); - assertTrue(cf.isDone() && cf.get() == - MoveManager.MoveResult.FAIL_UNEXPECTED_ERROR); - replicationManager.start(); - Thread.sleep(100L); - - //container in not in OPEN state - cf = replicationManager.move(id, dn1.getDatanodeDetails(), dn3); - assertTrue(cf.isDone() && cf.get() == - MoveManager.MoveResult.REPLICATION_FAIL_CONTAINER_NOT_CLOSED); - //open -> closing - containerStateManager.updateContainerState(id.getProtobuf(), - LifeCycleEvent.FINALIZE); - cf = replicationManager.move(id, dn1.getDatanodeDetails(), dn3); - assertTrue(cf.isDone() && cf.get() == - MoveManager.MoveResult.REPLICATION_FAIL_CONTAINER_NOT_CLOSED); - //closing -> quasi_closed - containerStateManager.updateContainerState(id.getProtobuf(), - LifeCycleEvent.QUASI_CLOSE); - cf = replicationManager.move(id, dn1.getDatanodeDetails(), dn3); - assertTrue(cf.isDone() && cf.get() == - MoveManager.MoveResult.REPLICATION_FAIL_CONTAINER_NOT_CLOSED); - - //quasi_closed -> closed - containerStateManager.updateContainerState(id.getProtobuf(), - LifeCycleEvent.FORCE_CLOSE); - assertSame(LifeCycleState.CLOSED, - containerStateManager.getContainer(id).getState()); - - //Node is not in healthy state - for (HddsProtos.NodeState state : HddsProtos.NodeState.values()) { - if (state != HEALTHY) { - nodeManager.setNodeStatus(dn3, - new NodeStatus(IN_SERVICE, state)); - cf = replicationManager.move(id, dn1.getDatanodeDetails(), dn3); - assertTrue(cf.isDone() && cf.get() == - MoveManager.MoveResult.REPLICATION_FAIL_NODE_UNHEALTHY); - cf = replicationManager.move(id, dn3, dn1.getDatanodeDetails()); - assertTrue(cf.isDone() && cf.get() == - MoveManager.MoveResult.REPLICATION_FAIL_NODE_UNHEALTHY); - } - } - nodeManager.setNodeStatus(dn3, new NodeStatus(IN_SERVICE, HEALTHY)); - - //Node is not in IN_SERVICE state - for (HddsProtos.NodeOperationalState state : - HddsProtos.NodeOperationalState.values()) { - if (state != IN_SERVICE) { - nodeManager.setNodeStatus(dn3, - new NodeStatus(state, HEALTHY)); - cf = replicationManager.move(id, dn1.getDatanodeDetails(), dn3); - assertTrue(cf.isDone() && cf.get() == - MoveManager.MoveResult.REPLICATION_FAIL_NODE_NOT_IN_SERVICE); - cf = replicationManager.move(id, dn3, dn1.getDatanodeDetails()); - assertTrue(cf.isDone() && cf.get() == - MoveManager.MoveResult.REPLICATION_FAIL_NODE_NOT_IN_SERVICE); - } - } - nodeManager.setNodeStatus(dn3, new NodeStatus(IN_SERVICE, HEALTHY)); - - //container exists in target datanode - cf = replicationManager.move(id, dn1.getDatanodeDetails(), - dn2.getDatanodeDetails()); - assertTrue(cf.isDone() && cf.get() == - MoveManager.MoveResult.REPLICATION_FAIL_EXIST_IN_TARGET); - - //container does not exist in source datanode - cf = replicationManager.move(id, dn3, dn3); - assertTrue(cf.isDone() && cf.get() == - MoveManager.MoveResult.REPLICATION_FAIL_NOT_EXIST_IN_SOURCE); - - //make container over relplicated to test the - // case that container is in inflightDeletion - ContainerReplica dn5 = addReplica(container, - new NodeStatus(IN_SERVICE, HEALTHY), State.CLOSED); - replicationManager.processAll(); - //waiting for inflightDeletion generation - eventQueue.processAll(1000); - cf = replicationManager.move(id, dn1.getDatanodeDetails(), dn3); - assertTrue(cf.isDone() && cf.get() == - MoveManager.MoveResult.REPLICATION_FAIL_INFLIGHT_DELETION); - resetReplicationManager(); - - //make the replica num be 2 to test the case - //that container is in inflightReplication - containerStateManager.removeContainerReplica(id, dn5); - containerStateManager.removeContainerReplica(id, dn4); - //replication manager should generate inflightReplication - replicationManager.processAll(); - //waiting for inflightReplication generation - eventQueue.processAll(1000); - cf = replicationManager.move(id, dn1.getDatanodeDetails(), dn3); - assertTrue(cf.isDone() && cf.get() == - MoveManager.MoveResult.REPLICATION_FAIL_INFLIGHT_REPLICATION); - } - } - - /** - * Tests mis-replicated containers with rack topology information. - */ - @Nested - class MisReplicated { - - @Test - public void additionalReplicaScheduledWhenMisReplicated() - throws IOException, TimeoutException { - final ContainerInfo container = getContainer(LifeCycleState.CLOSED); - container.setUsedBytes(100); - final ContainerID id = container.containerID(); - final UUID originNodeId = UUID.randomUUID(); - final ContainerReplica replicaOne = getReplicas( - id, State.CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaTwo = getReplicas( - id, State.CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaThree = getReplicas( - id, State.CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, replicaOne); - containerStateManager.updateContainerReplica(id, replicaTwo); - containerStateManager.updateContainerReplica( - id, replicaThree); - - // Ensure a mis-replicated status is returned for any containers in this - // test where there are 3 replicas. When there are 2 or 4 replicas - // the status returned will be healthy. - when(ratisContainerPlacementPolicy.validateContainerPlacement( - argThat(list -> list.size() == 3), - anyInt() - )).thenAnswer(invocation -> { - return new ContainerPlacementStatusDefault(1, 2, 3); - }); - - int currentReplicateCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.replicateContainerCommand); - final long currentBytesToReplicate = replicationManager.getMetrics() - .getReplicationBytesTotal(); - - replicationManager.processAll(); - eventQueue.processAll(1000); - // At this stage, due to the mocked calls to validateContainerPlacement - // the policy will not be satisfied, and replication will be triggered. - - assertEquals(currentReplicateCommandCount + 1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - assertEquals(currentReplicateCommandCount + 1, - replicationManager.getMetrics().getReplicationCmdsSentTotal()); - assertEquals(currentBytesToReplicate + 100, - replicationManager.getMetrics().getReplicationBytesTotal()); - assertEquals(1, getInflightCount(InflightType.REPLICATION)); - assertEquals(1, replicationManager.getMetrics() - .getInflightReplication()); - - ReplicationManagerReport report = replicationManager.getContainerReport(); - assertEquals(1, report.getStat(LifeCycleState.CLOSED)); - assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.MIS_REPLICATED)); - - // Now make it so that all containers seem mis-replicated no matter how - // many replicas. This will test replicas are not scheduled if the new - // replica does not fix the mis-replication. - when(ratisContainerPlacementPolicy.validateContainerPlacement( - anyList(), - anyInt() - )).thenAnswer(invocation -> { - return new ContainerPlacementStatusDefault(1, 2, 3); - }); - - currentReplicateCommandCount = datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand); - - replicationManager.processAll(); - eventQueue.processAll(1000); - // At this stage, due to the mocked calls to validateContainerPlacement - // the mis-replicated racks will not have improved, so expect to see - // nothing scheduled. - assertEquals(currentReplicateCommandCount, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - assertEquals(currentReplicateCommandCount, - replicationManager.getMetrics().getReplicationCmdsSentTotal()); - assertEquals(1, getInflightCount(InflightType.REPLICATION)); - assertEquals(1, replicationManager.getMetrics() - .getInflightReplication()); - } - - @Test - public void overReplicatedButRemovingMakesMisReplicated() - throws IOException, TimeoutException { - // In this test, the excess replica should not be removed. - final ContainerInfo container = getContainer(LifeCycleState.CLOSED); - final ContainerID id = container.containerID(); - final UUID originNodeId = UUID.randomUUID(); - final ContainerReplica replicaOne = getReplicas( - id, State.CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaTwo = getReplicas( - id, State.CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaThree = getReplicas( - id, State.CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaFour = getReplicas( - id, State.CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, replicaOne); - containerStateManager.updateContainerReplica(id, replicaTwo); - containerStateManager.updateContainerReplica( - id, replicaThree); - containerStateManager.updateContainerReplica(id, replicaFour); - - // Ensure a mis-replicated status is returned for any containers in this - // test where there are exactly 3 replicas checked. - when(ratisContainerPlacementPolicy.validateContainerPlacement( - argThat(list -> list.size() == 3), - anyInt() - )).thenAnswer( - invocation -> new ContainerPlacementStatusDefault(1, 2, 3)); - - int currentDeleteCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand); - - replicationManager.processAll(); - eventQueue.processAll(1000); - // TODO the new (non-legacy) RM needs a separate handler for - // topology status to make progress in this case by: - // 1. Deleting the closed replica to restore proper replica count. - // 2. Deleting the unhealthy replica since there are adequate healthy - // replicas. - // 3. Fixing topology issues left by the previous cleanup tasks. - // Current legacy RM implementation will take no action in this case - // because deletion would compromise topology. - assertEquals(currentDeleteCommandCount, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.deleteContainerCommand)); - assertEquals(currentDeleteCommandCount, - replicationManager.getMetrics().getDeletionCmdsSentTotal()); - - assertEquals(0, getInflightCount(InflightType.DELETION)); - assertEquals(0, replicationManager.getMetrics() - .getInflightDeletion()); - assertOverReplicatedCount(1); - } - - @Test - public void testOverReplicatedAndPolicySatisfied() - throws IOException, TimeoutException { - final ContainerInfo container = getContainer(LifeCycleState.CLOSED); - final ContainerID id = container.containerID(); - final UUID originNodeId = UUID.randomUUID(); - final ContainerReplica replicaOne = getReplicas( - id, State.CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaTwo = getReplicas( - id, State.CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaThree = getReplicas( - id, State.CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaFour = getReplicas( - id, State.CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, replicaOne); - containerStateManager.updateContainerReplica(id, replicaTwo); - containerStateManager.updateContainerReplica( - id, replicaThree); - containerStateManager.updateContainerReplica(id, replicaFour); - - when(ratisContainerPlacementPolicy.validateContainerPlacement( - argThat(list -> list.size() == 3), - anyInt() - )).thenAnswer( - invocation -> new ContainerPlacementStatusDefault(2, 2, 3)); - - final int currentDeleteCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand); - - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(currentDeleteCommandCount + 1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.deleteContainerCommand)); - assertEquals(currentDeleteCommandCount + 1, - replicationManager.getMetrics().getDeletionCmdsSentTotal()); - assertEquals(1, getInflightCount(InflightType.DELETION)); - assertEquals(1, replicationManager.getMetrics() - .getInflightDeletion()); - - assertOverReplicatedCount(1); - } - - - @Test - @Unhealthy("This test doesn't properly test Rack Placement Policy as" + - " LegacyReplicationManager doesn't handle rack aware delete properly.") - public void testOverReplicatedAndPolicyUnSatisfiedAndDeleted() - throws IOException, TimeoutException { - final ContainerInfo container = getContainer(LifeCycleState.CLOSED); - final ContainerID id = container.containerID(); - final UUID originNodeId = UUID.randomUUID(); - final ContainerReplica replicaOne = getReplicas( - id, State.CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaTwo = getReplicas( - id, State.CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaThree = getReplicas( - id, State.CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaFour = getReplicas( - id, State.CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaFive = getReplicas( - id, QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, replicaOne); - containerStateManager.updateContainerReplica(id, replicaTwo); - containerStateManager.updateContainerReplica( - id, replicaThree); - containerStateManager.updateContainerReplica(id, replicaFour); - containerStateManager.updateContainerReplica(id, replicaFive); - - when(ratisContainerPlacementPolicy.validateContainerPlacement( - argThat(list -> list != null && list.size() <= 4), - anyInt() - )).thenAnswer( - invocation -> new ContainerPlacementStatusDefault(1, 2, 3)); - - int currentDeleteCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand); - - // On the first run, RM will delete one of the extra closed replicas. - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(currentDeleteCommandCount + 1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.deleteContainerCommand)); - assertEquals(currentDeleteCommandCount + 1, - replicationManager.getMetrics().getDeletionCmdsSentTotal()); - assertEquals(1, getInflightCount(InflightType.DELETION)); - assertEquals(1, replicationManager.getMetrics() - .getInflightDeletion()); - - assertAnyDeleteTargets( - replicaOne.getDatanodeDetails(), - replicaTwo.getDatanodeDetails(), - replicaThree.getDatanodeDetails(), - replicaFour.getDatanodeDetails() - ); - - currentDeleteCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand); - - // One the second run, the container is now properly replicated when - // counting in flight deletes. This allows the quasi closed container to - // be deleted by the unhealthy container handler. - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(currentDeleteCommandCount + 1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.deleteContainerCommand)); - assertEquals(currentDeleteCommandCount + 1, - replicationManager.getMetrics().getDeletionCmdsSentTotal()); - assertEquals(1, getInflightCount(InflightType.DELETION)); - assertEquals(1, replicationManager.getMetrics() - .getInflightDeletion()); - - assertDeleteTargetsContain(replicaFive.getDatanodeDetails()); - } - } - - void runTestLimit(int replicationLimit, int deletionLimit, - int expectedReplicationSkipped, int expectedDeletionSkipped, - Callable testcase) throws Exception { - createReplicationManager(replicationLimit, deletionLimit); - - final ReplicationManagerMetrics metrics = replicationManager.getMetrics(); - final long replicationSkipped = metrics.getInflightReplicationSkipped(); - final long deletionSkipped = metrics.getInflightDeletionSkipped(); - - testcase.call(); - - assertEquals(replicationSkipped + expectedReplicationSkipped, - metrics.getInflightReplicationSkipped()); - assertEquals(deletionSkipped + expectedDeletionSkipped, - metrics.getInflightDeletionSkipped()); - - //reset limits for other tests. - createReplicationManager(0, 0); - } - - /** - * Checks that the set of datanodes given delete commands exactly matches - * targetDNs. - */ - private void assertExactDeleteTargets(DatanodeDetails... targetDNs) { - List deleteCommands = datanodeCommandHandler - .getReceivedCommands().stream() - .filter(c -> c.getCommand().getType() == - SCMCommandProto.Type.deleteContainerCommand) - .collect(Collectors.toList()); - - assertEquals(targetDNs.length, deleteCommands.size()); - - Set targetDNIDs = Arrays.stream(targetDNs) - .map(DatanodeDetails::getUuid) - .collect(Collectors.toSet()); - Set chosenDNIDs = deleteCommands.stream() - .map(CommandForDatanode::getDatanodeId) - .collect(Collectors.toSet()); - - assertEquals(targetDNIDs, chosenDNIDs); - } - - /** - * Checks if the set of nodes with deletions scheduled were taken from the - * provided set of DNs. - */ - private void assertAnyDeleteTargets(DatanodeDetails... validDeleteDNs) { - List deleteCommands = datanodeCommandHandler - .getReceivedCommands().stream() - .filter(c -> c.getCommand().getType() == - SCMCommandProto.Type.deleteContainerCommand) - .collect(Collectors.toList()); - - Set deleteCandidateIDs = Arrays.stream(validDeleteDNs) - .map(DatanodeDetails::getUuid) - .collect(Collectors.toSet()); - Set chosenDNIDs = deleteCommands.stream() - .map(CommandForDatanode::getDatanodeId) - .collect(Collectors.toSet()); - - assertThat(deleteCandidateIDs).containsAll(chosenDNIDs); - } - - /** - * Checks if the set of nodes with deletions scheduled contains all of the - * provided DNs. - */ - private void assertDeleteTargetsContain(DatanodeDetails... deleteDN) { - List deleteCommands = datanodeCommandHandler - .getReceivedCommands().stream() - .filter(c -> c.getCommand().getType() == - SCMCommandProto.Type.deleteContainerCommand) - .collect(Collectors.toList()); - - Set deleteDNIDs = Arrays.stream(deleteDN) - .map(DatanodeDetails::getUuid) - .collect(Collectors.toSet()); - Set chosenDNIDs = deleteCommands.stream() - .map(CommandForDatanode::getDatanodeId) - .collect(Collectors.toSet()); - - assertThat(chosenDNIDs).containsAll(deleteDNIDs); - } - - private ContainerInfo createContainer(LifeCycleState containerState) - throws IOException, TimeoutException { - return createContainer(containerState, CONTAINER_USED_BYTES_DEFAULT, - CONTAINER_NUM_KEYS_DEFAULT); - } - - private ContainerInfo createContainer(LifeCycleState containerState, - long usedBytes, long numKeys) throws IOException, TimeoutException { - final ContainerInfo container = getContainer(containerState); - container.setUsedBytes(usedBytes); - container.setNumberOfKeys(numKeys); - containerStateManager.addContainer(container.getProtobuf()); - return container; - } - - private DatanodeDetails addNode(NodeStatus nodeStatus) { - DatanodeDetails dn = randomDatanodeDetails(); - dn.setPersistedOpState(nodeStatus.getOperationalState()); - dn.setPersistedOpStateExpiryEpochSec( - nodeStatus.getOpStateExpiryEpochSeconds()); - nodeManager.register(dn, nodeStatus); - return dn; - } - - private void resetReplicationManager() throws InterruptedException { - replicationManager.stop(); - Thread.sleep(100L); - replicationManager.start(); - Thread.sleep(100L); - } - - private ContainerReplica addReplica(ContainerInfo container, - NodeStatus nodeStatus, State replicaState) - throws ContainerNotFoundException { - DatanodeDetails dn = addNode(nodeStatus); - return addReplicaToDn(container, dn, replicaState); - } - - private ContainerReplica addReplica(ContainerInfo container, - NodeStatus nodeStatus, State replicaState, long usedBytes, long numOfKeys) - throws ContainerNotFoundException { - DatanodeDetails dn = addNode(nodeStatus); - return addReplicaToDn(container, dn, replicaState, usedBytes, numOfKeys); - } - - private ContainerReplica addReplicaToDn(ContainerInfo container, - DatanodeDetails dn, State replicaState) - throws ContainerNotFoundException { - return addReplicaToDn(container, dn, replicaState, 1000L); - } - - private ContainerReplica addReplicaToDn(ContainerInfo container, - DatanodeDetails dn, State replicaState, long bcsId) - throws ContainerNotFoundException { - // Using the same originID for all replica in the container set. If each - // replica has a unique originID, it causes problems in ReplicationManager - // when processing over-replicated containers. - final UUID originNodeId = getUUID(container.getContainerID()); - final ContainerReplica replica = getReplicas(container.containerID(), - replicaState, container.getUsedBytes(), container.getNumberOfKeys(), - bcsId, originNodeId, dn); - containerStateManager - .updateContainerReplica(container.containerID(), replica); - return replica; - } - - static UUID getUUID(long id) { - return UUID.nameUUIDFromBytes(LongCodec.get().toPersistedFormat(id)); - } - - private ContainerReplica addReplicaToDn(ContainerInfo container, - DatanodeDetails dn, State replicaState, long usedBytes, long numOfKeys) { - // Using the same originID for all replica in the container set. If each - // replica has a unique originID, it causes problems in ReplicationManager - // when processing over-replicated containers. - final UUID originNodeId = getUUID(container.getContainerID()); - final ContainerReplica replica = getReplicas(container.containerID(), - replicaState, usedBytes, numOfKeys, 1000L, originNodeId, dn); - containerStateManager - .updateContainerReplica(container.containerID(), replica); - return replica; - } - - private void assertReplicaScheduled(int delta) { - final int currentReplicateCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.replicateContainerCommand); - - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(currentReplicateCommandCount + delta, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - assertEquals(currentReplicateCommandCount + delta, - replicationManager.getMetrics().getReplicationCmdsSentTotal()); - } - - private void assertDeleteScheduled(int delta) { - final int currentDeleteCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand); - - replicationManager.processAll(); - eventQueue.processAll(1000); - assertEquals(currentDeleteCommandCount + delta, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.deleteContainerCommand)); - assertEquals(currentDeleteCommandCount + delta, - replicationManager.getMetrics().getDeletionCmdsSentTotal()); - } - - private void assertUnderReplicatedCount(int count) { - ReplicationManagerReport report = replicationManager.getContainerReport(); - assertEquals(count, report.getStat( - ReplicationManagerReport.HealthState.UNDER_REPLICATED)); - } - - private void assertMissingCount(int count) { - ReplicationManagerReport report = replicationManager.getContainerReport(); - assertEquals(count, report.getStat(ReplicationManagerReport.HealthState.MISSING)); - } - - private void assertOverReplicatedCount(int count) { - ReplicationManagerReport report = replicationManager.getContainerReport(); - assertEquals(count, report.getStat(ReplicationManagerReport.HealthState.OVER_REPLICATED)); - } - - private static class DatanodeCommandHandler implements - EventHandler { - - private AtomicInteger invocation = new AtomicInteger(0); - private Map commandInvocation = - new HashMap<>(); - private List commands = new ArrayList<>(); - - @Override - public void onMessage(final CommandForDatanode command, - final EventPublisher publisher) { - final SCMCommandProto.Type type = command.getCommand().getType(); - commandInvocation.computeIfAbsent(type, k -> new AtomicInteger(0)); - commandInvocation.get(type).incrementAndGet(); - invocation.incrementAndGet(); - commands.add(command); - } - - private int getInvocation() { - return invocation.get(); - } - - private int getInvocationCount(SCMCommandProto.Type type) { - return commandInvocation.containsKey(type) ? - commandInvocation.get(type).get() : 0; - } - - private List getReceivedCommands() { - return commands; - } - - /** - * Returns true if the command handler has received the given - * command type for the provided datanode. - * - * @param type Command Type - * @param datanode DatanodeDetails - * @return True if command was received, false otherwise - */ - private boolean received(final SCMCommandProto.Type type, - final DatanodeDetails datanode) { - return commands.stream().anyMatch(dc -> - dc.getCommand().getType().equals(type) && - dc.getDatanodeId().equals(datanode.getUuid())); - } - - private void clearState() { - commands.clear(); - invocation.set(0); - commandInvocation.clear(); - } - } -} diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java index 568eba57154e..d33539445106 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java @@ -115,7 +115,6 @@ public class TestReplicationManager { private OzoneConfiguration configuration; private ReplicationManager replicationManager; - private LegacyReplicationManager legacyReplicationManager; private ContainerManager containerManager; private PlacementPolicy ratisPlacementPolicy; private PlacementPolicy ecPlacementPolicy; @@ -156,7 +155,6 @@ public void setup() throws IOException { return null; }).when(nodeManager).addDatanodeCommand(any(), any()); - legacyReplicationManager = mock(LegacyReplicationManager.class); clock = new TestClock(Instant.now(), ZoneId.systemDefault()); containerReplicaPendingOps = new ContainerReplicaPendingOps(clock); @@ -209,7 +207,6 @@ private ReplicationManager createReplicationManager() throws IOException { scmContext, nodeManager, clock, - legacyReplicationManager, containerReplicaPendingOps) { @Override protected void startSubServices() { diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManagerMetrics.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManagerMetrics.java index 31e2384d90df..53e767146242 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManagerMetrics.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManagerMetrics.java @@ -27,7 +27,6 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import static org.mockito.Mockito.any; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.apache.ozone.test.MetricsAsserts.getLongGauge; import static org.apache.ozone.test.MetricsAsserts.getMetrics; @@ -57,16 +56,11 @@ public void setup() { report.increment(s); } } - final LegacyReplicationManager lrm = mock( - LegacyReplicationManager.class); - when(lrm.getInflightCount(any(InflightType.class))) - .thenReturn(0); ConfigurationSource conf = new OzoneConfiguration(); ReplicationManager.ReplicationManagerConfiguration rmConf = conf .getObject(ReplicationManager.ReplicationManagerConfiguration.class); ReplicationManager replicationManager = mock(ReplicationManager.class); when(replicationManager.getConfig()).thenReturn(rmConf); - when(replicationManager.getLegacyReplicationManager()).thenReturn(lrm); when(replicationManager.getContainerReport()).thenReturn(report); when(replicationManager.getContainerReplicaPendingOps()) .thenReturn(mock(ContainerReplicaPendingOps.class)); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManagerScenarios.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManagerScenarios.java index 437fdc1c06fc..d54011859900 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManagerScenarios.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManagerScenarios.java @@ -105,7 +105,6 @@ public class TestReplicationManagerScenarios { private OzoneConfiguration configuration; private ReplicationManager replicationManager; - private LegacyReplicationManager legacyReplicationManager; private ContainerManager containerManager; private PlacementPolicy ratisPlacementPolicy; private PlacementPolicy ecPlacementPolicy; @@ -184,7 +183,6 @@ public void setup() throws IOException, NodeNotFoundException { return null; }).when(nodeManager).addDatanodeCommand(any(), any()); - legacyReplicationManager = mock(LegacyReplicationManager.class); clock = new TestClock(Instant.now(), ZoneId.systemDefault()); containerReplicaPendingOps = new ContainerReplicaPendingOps(clock); @@ -232,7 +230,6 @@ private ReplicationManager createReplicationManager() throws IOException { scmContext, nodeManager, clock, - legacyReplicationManager, containerReplicaPendingOps) { @Override protected void startSubServices() { diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestDatanodeAdminMonitor.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestDatanodeAdminMonitor.java index 5c04ad63210e..eebe523f4fdd 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestDatanodeAdminMonitor.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestDatanodeAdminMonitor.java @@ -30,7 +30,6 @@ import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException; import org.apache.hadoop.hdds.scm.container.ContainerReplica; -import org.apache.hadoop.hdds.scm.container.replication.LegacyRatisContainerReplicaCount; import org.apache.hadoop.hdds.scm.container.replication.RatisContainerReplicaCount; import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager; import org.apache.hadoop.hdds.scm.container.SimpleMockNodeManager; @@ -223,85 +222,6 @@ public void testDecommissionNodeWaitsForContainersToReplicate() nodeManager.getNodeStatus(dn1).getOperationalState()); } - /** - * Situation: A QUASI_CLOSED container has an UNHEALTHY replica with the - * greatest BCSID, and three QUASI_CLOSED replicas with a smaller BCSID. The - * UNHEALTHY container is on a decommissioning node, and there are no other - * copies of this replica, that is, replicas with the same Origin ID as - * this replica. - * - * Expectation: Decommissioning should not complete until the UNHEALTHY - * replica has been replicated to another node. - * - * Note: This test currently uses the LegacyReplicationManager, as the new - * one doesn't support this behaviour yet. - * @throws NodeNotFoundException - * @throws ContainerNotFoundException - */ - @Test - public void testDecommissionWaitsForUnhealthyReplicaToReplicate() - throws NodeNotFoundException, ContainerNotFoundException { - conf.setBoolean("hdds.scm.replication.enable.legacy", true); - - DatanodeDetails dn1 = MockDatanodeDetails.randomDatanodeDetails(); - nodeManager.register(dn1, - new NodeStatus(HddsProtos.NodeOperationalState.DECOMMISSIONING, - HddsProtos.NodeState.HEALTHY)); - - // create 3 QUASI_CLOSED replicas with containerID 1 and same origin ID - ContainerID containerID = ContainerID.valueOf(1); - Set replicas = - ReplicationTestUtil.createReplicasWithSameOrigin(containerID, - State.QUASI_CLOSED, 0, 0, 0); - - // the container's sequence id is greater than the healthy replicas' - ContainerInfo container = ReplicationTestUtil.createContainerInfo( - RatisReplicationConfig.getInstance( - HddsProtos.ReplicationFactor.THREE), containerID.getId(), - HddsProtos.LifeCycleState.QUASI_CLOSED, - replicas.iterator().next().getSequenceId() + 1); - // UNHEALTHY replica is on a unique origin and has same sequence id as - // the container - ContainerReplica unhealthy = - ReplicationTestUtil.createContainerReplica(containerID, 0, - dn1.getPersistedOpState(), State.UNHEALTHY, - container.getNumberOfKeys(), container.getUsedBytes(), dn1, - dn1.getUuid(), container.getSequenceId()); - replicas.add(unhealthy); - nodeManager.setContainers(dn1, ImmutableSet.of(containerID)); - - when(repManager.getContainerReplicaCount(eq(containerID))) - .thenReturn(new LegacyRatisContainerReplicaCount(container, replicas, - 0, 0, 3, 2)); - - // start monitoring dn1 - monitor.startMonitoring(dn1); - monitor.run(); - assertEquals(1, monitor.getTrackedNodeCount()); - assertEquals(HddsProtos.NodeOperationalState.DECOMMISSIONING, - nodeManager.getNodeStatus(dn1).getOperationalState()); - - // Running the monitor again causes it to remain DECOMMISSIONING - // as nothing has changed. - monitor.run(); - assertEquals(HddsProtos.NodeOperationalState.DECOMMISSIONING, - nodeManager.getNodeStatus(dn1).getOperationalState()); - - // add a copy of the UNHEALTHY replica on a new node, dn1 should get - // decommissioned now - ContainerReplica copyOfUnhealthyOnNewNode = unhealthy.toBuilder() - .setDatanodeDetails(MockDatanodeDetails.randomDatanodeDetails()) - .build(); - replicas.add(copyOfUnhealthyOnNewNode); - when(repManager.getContainerReplicaCount(eq(containerID))) - .thenReturn(new LegacyRatisContainerReplicaCount(container, replicas, - 0, 0, 3, 2)); - monitor.run(); - assertEquals(0, monitor.getTrackedNodeCount()); - assertEquals(HddsProtos.NodeOperationalState.DECOMMISSIONED, - nodeManager.getNodeStatus(dn1).getOperationalState()); - } - /** * Situation: A QUASI_CLOSED container has an UNHEALTHY replica with the * greatest BCSID, and three QUASI_CLOSED replicas with a smaller BCSID. The @@ -440,72 +360,6 @@ public void testDecommissionWaitsForUnhealthyReplicaWithUniqueOriginToReplicateN nodeManager.getNodeStatus(dn1).getOperationalState()); } - /** - * Consider a QUASI_CLOSED container with only UNHEALTHY replicas. If one - * of its nodes is decommissioned, the decommissioning should succeed. - */ - @Test - public void testQuasiClosedContainerWithAllUnhealthyReplicas() - throws NodeNotFoundException, ContainerNotFoundException { - conf.setBoolean("hdds.scm.replication.enable.legacy", true); - - DatanodeDetails decommissioningNode = - MockDatanodeDetails.randomDatanodeDetails(); - nodeManager.register(decommissioningNode, - new NodeStatus(HddsProtos.NodeOperationalState.DECOMMISSIONING, - HddsProtos.NodeState.HEALTHY)); - ContainerInfo container = ReplicationTestUtil.createContainer( - HddsProtos.LifeCycleState.QUASI_CLOSED, - RatisReplicationConfig.getInstance( - HddsProtos.ReplicationFactor.THREE)); - Set replicas = - ReplicationTestUtil.createReplicas(container.containerID(), - State.UNHEALTHY, 0, 0); - - ContainerReplica decommissioningReplica = - ReplicationTestUtil.createContainerReplica(container.containerID(), 0, - DECOMMISSIONING, State.UNHEALTHY, container.getNumberOfKeys(), - container.getUsedBytes(), decommissioningNode, - decommissioningNode.getUuid()); - replicas.add(decommissioningReplica); - nodeManager.setContainers(decommissioningNode, - ImmutableSet.of(container.containerID())); - - when(repManager.getContainerReplicaCount( - eq(container.containerID()))) - .thenReturn(new LegacyRatisContainerReplicaCount(container, replicas, - Collections.emptyList(), 2, true)); - - // start monitoring dn1 - monitor.startMonitoring(decommissioningNode); - monitor.run(); - assertEquals(1, monitor.getTrackedNodeCount()); - assertEquals(HddsProtos.NodeOperationalState.DECOMMISSIONING, - nodeManager.getNodeStatus(decommissioningNode).getOperationalState()); - - // Running the monitor again causes it to remain DECOMMISSIONING - // as nothing has changed. - monitor.run(); - assertEquals(HddsProtos.NodeOperationalState.DECOMMISSIONING, - nodeManager.getNodeStatus(decommissioningNode).getOperationalState()); - - // add a copy of the UNHEALTHY replica on a new node, decommissioningNode - // should get decommissioned now - ContainerReplica copyOfUnhealthyOnNewNode = - decommissioningReplica.toBuilder() - .setDatanodeDetails(MockDatanodeDetails.randomDatanodeDetails()) - .build(); - replicas.add(copyOfUnhealthyOnNewNode); - when(repManager.getContainerReplicaCount( - eq(container.containerID()))) - .thenReturn(new LegacyRatisContainerReplicaCount(container, replicas, - Collections.emptyList(), 3, true)); - monitor.run(); - assertEquals(0, monitor.getTrackedNodeCount()); - assertEquals(HddsProtos.NodeOperationalState.DECOMMISSIONED, - nodeManager.getNodeStatus(decommissioningNode).getOperationalState()); - } - @Test public void testDecommissionNotBlockedByDeletingContainers() throws NodeNotFoundException, ContainerNotFoundException { @@ -840,7 +694,6 @@ public void testCancelledNodesMovedToInService() @Test public void testContainersReplicatedOnDecomDnAPI() throws NodeNotFoundException, ContainerNotFoundException { - conf.setBoolean("hdds.scm.replication.enable.legacy", false); DatanodeDetails dn1 = MockDatanodeDetails.randomDatanodeDetails(); nodeManager.register(dn1, diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestFailoverWithSCMHA.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestFailoverWithSCMHA.java index 9db501edb721..4f6927d6faed 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestFailoverWithSCMHA.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestFailoverWithSCMHA.java @@ -18,16 +18,13 @@ import com.google.protobuf.ByteString; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient; import org.apache.hadoop.hdds.scm.client.ScmClient; -import org.apache.hadoop.hdds.scm.container.ContainerID; import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancer; import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerConfiguration; import org.apache.hadoop.hdds.scm.container.balancer.IllegalContainerBalancerStateException; import org.apache.hadoop.hdds.scm.container.balancer.InvalidContainerBalancerConfigurationException; -import org.apache.hadoop.hdds.scm.container.common.helpers.MoveDataNodePair; import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol; import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol; import org.apache.hadoop.hdds.scm.protocolPB.ScmBlockLocationProtocolClientSideTranslatorPB; @@ -37,7 +34,6 @@ import org.apache.hadoop.hdds.scm.proxy.SCMContainerLocationFailoverProxyProvider; import org.apache.hadoop.hdds.scm.server.StorageContainerManager; import org.apache.hadoop.hdds.tracing.TracingUtil; -import org.apache.hadoop.ozone.ClientVersion; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.MiniOzoneHAClusterImpl; import org.apache.ozone.test.GenericTestUtils; @@ -47,12 +43,9 @@ import org.slf4j.event.Level; import java.io.IOException; -import java.util.Map; import java.util.concurrent.TimeoutException; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ContainerBalancerConfigurationProto; -import static org.apache.hadoop.hdds.scm.HddsTestUtils.getContainer; -import static org.apache.hadoop.hdds.protocol.MockDatanodeDetails.randomDatanodeDetails; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; @@ -148,90 +141,6 @@ public void testFailover() throws Exception { .contains("Performing failover to suggested leader"); } - @Test - public void testMoveFailover() throws Exception { - SCMClientConfig scmClientConfig = - conf.getObject(SCMClientConfig.class); - scmClientConfig.setRetryCount(1); - scmClientConfig.setRetryInterval(100); - scmClientConfig.setMaxRetryTimeout(1500); - assertEquals(15, scmClientConfig.getRetryCount()); - conf.setFromObject(scmClientConfig); - StorageContainerManager scm = getLeader(cluster); - assertNotNull(scm); - - final ContainerID id = - getContainer(HddsProtos.LifeCycleState.CLOSED).containerID(); - DatanodeDetails dn1 = randomDatanodeDetails(); - DatanodeDetails dn2 = randomDatanodeDetails(); - - //here we just want to test whether the new leader will get the same - //inflight move after failover, so no need to create container and datanode, - //just mock them bypassing all the pre checks. - scm.getReplicationManager().getMoveScheduler().startMove(id.getProtobuf(), - (new MoveDataNodePair(dn1, dn2)) - .getProtobufMessage(ClientVersion.CURRENT_VERSION)); - - SCMBlockLocationFailoverProxyProvider failoverProxyProvider = - new SCMBlockLocationFailoverProxyProvider(conf); - failoverProxyProvider.changeCurrentProxy(scm.getSCMNodeId()); - ScmBlockLocationProtocolClientSideTranslatorPB scmBlockLocationClient = - new ScmBlockLocationProtocolClientSideTranslatorPB( - failoverProxyProvider); - GenericTestUtils - .setLogLevel(SCMBlockLocationFailoverProxyProvider.LOG, Level.DEBUG); - GenericTestUtils.LogCapturer logCapture = GenericTestUtils.LogCapturer - .captureLogs(SCMBlockLocationFailoverProxyProvider.LOG); - ScmBlockLocationProtocol scmBlockLocationProtocol = TracingUtil - .createProxy(scmBlockLocationClient, ScmBlockLocationProtocol.class, - conf); - scmBlockLocationProtocol.getScmInfo(); - assertThat(logCapture.getOutput()) - .contains("Performing failover to suggested leader"); - scm = getLeader(cluster); - assertNotNull(scm); - - //switch to the new leader successfully, new leader should - //get the same inflightMove - Map inflightMove = - scm.getReplicationManager().getMoveScheduler().getInflightMove(); - assertThat(inflightMove).containsKey(id); - MoveDataNodePair mp = inflightMove.get(id); - assertEquals(dn2, mp.getTgt()); - assertEquals(dn1, mp.getSrc()); - - //complete move in the new leader - scm.getReplicationManager().getMoveScheduler() - .completeMove(id.getProtobuf()); - - - SCMContainerLocationFailoverProxyProvider proxyProvider = - new SCMContainerLocationFailoverProxyProvider(conf, null); - GenericTestUtils.setLogLevel(SCMContainerLocationFailoverProxyProvider.LOG, - Level.DEBUG); - logCapture = GenericTestUtils.LogCapturer - .captureLogs(SCMContainerLocationFailoverProxyProvider.LOG); - proxyProvider.changeCurrentProxy(scm.getSCMNodeId()); - StorageContainerLocationProtocol scmContainerClient = - TracingUtil.createProxy( - new StorageContainerLocationProtocolClientSideTranslatorPB( - proxyProvider), StorageContainerLocationProtocol.class, conf); - - scmContainerClient.allocateContainer(HddsProtos.ReplicationType.RATIS, - HddsProtos.ReplicationFactor.ONE, "ozone"); - assertThat(logCapture.getOutput()) - .contains("Performing failover to suggested leader"); - - //switch to the new leader successfully, new leader should - //get the same inflightMove , which should not contains - //that container. - scm = getLeader(cluster); - assertNotNull(scm); - inflightMove = scm.getReplicationManager() - .getMoveScheduler().getInflightMove(); - assertThat(inflightMove).doesNotContainKey(id); - } - /** * Starts ContainerBalancer when the cluster is already balanced. * ContainerBalancer will identify that no unbalanced nodes are present and diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestStorageContainerManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestStorageContainerManager.java index 95d7faa91740..94c8f9142942 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestStorageContainerManager.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestStorageContainerManager.java @@ -58,7 +58,6 @@ import org.apache.hadoop.hdds.scm.server.SCMStorageConfig; import org.apache.hadoop.hdds.scm.server.StorageContainerManager; import org.apache.hadoop.hdds.server.events.EventExecutor; -import org.apache.hadoop.hdds.server.events.EventPublisher; import org.apache.hadoop.hdds.server.events.EventQueue; import org.apache.hadoop.hdds.server.events.FixedThreadPoolWithAffinityExecutor; import org.apache.hadoop.hdds.utils.HddsVersionInfo; @@ -155,7 +154,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.any; -import static org.mockito.Mockito.argThat; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; @@ -788,10 +786,6 @@ public void testCloseContainerCommandOnRestart() throws Exception { NodeManager nodeManager = mock(NodeManager.class); setInternalState(rm, "nodeManager", nodeManager); - EventPublisher publisher = mock(EventPublisher.class); - setInternalState(rm.getLegacyReplicationManager(), - "eventPublisher", publisher); - UUID dnUuid = cluster.getHddsDatanodes().iterator().next() .getDatanodeDetails().getUuid(); @@ -811,15 +805,7 @@ public void testCloseContainerCommandOnRestart() throws Exception { cluster.getStorageContainerManager() .getReplicationManager().processAll(); Thread.sleep(5000); - - if (rm.getConfig().isLegacyEnabled()) { - CommandForDatanode commandForDatanode = new CommandForDatanode( - dnUuid, closeContainerCommand); - verify(publisher).fireEvent(eq(SCMEvents.DATANODE_COMMAND), argThat(new - CloseContainerCommandMatcher(dnUuid, commandForDatanode))); - } else { - verify(nodeManager).addDatanodeCommand(dnUuid, closeContainerCommand); - } + verify(nodeManager).addDatanodeCommand(dnUuid, closeContainerCommand); } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java index 50bbd81ea6b6..b602ce59a5ba 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java @@ -179,7 +179,6 @@ private static OzoneConfiguration createConfiguration(boolean enableLegacy) { ReplicationManagerConfiguration repConf = conf.getObject(ReplicationManagerConfiguration.class); - repConf.setEnableLegacy(enableLegacy); repConf.setInterval(Duration.ofSeconds(1)); repConf.setUnderReplicatedInterval(Duration.ofSeconds(1)); conf.setFromObject(repConf); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java index 719715ac8b3d..d264c265f328 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java @@ -50,7 +50,6 @@ import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException; import org.apache.hadoop.hdds.scm.container.ContainerReplica; import org.apache.hadoop.hdds.scm.container.ContainerStateManager; -import org.apache.hadoop.hdds.scm.container.replication.LegacyReplicationManager; import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager; import org.apache.hadoop.hdds.scm.server.StorageContainerManager; import org.apache.hadoop.hdds.server.events.EventQueue; @@ -132,7 +131,6 @@ public void init() throws Exception { conf = new OzoneConfiguration(); GenericTestUtils.setLogLevel(DeletedBlockLogImpl.LOG, Level.DEBUG); GenericTestUtils.setLogLevel(SCMBlockDeletingService.LOG, Level.DEBUG); - GenericTestUtils.setLogLevel(LegacyReplicationManager.LOG, Level.DEBUG); GenericTestUtils.setLogLevel(ReplicationManager.LOG, Level.DEBUG); conf.set("ozone.replication.allowed-configs", @@ -340,7 +338,6 @@ public void testBlockDeletion(ReplicationConfig repConfig) throws Exception { @Test public void testContainerStatisticsAfterDelete() throws Exception { ReplicationManager replicationManager = scm.getReplicationManager(); - boolean legacyEnabled = replicationManager.getConfig().isLegacyEnabled(); String volumeName = UUID.randomUUID().toString(); String bucketName = UUID.randomUUID().toString(); @@ -403,16 +400,13 @@ public void testContainerStatisticsAfterDelete() throws Exception { containerInfos.stream().forEach(container -> assertEquals(HddsProtos.LifeCycleState.DELETING, container.getState())); - LogCapturer logCapturer = LogCapturer.captureLogs( - legacyEnabled ? LegacyReplicationManager.LOG : ReplicationManager.LOG); + LogCapturer logCapturer = LogCapturer.captureLogs(ReplicationManager.LOG); logCapturer.clearOutput(); Thread.sleep(5000); replicationManager.processAll(); ((EventQueue) scm.getEventQueue()).processAll(1000); - String expectedOutput = legacyEnabled - ? "Resend delete Container" - : "Sending delete command for container"; + String expectedOutput = "Sending delete command for container"; GenericTestUtils.waitFor(() -> logCapturer.getOutput() .contains(expectedOutput), 500, 5000);