diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java index 3d4d783212d7..2ff50aab757d 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java @@ -209,8 +209,10 @@ public void onMessage(final ContainerReportFromDatanode reportFromDatanode, } /** - * Processes the ContainerReport, unknown container reported - * that will be deleted by SCM. + * Processes the ContainerReport. + * Any unknown container reported by DN and not present in SCM + * containerSet will either be logged as an error or deleted based on + * unknownContainerHandleAction. * * @param datanodeDetails Datanode from which this report was received * @param container ContainerInfo representing the container 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 a3160208e33e..82b883ce716f 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 @@ -21,7 +21,7 @@ import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.container.ContainerReplica; -import java.util.Set; +import java.util.List; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_SERVICE; @@ -32,7 +32,7 @@ public interface ContainerReplicaCount { ContainerInfo getContainer(); - Set getReplicas(); + List getReplicas(); boolean isSufficientlyReplicated(); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECContainerReplicaCount.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECContainerReplicaCount.java index 1360c8401653..967bec211933 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECContainerReplicaCount.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECContainerReplicaCount.java @@ -25,6 +25,7 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -74,14 +75,20 @@ public class ECContainerReplicaCount implements ContainerReplicaCount { private final Map healthyIndexes = new HashMap<>(); private final Map decommissionIndexes = new HashMap<>(); private final Map maintenanceIndexes = new HashMap<>(); - private final Set replicas; + private final List replicas; public ECContainerReplicaCount(ContainerInfo containerInfo, Set replicas, List replicaPendingOps, int remainingMaintenanceRedundancy) { this.containerInfo = containerInfo; - this.replicas = replicas; + // Iterate replicas in deterministic order to avoid potential data loss + // on delete. + // See https://issues.apache.org/jira/browse/HDDS-4589. + // N.B., sort replicas by (containerID, datanodeDetails). + this.replicas = replicas.stream() + .sorted(Comparator.comparingLong(ContainerReplica::hashCode)) + .collect(Collectors.toList()); this.repConfig = (ECReplicationConfig)containerInfo.getReplicationConfig(); this.pendingAdd = new ArrayList<>(); this.pendingDelete = new ArrayList<>(); @@ -158,7 +165,7 @@ public ContainerInfo getContainer() { } @Override - public Set getReplicas() { + public List getReplicas() { return replicas; } 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 new file mode 100644 index 000000000000..a2139707ca24 --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/LegacyRatisContainerReplicaCount.java @@ -0,0 +1,51 @@ +/* + * 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.scm.container.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.ContainerReplica; + +import java.util.Set; + +/** + * 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); + } + + @Override + protected int healthyReplicaCountAdapter() { + return 0; + } +} 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 index 7e86455357c4..80a61ee5db31 100644 --- 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 @@ -72,11 +72,11 @@ import java.lang.reflect.Proxy; import java.time.Clock; 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.LinkedHashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -93,6 +93,7 @@ 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; /** @@ -522,24 +523,32 @@ protected void processContainer(ContainerInfo container, */ 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) { - if (!sufficientlyReplicated) { - report.incrementAndSample( - HealthState.UNDER_REPLICATED, container.containerID()); + // Replicate container if needed. + if (!inflightReplication.isFull() || !inflightDeletion.isFull()) { if (replicaSet.isUnrecoverable()) { - report.incrementAndSample(HealthState.MISSING, - container.containerID()); + // 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); + } } } - if (!placementSatisfied) { - report.incrementAndSample(HealthState.MIS_REPLICATED, - container.containerID()); - - } - if (!inflightReplication.isFull() || !inflightDeletion.isFull()) { - handleUnderReplicatedContainer(container, - replicaSet, placementStatus); - } return; } @@ -548,9 +557,7 @@ protected void processContainer(ContainerInfo container, * action. */ if (replicaSet.isOverReplicated()) { - report.incrementAndSample(HealthState.OVER_REPLICATED, - container.containerID()); - handleOverReplicatedContainer(container, replicaSet); + handleOverReplicatedHealthy(container, replicaSet, report); return; } @@ -560,9 +567,7 @@ protected void processContainer(ContainerInfo container, are not in the same state as the container itself. */ if (!replicaSet.isHealthy()) { - report.incrementAndSample(HealthState.UNHEALTHY, - container.containerID()); - handleUnstableContainer(container, replicas); + handleOverReplicatedExcessUnhealthy(container, replicaSet, report); } } } catch (ContainerNotFoundException ex) { @@ -997,7 +1002,7 @@ public ContainerReplicaCount getContainerReplicaCount(ContainerInfo container) */ private RatisContainerReplicaCount getContainerReplicaCount( ContainerInfo container, Set replica) { - return new RatisContainerReplicaCount( + return new LegacyRatisContainerReplicaCount( container, replica, getInflightAdd(container.containerID()), @@ -1126,180 +1131,209 @@ private void forceCloseContainer(final ContainerInfo container, * @param replicaSet An instance of ContainerReplicaCount, containing the * current replica count and inflight adds and deletes */ - private void handleUnderReplicatedContainer(final ContainerInfo container, + private void handleUnderReplicatedHealthy(final ContainerInfo container, final RatisContainerReplicaCount replicaSet, - final ContainerPlacementStatus placementStatus) { + final ContainerPlacementStatus placementStatus, + ReplicationManagerReport report) { LOG.debug("Handling under-replicated container: {}", container); - Set replicas = replicaSet.getReplicas(); - try { + if (replicaSet.isSufficientlyReplicated() + && placementStatus.isPolicySatisfied()) { + LOG.info("The container {} with replicas {} is sufficiently " + + "replicated and is not mis-replicated", + container.getContainerID(), replicaSet); + return; + } - if (replicaSet.isSufficientlyReplicated() - && placementStatus.isPolicySatisfied()) { - LOG.info("The container {} with replicas {} is sufficiently " + - "replicated and is not mis-replicated", - container.getContainerID(), replicaSet); - return; - } - int repDelta = replicaSet.additionalReplicaNeeded(); - final ContainerID id = container.containerID(); - final List deletionInFlight - = inflightDeletion.getDatanodeDetails(id); - final List replicationInFlight - = inflightReplication.getDatanodeDetails(id); - final List source = replicas.stream() - .filter(r -> - r.getState() == State.QUASI_CLOSED || - r.getState() == State.CLOSED) - // 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()) - .filter(r -> !deletionInFlight.contains(r.getDatanodeDetails())) - .sorted((r1, r2) -> r2.getSequenceId().compareTo(r1.getSequenceId())) - .map(ContainerReplica::getDatanodeDetails) - .collect(Collectors.toList()); - if (source.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<>(source); - // Then add any pending additions - targetReplicas.addAll(replicationInFlight); - final ContainerPlacementStatus inFlightplacementStatus = - containerPlacement.validateContainerPlacement( - targetReplicas, replicationFactor); - final int misRepDelta = inFlightplacementStatus.misReplicationCount(); - final int replicasNeeded - = repDelta < misRepDelta ? misRepDelta : repDelta; - if (replicasNeeded <= 0) { - LOG.debug("Container {} meets replication requirement with " + - "inflight replicas", id); - return; - } + List allReplicas = replicaSet.getReplicas(); + int numCloseCommandsSent = closeReplicasIfPossible(container, allReplicas); + int replicasNeeded = + replicaSet.additionalReplicaNeeded() - numCloseCommandsSent; - // We should ensure that the target datanode has enough space - // for a complete container to be created, but since the container - // size may be changed smaller than origin, we should be defensive. - final long dataSizeRequired = Math.max(container.getUsedBytes(), - currentContainerSize); - final List excludeList = replicas.stream() - .map(ContainerReplica::getDatanodeDetails) - .collect(Collectors.toList()); - excludeList.addAll(replicationInFlight); - final List selectedDatanodes = containerPlacement - .chooseDatanodes(excludeList, null, replicasNeeded, - 0, dataSizeRequired); - if (repDelta > 0) { - LOG.info("Container {} is under replicated. Expected replica count" + - " is {}, but found {}.", id, replicationFactor, - replicationFactor - repDelta); - } - 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 (repDelta > 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, source); - } - } 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 replica found.", - container.containerID()); - } - } catch (IOException | IllegalStateException ex) { - LOG.warn("Exception while replicating container {}.", - container.getContainerID(), ex); + if (replicasNeeded > 0) { + report.incrementAndSample(HealthState.UNDER_REPLICATED, + container.containerID()); } + + List replicationSources = getReplicationSources(container, + replicaSet.getReplicas(), State.CLOSED, State.QUASI_CLOSED); + // This method will handle topology even if replicasNeeded <= 0. + replicateAnyWithTopology(container, replicationSources, + placementStatus, replicasNeeded); } /** * 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). + * 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 handleOverReplicatedContainer(final ContainerInfo container, - final RatisContainerReplicaCount replicaSet) { + private void handleOverReplicatedHealthy(final ContainerInfo container, + final RatisContainerReplicaCount replicaSet, + ReplicationManagerReport report) { - final Set replicas = replicaSet.getReplicas(); 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); + " 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); + } + } + } - final List eligibleReplicas = new ArrayList<>(replicas); + /** + * 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) { - // Iterate replicas in deterministic order to avoid potential data loss. - // See https://issues.apache.org/jira/browse/HDDS-4589. - // N.B., sort replicas by (containerID, datanodeDetails). - eligibleReplicas.sort( - Comparator.comparingLong(ContainerReplica::hashCode)); + List replicas = replicaSet.getReplicas(); + int excessReplicas = replicas.size() - + container.getReplicationConfig().getRequiredNodes(); + int missingReplicas = excessReplicas * -1; + + if (missingReplicas > 0) { + handleUnderReplicatedAllUnhealthy(container, replicas, + placementStatus, missingReplicas, report); + } else if (excessReplicas > 0) { + handleOverReplicatedAllUnhealthy(container, replicas, + excessReplicas, report); + } else { + // We have the correct number of unhealthy replicas. See if any of them + // can be closed. + closeReplicasIfPossible(container, replicas); + } + } - final Map uniqueReplicas = - new LinkedHashMap<>(); + /** + * 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()) { + LOG.info("Container {} has {} excess unhealthy replicas. Excess " + + "unhealthy replicas will be deleted.", + container.getContainerID(), unhealthyReplicas.size()); + + report.incrementAndSample(HealthState.OVER_REPLICATED, + container.containerID()); + + int excessReplicaCount = replicas.size() - + container.getReplicationConfig().getRequiredNodes(); + if (container.getState() == LifeCycleState.CLOSED) { + // The container is already closed. The unhealthy replicas are extras + // and unnecessary. + deleteExcess(container, unhealthyReplicas, excessReplicaCount); + } 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. + deleteExcessWithNonUniqueOriginNodeIDs(container, + replicaSet.getReplicas(), unhealthyReplicas, excessReplicaCount); + } + } + } - if (container.getState() != LifeCycleState.CLOSED) { - replicas.stream() - .filter(r -> compareState(container.getState(), r.getState())) - .forEach(r -> uniqueReplicas - .putIfAbsent(r.getOriginDatanodeId(), r)); + /** + * 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()); + } - eligibleReplicas.removeAll(uniqueReplicas.values()); - } - // Replica which are maintenance or decommissioned are not eligible to - // be removed, as they do not count toward over-replication and they - // also many not be available - eligibleReplicas.removeIf(r -> - r.getDatanodeDetails().getPersistedOpState() != - NodeOperationalState.IN_SERVICE); - - final List unhealthyReplicas = eligibleReplicas - .stream() - .filter(r -> !compareState(container.getState(), r.getState())) - .collect(Collectors.toList()); + private List getHealthyDeletionCandidates( + ContainerInfo container, List replicas) { + return getDeletionCandidates(container, replicas, true); + } - // If there are unhealthy replicas, then we should remove them even if it - // makes the container violate the placement policy, as excess unhealthy - // containers are not really useful. It will be corrected later as a - // mis-replicated container will be seen as under-replicated. - for (ContainerReplica r : unhealthyReplicas) { - if (excess > 0) { - sendDeleteCommand(container, r.getDatanodeDetails(), true); - excess -= 1; - } else { - break; - } - } - eligibleReplicas.removeAll(unhealthyReplicas); - removeExcessReplicasIfNeeded(excess, container, eligibleReplicas); - } + 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()); } /** @@ -1355,57 +1389,6 @@ private void deleteSrcDnForMove(final ContainerInfo cif, } } - /** - * 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 removeExcessReplicasIfNeeded(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); - } - } - } - /** * whether the given two ContainerPlacementStatus are actually equal. * @@ -1436,65 +1419,6 @@ private ContainerPlacementStatus getPlacementStatus( replicaDns, replicationFactor); } - /** - * Handles unstable container. - * A container is inconsistent if any of the replica state doesn't - * match the container state. We have to take appropriate action - * based on state of the replica. - * - * @param container ContainerInfo - * @param replicas Set of ContainerReplicas - */ - private void handleUnstableContainer(final ContainerInfo container, - final Set replicas) { - // Find unhealthy replicas - List unhealthyReplicas = replicas.stream() - .filter(r -> !compareState(container.getState(), r.getState())) - .collect(Collectors.toList()); - - Iterator iterator = unhealthyReplicas.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); - iterator.remove(); - } - - if (state == State.QUASI_CLOSED) { - // Send force close command if the BCSID matches - if (container.getSequenceId() == replica.getSequenceId()) { - sendCloseCommand(container, replica.getDatanodeDetails(), true); - iterator.remove(); - } - } - } - - // Now we are left with the replicas which are either unhealthy or - // the BCSID doesn't match. These replicas should be deleted. - - /* - * If we have unhealthy replicas we go under replicated and then - * replicate the healthy copy. - * - * We also make sure that we delete only one unhealthy replica at a time. - * - * If there are two unhealthy replica: - * - Delete first unhealthy replica - * - Re-replicate the healthy copy - * - Delete second unhealthy replica - * - Re-replicate the healthy copy - * - * Note: Only one action will be executed in a single ReplicationMonitor - * iteration. So to complete all the above actions we need four - * ReplicationMonitor iterations. - */ - - unhealthyReplicas.stream().findFirst().ifPresent(replica -> - sendDeleteCommand(container, replica.getDatanodeDetails(), true)); - - } - /** * Sends close container command for the given container to the given * datanode. @@ -2009,5 +1933,325 @@ private void compleleteMoveFutureWithResult(ContainerID cid, MoveResult 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) { + // 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) { + 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); + } + + 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); + } 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. + deleteExcessWithNonUniqueOriginNodeIDs(container, + replicas, deleteCandidates, 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. + replicateAnyWithTopology(container, + getReplicationSources(container, replicas), placementStatus, + additionalReplicasNeeded); + } + } + + /* 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); + } + } + } + + private void 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. + + // Gather the origin node IDs of replicas which are not candidates for + // deletion. + Set existingOriginNodeIDs = allReplicas.stream() + .filter(r -> !deleteCandidates.contains(r)) + .map(ContainerReplica::getOriginDatanodeId) + .collect(Collectors.toSet()); + + List nonUniqueDeleteCandidates = new ArrayList<>(); + for (ContainerReplica replica: deleteCandidates) { + if (existingOriginNodeIDs.contains(replica.getOriginDatanodeId())) { + nonUniqueDeleteCandidates.add(replica); + } else { + // Spare this replica with this new origin node ID from deletion. + // delete candidates seen later in the loop with this same origin + // node ID can be deleted. + existingOriginNodeIDs.add(replica.getOriginDatanodeId()); + } + } + + 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()); + } + deleteExcess(container, nonUniqueDeleteCandidates, excess); + } + + /** + * 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).reversed()); + 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 replicas, + ContainerPlacementStatus placementStatus, int additionalReplicasNeeded) { + try { + final ContainerID id = container.containerID(); + + final List sourceDNs = replicas.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; + } + + // We should ensure that the target datanode has enough space + // for a complete container to be created, but since the container + // size may be changed smaller than origin, we should be defensive. + final long dataSizeRequired = Math.max(container.getUsedBytes(), + currentContainerSize); + final List excludeList = replicas.stream() + .map(ContainerReplica::getDatanodeDetails) + .collect(Collectors.toList()); + excludeList.addAll(replicationInFlight); + final List selectedDatanodes = containerPlacement + .chooseDatanodes(excludeList, null, replicasNeeded, + 0, dataSizeRequired); + 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 (IOException | IllegalStateException ex) { + LOG.warn("Exception while replicating container {}.", + container.getContainerID(), ex); + } + } +} 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 577fc6004d0c..8e54849fdf7b 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 @@ -21,7 +21,11 @@ import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.container.ContainerReplica; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; import java.util.Set; +import java.util.stream.Collectors; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONED; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONING; @@ -37,7 +41,8 @@ */ public class RatisContainerReplicaCount implements ContainerReplicaCount { - private int healthyCount; + private int healthyReplicaCount; + private int unhealthyReplicaCount; private int decommissionCount; private int maintenanceCount; private final int inFlightAdd; @@ -45,24 +50,32 @@ public class RatisContainerReplicaCount implements ContainerReplicaCount { private final int repFactor; private final int minHealthyForMaintenance; private final ContainerInfo container; - private final Set replica; + private final List replicas; public RatisContainerReplicaCount(ContainerInfo container, - Set replica, int inFlightAdd, + Set replicas, + int inFlightAdd, int inFlightDelete, int replicationFactor, int minHealthyForMaintenance) { - this.healthyCount = 0; + this.unhealthyReplicaCount = 0; + this.healthyReplicaCount = 0; this.decommissionCount = 0; this.maintenanceCount = 0; this.inFlightAdd = inFlightAdd; this.inFlightDel = inFlightDelete; this.repFactor = replicationFactor; - this.replica = replica; + // Iterate replicas in deterministic order to avoid potential data loss + // on delete. + // See https://issues.apache.org/jira/browse/HDDS-4589. + // N.B., sort replicas by (containerID, datanodeDetails). + this.replicas = replicas.stream() + .sorted(Comparator.comparingLong(ContainerReplica::hashCode)) + .collect(Collectors.toList()); this.minHealthyForMaintenance = Math.min(this.repFactor, minHealthyForMaintenance); this.container = container; - for (ContainerReplica cr : this.replica) { + for (ContainerReplica cr : this.replicas) { HddsProtos.NodeOperationalState state = cr.getDatanodeDetails().getPersistedOpState(); if (state == DECOMMISSIONED || state == DECOMMISSIONING) { @@ -70,13 +83,33 @@ public RatisContainerReplicaCount(ContainerInfo container, } else if (state == IN_MAINTENANCE || state == ENTERING_MAINTENANCE) { maintenanceCount++; } else { - healthyCount++; + if (LegacyReplicationManager.compareState(container.getState(), + cr.getState())) { + healthyReplicaCount++; + } else { + unhealthyReplicaCount++; + } } } } - public int getHealthyCount() { - return healthyCount; + public int getHealthyReplicaCount() { + return healthyReplicaCount + healthyReplicaCountAdapter(); + } + + public int getUnhealthyReplicaCount() { + return unhealthyReplicaCount; + } + + /** + * The new replication manager currently counts unhealthy and healthy + * replicas together. This should be updated when changes from HDDS-6447 + * are integrated into the new replication manager. See + * {@link LegacyRatisContainerReplicaCount}, which overrides this method, for + * details. + */ + protected int healthyReplicaCountAdapter() { + return getUnhealthyReplicaCount(); } @Override @@ -99,15 +132,16 @@ public ContainerInfo getContainer() { } @Override - public Set getReplicas() { - return replica; + public List getReplicas() { + return new ArrayList<>(replicas); } @Override public String toString() { return "Container State: " + container.getState() + - " Replica Count: " + replica.size() + - " Healthy Count: " + healthyCount + + " Replica Count: " + replicas.size() + + " Healthy Count: " + healthyReplicaCount + + " Unhealthy Count: " + unhealthyReplicaCount + " Decommission Count: " + decommissionCount + " Maintenance Count: " + maintenanceCount + " inFlightAdd Count: " + inFlightAdd + @@ -210,7 +244,7 @@ public int additionalReplicaNeeded() { * for under replicated and a negative value for over replicated. */ private int missingReplicas() { - int delta = repFactor - healthyCount; + int delta = repFactor - getHealthyReplicaCount(); if (delta < 0) { // Over replicated, so may need to remove a container. @@ -219,7 +253,7 @@ private int missingReplicas() { // May be under-replicated, depending on maintenance. delta = Math.max(0, delta - maintenanceCount); int neededHealthy = - Math.max(0, minHealthyForMaintenance - healthyCount); + Math.max(0, minHealthyForMaintenance - getHealthyReplicaCount()); delta = Math.max(neededHealthy, delta); return delta; } else { // delta == 0 @@ -334,7 +368,8 @@ private int redundancyDelta(boolean includePendingDelete, */ public int getRemainingRedundancy() { return Math.max(0, - healthyCount + decommissionCount + maintenanceCount - inFlightDel - 1); + getHealthyReplicaCount() + decommissionCount + maintenanceCount + - inFlightDel - 1); } /** 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 4c446acce7ff..26a0905a9f16 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 @@ -37,6 +37,7 @@ import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.HashMap; @@ -398,7 +399,7 @@ private boolean checkContainersReplicatedOnNode(DatanodeDetails dn) return underReplicated == 0 && unhealthy == 0; } - private String replicaDetails(Set replicas) { + private String replicaDetails(Collection replicas) { StringBuilder sb = new StringBuilder(); sb.append("Replicas{"); sb.append(replicas.stream() 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 index 69ed758b4042..8c46db43ac3d 100644 --- 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 @@ -20,7 +20,6 @@ import com.google.common.primitives.Longs; import org.apache.commons.io.FileUtils; -import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; @@ -71,7 +70,10 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Nested; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import org.mockito.Mockito; import java.io.File; @@ -79,7 +81,9 @@ 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; @@ -103,6 +107,8 @@ 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; @@ -282,1726 +288,2132 @@ void createReplicationManager(ReplicationManagerConfiguration rmConf, } @AfterEach - public void tearDown() throws Exception { + public void teardown() throws Exception { containerStateManager.close(); + replicationManager.stop(); if (dbStore != null) { dbStore.close(); } - - FileUtil.fullyDelete(testDir); - } - - /** - * Checks if restarting of replication manager works. - */ - @Test - public void testReplicationManagerRestart() throws InterruptedException { - Assertions.assertTrue(replicationManager.isRunning()); - replicationManager.stop(); - // Stop is a non-blocking call, it might take sometime for the - // ReplicationManager to shutdown - Thread.sleep(500); - Assertions.assertFalse(replicationManager.isRunning()); - replicationManager.start(); - Assertions.assertTrue(replicationManager.isRunning()); - } - - /** - * 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(); - Assertions.assertEquals(1, report.getStat(LifeCycleState.OPEN)); - Assertions.assertEquals(0, datanodeCommandHandler.getInvocation()); + FileUtils.deleteDirectory(testDir); } - /** - * If the container is in CLOSING state we resend close container command - * to all the datanodes. - */ - @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); + @Nested + class Misc { + /** + * Checks if restarting of replication manager works. + */ + @Test + public void testReplicationManagerRestart() throws InterruptedException { + Assertions.assertTrue(replicationManager.isRunning()); + replicationManager.stop(); + // Stop is a non-blocking call, it might take sometime for the + // ReplicationManager to shutdown + Thread.sleep(500); + Assertions.assertFalse(replicationManager.isRunning()); + replicationManager.start(); + Assertions.assertTrue(replicationManager.isRunning()); } - final int currentCloseCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.closeContainerCommand); + @Test + public void testGeneratedConfig() { + ReplicationManagerConfiguration rmc = OzoneConfiguration.newInstanceOf( + ReplicationManagerConfiguration.class); - replicationManager.processAll(); - eventQueue.processAll(1000); - Assertions.assertEquals(currentCloseCommandCount + 3, datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.closeContainerCommand)); + //default is not included in ozone-site.xml but generated from annotation + //to the ozone-site-generated.xml which should be loaded by the + // OzoneConfiguration. + Assertions.assertEquals(1800000, rmc.getEventTimeout()); - // Update the OPEN to CLOSING - for (ContainerReplica replica : getReplicas(id, State.CLOSING, datanode)) { - containerStateManager.updateContainerReplica(id, replica); } - - replicationManager.processAll(); - eventQueue.processAll(1000); - Assertions.assertEquals(currentCloseCommandCount + 6, datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.closeContainerCommand)); - ReplicationManagerReport report = replicationManager.getContainerReport(); - Assertions.assertEquals(1, report.getStat(LifeCycleState.CLOSING)); - } - - - /** - * The container is QUASI_CLOSED but two of the replica is still in - * open state. ReplicationManager should resend close command to those - * datanodes. - */ - @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, State.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); - - final int currentCloseCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.closeContainerCommand); - // Two of the replicas are in OPEN state - replicationManager.processAll(); - eventQueue.processAll(1000); - Assertions.assertEquals(currentCloseCommandCount + 2, datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.closeContainerCommand)); - Assertions.assertTrue(datanodeCommandHandler.received( - SCMCommandProto.Type.closeContainerCommand, - replicaTwo.getDatanodeDetails())); - Assertions.assertTrue(datanodeCommandHandler.received( - SCMCommandProto.Type.closeContainerCommand, - replicaThree.getDatanodeDetails())); - ReplicationManagerReport report = replicationManager.getContainerReport(); - Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); } /** - * 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. + * Tests replication manager with healthy open and closed containers. No + * quasi closed or unhealthy containers are involved. */ - @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, State.QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaTwo = getReplicas( - id, State.QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaThree = getReplicas( - id, State.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); - Assertions.assertEquals(0, datanodeCommandHandler.getInvocation()); - ReplicationManagerReport report = replicationManager.getContainerReport(); - Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - Assertions.assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - } - - /** - * 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, State.QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaTwo = getReplicas( - id, State.QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaThree = getReplicas( - id, State.QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, replicaOne); - containerStateManager.updateContainerReplica(id, replicaTwo); - containerStateManager.updateContainerReplica( - id, replicaThree); - - final int currentDeleteCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand); - final 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); - Assertions.assertEquals(0, datanodeCommandHandler.getInvocation()); - - // Make the first replica unhealthy - final ContainerReplica unhealthyReplica = getReplicas( - id, State.UNHEALTHY, 1000L, originNodeId, - replicaOne.getDatanodeDetails()); - containerStateManager.updateContainerReplica( - id, unhealthyReplica); - - replicationManager.processAll(); - eventQueue.processAll(1000); - Assertions.assertEquals(currentDeleteCommandCount + 1, - datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand)); - Assertions.assertTrue(datanodeCommandHandler.received( - SCMCommandProto.Type.deleteContainerCommand, - replicaOne.getDatanodeDetails())); - Assertions.assertEquals(currentDeleteCommandCount + 1, - replicationManager.getMetrics().getNumDeletionCmdsSent()); - - // Now we will delete the unhealthy replica from in-memory. - containerStateManager.removeContainerReplica(id, replicaOne); - - final long currentBytesToReplicate = replicationManager.getMetrics() - .getNumReplicationBytesTotal(); - - // The container is under replicated as unhealthy replica is removed - replicationManager.processAll(); - eventQueue.processAll(1000); - - // We should get replicate command - Assertions.assertEquals(currentReplicateCommandCount + 1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - Assertions.assertEquals(currentReplicateCommandCount + 1, - replicationManager.getMetrics().getNumReplicationCmdsSent()); - Assertions.assertEquals(currentBytesToReplicate + 100L, - replicationManager.getMetrics().getNumReplicationBytesTotal()); - Assertions.assertEquals(1, getInflightCount(InflightType.REPLICATION)); - Assertions.assertEquals(1, replicationManager.getMetrics() - .getInflightReplication()); - - // We should have one under replicated and one quasi_closed_stuck - ReplicationManagerReport report = replicationManager.getContainerReport(); - Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - Assertions.assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - Assertions.assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.UNDER_REPLICATED)); - - // Now we add the missing replica back - DatanodeDetails targetDn = replicationManager.getLegacyReplicationManager() - .getFirstDatanode(InflightType.REPLICATION, id); - final ContainerReplica replicatedReplicaOne = getReplicas( - id, State.CLOSED, 1000L, originNodeId, targetDn); - containerStateManager.updateContainerReplica( - id, replicatedReplicaOne); - - final long currentReplicationCommandCompleted = replicationManager - .getMetrics().getNumReplicationCmdsCompleted(); - final long currentBytesCompleted = replicationManager.getMetrics() - .getNumReplicationBytesCompleted(); - - replicationManager.processAll(); - eventQueue.processAll(1000); - - Assertions.assertEquals(0, getInflightCount(InflightType.REPLICATION)); - Assertions.assertEquals(0, replicationManager.getMetrics() - .getInflightReplication()); - Assertions.assertEquals(currentReplicationCommandCompleted + 1, - replicationManager.getMetrics().getNumReplicationCmdsCompleted()); - Assertions.assertEquals(currentBytesCompleted + 100L, - replicationManager.getMetrics().getNumReplicationBytesCompleted()); - - report = replicationManager.getContainerReport(); - Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - Assertions.assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - Assertions.assertEquals(0, report.getStat( - ReplicationManagerReport.HealthState.UNDER_REPLICATED)); - } - - /** - * When a QUASI_CLOSED container is over replicated, ReplicationManager - * deletes the excess replicas. - */ - @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, State.QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaTwo = getReplicas( - id, State.QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaThree = getReplicas( - id, State.QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaFour = getReplicas( - id, State.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); - Assertions.assertEquals(currentDeleteCommandCount + 1, - datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand)); - Assertions.assertEquals(currentDeleteCommandCount + 1, - replicationManager.getMetrics().getNumDeletionCmdsSent()); - Assertions.assertEquals(1, getInflightCount(InflightType.DELETION)); - Assertions.assertEquals(1, replicationManager.getMetrics() - .getInflightDeletion()); - - ReplicationManagerReport report = replicationManager.getContainerReport(); - Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - Assertions.assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - Assertions.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); + @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(); + Assertions.assertEquals(1, report.getStat(LifeCycleState.OPEN)); + Assertions.assertEquals(0, datanodeCommandHandler.getInvocation()); } - final long currentDeleteCommandCompleted = replicationManager.getMetrics() - .getNumDeletionCmdsCompleted(); - final long deleteBytesCompleted = - replicationManager.getMetrics().getNumDeletionBytesCompleted(); - - replicationManager.processAll(); - eventQueue.processAll(1000); - Assertions.assertEquals(0, getInflightCount(InflightType.DELETION)); - Assertions.assertEquals(0, replicationManager.getMetrics() - .getInflightDeletion()); - Assertions.assertEquals(currentDeleteCommandCompleted + 1, - replicationManager.getMetrics().getNumDeletionCmdsCompleted()); - Assertions.assertEquals(deleteBytesCompleted + 101, - replicationManager.getMetrics().getNumDeletionBytesCompleted()); - - report = replicationManager.getContainerReport(); - Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - Assertions.assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - Assertions.assertEquals(0, report.getStat( - ReplicationManagerReport.HealthState.OVER_REPLICATED)); - } - - /** - * When a QUASI_CLOSED container is over replicated, ReplicationManager - * deletes the excess replicas. While choosing the replica for deletion - * ReplicationManager should prioritize unhealthy replica over QUASI_CLOSED - * replica. - */ - @Test - public void testOverReplicatedQuasiClosedContainerWithUnhealthyReplica() - 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, State.UNHEALTHY, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaTwo = getReplicas( - id, State.QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaThree = getReplicas( - id, State.QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaFour = getReplicas( - id, State.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); - Assertions.assertEquals(currentDeleteCommandCount + 1, - datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand)); - Assertions.assertTrue(datanodeCommandHandler.received( - SCMCommandProto.Type.deleteContainerCommand, - replicaOne.getDatanodeDetails())); - Assertions.assertEquals(currentDeleteCommandCount + 1, - replicationManager.getMetrics().getNumDeletionCmdsSent()); - Assertions.assertEquals(1, getInflightCount(InflightType.DELETION)); - Assertions.assertEquals(1, replicationManager.getMetrics() - .getInflightDeletion()); - - ReplicationManagerReport report = replicationManager.getContainerReport(); - Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - Assertions.assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - Assertions.assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.OVER_REPLICATED)); - - final long currentDeleteCommandCompleted = replicationManager.getMetrics() - .getNumDeletionCmdsCompleted(); - // Now we remove the replica to simulate deletion complete - containerStateManager.removeContainerReplica(id, replicaOne); - - replicationManager.processAll(); - eventQueue.processAll(1000); - - Assertions.assertEquals(currentDeleteCommandCompleted + 1, - replicationManager.getMetrics().getNumDeletionCmdsCompleted()); - Assertions.assertEquals(0, getInflightCount(InflightType.DELETION)); - Assertions.assertEquals(0, replicationManager.getMetrics() - .getInflightDeletion()); - - report = replicationManager.getContainerReport(); - Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - Assertions.assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - Assertions.assertEquals(0, report.getStat( - ReplicationManagerReport.HealthState.OVER_REPLICATED)); - } - - /** - * ReplicationManager should replicate an QUASI_CLOSED replica if it is - * under replicated. - */ - @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, State.QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaTwo = getReplicas( - id, State.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() - .getNumReplicationBytesTotal(); - - replicationManager.processAll(); - eventQueue.processAll(1000); - Assertions.assertEquals(currentReplicateCommandCount + 1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - Assertions.assertEquals(currentReplicateCommandCount + 1, - replicationManager.getMetrics().getNumReplicationCmdsSent()); - Assertions.assertEquals(currentBytesToReplicate + 100, - replicationManager.getMetrics().getNumReplicationBytesTotal()); - Assertions.assertEquals(1, getInflightCount(InflightType.REPLICATION)); - Assertions.assertEquals(1, replicationManager.getMetrics() - .getInflightReplication()); + /** + * 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(); - ReplicationManagerReport report = replicationManager.getContainerReport(); - Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - Assertions.assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - Assertions.assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.UNDER_REPLICATED)); + containerStateManager.addContainer(container.getProtobuf()); - final long currentReplicateCommandCompleted = replicationManager - .getMetrics().getNumReplicationCmdsCompleted(); - final long currentReplicateBytesCompleted = replicationManager - .getMetrics().getNumReplicationBytesCompleted(); + // Two replicas in CLOSING state + final Set replicas = getReplicas(id, State.CLOSING, + randomDatanodeDetails(), + randomDatanodeDetails()); - // Now we add the replicated new replica - DatanodeDetails targetDn = replicationManager.getLegacyReplicationManager() - .getFirstDatanode(InflightType.REPLICATION, id); - final ContainerReplica replicatedReplicaThree = getReplicas( - id, State.CLOSED, 1000L, originNodeId, targetDn); - containerStateManager.updateContainerReplica( - id, replicatedReplicaThree); + // One replica in OPEN state + final DatanodeDetails datanode = randomDatanodeDetails(); + replicas.addAll(getReplicas(id, State.OPEN, datanode)); - replicationManager.processAll(); - eventQueue.processAll(1000); + for (ContainerReplica replica : replicas) { + containerStateManager.updateContainerReplica(id, replica); + } - Assertions.assertEquals(currentReplicateCommandCompleted + 1, - replicationManager.getMetrics().getNumReplicationCmdsCompleted()); - Assertions.assertEquals(currentReplicateBytesCompleted + 100, - replicationManager.getMetrics().getNumReplicationBytesCompleted()); - Assertions.assertEquals(0, getInflightCount(InflightType.REPLICATION)); - Assertions.assertEquals(0, replicationManager.getMetrics() - .getInflightReplication()); - - report = replicationManager.getContainerReport(); - Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - Assertions.assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - Assertions.assertEquals(0, report.getStat( - ReplicationManagerReport.HealthState.UNDER_REPLICATED)); - } + final int currentCloseCommandCount = datanodeCommandHandler + .getInvocationCount(SCMCommandProto.Type.closeContainerCommand); - /** - * When a QUASI_CLOSED container is under replicated, ReplicationManager - * should re-replicate it. If there are any unhealthy replica, it has to - * be deleted. - * - * In this test case, the container is QUASI_CLOSED and is under replicated - * and also has an unhealthy replica. - * - * In the first iteration of ReplicationManager, it should re-replicate - * the container so that it has enough replicas. - * - * In the second iteration, ReplicationManager should delete the unhealthy - * replica. - * - * In the third iteration, ReplicationManager will re-replicate as the - * container has again become under replicated after the unhealthy - * replica has been deleted. - * - */ - @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, State.QUASI_CLOSED, 1000L, originNodeId, randomDatanodeDetails()); - final ContainerReplica replicaTwo = getReplicas( - id, State.UNHEALTHY, 1000L, originNodeId, randomDatanodeDetails()); + replicationManager.processAll(); + eventQueue.processAll(1000); + Assertions.assertEquals(currentCloseCommandCount + 3, + datanodeCommandHandler.getInvocationCount( + SCMCommandProto.Type.closeContainerCommand)); - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, replicaOne); - containerStateManager.updateContainerReplica(id, replicaTwo); + // Update the OPEN to CLOSING + for (ContainerReplica replica: getReplicas(id, State.CLOSING, datanode)) { + containerStateManager.updateContainerReplica(id, replica); + } - final int currentReplicateCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.replicateContainerCommand); - final int currentDeleteCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand); - final long currentBytesToDelete = replicationManager.getMetrics() - .getNumDeletionBytesTotal(); + replicationManager.processAll(); + eventQueue.processAll(1000); + Assertions.assertEquals(currentCloseCommandCount + 6, + datanodeCommandHandler.getInvocationCount( + SCMCommandProto.Type.closeContainerCommand)); + ReplicationManagerReport report = replicationManager.getContainerReport(); + Assertions.assertEquals(1, report.getStat(LifeCycleState.CLOSING)); + } - replicationManager.processAll(); - GenericTestUtils.waitFor( - () -> (currentReplicateCommandCount + 1) == datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.replicateContainerCommand), - 50, 5000); + @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); + Assertions.assertEquals(1, replicationManager.getMetrics() + .getNumReplicationCmdsTimeout()); + } - Optional replicateCommand = datanodeCommandHandler - .getReceivedCommands().stream() - .filter(c -> c.getCommand().getType() - .equals(SCMCommandProto.Type.replicateContainerCommand)) - .findFirst(); + @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); + Assertions.assertEquals(1, replicationManager.getMetrics() + .getNumDeletionCmdsTimeout()); + } - Assertions.assertTrue(replicateCommand.isPresent()); + /** + * 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); + } - DatanodeDetails newNode = createDatanodeDetails( - replicateCommand.get().getDatanodeId()); - ContainerReplica newReplica = getReplicas( - id, State.QUASI_CLOSED, 1000L, originNodeId, newNode); - containerStateManager.updateContainerReplica(id, newReplica); + 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; + } - ReplicationManagerReport report = replicationManager.getContainerReport(); - Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - Assertions.assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - Assertions.assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.UNDER_REPLICATED)); - Assertions.assertEquals(0, report.getStat( - ReplicationManagerReport.HealthState.UNHEALTHY)); + @Test + public void testDeletionLimit() throws Exception { + runTestLimit(0, 2, 0, 1, + () -> runTestDeleteEmptyContainer(2)); + } - /* - * We have report the replica to SCM, in the next ReplicationManager - * iteration it should delete the unhealthy replica. + /** + * 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.processAll(); - eventQueue.processAll(1000); - Assertions.assertEquals(currentDeleteCommandCount + 1, - datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand)); - // ReplicaTwo should be deleted, that is the unhealthy one - Assertions.assertTrue(datanodeCommandHandler.received( - SCMCommandProto.Type.deleteContainerCommand, - replicaTwo.getDatanodeDetails())); - Assertions.assertEquals(currentDeleteCommandCount + 1, - replicationManager.getMetrics().getNumDeletionCmdsSent()); - Assertions.assertEquals(currentBytesToDelete + 99, - replicationManager.getMetrics().getNumDeletionBytesTotal()); - Assertions.assertEquals(1, getInflightCount(InflightType.DELETION)); - Assertions.assertEquals(1, replicationManager.getMetrics() - .getInflightDeletion()); - - containerStateManager.removeContainerReplica(id, replicaTwo); - - final long currentDeleteCommandCompleted = replicationManager.getMetrics() - .getNumDeletionCmdsCompleted(); - - report = replicationManager.getContainerReport(); - Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - Assertions.assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - Assertions.assertEquals(0, report.getStat( - ReplicationManagerReport.HealthState.UNDER_REPLICATED)); - Assertions.assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.UNHEALTHY)); - /* - * We have now removed unhealthy replica, next iteration of - * ReplicationManager should re-replicate the container as it - * is under replicated now + /** + * ReplicationManager should replicate zero replica when all copies + * are missing. */ - - replicationManager.processAll(); - eventQueue.processAll(1000); - - Assertions.assertEquals(0, getInflightCount(InflightType.DELETION)); - Assertions.assertEquals(0, replicationManager.getMetrics() - .getInflightDeletion()); - Assertions.assertEquals(currentDeleteCommandCompleted + 1, - replicationManager.getMetrics().getNumDeletionCmdsCompleted()); - - Assertions.assertEquals(currentReplicateCommandCount + 2, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - Assertions.assertEquals(currentReplicateCommandCount + 2, - replicationManager.getMetrics().getNumReplicationCmdsSent()); - Assertions.assertEquals(1, getInflightCount(InflightType.REPLICATION)); - Assertions.assertEquals(1, replicationManager.getMetrics() - .getInflightReplication()); - - report = replicationManager.getContainerReport(); - Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - Assertions.assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); - Assertions.assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.UNDER_REPLICATED)); - Assertions.assertEquals(0, report.getStat( - ReplicationManagerReport.HealthState.UNHEALTHY)); - } - - - /** - * 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, State.QUASI_CLOSED, - randomDatanodeDetails(), - randomDatanodeDetails(), - randomDatanodeDetails()); - containerStateManager.addContainer(container.getProtobuf()); - for (ContainerReplica replica : replicas) { - containerStateManager.updateContainerReplica(id, replica); + @Test + public void testContainerWithMissingReplicas() + throws IOException, TimeoutException { + createContainer(LifeCycleState.CLOSED); + assertReplicaScheduled(0); + assertUnderReplicatedCount(1); + assertMissingCount(1); } - final int currentCloseCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.closeContainerCommand); - - replicationManager.processAll(); - eventQueue.processAll(1000); + /** + * 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); + } - // All the replicas have same BCSID, so all of them will be closed. - Assertions.assertEquals(currentCloseCommandCount + 3, datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.closeContainerCommand)); + replicationManager.processAll(); + eventQueue.processAll(1000); + Assertions.assertEquals(0, datanodeCommandHandler.getInvocation()); - ReplicationManagerReport report = replicationManager.getContainerReport(); - Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); - Assertions.assertEquals(0, report.getStat( - ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); + ReplicationManagerReport report = replicationManager.getContainerReport(); + Assertions.assertEquals(1, report.getStat(LifeCycleState.CLOSED)); + for (ReplicationManagerReport.HealthState s : + ReplicationManagerReport.HealthState.values()) { + Assertions.assertEquals(0, report.getStat(s)); + } + } } - /** - * ReplicationManager should not take any action if the container is - * CLOSED and healthy. + * Tests replication manager with unhealthy and quasi-closed container + * replicas. */ - @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()); + @Nested + class UnstableReplicas { + /** + * 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); - containerStateManager.addContainer(container.getProtobuf()); - for (ContainerReplica replica : replicas) { - containerStateManager.updateContainerReplica(id, replica); + // First iteration + + final int currentCloseCommandCount = datanodeCommandHandler + .getInvocationCount(SCMCommandProto.Type.closeContainerCommand); + // Two of the replicas are in OPEN state + replicationManager.processAll(); + eventQueue.processAll(1000); + Assertions.assertEquals(currentCloseCommandCount + 2, + datanodeCommandHandler.getInvocationCount( + SCMCommandProto.Type.closeContainerCommand)); + Assertions.assertTrue(datanodeCommandHandler.received( + SCMCommandProto.Type.closeContainerCommand, + replicaTwo.getDatanodeDetails())); + Assertions.assertTrue(datanodeCommandHandler.received( + SCMCommandProto.Type.closeContainerCommand, + replicaThree.getDatanodeDetails())); + ReplicationManagerReport report = replicationManager.getContainerReport(); + Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); } - replicationManager.processAll(); - eventQueue.processAll(1000); - Assertions.assertEquals(0, datanodeCommandHandler.getInvocation()); + /** + * 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); - ReplicationManagerReport report = replicationManager.getContainerReport(); - Assertions.assertEquals(1, report.getStat(LifeCycleState.CLOSED)); - for (ReplicationManagerReport.HealthState s : - ReplicationManagerReport.HealthState.values()) { - Assertions.assertEquals(0, report.getStat(s)); + // 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); + Assertions.assertEquals(0, datanodeCommandHandler.getInvocation()); + ReplicationManagerReport report = replicationManager.getContainerReport(); + Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); + Assertions.assertEquals(1, report.getStat( + ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); } - } - /** - * 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, State.UNHEALTHY, randomDatanodeDetails())); + /** + * 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); + Assertions.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() + .getNumReplicationBytesTotal(); + 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; + Assertions.assertEquals(currentReplicateCommandCount, + datanodeCommandHandler.getInvocationCount( + SCMCommandProto.Type.replicateContainerCommand)); + Assertions.assertEquals(currentReplicateCommandCount, + replicationManager.getMetrics().getNumReplicationCmdsSent()); + Assertions.assertEquals(currentBytesToReplicate, + replicationManager.getMetrics().getNumReplicationBytesTotal()); + Assertions.assertEquals(1, getInflightCount(InflightType.REPLICATION)); + Assertions.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(); + Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); + Assertions.assertEquals(1, report.getStat( + ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); + Assertions.assertEquals(0, report.getStat( + ReplicationManagerReport.HealthState.UNDER_REPLICATED)); + Assertions.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); + } - containerStateManager.addContainer(container.getProtobuf()); - for (ContainerReplica replica : replicas) { - containerStateManager.updateContainerReplica(id, replica); + // 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. + Assertions.assertEquals(currentReplicateCommandCount, + datanodeCommandHandler.getInvocationCount( + SCMCommandProto.Type.replicateContainerCommand)); + Assertions.assertEquals(currentReplicateCommandCount, + replicationManager.getMetrics().getNumReplicationCmdsSent()); + Assertions.assertEquals(currentBytesToReplicate, + replicationManager.getMetrics().getNumReplicationBytesTotal()); + Assertions.assertEquals(0, getInflightCount(InflightType.REPLICATION)); + Assertions.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. + Assertions.assertEquals(2, datanodeCommandHandler.getInvocation()); } - final CloseContainerEventHandler closeContainerHandler = - Mockito.mock(CloseContainerEventHandler.class); - eventQueue.addHandler(SCMEvents.CLOSE_CONTAINER, closeContainerHandler); - - replicationManager.processAll(); - eventQueue.processAll(1000); - Mockito.verify(closeContainerHandler, Mockito.times(1)) - .onMessage(id, eventQueue); - - ReplicationManagerReport report = replicationManager.getContainerReport(); - Assertions.assertEquals(1, report.getStat(LifeCycleState.OPEN)); - Assertions.assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.OPEN_UNHEALTHY)); - } - /** - * 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, State.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); + /** + * 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); + + Assertions.assertEquals(0, + datanodeCommandHandler.getInvocationCount( + SCMCommandProto.Type.replicateContainerCommand)); + + Assertions.assertEquals(0, + datanodeCommandHandler.getInvocationCount( + SCMCommandProto.Type.deleteContainerCommand)); } - replicationManager.processAll(); - // Wait for EventQueue to call the event handler - eventQueue.processAll(1000); - Assertions.assertEquals(2, datanodeCommandHandler.getInvocation()); - } - - @Test - public void testGeneratedConfig() { - ReplicationManagerConfiguration rmc = - OzoneConfiguration.newInstanceOf(ReplicationManagerConfiguration.class); - - //default is not included in ozone-site.xml but generated from annotation - //to the ozone-site-generated.xml which should be loaded by the - // OzoneConfiguration. - Assertions.assertEquals(1800000, rmc.getEventTimeout()); + /** + * 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); + Assertions.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); + Assertions.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); + Assertions.assertEquals(1, + datanodeCommandHandler.getInvocationCount( + SCMCommandProto.Type.deleteContainerCommand)); + assertExactDeleteTargets(unhealthyReplica.getDatanodeDetails()); + } - @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. - Mockito.when(ratisContainerPlacementPolicy.validateContainerPlacement( - Mockito.argThat(list -> list.size() == 3), - Mockito.anyInt() - )).thenAnswer(invocation -> { - return new ContainerPlacementStatusDefault(1, 2, 3); - }); + /** + * $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); + } - int currentReplicateCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.replicateContainerCommand); - final long currentBytesToReplicate = replicationManager.getMetrics() - .getNumReplicationBytesTotal(); + /** + * 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; + UUID closedDeletedUUID = null; + + for (CommandForDatanode command : + datanodeCommandHandler.getReceivedCommands()) { + if (command.getCommand().getType() == + SCMCommandProto.Type.deleteContainerCommand) { + if (command.getDatanodeId() == + unhealthy.getDatanodeDetails().getUuid()) { + unhealthyDeleted = true; + } else { + closedDeleted = true; + closedDeletedUUID = command.getDatanodeId(); + } + } + } - 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. + Assertions.assertFalse(unhealthyDeleted); + Assertions.assertTrue(closedDeleted); + + // Do a second run. + assertReplicaScheduled(0); + assertUnderReplicatedCount(0); + unhealthyDeleted = false; + 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; + // The delete command should have been left over from the last run. + Assertions.assertEquals(closedDeletedUUID, command.getDatanodeId()); + } + } + } - Assertions.assertEquals(currentReplicateCommandCount + 1, - datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - Assertions.assertEquals(currentReplicateCommandCount + 1, - replicationManager.getMetrics().getNumReplicationCmdsSent()); - Assertions.assertEquals(currentBytesToReplicate + 100, - replicationManager.getMetrics().getNumReplicationBytesTotal()); - Assertions.assertEquals(1, getInflightCount(InflightType.REPLICATION)); - Assertions.assertEquals(1, replicationManager.getMetrics() - .getInflightReplication()); + Assertions.assertTrue(unhealthyDeleted); + Assertions.assertTrue(closedDeleted); + } - ReplicationManagerReport report = replicationManager.getContainerReport(); - Assertions.assertEquals(1, report.getStat(LifeCycleState.CLOSED)); - Assertions.assertEquals(1, report.getStat( - ReplicationManagerReport.HealthState.MIS_REPLICATED)); + /** + * 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()); + } - // 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. - Mockito.when(ratisContainerPlacementPolicy.validateContainerPlacement( - Mockito.anyList(), - Mockito.anyInt() - )).thenAnswer(invocation -> { - return new ContainerPlacementStatusDefault(1, 2, 3); - }); + // No replications should be scheduled. + replicationManager.processAll(); + eventQueue.processAll(1000); + Assertions.assertEquals(0, + datanodeCommandHandler.getInvocationCount( + SCMCommandProto.Type.replicateContainerCommand)); + assertUnderReplicatedCount(0); + + // One replica should be deleted. + Assertions.assertEquals(1, + datanodeCommandHandler.getInvocationCount( + SCMCommandProto.Type.deleteContainerCommand)); + Assertions.assertTrue( + datanodeCommandHandler.getReceivedCommands().stream() + .anyMatch(c -> c.getCommand().getType() == + SCMCommandProto.Type.deleteContainerCommand && + unhealthyContainerDNIDs.contains(c.getDatanodeId()))); + } - currentReplicateCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.replicateContainerCommand); + /** + * 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); + Assertions.assertEquals(currentDeleteCommandCount + 1, + datanodeCommandHandler + .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand)); + Assertions.assertEquals(currentDeleteCommandCount + 1, + replicationManager.getMetrics().getNumDeletionCmdsSent()); + Assertions.assertEquals(1, getInflightCount(InflightType.DELETION)); + Assertions.assertEquals(1, replicationManager.getMetrics() + .getInflightDeletion()); + + ReplicationManagerReport report = replicationManager.getContainerReport(); + Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); + Assertions.assertEquals(1, report.getStat( + ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); + Assertions.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); + } - 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. - Assertions.assertEquals(currentReplicateCommandCount, datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.replicateContainerCommand)); - Assertions.assertEquals(currentReplicateCommandCount, - replicationManager.getMetrics().getNumReplicationCmdsSent()); - Assertions.assertEquals(1, getInflightCount(InflightType.REPLICATION)); - Assertions.assertEquals(1, replicationManager.getMetrics() - .getInflightReplication()); - } + final long currentDeleteCommandCompleted = replicationManager.getMetrics() + .getNumDeletionCmdsCompleted(); + final long deleteBytesCompleted = + replicationManager.getMetrics().getNumDeletionBytesCompleted(); + + replicationManager.processAll(); + eventQueue.processAll(1000); + Assertions.assertEquals(0, getInflightCount(InflightType.DELETION)); + Assertions.assertEquals(0, replicationManager.getMetrics() + .getInflightDeletion()); + Assertions.assertEquals(currentDeleteCommandCompleted + 1, + replicationManager.getMetrics().getNumDeletionCmdsCompleted()); + Assertions.assertEquals(deleteBytesCompleted + 101, + replicationManager.getMetrics().getNumDeletionBytesCompleted()); + + report = replicationManager.getContainerReport(); + Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); + Assertions.assertEquals(1, report.getStat( + ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); + Assertions.assertEquals(0, report.getStat( + ReplicationManagerReport.HealthState.OVER_REPLICATED)); + } - @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()); - final ContainerReplica replicaFive = getReplicas( - id, State.UNHEALTHY, 1000L, originNodeId, randomDatanodeDetails()); + /** + * 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); + } - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, replicaOne); - containerStateManager.updateContainerReplica(id, replicaTwo); - containerStateManager.updateContainerReplica( - id, replicaThree); - containerStateManager.updateContainerReplica(id, replicaFour); - containerStateManager.updateContainerReplica(id, replicaFive); - - // Ensure a mis-replicated status is returned for any containers in this - // test where there are exactly 3 replicas checked. - Mockito.when(ratisContainerPlacementPolicy.validateContainerPlacement( - Mockito.argThat(list -> list.size() == 3), - Mockito.anyInt() - )).thenAnswer( - invocation -> new ContainerPlacementStatusDefault(1, 2, 3)); + final CloseContainerEventHandler closeContainerHandler = + Mockito.mock(CloseContainerEventHandler.class); + eventQueue.addHandler(SCMEvents.CLOSE_CONTAINER, closeContainerHandler); - int currentDeleteCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand); + replicationManager.processAll(); + eventQueue.processAll(1000); + Mockito.verify(closeContainerHandler, Mockito.times(1)) + .onMessage(id, eventQueue); - replicationManager.processAll(); - eventQueue.processAll(1000); - // The unhealthy replica should be removed, but not the other replica - // as each time we test with 3 replicas, Mockito ensures it returns - // mis-replicated - Assertions.assertEquals(currentDeleteCommandCount + 1, - datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand)); - Assertions.assertEquals(currentDeleteCommandCount + 1, - replicationManager.getMetrics().getNumDeletionCmdsSent()); + ReplicationManagerReport report = replicationManager.getContainerReport(); + Assertions.assertEquals(1, report.getStat(LifeCycleState.OPEN)); + Assertions.assertEquals(1, report.getStat( + ReplicationManagerReport.HealthState.OPEN_UNHEALTHY)); + } - Assertions.assertTrue(datanodeCommandHandler.received( - SCMCommandProto.Type.deleteContainerCommand, - replicaFive.getDatanodeDetails())); - Assertions.assertEquals(1, getInflightCount(InflightType.DELETION)); - Assertions.assertEquals(1, replicationManager.getMetrics() - .getInflightDeletion()); - assertOverReplicatedCount(1); - } + /** + * 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); + } - @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()); + replicationManager.processAll(); + // Wait for EventQueue to call the event handler + eventQueue.processAll(1000); + Assertions.assertEquals(2, + datanodeCommandHandler.getInvocation()); + } - containerStateManager.addContainer(container.getProtobuf()); - containerStateManager.updateContainerReplica(id, replicaOne); - containerStateManager.updateContainerReplica(id, replicaTwo); - containerStateManager.updateContainerReplica( - id, replicaThree); - containerStateManager.updateContainerReplica(id, replicaFour); + /** + * 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); + Assertions.assertTrue( + datanodeCommandHandler.getReceivedCommands().stream() + .anyMatch(c -> c.getCommand().getType() == + SCMCommandProto.Type.deleteContainerCommand && + c.getDatanodeId().equals( + unhealthyReplica.getDatanodeDetails().getUuid()))); + + ReplicationManagerReport report = replicationManager.getContainerReport(); + Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); + Assertions.assertEquals(1, report.getStat( + ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); + // Container should have been considered over replicated including the + // unhealthy replica. + Assertions.assertEquals(1, report.getStat( + ReplicationManagerReport.HealthState.OVER_REPLICATED)); + + final long currentDeleteCommandCompleted = replicationManager.getMetrics() + .getNumDeletionCmdsCompleted(); + // 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); + + Assertions.assertEquals(currentDeleteCommandCompleted + 1, + replicationManager.getMetrics().getNumDeletionCmdsCompleted()); + Assertions.assertEquals(0, getInflightCount(InflightType.DELETION)); + Assertions.assertEquals(0, replicationManager.getMetrics() + .getInflightDeletion()); + + report = replicationManager.getContainerReport(); + Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); + Assertions.assertEquals(1, report.getStat( + ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); + Assertions.assertEquals(0, report.getStat( + ReplicationManagerReport.HealthState.OVER_REPLICATED)); + } - Mockito.when(ratisContainerPlacementPolicy.validateContainerPlacement( - Mockito.argThat(list -> list.size() == 3), - Mockito.anyInt() - )).thenAnswer( - invocation -> new ContainerPlacementStatusDefault(2, 2, 3)); + /** + * 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() + .getNumReplicationBytesTotal(); + + // On the first iteration, one of the quasi closed replicas should be + // replicated. + replicationManager.processAll(); + eventQueue.processAll(1000); + Assertions.assertEquals(currentReplicateCommandCount + 1, + datanodeCommandHandler.getInvocationCount( + SCMCommandProto.Type.replicateContainerCommand)); + Assertions.assertEquals(currentReplicateCommandCount + 1, + replicationManager.getMetrics().getNumReplicationCmdsSent()); + Assertions.assertEquals(currentBytesToReplicate + 100, + replicationManager.getMetrics().getNumReplicationBytesTotal()); + Assertions.assertEquals(1, getInflightCount(InflightType.REPLICATION)); + Assertions.assertEquals(1, replicationManager.getMetrics() + .getInflightReplication()); + + ReplicationManagerReport report = replicationManager.getContainerReport(); + Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); + Assertions.assertEquals(1, report.getStat( + ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); + Assertions.assertEquals(1, report.getStat( + ReplicationManagerReport.HealthState.UNDER_REPLICATED)); + + final long currentReplicateCommandCompleted = replicationManager + .getMetrics().getNumReplicationCmdsCompleted(); + final long currentReplicateBytesCompleted = replicationManager + .getMetrics().getNumReplicationBytesCompleted(); + + // 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); + + Assertions.assertEquals(currentReplicateCommandCompleted + 1, + replicationManager.getMetrics().getNumReplicationCmdsCompleted()); + Assertions.assertEquals(currentReplicateBytesCompleted + 100, + replicationManager.getMetrics().getNumReplicationBytesCompleted()); + Assertions.assertEquals(0, getInflightCount(InflightType.REPLICATION)); + Assertions.assertEquals(0, replicationManager.getMetrics() + .getInflightReplication()); + + report = replicationManager.getContainerReport(); + Assertions.assertEquals(1, report.getStat(LifeCycleState.QUASI_CLOSED)); + Assertions.assertEquals(1, report.getStat( + ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); + Assertions.assertEquals(0, report.getStat( + ReplicationManagerReport.HealthState.UNDER_REPLICATED)); + } - final int currentDeleteCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand); + /** + * 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() + .getNumDeletionBytesTotal(); + + // Run first iteraiton + + replicationManager.processAll(); + GenericTestUtils.waitFor( + () -> (currentReplicateCommandCount + 2) == datanodeCommandHandler + .getInvocationCount( + SCMCommandProto.Type.replicateContainerCommand), + 50, 5000); + + ReplicationManagerReport report = replicationManager.getContainerReport(); + Assertions.assertEquals(1, + report.getStat(LifeCycleState.QUASI_CLOSED)); + Assertions.assertEquals(1, report.getStat( + ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); + Assertions.assertEquals(1, report.getStat( + ReplicationManagerReport.HealthState.UNDER_REPLICATED)); + Assertions.assertEquals(1, report.getStat( + ReplicationManagerReport.HealthState.UNHEALTHY)); + + List replicateCommands = datanodeCommandHandler + .getReceivedCommands().stream() + .filter(c -> c.getCommand().getType() + .equals(SCMCommandProto.Type.replicateContainerCommand)) + .collect(Collectors.toList()); + + Assertions.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); + } - replicationManager.processAll(); - eventQueue.processAll(1000); - Assertions.assertEquals(currentDeleteCommandCount + 1, - datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand)); - Assertions.assertEquals(currentDeleteCommandCount + 1, - replicationManager.getMetrics().getNumDeletionCmdsSent()); - Assertions.assertEquals(1, getInflightCount(InflightType.DELETION)); - Assertions.assertEquals(1, replicationManager.getMetrics() - .getInflightDeletion()); + // Run second iteration. + // Now that the quasi closed replica is sufficiently replicated, SCM + // should delete the unhealthy replica on the next iteration. - assertOverReplicatedCount(1); - } + replicationManager.processAll(); + eventQueue.processAll(1000); + Assertions.assertEquals(currentDeleteCommandCount + 1, + datanodeCommandHandler + .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand)); + Assertions.assertTrue(datanodeCommandHandler.received( + SCMCommandProto.Type.deleteContainerCommand, + unhealthyReplica.getDatanodeDetails())); + Assertions.assertEquals(currentDeleteCommandCount + 1, + replicationManager.getMetrics().getNumDeletionCmdsSent()); + Assertions.assertEquals(currentBytesToDelete + 99, + replicationManager.getMetrics().getNumDeletionBytesTotal()); + Assertions.assertEquals(1, + getInflightCount(InflightType.DELETION)); + Assertions.assertEquals(1, replicationManager.getMetrics() + .getInflightDeletion()); + + containerStateManager.removeContainerReplica(id, unhealthyReplica); + + report = replicationManager.getContainerReport(); + Assertions.assertEquals(1, + report.getStat(LifeCycleState.QUASI_CLOSED)); + Assertions.assertEquals(1, report.getStat( + ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); + Assertions.assertEquals(0, report.getStat( + ReplicationManagerReport.HealthState.UNDER_REPLICATED)); + Assertions.assertEquals(1, report.getStat( + ReplicationManagerReport.HealthState.UNHEALTHY)); + } - @Test - 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, State.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); + /** + * 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); + } - Mockito.when(ratisContainerPlacementPolicy.validateContainerPlacement( - Mockito.argThat(list -> list != null && list.size() <= 4), - Mockito.anyInt() - )).thenAnswer( - invocation -> new ContainerPlacementStatusDefault(1, 2, 3)); + final int currentCloseCommandCount = datanodeCommandHandler + .getInvocationCount(SCMCommandProto.Type.closeContainerCommand); - final int currentDeleteCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand); + replicationManager.processAll(); + eventQueue.processAll(1000); - replicationManager.processAll(); - eventQueue.processAll(1000); - Assertions.assertEquals(currentDeleteCommandCount + 2, - datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand)); - Assertions.assertEquals(currentDeleteCommandCount + 2, - replicationManager.getMetrics().getNumDeletionCmdsSent()); - Assertions.assertEquals(1, getInflightCount(InflightType.DELETION)); - Assertions.assertEquals(1, replicationManager.getMetrics() - .getInflightDeletion()); - } + // All the replicas have same BCSID, so all of them will be closed. + Assertions.assertEquals(currentCloseCommandCount + 3, + datanodeCommandHandler.getInvocationCount( + SCMCommandProto.Type.closeContainerCommand)); - /** - * ReplicationManager should replicate an additional replica if there are - * decommissioned replicas. - */ - @Test - public void testUnderReplicatedDueToDecommission() - 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); + ReplicationManagerReport report = replicationManager.getContainerReport(); + Assertions.assertEquals(1, + report.getStat(LifeCycleState.QUASI_CLOSED)); + Assertions.assertEquals(0, report.getStat( + ReplicationManagerReport.HealthState.QUASI_CLOSED_STUCK)); + } } /** - * ReplicationManager should replicate an additional replica when all copies - * are decommissioning. + * Tests replication manager handling of decommissioning and maintainence + * mode datanodes. */ - @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; - } + @Nested + class DecomAndMaintenance { + /** + * ReplicationManager should replicate an additional replica if there are + * decommissioned replicas. + */ + @Test + public void testUnderReplicatedDueToDecommission() + 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); + } - @Test - public void testReplicationLimit() throws Exception { - runTestLimit(1, 0, 2, 0, - () -> runTestUnderReplicatedDueToAllDecommission(1)); - } + /** + * ReplicationManager should replicate an additional replica when all copies + * are decommissioning. + */ + @Test + public void testUnderReplicatedDueToAllDecommission() + throws IOException, TimeoutException { + runTestUnderReplicatedDueToAllDecommission(3); + } - void runTestLimit(int replicationLimit, int deletionLimit, - int expectedReplicationSkipped, int expectedDeletionSkipped, - Callable testcase) throws Exception { - createReplicationManager(replicationLimit, deletionLimit); + 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; + } - final ReplicationManagerMetrics metrics = replicationManager.getMetrics(); - final long replicationSkipped = metrics.getInflightReplicationSkipped(); - final long deletionSkipped = metrics.getInflightDeletionSkipped(); + @Test + public void testReplicationLimit() throws Exception { + runTestLimit(1, 0, 2, 0, + () -> runTestUnderReplicatedDueToAllDecommission(1)); + } - testcase.call(); + /** + * 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); + } - Assertions.assertEquals(replicationSkipped + expectedReplicationSkipped, - metrics.getInflightReplicationSkipped()); - Assertions.assertEquals(deletionSkipped + expectedDeletionSkipped, - metrics.getInflightDeletionSkipped()); + /** + * 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); + } - //reset limits for other tests. - createReplicationManager(0, 0); - } + /** + * 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 = + new ReplicationManagerConfiguration(); + 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 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 all copies + * are going off line and min rep is 1. + */ + @Test + public void testUnderReplicatedDueToMaintenanceMinRepOne() + throws Exception { + replicationManager.stop(); + ReplicationManagerConfiguration newConf = + new ReplicationManagerConfiguration(); + 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 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 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 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 = - new ReplicationManagerConfiguration(); - 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 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 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 = - new ReplicationManagerConfiguration(); - 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 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); + } - /** - * 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); - } + /** + * 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); + Assertions.assertEquals(currentDeleteCommandCount + 2, + datanodeCommandHandler.getInvocationCount( + SCMCommandProto.Type.deleteContainerCommand)); + Assertions.assertEquals(currentDeleteCommandCount + 2, + replicationManager.getMetrics().getNumDeletionCmdsSent()); + Assertions.assertEquals(1, getInflightCount(InflightType.DELETION)); + Assertions.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) { + Assertions.assertFalse(datanodeCommandHandler.received( + SCMCommandProto.Type.deleteContainerCommand, + r.getDatanodeDetails())); + } + assertOverReplicatedCount(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); + /** + * 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); + } } /** - * ReplicationManager should replicate additional replica when all copies - * are decommissioning or maintenance. + * Tests replication manager move command. */ - @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); - } + @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); + Assertions.assertTrue(scmLogs.getOutput().contains( + "receive a move request about container")); + Thread.sleep(100L); + Assertions.assertTrue(datanodeCommandHandler.received( + SCMCommandProto.Type.replicateContainerCommand, dn3)); + Assertions.assertEquals(1, datanodeCommandHandler.getInvocationCount( + SCMCommandProto.Type.replicateContainerCommand)); + + //replicate container to dn3 + addReplicaToDn(container, dn3, CLOSED); + replicationManager.processAll(); + eventQueue.processAll(1000); + + Assertions.assertTrue(datanodeCommandHandler.received( + SCMCommandProto.Type.deleteContainerCommand, + dn1.getDatanodeDetails())); + Assertions.assertEquals(1, datanodeCommandHandler.getInvocationCount( + SCMCommandProto.Type.deleteContainerCommand)); + containerStateManager.removeContainerReplica(id, dn1); - /** - * 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); - } - /** - * 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); + replicationManager.processAll(); + eventQueue.processAll(1000); - final int currentDeleteCommandCount = datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand); + Assertions.assertTrue(cf.isDone() && cf.get() == MoveResult.COMPLETED); + } - replicationManager.processAll(); - eventQueue.processAll(1000); - Assertions.assertEquals(currentDeleteCommandCount + 2, - datanodeCommandHandler - .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand)); - Assertions.assertEquals(currentDeleteCommandCount + 2, - replicationManager.getMetrics().getNumDeletionCmdsSent()); - Assertions.assertEquals(1, getInflightCount(InflightType.DELETION)); - Assertions.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) { + /** + * 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); + Assertions.assertTrue(scmLogs.getOutput().contains( + "receive a move request about container")); + Thread.sleep(100L); + Assertions.assertTrue(datanodeCommandHandler.received( + SCMCommandProto.Type.replicateContainerCommand, dn3)); + Assertions.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)); + Assertions.assertTrue(replicationManager.getMoveScheduler() + .getInflightMove().containsKey(id)); + MoveDataNodePair kv = replicationManager.getMoveScheduler() + .getInflightMove().get(id); + Assertions.assertEquals(kv.getSrc(), dn1.getDatanodeDetails()); + Assertions.assertEquals(kv.getTgt(), dn3); + serviceManager.notifyStatusChanged(); + + Thread.sleep(100L); + // now, the container is not over-replicated, + // so no deleteContainerCommand will be sent Assertions.assertFalse(datanodeCommandHandler.received( SCMCommandProto.Type.deleteContainerCommand, - r.getDatanodeDetails())); + dn1.getDatanodeDetails())); + //replica does not exist in target datanode, so a + // replicateContainerCommand will be sent again at + // notifyStatusChanged#onLeaderReadyAndOutOfSafeMode + Assertions.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 + Assertions.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)); + Assertions.assertTrue(replicationManager.getMoveScheduler() + .getInflightMove().containsKey(id)); + kv = replicationManager.getMoveScheduler() + .getInflightMove().get(id); + Assertions.assertEquals(kv.getSrc(), dn1.getDatanodeDetails()); + Assertions.assertEquals(kv.getTgt(), dn3); + serviceManager.notifyStatusChanged(); + + //after restart and the container is over-replicated now, + //deleteContainerCommand will be sent again + Assertions.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)); + Assertions.assertFalse(replicationManager.getMoveScheduler() + .getInflightMove().containsKey(id)); + + //completeableFuture is not stored in DB, so after scm crash and + //restart ,completeableFuture is missing } - 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); - } - - /** - * 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); - Assertions.assertTrue(scmLogs.getOutput().contains( - "receive a move request about container")); - Thread.sleep(100L); - Assertions.assertTrue(datanodeCommandHandler.received( - SCMCommandProto.Type.replicateContainerCommand, dn3)); - Assertions.assertEquals(1, datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - //replicate container to dn3 - addReplicaToDn(container, dn3, CLOSED); - replicationManager.processAll(); - eventQueue.processAll(1000); + /** + * 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); + Assertions.assertTrue(scmLogs.getOutput().contains( + "receive a move request about container")); + Thread.sleep(100L); + Assertions.assertTrue(datanodeCommandHandler.received( + SCMCommandProto.Type.replicateContainerCommand, dn4)); + Assertions.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); + + Assertions.assertFalse( + datanodeCommandHandler.received( + SCMCommandProto.Type.deleteContainerCommand, + dn1.getDatanodeDetails())); + + Assertions.assertTrue(cf.isDone() && + cf.get() == MoveResult.DELETE_FAIL_POLICY); + } - Assertions.assertTrue(datanodeCommandHandler.received( - SCMCommandProto.Type.deleteContainerCommand, dn1.getDatanodeDetails())); - Assertions.assertEquals(1, datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.deleteContainerCommand)); - containerStateManager.removeContainerReplica(id, dn1); - replicationManager.processAll(); - eventQueue.processAll(1000); + /** + * 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); + Assertions.assertTrue(scmLogs.getOutput().contains( + "receive a move request about container")); + + nodeManager.setNodeStatus(dn3, new NodeStatus(IN_SERVICE, STALE)); + replicationManager.processAll(); + eventQueue.processAll(1000); + + Assertions.assertTrue(cf.isDone() && cf.get() == + 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); + + Assertions.assertTrue(cf.isDone() && cf.get() == + MoveResult.DELETION_FAIL_NODE_UNHEALTHY); + } - Assertions.assertTrue(cf.isDone() && cf.get() == MoveResult.COMPLETED); + /** + * 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); + Assertions.assertTrue(cf.isDone() && cf.get() == + MoveResult.FAIL_NOT_RUNNING); + replicationManager.start(); + Thread.sleep(100L); + + //container in not in OPEN state + cf = replicationManager.move(id, dn1.getDatanodeDetails(), dn3); + Assertions.assertTrue(cf.isDone() && cf.get() == + MoveResult.REPLICATION_FAIL_CONTAINER_NOT_CLOSED); + //open -> closing + containerStateManager.updateContainerState(id.getProtobuf(), + LifeCycleEvent.FINALIZE); + cf = replicationManager.move(id, dn1.getDatanodeDetails(), dn3); + Assertions.assertTrue(cf.isDone() && cf.get() == + MoveResult.REPLICATION_FAIL_CONTAINER_NOT_CLOSED); + //closing -> quasi_closed + containerStateManager.updateContainerState(id.getProtobuf(), + LifeCycleEvent.QUASI_CLOSE); + cf = replicationManager.move(id, dn1.getDatanodeDetails(), dn3); + Assertions.assertTrue(cf.isDone() && cf.get() == + MoveResult.REPLICATION_FAIL_CONTAINER_NOT_CLOSED); + + //quasi_closed -> closed + containerStateManager.updateContainerState(id.getProtobuf(), + LifeCycleEvent.FORCE_CLOSE); + Assertions.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); + Assertions.assertTrue(cf.isDone() && cf.get() == + MoveResult.REPLICATION_FAIL_NODE_UNHEALTHY); + cf = replicationManager.move(id, dn3, dn1.getDatanodeDetails()); + Assertions.assertTrue(cf.isDone() && cf.get() == + 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); + Assertions.assertTrue(cf.isDone() && cf.get() == + MoveResult.REPLICATION_FAIL_NODE_NOT_IN_SERVICE); + cf = replicationManager.move(id, dn3, dn1.getDatanodeDetails()); + Assertions.assertTrue(cf.isDone() && cf.get() == + 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()); + Assertions.assertTrue(cf.isDone() && cf.get() == + MoveResult.REPLICATION_FAIL_EXIST_IN_TARGET); + + //container does not exist in source datanode + cf = replicationManager.move(id, dn3, dn3); + Assertions.assertTrue(cf.isDone() && cf.get() == + 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); + Assertions.assertTrue(cf.isDone() && cf.get() == + 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); + Assertions.assertTrue(cf.isDone() && cf.get() == + MoveResult.REPLICATION_FAIL_INFLIGHT_REPLICATION); + } } /** - * if crash happened and restarted, move option should work as expected. + * Tests mis-replicated containers with rack topology information. */ - @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); - Assertions.assertTrue(scmLogs.getOutput().contains( - "receive a move request about container")); - Thread.sleep(100L); - Assertions.assertTrue(datanodeCommandHandler.received( - SCMCommandProto.Type.replicateContainerCommand, dn3)); - Assertions.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)); - Assertions.assertTrue(replicationManager.getMoveScheduler() - .getInflightMove().containsKey(id)); - MoveDataNodePair kv = replicationManager.getMoveScheduler() - .getInflightMove().get(id); - Assertions.assertEquals(kv.getSrc(), dn1.getDatanodeDetails()); - Assertions.assertEquals(kv.getTgt(), dn3); - serviceManager.notifyStatusChanged(); + @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. + Mockito.when(ratisContainerPlacementPolicy.validateContainerPlacement( + Mockito.argThat(list -> list.size() == 3), + Mockito.anyInt() + )).thenAnswer(invocation -> { + return new ContainerPlacementStatusDefault(1, 2, 3); + }); + + int currentReplicateCommandCount = datanodeCommandHandler + .getInvocationCount(SCMCommandProto.Type.replicateContainerCommand); + final long currentBytesToReplicate = replicationManager.getMetrics() + .getNumReplicationBytesTotal(); + + 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. + + Assertions.assertEquals(currentReplicateCommandCount + 1, + datanodeCommandHandler.getInvocationCount( + SCMCommandProto.Type.replicateContainerCommand)); + Assertions.assertEquals(currentReplicateCommandCount + 1, + replicationManager.getMetrics().getNumReplicationCmdsSent()); + Assertions.assertEquals(currentBytesToReplicate + 100, + replicationManager.getMetrics().getNumReplicationBytesTotal()); + Assertions.assertEquals(1, getInflightCount(InflightType.REPLICATION)); + Assertions.assertEquals(1, replicationManager.getMetrics() + .getInflightReplication()); + + ReplicationManagerReport report = replicationManager.getContainerReport(); + Assertions.assertEquals(1, report.getStat(LifeCycleState.CLOSED)); + Assertions.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. + Mockito.when(ratisContainerPlacementPolicy.validateContainerPlacement( + Mockito.anyList(), + Mockito.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. + Assertions.assertEquals(currentReplicateCommandCount, + datanodeCommandHandler.getInvocationCount( + SCMCommandProto.Type.replicateContainerCommand)); + Assertions.assertEquals(currentReplicateCommandCount, + replicationManager.getMetrics().getNumReplicationCmdsSent()); + Assertions.assertEquals(1, getInflightCount(InflightType.REPLICATION)); + Assertions.assertEquals(1, replicationManager.getMetrics() + .getInflightReplication()); + } - Thread.sleep(100L); - // now, the container is not over-replicated, - // so no deleteContainerCommand will be sent - Assertions.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 - Assertions.assertEquals(2, datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.replicateContainerCommand)); - - - //replicate container to dn3, now, over-replicated - addReplicaToDn(container, dn3, CLOSED); - replicationManager.processAll(); - eventQueue.processAll(1000); + @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()); + final ContainerReplica replicaFive = getReplicas( + id, UNHEALTHY, 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); + + // Ensure a mis-replicated status is returned for any containers in this + // test where there are exactly 3 replicas checked. + Mockito.when(ratisContainerPlacementPolicy.validateContainerPlacement( + Mockito.argThat(list -> list.size() == 3), + Mockito.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. + Assertions.assertEquals(currentDeleteCommandCount, + datanodeCommandHandler.getInvocationCount( + SCMCommandProto.Type.deleteContainerCommand)); + Assertions.assertEquals(currentDeleteCommandCount, + replicationManager.getMetrics().getNumDeletionCmdsSent()); - //deleteContainerCommand is sent, but the src replica is not deleted now - Assertions.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)); - Assertions.assertTrue(replicationManager.getMoveScheduler() - .getInflightMove().containsKey(id)); - kv = replicationManager.getMoveScheduler() - .getInflightMove().get(id); - Assertions.assertEquals(kv.getSrc(), dn1.getDatanodeDetails()); - Assertions.assertEquals(kv.getTgt(), dn3); - serviceManager.notifyStatusChanged(); + Assertions.assertFalse(datanodeCommandHandler.received( + SCMCommandProto.Type.deleteContainerCommand, + replicaFive.getDatanodeDetails())); + Assertions.assertEquals(0, getInflightCount(InflightType.DELETION)); + Assertions.assertEquals(0, replicationManager.getMetrics() + .getInflightDeletion()); + assertOverReplicatedCount(1); + } - //after restart and the container is over-replicated now, - //deleteContainerCommand will be sent again - Assertions.assertEquals(2, datanodeCommandHandler.getInvocationCount( - SCMCommandProto.Type.deleteContainerCommand)); - containerStateManager.removeContainerReplica(id, dn1); + @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); + + Mockito.when(ratisContainerPlacementPolicy.validateContainerPlacement( + Mockito.argThat(list -> list.size() == 3), + Mockito.anyInt() + )).thenAnswer( + invocation -> new ContainerPlacementStatusDefault(2, 2, 3)); + + final int currentDeleteCommandCount = datanodeCommandHandler + .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand); + + replicationManager.processAll(); + eventQueue.processAll(1000); + Assertions.assertEquals(currentDeleteCommandCount + 1, + datanodeCommandHandler.getInvocationCount( + SCMCommandProto.Type.deleteContainerCommand)); + Assertions.assertEquals(currentDeleteCommandCount + 1, + replicationManager.getMetrics().getNumDeletionCmdsSent()); + Assertions.assertEquals(1, getInflightCount(InflightType.DELETION)); + Assertions.assertEquals(1, replicationManager.getMetrics() + .getInflightDeletion()); + + assertOverReplicatedCount(1); + } - //replica in src datanode is deleted now - containerStateManager.removeContainerReplica(id, dn1); - replicationManager.processAll(); - eventQueue.processAll(1000); + @Test + 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); + + Mockito.when(ratisContainerPlacementPolicy.validateContainerPlacement( + Mockito.argThat(list -> list != null && list.size() <= 4), + Mockito.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); + Assertions.assertEquals(currentDeleteCommandCount + 1, + datanodeCommandHandler.getInvocationCount( + SCMCommandProto.Type.deleteContainerCommand)); + Assertions.assertEquals(currentDeleteCommandCount + 1, + replicationManager.getMetrics().getNumDeletionCmdsSent()); + Assertions.assertEquals(1, getInflightCount(InflightType.DELETION)); + Assertions.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); + Assertions.assertEquals(currentDeleteCommandCount + 1, + datanodeCommandHandler.getInvocationCount( + SCMCommandProto.Type.deleteContainerCommand)); + Assertions.assertEquals(currentDeleteCommandCount + 1, + replicationManager.getMetrics().getNumDeletionCmdsSent()); + Assertions.assertEquals(1, getInflightCount(InflightType.DELETION)); + Assertions.assertEquals(1, replicationManager.getMetrics() + .getInflightDeletion()); + + assertDeleteTargetsContain(replicaFive.getDatanodeDetails()); + } + } - //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)); - Assertions.assertFalse(replicationManager.getMoveScheduler() - .getInflightMove().containsKey(id)); + void runTestLimit(int replicationLimit, int deletionLimit, + int expectedReplicationSkipped, int expectedDeletionSkipped, + Callable testcase) throws Exception { + createReplicationManager(replicationLimit, deletionLimit); - //completeableFuture is not stored in DB, so after scm crash and - //restart ,completeableFuture is missing - } + final ReplicationManagerMetrics metrics = replicationManager.getMetrics(); + final long replicationSkipped = metrics.getInflightReplicationSkipped(); + final long deletionSkipped = metrics.getInflightDeletionSkipped(); - /** - * 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); - Assertions.assertTrue(scmLogs.getOutput().contains( - "receive a move request about container")); - Thread.sleep(100L); - Assertions.assertTrue(datanodeCommandHandler.received( - SCMCommandProto.Type.replicateContainerCommand, dn4)); - Assertions.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); + testcase.call(); - Assertions.assertFalse(datanodeCommandHandler.received( - SCMCommandProto.Type.deleteContainerCommand, dn1.getDatanodeDetails())); + Assertions.assertEquals(replicationSkipped + expectedReplicationSkipped, + metrics.getInflightReplicationSkipped()); + Assertions.assertEquals(deletionSkipped + expectedDeletionSkipped, + metrics.getInflightDeletionSkipped()); - Assertions.assertTrue(cf.isDone() && - cf.get() == MoveResult.DELETE_FAIL_POLICY); + //reset limits for other tests. + createReplicationManager(0, 0); } - /** - * test src and target datanode become unhealthy when moving. + * Checks that the set of datanodes given delete commands exactly matches + * targetDNs. */ - @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); - Assertions.assertTrue(scmLogs.getOutput().contains( - "receive a move request about container")); - - nodeManager.setNodeStatus(dn3, new NodeStatus(IN_SERVICE, STALE)); - replicationManager.processAll(); - eventQueue.processAll(1000); + private void assertExactDeleteTargets(DatanodeDetails... targetDNs) { + List deleteCommands = datanodeCommandHandler + .getReceivedCommands().stream() + .filter(c -> c.getCommand().getType() == + SCMCommandProto.Type.deleteContainerCommand) + .collect(Collectors.toList()); - Assertions.assertTrue(cf.isDone() && cf.get() == - MoveResult.REPLICATION_FAIL_NODE_UNHEALTHY); + Assertions.assertEquals(targetDNs.length, deleteCommands.size()); - 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); + Set targetDNIDs = Arrays.stream(targetDNs) + .map(DatanodeDetails::getUuid) + .collect(Collectors.toSet()); + Set chosenDNIDs = deleteCommands.stream() + .map(CommandForDatanode::getDatanodeId) + .collect(Collectors.toSet()); - Assertions.assertTrue(cf.isDone() && cf.get() == - MoveResult.DELETION_FAIL_NODE_UNHEALTHY); + Assertions.assertEquals(targetDNIDs, chosenDNIDs); } /** - * before Replication Manager generates a completablefuture for a move option, - * some Prerequisites should be satisfied. + * Checks if the set of nodes with deletions scheduled were taken from the + * provided set of DNs. */ - @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); - Assertions.assertTrue(cf.isDone() && cf.get() == - MoveResult.FAIL_NOT_RUNNING); - replicationManager.start(); - Thread.sleep(100L); - - //container in not in OPEN state - cf = replicationManager.move(id, dn1.getDatanodeDetails(), dn3); - Assertions.assertTrue(cf.isDone() && cf.get() == - MoveResult.REPLICATION_FAIL_CONTAINER_NOT_CLOSED); - //open -> closing - containerStateManager.updateContainerState(id.getProtobuf(), - LifeCycleEvent.FINALIZE); - cf = replicationManager.move(id, dn1.getDatanodeDetails(), dn3); - Assertions.assertTrue(cf.isDone() && cf.get() == - MoveResult.REPLICATION_FAIL_CONTAINER_NOT_CLOSED); - //closing -> quasi_closed - containerStateManager.updateContainerState(id.getProtobuf(), - LifeCycleEvent.QUASI_CLOSE); - cf = replicationManager.move(id, dn1.getDatanodeDetails(), dn3); - Assertions.assertTrue(cf.isDone() && cf.get() == - MoveResult.REPLICATION_FAIL_CONTAINER_NOT_CLOSED); - - //quasi_closed -> closed - containerStateManager.updateContainerState(id.getProtobuf(), - LifeCycleEvent.FORCE_CLOSE); - Assertions.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); - Assertions.assertTrue(cf.isDone() && cf.get() == - MoveResult.REPLICATION_FAIL_NODE_UNHEALTHY); - cf = replicationManager.move(id, dn3, dn1.getDatanodeDetails()); - Assertions.assertTrue(cf.isDone() && cf.get() == - 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); - Assertions.assertTrue(cf.isDone() && cf.get() == - MoveResult.REPLICATION_FAIL_NODE_NOT_IN_SERVICE); - cf = replicationManager.move(id, dn3, dn1.getDatanodeDetails()); - Assertions.assertTrue(cf.isDone() && cf.get() == - 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()); - Assertions.assertTrue(cf.isDone() && cf.get() == - MoveResult.REPLICATION_FAIL_EXIST_IN_TARGET); - - //container does not exist in source datanode - cf = replicationManager.move(id, dn3, dn3); - Assertions.assertTrue(cf.isDone() && cf.get() == - 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); - Assertions.assertTrue(cf.isDone() && cf.get() == - 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); - Assertions.assertTrue(cf.isDone() && cf.get() == - MoveResult.REPLICATION_FAIL_INFLIGHT_REPLICATION); - } + private void assertAnyDeleteTargets(DatanodeDetails... validDeleteDNs) { + List deleteCommands = datanodeCommandHandler + .getReceivedCommands().stream() + .filter(c -> c.getCommand().getType() == + SCMCommandProto.Type.deleteContainerCommand) + .collect(Collectors.toList()); - @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); - Assertions.assertEquals(1, replicationManager.getMetrics() - .getNumReplicationCmdsTimeout()); - } + Set deleteCandidateIDs = Arrays.stream(validDeleteDNs) + .map(DatanodeDetails::getUuid) + .collect(Collectors.toSet()); + Set chosenDNIDs = deleteCommands.stream() + .map(CommandForDatanode::getDatanodeId) + .collect(Collectors.toSet()); - @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); - Assertions.assertEquals(1, replicationManager.getMetrics() - .getNumDeletionCmdsTimeout()); + Assertions.assertTrue(deleteCandidateIDs.containsAll(chosenDNIDs)); } /** - * 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). + * Checks if the set of nodes with deletions scheduled contains all of the + * provided DNs. */ - @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; - } + private void assertDeleteTargetsContain(DatanodeDetails... deleteDN) { + List deleteCommands = datanodeCommandHandler + .getReceivedCommands().stream() + .filter(c -> c.getCommand().getType() == + SCMCommandProto.Type.deleteContainerCommand) + .collect(Collectors.toList()); - @Test - public void testDeletionLimit() throws Exception { - runTestLimit(0, 2, 0, 1, - () -> runTestDeleteEmptyContainer(2)); - } + Set deleteDNIDs = Arrays.stream(deleteDN) + .map(DatanodeDetails::getUuid) + .collect(Collectors.toSet()); + Set chosenDNIDs = deleteCommands.stream() + .map(CommandForDatanode::getDatanodeId) + .collect(Collectors.toSet()); - /** - * 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); + Assertions.assertTrue(chosenDNIDs.containsAll(deleteDNIDs)); } private ContainerInfo createContainer(LifeCycleState containerState) @@ -2052,6 +2464,12 @@ private ContainerReplica addReplica(ContainerInfo container, 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. @@ -2059,7 +2477,7 @@ private ContainerReplica addReplicaToDn(ContainerInfo container, UUID.nameUUIDFromBytes(Longs.toByteArray(container.getContainerID())); final ContainerReplica replica = getReplicas(container.containerID(), replicaState, container.getUsedBytes(), container.getNumberOfKeys(), - 1000L, originNodeId, dn); + bcsId, originNodeId, dn); containerStateManager .updateContainerReplica(container.containerID(), replica); return replica; @@ -2124,16 +2542,6 @@ private void assertOverReplicatedCount(int count) { ReplicationManagerReport.HealthState.OVER_REPLICATED)); } - @AfterEach - public void teardown() throws Exception { - containerStateManager.close(); - replicationManager.stop(); - if (dbStore != null) { - dbStore.close(); - } - FileUtils.deleteDirectory(testDir); - } - private static class DatanodeCommandHandler implements EventHandler {