You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2022/10/06 11:10:24 UTC

[GitHub] [ozone] sodonnel opened a new pull request, #3802: HDDS-7058. EC: ReplicationManager - Implement ratis container replication check handler

sodonnel opened a new pull request, #3802:
URL: https://github.com/apache/ozone/pull/3802

   ## What changes were proposed in this pull request?
   
   Create a handler for the new replication manager to process Ratis container and detect under / over / mis-replication issues.
   
   The logic is largely unchanged from the LegacyReplication manager - simply packaged into the new "handler" structure.
   
   At the moment, this code will not be executed by the new replication manager, as all non-EC container will be directed to the Legacy Replication Manager for processing.
   
   This Jira is part of the work to remove the Legacy Replication Manager.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-7058
   
   ## How was this patch tested?
   
   New unit tests
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] siddhantsangwan commented on a diff in pull request #3802: HDDS-7058. EC: ReplicationManager - Implement ratis container replication check handler

Posted by GitBox <gi...@apache.org>.
siddhantsangwan commented on code in PR #3802:
URL: https://github.com/apache/ozone/pull/3802#discussion_r991937392


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java:
##########
@@ -210,7 +218,8 @@ public ReplicationManager(final ConfigurationSource conf,
         .addNext(new ClosingContainerHandler(this))
         .addNext(new QuasiClosedContainerHandler(this))
         .addNext(new ClosedWithMismatchedReplicasHandler(this))
-        .addNext(ecReplicationCheckHandler);
+        .addNext(ecReplicationCheckHandler)
+        .addNext(ratisReplicationCheckHandler);

Review Comment:
   > For the case of EC, the handleChain function would always return false.
   
   I didn't understand this part. If the EC handler returns true, then `handleChain` will also return true. Right?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #3802: HDDS-7058. EC: ReplicationManager - Implement ratis container replication check handler

Posted by GitBox <gi...@apache.org>.
swamirishi commented on code in PR #3802:
URL: https://github.com/apache/ozone/pull/3802#discussion_r991874415


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ContainerHealthResult.java:
##########
@@ -106,6 +106,9 @@ public static class UnHealthyResult extends ContainerHealthResult {
     private final int remainingRedundancy;
     private final boolean dueToDecommission;
     private final boolean sufficientlyReplicatedAfterPending;
+    private boolean dueToMisReplication = false;
+    private boolean isMisReplicated = false;
+    private boolean isMisReplicatedAfterPending = false;

Review Comment:
   Should we have another constructor which initializes the following arguments?
   public UnderReplicatedHealthResult(ContainerInfo containerInfo,
           int remainingRedundancy, boolean dueToDecommission, boolean replicatedOkWithPending, boolean unrecoverable,boolean dueToMisReplication, boolean isMisReplicated, boolean isMisReplicatedAfterPending)



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java:
##########
@@ -210,7 +218,8 @@ public ReplicationManager(final ConfigurationSource conf,
         .addNext(new ClosingContainerHandler(this))
         .addNext(new QuasiClosedContainerHandler(this))
         .addNext(new ClosedWithMismatchedReplicasHandler(this))
-        .addNext(ecReplicationCheckHandler);
+        .addNext(ecReplicationCheckHandler)
+        .addNext(ratisReplicationCheckHandler);

Review Comment:
   For the case of EC, the handleChain function would always return false. See line 465 of this file
   boolean handled = containerCheckChain.handleChain(checkRequest);
       if (!handled) {
         LOG.debug("Container {} had no actions after passing through the " +
             "check chain", containerInfo.containerID());
       }
   This would be always logged in debug mode which is wrong.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisContainerReplicaCount.java:
##########
@@ -257,7 +277,50 @@ public boolean isSufficientlyReplicated() {
    */
   @Override
   public boolean isOverReplicated() {
-    return missingReplicas() + inFlightDel < 0;
+    return isOverReplicated(true);
+  }
+
+  /**
+   * Return true if the container is over replicated. Decommission and
+   * maintenance containers are ignored for this check.
+   * The check ignores inflight additions, as they may fail, but it does
+   * consider inflight deletes if includePendingDelete is true.
+   *
+   * @return True if the container is over replicated, false otherwise.
+   */
+  public boolean isOverReplicated(boolean includePendingDelete) {

Review Comment:
   Could be changed to getExcessRedundancyCanBeCalled(includePending)>0 to avoid redundancy of logic.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/RatisReplicationCheckHandler.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.health;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.RatisContainerReplicaCount;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
+
+/**
+ * Class to determine the health state of a Ratis Container. Given the container
+ * and current replica details, along with replicas pending add and delete,
+ * this class will return a ContainerHealthResult indicating if the container
+ * is healthy, or under / over replicated etc.
+ */
+public class RatisReplicationCheckHandler extends AbstractCheck {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(RatisReplicationCheckHandler.class);
+
+  /**
+   * PlacementPolicy which is used to identify where a container
+   * should be replicated.
+   */
+  private final PlacementPolicy ratisContainerPlacement;
+
+  public RatisReplicationCheckHandler(PlacementPolicy containerPlacement) {
+    this.ratisContainerPlacement = containerPlacement;
+  }
+
+  @Override
+  public boolean handle(ContainerCheckRequest request) {
+    if (request.getContainerInfo().getReplicationType() != RATIS) {
+      // This handler is only for Ratis containers.
+      return false;
+    }
+    ReplicationManagerReport report = request.getReport();
+    ContainerInfo container = request.getContainerInfo();
+    ContainerHealthResult health = checkHealth(request);
+    if (health.getHealthState() == ContainerHealthResult.HealthState.HEALTHY) {
+      // If the container is healthy, there is nothing else to do in this
+      // handler so return as unhandled so any further handlers will be tried.
+      return false;
+    }
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.UNDER_REPLICATED) {
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.UNDER_REPLICATED,
+          container.containerID());
+      ContainerHealthResult.UnderReplicatedHealthResult underHealth
+          = ((ContainerHealthResult.UnderReplicatedHealthResult) health);
+      if (underHealth.isUnrecoverable()) {
+        report.incrementAndSample(ReplicationManagerReport.HealthState.MISSING,
+            container.containerID());
+      }
+      if (underHealth.isMisReplicated()) {
+        report.incrementAndSample(
+            ReplicationManagerReport.HealthState.MIS_REPLICATED,
+            container.containerID());
+      }
+      // TODO - if it is unrecoverable, should we return false to other
+      //        handlers can be tried?
+      if (!underHealth.isUnrecoverable() &&
+          (underHealth.isMisReplicatedAfterPending() ||
+              !underHealth.isSufficientlyReplicatedAfterPending())) {
+        request.getReplicationQueue().enqueue(underHealth);
+      }
+      return true;
+    }
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.OVER_REPLICATED) {
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.OVER_REPLICATED,
+          container.containerID());
+      ContainerHealthResult.OverReplicatedHealthResult overHealth
+          = ((ContainerHealthResult.OverReplicatedHealthResult) health);
+      if (!overHealth.isSufficientlyReplicatedAfterPending()) {
+        request.getReplicationQueue().enqueue(overHealth);
+      }
+      return true;
+    }
+    return false;
+  }
+
+  public ContainerHealthResult checkHealth(ContainerCheckRequest request) {

Review Comment:
   Do we need to have this method as public?



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java:
##########
@@ -210,7 +218,8 @@ public ReplicationManager(final ConfigurationSource conf,
         .addNext(new ClosingContainerHandler(this))
         .addNext(new QuasiClosedContainerHandler(this))
         .addNext(new ClosedWithMismatchedReplicasHandler(this))
-        .addNext(ecReplicationCheckHandler);
+        .addNext(ecReplicationCheckHandler)
+        .addNext(ratisReplicationCheckHandler);

Review Comment:
   I guess we should have a Unit Test case for checking this for both EC & Ratis if it is not there.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/RatisReplicationCheckHandler.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.health;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.RatisContainerReplicaCount;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
+
+/**
+ * Class to determine the health state of a Ratis Container. Given the container
+ * and current replica details, along with replicas pending add and delete,
+ * this class will return a ContainerHealthResult indicating if the container
+ * is healthy, or under / over replicated etc.
+ */
+public class RatisReplicationCheckHandler extends AbstractCheck {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(RatisReplicationCheckHandler.class);
+
+  /**
+   * PlacementPolicy which is used to identify where a container
+   * should be replicated.
+   */
+  private final PlacementPolicy ratisContainerPlacement;
+
+  public RatisReplicationCheckHandler(PlacementPolicy containerPlacement) {
+    this.ratisContainerPlacement = containerPlacement;
+  }
+
+  @Override
+  public boolean handle(ContainerCheckRequest request) {
+    if (request.getContainerInfo().getReplicationType() != RATIS) {
+      // This handler is only for Ratis containers.
+      return false;
+    }
+    ReplicationManagerReport report = request.getReport();
+    ContainerInfo container = request.getContainerInfo();
+    ContainerHealthResult health = checkHealth(request);
+    if (health.getHealthState() == ContainerHealthResult.HealthState.HEALTHY) {
+      // If the container is healthy, there is nothing else to do in this
+      // handler so return as unhandled so any further handlers will be tried.
+      return false;
+    }
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.UNDER_REPLICATED) {
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.UNDER_REPLICATED,
+          container.containerID());
+      ContainerHealthResult.UnderReplicatedHealthResult underHealth
+          = ((ContainerHealthResult.UnderReplicatedHealthResult) health);
+      if (underHealth.isUnrecoverable()) {
+        report.incrementAndSample(ReplicationManagerReport.HealthState.MISSING,
+            container.containerID());
+      }
+      if (underHealth.isMisReplicated()) {
+        report.incrementAndSample(
+            ReplicationManagerReport.HealthState.MIS_REPLICATED,
+            container.containerID());
+      }
+      // TODO - if it is unrecoverable, should we return false to other
+      //        handlers can be tried?
+      if (!underHealth.isUnrecoverable() &&
+          (underHealth.isMisReplicatedAfterPending() ||
+              !underHealth.isSufficientlyReplicatedAfterPending())) {
+        request.getReplicationQueue().enqueue(underHealth);
+      }
+      return true;
+    }
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.OVER_REPLICATED) {
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.OVER_REPLICATED,
+          container.containerID());
+      ContainerHealthResult.OverReplicatedHealthResult overHealth
+          = ((ContainerHealthResult.OverReplicatedHealthResult) health);
+      if (!overHealth.isSufficientlyReplicatedAfterPending()) {
+        request.getReplicationQueue().enqueue(overHealth);
+      }
+      return true;
+    }
+    return false;
+  }
+
+  public ContainerHealthResult checkHealth(ContainerCheckRequest request) {
+    ContainerInfo container = request.getContainerInfo();
+    Set<ContainerReplica> replicas = request.getContainerReplicas();
+    List<ContainerReplicaOp> replicaPendingOps = request.getPendingOps();
+    // Note that this setting is minReplicasForMaintenance. For EC the variable
+    // is defined as remainingRedundancy which is subtly different.
+    int minReplicasForMaintenance = request.getMaintenanceRedundancy();
+    int pendingAdd = 0;
+    int pendingDelete = 0;
+    for (ContainerReplicaOp op : replicaPendingOps) {
+      if (op.getOpType() == ContainerReplicaOp.PendingOpType.ADD) {
+        pendingAdd++;
+      } else if (op.getOpType() == ContainerReplicaOp.PendingOpType.DELETE) {
+        pendingDelete++;
+      }
+    }
+    int requiredNodes = container.getReplicationConfig().getRequiredNodes();
+
+    // RatisContainerReplicaCount uses the minReplicasForMaintenance rather
+    // than remainingRedundancy which ECContainerReplicaCount uses.
+    RatisContainerReplicaCount replicaCount =
+        new RatisContainerReplicaCount(container, replicas, pendingAdd,
+            pendingDelete, requiredNodes, minReplicasForMaintenance);
+
+    ContainerPlacementStatus placementStatus =
+        getPlacementStatus(replicas, requiredNodes, Collections.EMPTY_LIST);

Review Comment:
   ```suggestion
           getPlacementStatus(replicas, requiredNodes, Collections.emptyList());
   ```



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/RatisReplicationCheckHandler.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.health;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.RatisContainerReplicaCount;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
+
+/**
+ * Class to determine the health state of a Ratis Container. Given the container
+ * and current replica details, along with replicas pending add and delete,
+ * this class will return a ContainerHealthResult indicating if the container
+ * is healthy, or under / over replicated etc.
+ */
+public class RatisReplicationCheckHandler extends AbstractCheck {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(RatisReplicationCheckHandler.class);
+
+  /**
+   * PlacementPolicy which is used to identify where a container
+   * should be replicated.
+   */
+  private final PlacementPolicy ratisContainerPlacement;
+
+  public RatisReplicationCheckHandler(PlacementPolicy containerPlacement) {
+    this.ratisContainerPlacement = containerPlacement;
+  }
+
+  @Override
+  public boolean handle(ContainerCheckRequest request) {
+    if (request.getContainerInfo().getReplicationType() != RATIS) {
+      // This handler is only for Ratis containers.
+      return false;
+    }
+    ReplicationManagerReport report = request.getReport();
+    ContainerInfo container = request.getContainerInfo();
+    ContainerHealthResult health = checkHealth(request);
+    if (health.getHealthState() == ContainerHealthResult.HealthState.HEALTHY) {
+      // If the container is healthy, there is nothing else to do in this
+      // handler so return as unhandled so any further handlers will be tried.
+      return false;
+    }
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.UNDER_REPLICATED) {
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.UNDER_REPLICATED,
+          container.containerID());
+      ContainerHealthResult.UnderReplicatedHealthResult underHealth
+          = ((ContainerHealthResult.UnderReplicatedHealthResult) health);
+      if (underHealth.isUnrecoverable()) {
+        report.incrementAndSample(ReplicationManagerReport.HealthState.MISSING,
+            container.containerID());
+      }
+      if (underHealth.isMisReplicated()) {
+        report.incrementAndSample(
+            ReplicationManagerReport.HealthState.MIS_REPLICATED,
+            container.containerID());
+      }
+      // TODO - if it is unrecoverable, should we return false to other
+      //        handlers can be tried?
+      if (!underHealth.isUnrecoverable() &&
+          (underHealth.isMisReplicatedAfterPending() ||
+              !underHealth.isSufficientlyReplicatedAfterPending())) {
+        request.getReplicationQueue().enqueue(underHealth);
+      }
+      return true;
+    }
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.OVER_REPLICATED) {
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.OVER_REPLICATED,
+          container.containerID());
+      ContainerHealthResult.OverReplicatedHealthResult overHealth
+          = ((ContainerHealthResult.OverReplicatedHealthResult) health);
+      if (!overHealth.isSufficientlyReplicatedAfterPending()) {
+        request.getReplicationQueue().enqueue(overHealth);
+      }
+      return true;
+    }
+    return false;
+  }
+
+  public ContainerHealthResult checkHealth(ContainerCheckRequest request) {
+    ContainerInfo container = request.getContainerInfo();
+    Set<ContainerReplica> replicas = request.getContainerReplicas();
+    List<ContainerReplicaOp> replicaPendingOps = request.getPendingOps();
+    // Note that this setting is minReplicasForMaintenance. For EC the variable
+    // is defined as remainingRedundancy which is subtly different.
+    int minReplicasForMaintenance = request.getMaintenanceRedundancy();
+    int pendingAdd = 0;
+    int pendingDelete = 0;
+    for (ContainerReplicaOp op : replicaPendingOps) {
+      if (op.getOpType() == ContainerReplicaOp.PendingOpType.ADD) {
+        pendingAdd++;
+      } else if (op.getOpType() == ContainerReplicaOp.PendingOpType.DELETE) {
+        pendingDelete++;
+      }
+    }
+    int requiredNodes = container.getReplicationConfig().getRequiredNodes();
+
+    // RatisContainerReplicaCount uses the minReplicasForMaintenance rather
+    // than remainingRedundancy which ECContainerReplicaCount uses.
+    RatisContainerReplicaCount replicaCount =
+        new RatisContainerReplicaCount(container, replicas, pendingAdd,
+            pendingDelete, requiredNodes, minReplicasForMaintenance);
+
+    ContainerPlacementStatus placementStatus =
+        getPlacementStatus(replicas, requiredNodes, Collections.EMPTY_LIST);
+
+    ContainerPlacementStatus placementStatusWithPending = placementStatus;
+    if (replicaPendingOps.size() > 0) {
+      placementStatusWithPending =
+          getPlacementStatus(replicas, requiredNodes, replicaPendingOps);
+    }
+    boolean sufficientlyReplicated
+        = replicaCount.isSufficientlyReplicated(false);
+    boolean isPolicySatisfied = placementStatus.isPolicySatisfied();
+    if (!sufficientlyReplicated || !isPolicySatisfied) {
+      ContainerHealthResult.UnderReplicatedHealthResult result =
+          new ContainerHealthResult.UnderReplicatedHealthResult(
+          container, replicaCount.getRemainingRedundancy(),
+          isPolicySatisfied && replicas.size() >= requiredNodes,
+          replicaCount.isSufficientlyReplicated(true),
+          replicaCount.isUnrecoverable());
+      result.setMisReplicated(!isPolicySatisfied)
+          .setMisReplicatedAfterPending(
+              !placementStatusWithPending.isPolicySatisfied())
+          .setDueToMisReplication(
+              !isPolicySatisfied && replicaCount.isSufficientlyReplicated());
+      return result;
+    }
+
+    boolean isOverReplicated = replicaCount.isOverReplicated(false);
+    if (isOverReplicated) {
+      boolean repOkWithPending = !replicaCount.isOverReplicated(true);
+      return new ContainerHealthResult.OverReplicatedHealthResult(
+          container, replicaCount.getExcessRedundancy(false), repOkWithPending);
+    }
+    // No issues detected, just return healthy.
+    return new ContainerHealthResult.HealthyResult(container);
+  }
+
+  /**
+   * Given a set of ContainerReplica, transform it to a list of DatanodeDetails
+   * and then check if the list meets the container placement policy.
+   * @param replicas List of containerReplica
+   * @param replicationFactor Expected Replication Factor of the containe
+   * @return ContainerPlacementStatus indicating if the policy is met or not
+   */
+  private ContainerPlacementStatus getPlacementStatus(
+      Set<ContainerReplica> replicas, int replicationFactor,
+      List<ContainerReplicaOp> pendingOps) {
+
+    Set<DatanodeDetails> replicaDns = replicas.stream()
+        .map(ContainerReplica::getDatanodeDetails)
+        .collect(Collectors.toSet());
+    for (ContainerReplicaOp op : pendingOps) {
+      if (op.getOpType() == ContainerReplicaOp.PendingOpType.ADD) {
+        replicaDns.add(op.getTarget());
+      } else if (op.getOpType() == ContainerReplicaOp.PendingOpType.DELETE) {
+        replicaDns.remove(op.getTarget());
+      }

Review Comment:
   Can there be a case where there is pending ADD & pending DELETE to the same node? Some kind of race condition.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisContainerReplicaCount.java:
##########
@@ -257,7 +277,50 @@ public boolean isSufficientlyReplicated() {
    */
   @Override
   public boolean isOverReplicated() {
-    return missingReplicas() + inFlightDel < 0;
+    return isOverReplicated(true);
+  }
+
+  /**
+   * Return true if the container is over replicated. Decommission and
+   * maintenance containers are ignored for this check.
+   * The check ignores inflight additions, as they may fail, but it does
+   * consider inflight deletes if includePendingDelete is true.
+   *
+   * @return True if the container is over replicated, false otherwise.
+   */
+  public boolean isOverReplicated(boolean includePendingDelete) {
+    int delta = missingReplicas();
+    if (includePendingDelete) {
+      delta += inFlightDel;
+    }
+    return delta < 0;
+  }
+
+  /**
+   * @return Return Excess Redundancy replica nums.
+   */
+  public int getExcessRedundancy(boolean includePendingDelete) {

Review Comment:
   We could add a boolean with includePendingAdd as well. I see redundant duplicate code logic in sufficientlyReplicated & isOverReplicated.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisContainerReplicaCount.java:
##########
@@ -257,7 +277,50 @@ public boolean isSufficientlyReplicated() {
    */
   @Override
   public boolean isOverReplicated() {
-    return missingReplicas() + inFlightDel < 0;
+    return isOverReplicated(true);
+  }
+
+  /**
+   * Return true if the container is over replicated. Decommission and
+   * maintenance containers are ignored for this check.
+   * The check ignores inflight additions, as they may fail, but it does
+   * consider inflight deletes if includePendingDelete is true.
+   *
+   * @return True if the container is over replicated, false otherwise.
+   */
+  public boolean isOverReplicated(boolean includePendingDelete) {

Review Comment:
   Could be changed to getExcessRedundancyCanBeCalled(includePending)>0 to avoid redundancy of logic.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] sodonnel commented on a diff in pull request #3802: HDDS-7058. EC: ReplicationManager - Implement ratis container replication check handler

Posted by GitBox <gi...@apache.org>.
sodonnel commented on code in PR #3802:
URL: https://github.com/apache/ozone/pull/3802#discussion_r992104414


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/RatisReplicationCheckHandler.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.health;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.RatisContainerReplicaCount;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
+
+/**
+ * Class to determine the health state of a Ratis Container. Given the container
+ * and current replica details, along with replicas pending add and delete,
+ * this class will return a ContainerHealthResult indicating if the container
+ * is healthy, or under / over replicated etc.
+ */
+public class RatisReplicationCheckHandler extends AbstractCheck {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(RatisReplicationCheckHandler.class);
+
+  /**
+   * PlacementPolicy which is used to identify where a container
+   * should be replicated.
+   */
+  private final PlacementPolicy ratisContainerPlacement;
+
+  public RatisReplicationCheckHandler(PlacementPolicy containerPlacement) {
+    this.ratisContainerPlacement = containerPlacement;
+  }
+
+  @Override
+  public boolean handle(ContainerCheckRequest request) {
+    if (request.getContainerInfo().getReplicationType() != RATIS) {
+      // This handler is only for Ratis containers.
+      return false;
+    }
+    ReplicationManagerReport report = request.getReport();
+    ContainerInfo container = request.getContainerInfo();
+    ContainerHealthResult health = checkHealth(request);
+    if (health.getHealthState() == ContainerHealthResult.HealthState.HEALTHY) {
+      // If the container is healthy, there is nothing else to do in this
+      // handler so return as unhandled so any further handlers will be tried.
+      return false;
+    }
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.UNDER_REPLICATED) {
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.UNDER_REPLICATED,
+          container.containerID());
+      ContainerHealthResult.UnderReplicatedHealthResult underHealth
+          = ((ContainerHealthResult.UnderReplicatedHealthResult) health);
+      if (underHealth.isUnrecoverable()) {
+        report.incrementAndSample(ReplicationManagerReport.HealthState.MISSING,
+            container.containerID());
+      }
+      if (underHealth.isMisReplicated()) {
+        report.incrementAndSample(
+            ReplicationManagerReport.HealthState.MIS_REPLICATED,
+            container.containerID());
+      }
+      // TODO - if it is unrecoverable, should we return false to other
+      //        handlers can be tried?
+      if (!underHealth.isUnrecoverable() &&
+          (underHealth.isMisReplicatedAfterPending() ||
+              !underHealth.isSufficientlyReplicatedAfterPending())) {
+        request.getReplicationQueue().enqueue(underHealth);
+      }
+      return true;
+    }
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.OVER_REPLICATED) {
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.OVER_REPLICATED,
+          container.containerID());
+      ContainerHealthResult.OverReplicatedHealthResult overHealth
+          = ((ContainerHealthResult.OverReplicatedHealthResult) health);
+      if (!overHealth.isSufficientlyReplicatedAfterPending()) {
+        request.getReplicationQueue().enqueue(overHealth);
+      }
+      return true;
+    }
+    return false;
+  }
+
+  public ContainerHealthResult checkHealth(ContainerCheckRequest request) {

Review Comment:
   It will need to be public when we implement the under / over replication handler to process the under / over replicated queue. This is the same as in the ECHandler, where it has this same method public for that reason.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] sodonnel commented on a diff in pull request #3802: HDDS-7058. EC: ReplicationManager - Implement ratis container replication check handler

Posted by GitBox <gi...@apache.org>.
sodonnel commented on code in PR #3802:
URL: https://github.com/apache/ozone/pull/3802#discussion_r992106926


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/RatisReplicationCheckHandler.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.health;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.RatisContainerReplicaCount;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
+
+/**
+ * Class to determine the health state of a Ratis Container. Given the container
+ * and current replica details, along with replicas pending add and delete,
+ * this class will return a ContainerHealthResult indicating if the container
+ * is healthy, or under / over replicated etc.
+ */
+public class RatisReplicationCheckHandler extends AbstractCheck {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(RatisReplicationCheckHandler.class);
+
+  /**
+   * PlacementPolicy which is used to identify where a container
+   * should be replicated.
+   */
+  private final PlacementPolicy ratisContainerPlacement;
+
+  public RatisReplicationCheckHandler(PlacementPolicy containerPlacement) {
+    this.ratisContainerPlacement = containerPlacement;
+  }
+
+  @Override
+  public boolean handle(ContainerCheckRequest request) {
+    if (request.getContainerInfo().getReplicationType() != RATIS) {
+      // This handler is only for Ratis containers.
+      return false;
+    }
+    ReplicationManagerReport report = request.getReport();
+    ContainerInfo container = request.getContainerInfo();
+    ContainerHealthResult health = checkHealth(request);
+    if (health.getHealthState() == ContainerHealthResult.HealthState.HEALTHY) {
+      // If the container is healthy, there is nothing else to do in this
+      // handler so return as unhandled so any further handlers will be tried.
+      return false;
+    }
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.UNDER_REPLICATED) {
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.UNDER_REPLICATED,
+          container.containerID());
+      ContainerHealthResult.UnderReplicatedHealthResult underHealth
+          = ((ContainerHealthResult.UnderReplicatedHealthResult) health);
+      if (underHealth.isUnrecoverable()) {
+        report.incrementAndSample(ReplicationManagerReport.HealthState.MISSING,
+            container.containerID());
+      }
+      if (underHealth.isMisReplicated()) {
+        report.incrementAndSample(
+            ReplicationManagerReport.HealthState.MIS_REPLICATED,
+            container.containerID());
+      }
+      // TODO - if it is unrecoverable, should we return false to other
+      //        handlers can be tried?
+      if (!underHealth.isUnrecoverable() &&
+          (underHealth.isMisReplicatedAfterPending() ||
+              !underHealth.isSufficientlyReplicatedAfterPending())) {
+        request.getReplicationQueue().enqueue(underHealth);
+      }
+      return true;
+    }

Review Comment:
   ok - I added that in.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kerneltime commented on pull request #3802: HDDS-7058. EC: ReplicationManager - Implement ratis container replication check handler

Posted by GitBox <gi...@apache.org>.
kerneltime commented on PR #3802:
URL: https://github.com/apache/ozone/pull/3802#issuecomment-1273532266

   @aswinshakil 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] sodonnel commented on a diff in pull request #3802: HDDS-7058. EC: ReplicationManager - Implement ratis container replication check handler

Posted by GitBox <gi...@apache.org>.
sodonnel commented on code in PR #3802:
URL: https://github.com/apache/ozone/pull/3802#discussion_r998377553


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestRatisReplicationCheckHandler.java:
##########
@@ -0,0 +1,560 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.health;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementStatusDefault;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult.HealthState;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult.OverReplicatedHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult.UnderReplicatedHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationQueue;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONING;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_MAINTENANCE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_SERVICE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
+import static org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp.PendingOpType.ADD;
+import static org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp.PendingOpType.DELETE;
+import static org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil.createContainerInfo;
+import static org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil.createReplicas;
+
+/**
+ * Tests for the RatisReplicationCheckHandler class.
+ */
+public class TestRatisReplicationCheckHandler {
+
+  private RatisReplicationCheckHandler healthCheck;
+  private ReplicationConfig repConfig;
+  private PlacementPolicy containerPlacementPolicy;
+  private ReplicationQueue repQueue;
+  private ContainerCheckRequest.Builder requestBuilder;
+  private ReplicationManagerReport report;
+  private int maintenanceRedundancy = 2;
+
+  @Before
+  public void setup() throws IOException {
+    containerPlacementPolicy = Mockito.mock(PlacementPolicy.class);
+    Mockito.when(containerPlacementPolicy.validateContainerPlacement(
+        Mockito.any(),
+        Mockito.anyInt()
+    )).thenAnswer(invocation ->
+        new ContainerPlacementStatusDefault(2, 2, 3));
+    healthCheck = new RatisReplicationCheckHandler(containerPlacementPolicy);
+    repConfig = RatisReplicationConfig.getInstance(THREE);
+    repQueue = new ReplicationQueue();
+    report = new ReplicationManagerReport();
+    requestBuilder = new ContainerCheckRequest.Builder()
+        .setReplicationQueue(repQueue)
+        .setMaintenanceRedundancy(maintenanceRedundancy)
+        .setPendingOps(Collections.emptyList())
+        .setReport(report);
+  }
+
+  @Test
+  public void testReturnFalseForNonRatis() {
+    ContainerInfo container =
+        createContainerInfo(new ECReplicationConfig(3, 2));
+    Set<ContainerReplica> replicas =
+        createReplicas(container.containerID(), 1, 2, 3, 4);
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    Assert.assertFalse(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+  }
+
+  @Test
+  public void testHealthyContainerIsHealthy() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas
+        = createReplicas(container.containerID(), 0, 0, 0);
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    ContainerHealthResult result =
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.HEALTHY, result.getHealthState());
+
+    Assert.assertFalse(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+  }
+
+  @Test
+  public void testUnderReplicatedContainerIsUnderReplicated() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas
+        = createReplicas(container.containerID(), 0, 0);
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getRemainingRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertFalse(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(1, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedContainerDueToPendingDelete() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas
+        = createReplicas(container.containerID(), 0, 0, 0);
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        DELETE, MockDatanodeDetails.randomDatanodeDetails(), 0));
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container)
+        .setPendingOps(pending);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getRemainingRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertFalse(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(1, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedContainerFixedWithPending() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas
+        = createReplicas(container.containerID(), 0, 0);
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        ADD, MockDatanodeDetails.randomDatanodeDetails(), 0));
+    requestBuilder.setContainerReplicas(replicas)
+        .setPendingOps(pending)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getRemainingRedundancy());
+    Assert.assertTrue(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertFalse(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    // Fixed with pending, so nothing added to the queue
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    // Still under replicated until the pending complete
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedDueToDecommission() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas = createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(DECOMMISSIONING, 0),
+        Pair.of(DECOMMISSIONED, 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(2, result.getRemainingRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertTrue(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(1, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedDueToDecommissionFixedWithPending() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas = createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(DECOMMISSIONED, 0));
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        ADD, MockDatanodeDetails.randomDatanodeDetails(), 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setPendingOps(pending)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(2, result.getRemainingRedundancy());
+    Assert.assertTrue(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertTrue(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    // Nothing queued as inflight replicas will fix it.
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    // Still under replicated in the report until pending complete
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedDueToDecommissionAndMissing() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas = createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(DECOMMISSIONED, 0));
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        ADD, MockDatanodeDetails.randomDatanodeDetails(), 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setPendingOps(pending)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getRemainingRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertFalse(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(1, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedAndUnrecoverable() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas = Collections.EMPTY_SET;
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(0, result.getRemainingRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertFalse(result.underReplicatedDueToDecommission());
+    Assert.assertTrue(result.isUnrecoverable());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    // Unrecoverable, so not added to the queue.
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.MISSING));
+  }
+
+  @Test
+  public void testOverReplicatedContainer() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas =  createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0));
+
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        DELETE, MockDatanodeDetails.randomDatanodeDetails(), 0));
+    pending.add(ContainerReplicaOp.create(
+        DELETE, MockDatanodeDetails.randomDatanodeDetails(), 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setPendingOps(pending)
+        .setContainerInfo(container);
+    OverReplicatedHealthResult result = (OverReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.OVER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(4, result.getExcessRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(1, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.OVER_REPLICATED));
+  }
+
+  @Test
+  public void testOverReplicatedContainerFixedByPending() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas =  createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0));
+
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        DELETE, MockDatanodeDetails.randomDatanodeDetails(), 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setPendingOps(pending)
+        .setContainerInfo(container);
+    OverReplicatedHealthResult result = (OverReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.OVER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getExcessRedundancy());
+    Assert.assertTrue(result.isSufficientlyReplicatedAfterPending());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    // Fixed by pending so nothing queued.
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    // Still over replicated, so the report should contain it
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.OVER_REPLICATED));
+  }
+
+  @Test
+  public void testOverReplicatedContainerWithMaintenance() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas =  createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_MAINTENANCE, 0), Pair.of(DECOMMISSIONED, 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    OverReplicatedHealthResult result = (OverReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.OVER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getExcessRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(1, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.OVER_REPLICATED));
+  }
+
+  @Test
+  public void testOverReplicatedContainerDueToMaintenance() {

Review Comment:
   OK - I added `IsHealthy` to the end of it.



##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestRatisReplicationCheckHandler.java:
##########
@@ -0,0 +1,560 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.health;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementStatusDefault;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult.HealthState;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult.OverReplicatedHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult.UnderReplicatedHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationQueue;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONING;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_MAINTENANCE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_SERVICE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
+import static org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp.PendingOpType.ADD;
+import static org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp.PendingOpType.DELETE;
+import static org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil.createContainerInfo;
+import static org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil.createReplicas;
+
+/**
+ * Tests for the RatisReplicationCheckHandler class.
+ */
+public class TestRatisReplicationCheckHandler {
+
+  private RatisReplicationCheckHandler healthCheck;
+  private ReplicationConfig repConfig;
+  private PlacementPolicy containerPlacementPolicy;
+  private ReplicationQueue repQueue;
+  private ContainerCheckRequest.Builder requestBuilder;
+  private ReplicationManagerReport report;
+  private int maintenanceRedundancy = 2;
+
+  @Before
+  public void setup() throws IOException {
+    containerPlacementPolicy = Mockito.mock(PlacementPolicy.class);
+    Mockito.when(containerPlacementPolicy.validateContainerPlacement(
+        Mockito.any(),
+        Mockito.anyInt()
+    )).thenAnswer(invocation ->
+        new ContainerPlacementStatusDefault(2, 2, 3));
+    healthCheck = new RatisReplicationCheckHandler(containerPlacementPolicy);
+    repConfig = RatisReplicationConfig.getInstance(THREE);
+    repQueue = new ReplicationQueue();
+    report = new ReplicationManagerReport();
+    requestBuilder = new ContainerCheckRequest.Builder()
+        .setReplicationQueue(repQueue)
+        .setMaintenanceRedundancy(maintenanceRedundancy)
+        .setPendingOps(Collections.emptyList())
+        .setReport(report);
+  }
+
+  @Test
+  public void testReturnFalseForNonRatis() {
+    ContainerInfo container =
+        createContainerInfo(new ECReplicationConfig(3, 2));
+    Set<ContainerReplica> replicas =
+        createReplicas(container.containerID(), 1, 2, 3, 4);
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    Assert.assertFalse(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+  }
+
+  @Test
+  public void testHealthyContainerIsHealthy() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas
+        = createReplicas(container.containerID(), 0, 0, 0);
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    ContainerHealthResult result =
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.HEALTHY, result.getHealthState());
+
+    Assert.assertFalse(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+  }
+
+  @Test
+  public void testUnderReplicatedContainerIsUnderReplicated() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas
+        = createReplicas(container.containerID(), 0, 0);
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getRemainingRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertFalse(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(1, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedContainerDueToPendingDelete() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas
+        = createReplicas(container.containerID(), 0, 0, 0);
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        DELETE, MockDatanodeDetails.randomDatanodeDetails(), 0));
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container)
+        .setPendingOps(pending);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getRemainingRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertFalse(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(1, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedContainerFixedWithPending() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas
+        = createReplicas(container.containerID(), 0, 0);
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        ADD, MockDatanodeDetails.randomDatanodeDetails(), 0));
+    requestBuilder.setContainerReplicas(replicas)
+        .setPendingOps(pending)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getRemainingRedundancy());
+    Assert.assertTrue(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertFalse(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    // Fixed with pending, so nothing added to the queue
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    // Still under replicated until the pending complete
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedDueToDecommission() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas = createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(DECOMMISSIONING, 0),
+        Pair.of(DECOMMISSIONED, 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(2, result.getRemainingRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertTrue(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(1, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedDueToDecommissionFixedWithPending() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas = createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(DECOMMISSIONED, 0));
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        ADD, MockDatanodeDetails.randomDatanodeDetails(), 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setPendingOps(pending)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(2, result.getRemainingRedundancy());
+    Assert.assertTrue(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertTrue(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    // Nothing queued as inflight replicas will fix it.
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    // Still under replicated in the report until pending complete
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedDueToDecommissionAndMissing() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas = createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(DECOMMISSIONED, 0));
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        ADD, MockDatanodeDetails.randomDatanodeDetails(), 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setPendingOps(pending)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getRemainingRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertFalse(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(1, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedAndUnrecoverable() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas = Collections.EMPTY_SET;
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(0, result.getRemainingRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertFalse(result.underReplicatedDueToDecommission());
+    Assert.assertTrue(result.isUnrecoverable());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    // Unrecoverable, so not added to the queue.
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.MISSING));
+  }
+
+  @Test
+  public void testOverReplicatedContainer() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas =  createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0));
+
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        DELETE, MockDatanodeDetails.randomDatanodeDetails(), 0));
+    pending.add(ContainerReplicaOp.create(
+        DELETE, MockDatanodeDetails.randomDatanodeDetails(), 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setPendingOps(pending)
+        .setContainerInfo(container);
+    OverReplicatedHealthResult result = (OverReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.OVER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(4, result.getExcessRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(1, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.OVER_REPLICATED));
+  }
+
+  @Test
+  public void testOverReplicatedContainerFixedByPending() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas =  createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0));
+
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        DELETE, MockDatanodeDetails.randomDatanodeDetails(), 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setPendingOps(pending)
+        .setContainerInfo(container);
+    OverReplicatedHealthResult result = (OverReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.OVER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getExcessRedundancy());
+    Assert.assertTrue(result.isSufficientlyReplicatedAfterPending());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    // Fixed by pending so nothing queued.
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    // Still over replicated, so the report should contain it
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.OVER_REPLICATED));
+  }
+
+  @Test
+  public void testOverReplicatedContainerWithMaintenance() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas =  createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_MAINTENANCE, 0), Pair.of(DECOMMISSIONED, 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    OverReplicatedHealthResult result = (OverReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.OVER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getExcessRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(1, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.OVER_REPLICATED));
+  }
+
+  @Test
+  public void testOverReplicatedContainerDueToMaintenance() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas = createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_MAINTENANCE, 0),
+        Pair.of(IN_MAINTENANCE, 2));

Review Comment:
   yes, well spotted.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #3802: HDDS-7058. EC: ReplicationManager - Implement ratis container replication check handler

Posted by GitBox <gi...@apache.org>.
swamirishi commented on code in PR #3802:
URL: https://github.com/apache/ozone/pull/3802#discussion_r991908073


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/RatisReplicationCheckHandler.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.health;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.RatisContainerReplicaCount;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
+
+/**
+ * Class to determine the health state of a Ratis Container. Given the container
+ * and current replica details, along with replicas pending add and delete,
+ * this class will return a ContainerHealthResult indicating if the container
+ * is healthy, or under / over replicated etc.
+ */
+public class RatisReplicationCheckHandler extends AbstractCheck {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(RatisReplicationCheckHandler.class);
+
+  /**
+   * PlacementPolicy which is used to identify where a container
+   * should be replicated.
+   */
+  private final PlacementPolicy ratisContainerPlacement;
+
+  public RatisReplicationCheckHandler(PlacementPolicy containerPlacement) {
+    this.ratisContainerPlacement = containerPlacement;
+  }
+
+  @Override
+  public boolean handle(ContainerCheckRequest request) {
+    if (request.getContainerInfo().getReplicationType() != RATIS) {
+      // This handler is only for Ratis containers.
+      return false;
+    }
+    ReplicationManagerReport report = request.getReport();
+    ContainerInfo container = request.getContainerInfo();
+    ContainerHealthResult health = checkHealth(request);
+    if (health.getHealthState() == ContainerHealthResult.HealthState.HEALTHY) {
+      // If the container is healthy, there is nothing else to do in this
+      // handler so return as unhandled so any further handlers will be tried.
+      return false;
+    }
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.UNDER_REPLICATED) {
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.UNDER_REPLICATED,
+          container.containerID());
+      ContainerHealthResult.UnderReplicatedHealthResult underHealth
+          = ((ContainerHealthResult.UnderReplicatedHealthResult) health);
+      if (underHealth.isUnrecoverable()) {
+        report.incrementAndSample(ReplicationManagerReport.HealthState.MISSING,
+            container.containerID());
+      }
+      if (underHealth.isMisReplicated()) {
+        report.incrementAndSample(
+            ReplicationManagerReport.HealthState.MIS_REPLICATED,
+            container.containerID());
+      }
+      // TODO - if it is unrecoverable, should we return false to other
+      //        handlers can be tried?
+      if (!underHealth.isUnrecoverable() &&
+          (underHealth.isMisReplicatedAfterPending() ||
+              !underHealth.isSufficientlyReplicatedAfterPending())) {
+        request.getReplicationQueue().enqueue(underHealth);
+      }
+      return true;
+    }
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.OVER_REPLICATED) {
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.OVER_REPLICATED,
+          container.containerID());
+      ContainerHealthResult.OverReplicatedHealthResult overHealth
+          = ((ContainerHealthResult.OverReplicatedHealthResult) health);
+      if (!overHealth.isSufficientlyReplicatedAfterPending()) {
+        request.getReplicationQueue().enqueue(overHealth);
+      }
+      return true;
+    }
+    return false;
+  }
+
+  public ContainerHealthResult checkHealth(ContainerCheckRequest request) {

Review Comment:
   Probably could add @visibleForTesting if you want to add unit tests or could even change the access specifier using replication using reflection.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #3802: HDDS-7058. EC: ReplicationManager - Implement ratis container replication check handler

Posted by GitBox <gi...@apache.org>.
swamirishi commented on code in PR #3802:
URL: https://github.com/apache/ozone/pull/3802#discussion_r992002733


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java:
##########
@@ -210,7 +218,8 @@ public ReplicationManager(final ConfigurationSource conf,
         .addNext(new ClosingContainerHandler(this))
         .addNext(new QuasiClosedContainerHandler(this))
         .addNext(new ClosedWithMismatchedReplicasHandler(this))
-        .addNext(ecReplicationCheckHandler);
+        .addNext(ecReplicationCheckHandler)
+        .addNext(ratisReplicationCheckHandler);

Review Comment:
   My understanding of the thing was wrong. Please ignore.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #3802: HDDS-7058. EC: ReplicationManager - Implement ratis container replication check handler

Posted by GitBox <gi...@apache.org>.
swamirishi commented on code in PR #3802:
URL: https://github.com/apache/ozone/pull/3802#discussion_r991898560


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/RatisReplicationCheckHandler.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.health;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.RatisContainerReplicaCount;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
+
+/**
+ * Class to determine the health state of a Ratis Container. Given the container
+ * and current replica details, along with replicas pending add and delete,
+ * this class will return a ContainerHealthResult indicating if the container
+ * is healthy, or under / over replicated etc.
+ */
+public class RatisReplicationCheckHandler extends AbstractCheck {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(RatisReplicationCheckHandler.class);
+
+  /**
+   * PlacementPolicy which is used to identify where a container
+   * should be replicated.
+   */
+  private final PlacementPolicy ratisContainerPlacement;
+
+  public RatisReplicationCheckHandler(PlacementPolicy containerPlacement) {
+    this.ratisContainerPlacement = containerPlacement;
+  }
+
+  @Override
+  public boolean handle(ContainerCheckRequest request) {
+    if (request.getContainerInfo().getReplicationType() != RATIS) {
+      // This handler is only for Ratis containers.
+      return false;
+    }
+    ReplicationManagerReport report = request.getReport();
+    ContainerInfo container = request.getContainerInfo();
+    ContainerHealthResult health = checkHealth(request);
+    if (health.getHealthState() == ContainerHealthResult.HealthState.HEALTHY) {
+      // If the container is healthy, there is nothing else to do in this
+      // handler so return as unhandled so any further handlers will be tried.
+      return false;
+    }
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.UNDER_REPLICATED) {
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.UNDER_REPLICATED,
+          container.containerID());
+      ContainerHealthResult.UnderReplicatedHealthResult underHealth
+          = ((ContainerHealthResult.UnderReplicatedHealthResult) health);
+      if (underHealth.isUnrecoverable()) {
+        report.incrementAndSample(ReplicationManagerReport.HealthState.MISSING,
+            container.containerID());
+      }
+      if (underHealth.isMisReplicated()) {
+        report.incrementAndSample(
+            ReplicationManagerReport.HealthState.MIS_REPLICATED,
+            container.containerID());
+      }
+      // TODO - if it is unrecoverable, should we return false to other
+      //        handlers can be tried?
+      if (!underHealth.isUnrecoverable() &&
+          (underHealth.isMisReplicatedAfterPending() ||
+              !underHealth.isSufficientlyReplicatedAfterPending())) {
+        request.getReplicationQueue().enqueue(underHealth);
+      }
+      return true;
+    }
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.OVER_REPLICATED) {
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.OVER_REPLICATED,
+          container.containerID());
+      ContainerHealthResult.OverReplicatedHealthResult overHealth
+          = ((ContainerHealthResult.OverReplicatedHealthResult) health);
+      if (!overHealth.isSufficientlyReplicatedAfterPending()) {
+        request.getReplicationQueue().enqueue(overHealth);
+      }
+      return true;
+    }
+    return false;
+  }
+
+  public ContainerHealthResult checkHealth(ContainerCheckRequest request) {

Review Comment:
   Do we need to make this method as public?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] sodonnel commented on a diff in pull request #3802: HDDS-7058. EC: ReplicationManager - Implement ratis container replication check handler

Posted by GitBox <gi...@apache.org>.
sodonnel commented on code in PR #3802:
URL: https://github.com/apache/ozone/pull/3802#discussion_r992112125


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/RatisReplicationCheckHandler.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.health;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.RatisContainerReplicaCount;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
+
+/**
+ * Class to determine the health state of a Ratis Container. Given the container
+ * and current replica details, along with replicas pending add and delete,
+ * this class will return a ContainerHealthResult indicating if the container
+ * is healthy, or under / over replicated etc.
+ */
+public class RatisReplicationCheckHandler extends AbstractCheck {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(RatisReplicationCheckHandler.class);
+
+  /**
+   * PlacementPolicy which is used to identify where a container
+   * should be replicated.
+   */
+  private final PlacementPolicy ratisContainerPlacement;
+
+  public RatisReplicationCheckHandler(PlacementPolicy containerPlacement) {
+    this.ratisContainerPlacement = containerPlacement;
+  }
+
+  @Override
+  public boolean handle(ContainerCheckRequest request) {
+    if (request.getContainerInfo().getReplicationType() != RATIS) {
+      // This handler is only for Ratis containers.
+      return false;
+    }
+    ReplicationManagerReport report = request.getReport();
+    ContainerInfo container = request.getContainerInfo();
+    ContainerHealthResult health = checkHealth(request);
+    if (health.getHealthState() == ContainerHealthResult.HealthState.HEALTHY) {
+      // If the container is healthy, there is nothing else to do in this
+      // handler so return as unhandled so any further handlers will be tried.
+      return false;
+    }
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.UNDER_REPLICATED) {
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.UNDER_REPLICATED,
+          container.containerID());
+      ContainerHealthResult.UnderReplicatedHealthResult underHealth
+          = ((ContainerHealthResult.UnderReplicatedHealthResult) health);
+      if (underHealth.isUnrecoverable()) {
+        report.incrementAndSample(ReplicationManagerReport.HealthState.MISSING,
+            container.containerID());
+      }
+      if (underHealth.isMisReplicated()) {
+        report.incrementAndSample(
+            ReplicationManagerReport.HealthState.MIS_REPLICATED,
+            container.containerID());
+      }
+      // TODO - if it is unrecoverable, should we return false to other
+      //        handlers can be tried?
+      if (!underHealth.isUnrecoverable() &&
+          (underHealth.isMisReplicatedAfterPending() ||
+              !underHealth.isSufficientlyReplicatedAfterPending())) {
+        request.getReplicationQueue().enqueue(underHealth);
+      }
+      return true;
+    }
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.OVER_REPLICATED) {
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.OVER_REPLICATED,
+          container.containerID());
+      ContainerHealthResult.OverReplicatedHealthResult overHealth
+          = ((ContainerHealthResult.OverReplicatedHealthResult) health);
+      if (!overHealth.isSufficientlyReplicatedAfterPending()) {
+        request.getReplicationQueue().enqueue(overHealth);
+      }
+      return true;
+    }
+    return false;
+  }
+
+  public ContainerHealthResult checkHealth(ContainerCheckRequest request) {
+    ContainerInfo container = request.getContainerInfo();
+    Set<ContainerReplica> replicas = request.getContainerReplicas();
+    List<ContainerReplicaOp> replicaPendingOps = request.getPendingOps();
+    // Note that this setting is minReplicasForMaintenance. For EC the variable
+    // is defined as remainingRedundancy which is subtly different.
+    int minReplicasForMaintenance = request.getMaintenanceRedundancy();
+    int pendingAdd = 0;
+    int pendingDelete = 0;
+    for (ContainerReplicaOp op : replicaPendingOps) {
+      if (op.getOpType() == ContainerReplicaOp.PendingOpType.ADD) {
+        pendingAdd++;
+      } else if (op.getOpType() == ContainerReplicaOp.PendingOpType.DELETE) {
+        pendingDelete++;
+      }
+    }
+    int requiredNodes = container.getReplicationConfig().getRequiredNodes();
+
+    // RatisContainerReplicaCount uses the minReplicasForMaintenance rather
+    // than remainingRedundancy which ECContainerReplicaCount uses.
+    RatisContainerReplicaCount replicaCount =
+        new RatisContainerReplicaCount(container, replicas, pendingAdd,
+            pendingDelete, requiredNodes, minReplicasForMaintenance);
+
+    ContainerPlacementStatus placementStatus =
+        getPlacementStatus(replicas, requiredNodes, Collections.EMPTY_LIST);
+
+    ContainerPlacementStatus placementStatusWithPending = placementStatus;
+    if (replicaPendingOps.size() > 0) {
+      placementStatusWithPending =
+          getPlacementStatus(replicas, requiredNodes, replicaPendingOps);
+    }
+    boolean sufficientlyReplicated
+        = replicaCount.isSufficientlyReplicated(false);
+    boolean isPolicySatisfied = placementStatus.isPolicySatisfied();
+    if (!sufficientlyReplicated || !isPolicySatisfied) {
+      ContainerHealthResult.UnderReplicatedHealthResult result =
+          new ContainerHealthResult.UnderReplicatedHealthResult(
+          container, replicaCount.getRemainingRedundancy(),
+          isPolicySatisfied && replicas.size() >= requiredNodes,
+          replicaCount.isSufficientlyReplicated(true),
+          replicaCount.isUnrecoverable());
+      result.setMisReplicated(!isPolicySatisfied)
+          .setMisReplicatedAfterPending(
+              !placementStatusWithPending.isPolicySatisfied())
+          .setDueToMisReplication(
+              !isPolicySatisfied && replicaCount.isSufficientlyReplicated());
+      return result;
+    }
+
+    boolean isOverReplicated = replicaCount.isOverReplicated(false);
+    if (isOverReplicated) {
+      boolean repOkWithPending = !replicaCount.isOverReplicated(true);
+      return new ContainerHealthResult.OverReplicatedHealthResult(
+          container, replicaCount.getExcessRedundancy(false), repOkWithPending);
+    }
+    // No issues detected, just return healthy.
+    return new ContainerHealthResult.HealthyResult(container);
+  }
+
+  /**
+   * Given a set of ContainerReplica, transform it to a list of DatanodeDetails
+   * and then check if the list meets the container placement policy.
+   * @param replicas List of containerReplica
+   * @param replicationFactor Expected Replication Factor of the containe
+   * @return ContainerPlacementStatus indicating if the policy is met or not
+   */
+  private ContainerPlacementStatus getPlacementStatus(
+      Set<ContainerReplica> replicas, int replicationFactor,
+      List<ContainerReplicaOp> pendingOps) {
+
+    Set<DatanodeDetails> replicaDns = replicas.stream()
+        .map(ContainerReplica::getDatanodeDetails)
+        .collect(Collectors.toSet());
+    for (ContainerReplicaOp op : pendingOps) {
+      if (op.getOpType() == ContainerReplicaOp.PendingOpType.ADD) {
+        replicaDns.add(op.getTarget());
+      } else if (op.getOpType() == ContainerReplicaOp.PendingOpType.DELETE) {
+        replicaDns.remove(op.getTarget());
+      }

Review Comment:
   This should not be able to happen, as if a node has a replica it cannot get another copy of it. For a delete to be scheduled it must have a copy which will prevent an add etc.
   
   However I will change this to two IF statements rather than if .. else if



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] sodonnel commented on a diff in pull request #3802: HDDS-7058. EC: ReplicationManager - Implement ratis container replication check handler

Posted by GitBox <gi...@apache.org>.
sodonnel commented on code in PR #3802:
URL: https://github.com/apache/ozone/pull/3802#discussion_r992137787


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisContainerReplicaCount.java:
##########
@@ -247,7 +248,26 @@ public boolean isSufficientlyReplicated() {
   }
 
   /**
-   * Return true is the container is over replicated. Decommission and
+   * Return true if the container is sufficiently replicated. Decommissioning
+   * and Decommissioned containers are ignored in this check, assuming they will
+   * eventually be removed from the cluster.
+   * This check ignores inflight additions, if includePendingAdd is false,
+   * otherwise it will assume they complete ok.
+   *
+   * @return True if the container is sufficiently replicated and False
+   *         otherwise.
+   */
+  public boolean isSufficientlyReplicated(boolean includePendingAdd) {
+    // Positive for under-rep, negative for over-rep
+    int delta = missingReplicas();
+    if (includePendingAdd) {
+      delta -= inFlightAdd;
+    }
+    return delta <= 0;
+  }

Review Comment:
   I think you are correct - I have missed this. We should be removing the inflight deletes as per the original method defined just above this one. I will fix this and modidy a test to validate it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] sodonnel commented on a diff in pull request #3802: HDDS-7058. EC: ReplicationManager - Implement ratis container replication check handler

Posted by GitBox <gi...@apache.org>.
sodonnel commented on code in PR #3802:
URL: https://github.com/apache/ozone/pull/3802#discussion_r992152269


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisContainerReplicaCount.java:
##########
@@ -257,7 +277,50 @@ public boolean isSufficientlyReplicated() {
    */
   @Override
   public boolean isOverReplicated() {
-    return missingReplicas() + inFlightDel < 0;
+    return isOverReplicated(true);
+  }
+
+  /**
+   * Return true if the container is over replicated. Decommission and
+   * maintenance containers are ignored for this check.
+   * The check ignores inflight additions, as they may fail, but it does
+   * consider inflight deletes if includePendingDelete is true.
+   *
+   * @return True if the container is over replicated, false otherwise.
+   */
+  public boolean isOverReplicated(boolean includePendingDelete) {
+    int delta = missingReplicas();
+    if (includePendingDelete) {
+      delta += inFlightDel;
+    }
+    return delta < 0;
+  }
+
+  /**
+   * @return Return Excess Redundancy replica nums.
+   */
+  public int getExcessRedundancy(boolean includePendingDelete) {

Review Comment:
   Yea the logic is very similar. I have added a new private method both can call.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #3802: HDDS-7058. EC: ReplicationManager - Implement ratis container replication check handler

Posted by GitBox <gi...@apache.org>.
swamirishi commented on code in PR #3802:
URL: https://github.com/apache/ozone/pull/3802#discussion_r991910611


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/RatisReplicationCheckHandler.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.health;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.RatisContainerReplicaCount;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
+
+/**
+ * Class to determine the health state of a Ratis Container. Given the container
+ * and current replica details, along with replicas pending add and delete,
+ * this class will return a ContainerHealthResult indicating if the container
+ * is healthy, or under / over replicated etc.
+ */
+public class RatisReplicationCheckHandler extends AbstractCheck {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(RatisReplicationCheckHandler.class);
+
+  /**
+   * PlacementPolicy which is used to identify where a container
+   * should be replicated.
+   */
+  private final PlacementPolicy ratisContainerPlacement;
+
+  public RatisReplicationCheckHandler(PlacementPolicy containerPlacement) {
+    this.ratisContainerPlacement = containerPlacement;
+  }
+
+  @Override
+  public boolean handle(ContainerCheckRequest request) {
+    if (request.getContainerInfo().getReplicationType() != RATIS) {
+      // This handler is only for Ratis containers.
+      return false;
+    }
+    ReplicationManagerReport report = request.getReport();
+    ContainerInfo container = request.getContainerInfo();
+    ContainerHealthResult health = checkHealth(request);
+    if (health.getHealthState() == ContainerHealthResult.HealthState.HEALTHY) {
+      // If the container is healthy, there is nothing else to do in this
+      // handler so return as unhandled so any further handlers will be tried.
+      return false;
+    }
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.UNDER_REPLICATED) {
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.UNDER_REPLICATED,
+          container.containerID());
+      ContainerHealthResult.UnderReplicatedHealthResult underHealth
+          = ((ContainerHealthResult.UnderReplicatedHealthResult) health);
+      if (underHealth.isUnrecoverable()) {
+        report.incrementAndSample(ReplicationManagerReport.HealthState.MISSING,
+            container.containerID());
+      }
+      if (underHealth.isMisReplicated()) {
+        report.incrementAndSample(
+            ReplicationManagerReport.HealthState.MIS_REPLICATED,
+            container.containerID());
+      }
+      // TODO - if it is unrecoverable, should we return false to other
+      //        handlers can be tried?
+      if (!underHealth.isUnrecoverable() &&
+          (underHealth.isMisReplicatedAfterPending() ||
+              !underHealth.isSufficientlyReplicatedAfterPending())) {
+        request.getReplicationQueue().enqueue(underHealth);
+      }
+      return true;
+    }
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.OVER_REPLICATED) {
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.OVER_REPLICATED,
+          container.containerID());
+      ContainerHealthResult.OverReplicatedHealthResult overHealth
+          = ((ContainerHealthResult.OverReplicatedHealthResult) health);
+      if (!overHealth.isSufficientlyReplicatedAfterPending()) {
+        request.getReplicationQueue().enqueue(overHealth);
+      }
+      return true;
+    }
+    return false;
+  }
+
+  public ContainerHealthResult checkHealth(ContainerCheckRequest request) {
+    ContainerInfo container = request.getContainerInfo();
+    Set<ContainerReplica> replicas = request.getContainerReplicas();
+    List<ContainerReplicaOp> replicaPendingOps = request.getPendingOps();
+    // Note that this setting is minReplicasForMaintenance. For EC the variable
+    // is defined as remainingRedundancy which is subtly different.
+    int minReplicasForMaintenance = request.getMaintenanceRedundancy();
+    int pendingAdd = 0;
+    int pendingDelete = 0;
+    for (ContainerReplicaOp op : replicaPendingOps) {
+      if (op.getOpType() == ContainerReplicaOp.PendingOpType.ADD) {
+        pendingAdd++;
+      } else if (op.getOpType() == ContainerReplicaOp.PendingOpType.DELETE) {
+        pendingDelete++;
+      }
+    }
+    int requiredNodes = container.getReplicationConfig().getRequiredNodes();
+
+    // RatisContainerReplicaCount uses the minReplicasForMaintenance rather
+    // than remainingRedundancy which ECContainerReplicaCount uses.
+    RatisContainerReplicaCount replicaCount =
+        new RatisContainerReplicaCount(container, replicas, pendingAdd,
+            pendingDelete, requiredNodes, minReplicasForMaintenance);
+
+    ContainerPlacementStatus placementStatus =
+        getPlacementStatus(replicas, requiredNodes, Collections.EMPTY_LIST);
+
+    ContainerPlacementStatus placementStatusWithPending = placementStatus;
+    if (replicaPendingOps.size() > 0) {
+      placementStatusWithPending =
+          getPlacementStatus(replicas, requiredNodes, replicaPendingOps);
+    }
+    boolean sufficientlyReplicated
+        = replicaCount.isSufficientlyReplicated(false);
+    boolean isPolicySatisfied = placementStatus.isPolicySatisfied();
+    if (!sufficientlyReplicated || !isPolicySatisfied) {
+      ContainerHealthResult.UnderReplicatedHealthResult result =
+          new ContainerHealthResult.UnderReplicatedHealthResult(
+          container, replicaCount.getRemainingRedundancy(),
+          isPolicySatisfied && replicas.size() >= requiredNodes,
+          replicaCount.isSufficientlyReplicated(true),
+          replicaCount.isUnrecoverable());
+      result.setMisReplicated(!isPolicySatisfied)
+          .setMisReplicatedAfterPending(
+              !placementStatusWithPending.isPolicySatisfied())
+          .setDueToMisReplication(
+              !isPolicySatisfied && replicaCount.isSufficientlyReplicated());
+      return result;
+    }
+
+    boolean isOverReplicated = replicaCount.isOverReplicated(false);
+    if (isOverReplicated) {
+      boolean repOkWithPending = !replicaCount.isOverReplicated(true);
+      return new ContainerHealthResult.OverReplicatedHealthResult(
+          container, replicaCount.getExcessRedundancy(false), repOkWithPending);
+    }
+    // No issues detected, just return healthy.
+    return new ContainerHealthResult.HealthyResult(container);
+  }
+
+  /**
+   * Given a set of ContainerReplica, transform it to a list of DatanodeDetails
+   * and then check if the list meets the container placement policy.
+   * @param replicas List of containerReplica
+   * @param replicationFactor Expected Replication Factor of the containe
+   * @return ContainerPlacementStatus indicating if the policy is met or not
+   */
+  private ContainerPlacementStatus getPlacementStatus(
+      Set<ContainerReplica> replicas, int replicationFactor,
+      List<ContainerReplicaOp> pendingOps) {
+
+    Set<DatanodeDetails> replicaDns = replicas.stream()
+        .map(ContainerReplica::getDatanodeDetails)
+        .collect(Collectors.toSet());
+    for (ContainerReplicaOp op : pendingOps) {
+      if (op.getOpType() == ContainerReplicaOp.PendingOpType.ADD) {
+        replicaDns.add(op.getTarget());
+      } else if (op.getOpType() == ContainerReplicaOp.PendingOpType.DELETE) {
+        replicaDns.remove(op.getTarget());
+      }

Review Comment:
   Better to use a Map<DataNodeDetails,Integer> in that case.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] siddhantsangwan commented on a diff in pull request #3802: HDDS-7058. EC: ReplicationManager - Implement ratis container replication check handler

Posted by GitBox <gi...@apache.org>.
siddhantsangwan commented on code in PR #3802:
URL: https://github.com/apache/ozone/pull/3802#discussion_r991879550


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisContainerReplicaCount.java:
##########
@@ -247,7 +248,26 @@ public boolean isSufficientlyReplicated() {
   }
 
   /**
-   * Return true is the container is over replicated. Decommission and
+   * Return true if the container is sufficiently replicated. Decommissioning
+   * and Decommissioned containers are ignored in this check, assuming they will
+   * eventually be removed from the cluster.
+   * This check ignores inflight additions, if includePendingAdd is false,
+   * otherwise it will assume they complete ok.
+   *
+   * @return True if the container is sufficiently replicated and False
+   *         otherwise.
+   */
+  public boolean isSufficientlyReplicated(boolean includePendingAdd) {
+    // Positive for under-rep, negative for over-rep
+    int delta = missingReplicas();
+    if (includePendingAdd) {
+      delta -= inFlightAdd;
+    }
+    return delta <= 0;
+  }

Review Comment:
   Are we deliberately not considering pending deletes here?



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/RatisReplicationCheckHandler.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.health;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.RatisContainerReplicaCount;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
+
+/**
+ * Class to determine the health state of a Ratis Container. Given the container
+ * and current replica details, along with replicas pending add and delete,
+ * this class will return a ContainerHealthResult indicating if the container
+ * is healthy, or under / over replicated etc.
+ */
+public class RatisReplicationCheckHandler extends AbstractCheck {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(RatisReplicationCheckHandler.class);
+
+  /**
+   * PlacementPolicy which is used to identify where a container
+   * should be replicated.
+   */
+  private final PlacementPolicy ratisContainerPlacement;
+
+  public RatisReplicationCheckHandler(PlacementPolicy containerPlacement) {
+    this.ratisContainerPlacement = containerPlacement;
+  }
+
+  @Override
+  public boolean handle(ContainerCheckRequest request) {
+    if (request.getContainerInfo().getReplicationType() != RATIS) {
+      // This handler is only for Ratis containers.
+      return false;
+    }
+    ReplicationManagerReport report = request.getReport();
+    ContainerInfo container = request.getContainerInfo();
+    ContainerHealthResult health = checkHealth(request);
+    if (health.getHealthState() == ContainerHealthResult.HealthState.HEALTHY) {
+      // If the container is healthy, there is nothing else to do in this
+      // handler so return as unhandled so any further handlers will be tried.
+      return false;
+    }
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.UNDER_REPLICATED) {
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.UNDER_REPLICATED,
+          container.containerID());
+      ContainerHealthResult.UnderReplicatedHealthResult underHealth
+          = ((ContainerHealthResult.UnderReplicatedHealthResult) health);
+      if (underHealth.isUnrecoverable()) {
+        report.incrementAndSample(ReplicationManagerReport.HealthState.MISSING,
+            container.containerID());
+      }
+      if (underHealth.isMisReplicated()) {
+        report.incrementAndSample(
+            ReplicationManagerReport.HealthState.MIS_REPLICATED,
+            container.containerID());
+      }
+      // TODO - if it is unrecoverable, should we return false to other
+      //        handlers can be tried?
+      if (!underHealth.isUnrecoverable() &&
+          (underHealth.isMisReplicatedAfterPending() ||
+              !underHealth.isSufficientlyReplicatedAfterPending())) {
+        request.getReplicationQueue().enqueue(underHealth);
+      }
+      return true;
+    }

Review Comment:
   NIT: Let's add a new line after this brace for better readability. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] sodonnel commented on a diff in pull request #3802: HDDS-7058. EC: ReplicationManager - Implement ratis container replication check handler

Posted by GitBox <gi...@apache.org>.
sodonnel commented on code in PR #3802:
URL: https://github.com/apache/ozone/pull/3802#discussion_r992103112


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ContainerHealthResult.java:
##########
@@ -106,6 +106,9 @@ public static class UnHealthyResult extends ContainerHealthResult {
     private final int remainingRedundancy;
     private final boolean dueToDecommission;
     private final boolean sufficientlyReplicatedAfterPending;
+    private boolean dueToMisReplication = false;
+    private boolean isMisReplicated = false;
+    private boolean isMisReplicatedAfterPending = false;

Review Comment:
   I don't want to change the existing constructor, as then we need to change it everywhere it is used. Adding a new constructor starts a bad pattern where each new parameter needs a new constructor, and what we really need is a builder.
   
   At the moment I think these 3 parameters have good defaults for the common case and then using the settings when needed is a good compromise.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] sodonnel commented on a diff in pull request #3802: HDDS-7058. EC: ReplicationManager - Implement ratis container replication check handler

Posted by GitBox <gi...@apache.org>.
sodonnel commented on code in PR #3802:
URL: https://github.com/apache/ozone/pull/3802#discussion_r992140592


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisContainerReplicaCount.java:
##########
@@ -257,7 +277,50 @@ public boolean isSufficientlyReplicated() {
    */
   @Override
   public boolean isOverReplicated() {
-    return missingReplicas() + inFlightDel < 0;
+    return isOverReplicated(true);
+  }
+
+  /**
+   * Return true if the container is over replicated. Decommission and
+   * maintenance containers are ignored for this check.
+   * The check ignores inflight additions, as they may fail, but it does
+   * consider inflight deletes if includePendingDelete is true.
+   *
+   * @return True if the container is over replicated, false otherwise.
+   */
+  public boolean isOverReplicated(boolean includePendingDelete) {

Review Comment:
   Good point. I have changed this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #3802: HDDS-7058. EC: ReplicationManager - Implement ratis container replication check handler

Posted by GitBox <gi...@apache.org>.
swamirishi commented on code in PR #3802:
URL: https://github.com/apache/ozone/pull/3802#discussion_r992002733


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java:
##########
@@ -210,7 +218,8 @@ public ReplicationManager(final ConfigurationSource conf,
         .addNext(new ClosingContainerHandler(this))
         .addNext(new QuasiClosedContainerHandler(this))
         .addNext(new ClosedWithMismatchedReplicasHandler(this))
-        .addNext(ecReplicationCheckHandler);
+        .addNext(ecReplicationCheckHandler)
+        .addNext(ratisReplicationCheckHandler);

Review Comment:
   For EC case RatisReplicationCheckHandler would return false.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] sodonnel commented on a diff in pull request #3802: HDDS-7058. EC: ReplicationManager - Implement ratis container replication check handler

Posted by GitBox <gi...@apache.org>.
sodonnel commented on code in PR #3802:
URL: https://github.com/apache/ozone/pull/3802#discussion_r992108314


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/RatisReplicationCheckHandler.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.health;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.RatisContainerReplicaCount;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
+
+/**
+ * Class to determine the health state of a Ratis Container. Given the container
+ * and current replica details, along with replicas pending add and delete,
+ * this class will return a ContainerHealthResult indicating if the container
+ * is healthy, or under / over replicated etc.
+ */
+public class RatisReplicationCheckHandler extends AbstractCheck {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(RatisReplicationCheckHandler.class);
+
+  /**
+   * PlacementPolicy which is used to identify where a container
+   * should be replicated.
+   */
+  private final PlacementPolicy ratisContainerPlacement;
+
+  public RatisReplicationCheckHandler(PlacementPolicy containerPlacement) {
+    this.ratisContainerPlacement = containerPlacement;
+  }
+
+  @Override
+  public boolean handle(ContainerCheckRequest request) {
+    if (request.getContainerInfo().getReplicationType() != RATIS) {
+      // This handler is only for Ratis containers.
+      return false;
+    }
+    ReplicationManagerReport report = request.getReport();
+    ContainerInfo container = request.getContainerInfo();
+    ContainerHealthResult health = checkHealth(request);
+    if (health.getHealthState() == ContainerHealthResult.HealthState.HEALTHY) {
+      // If the container is healthy, there is nothing else to do in this
+      // handler so return as unhandled so any further handlers will be tried.
+      return false;
+    }
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.UNDER_REPLICATED) {
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.UNDER_REPLICATED,
+          container.containerID());
+      ContainerHealthResult.UnderReplicatedHealthResult underHealth
+          = ((ContainerHealthResult.UnderReplicatedHealthResult) health);
+      if (underHealth.isUnrecoverable()) {
+        report.incrementAndSample(ReplicationManagerReport.HealthState.MISSING,
+            container.containerID());
+      }
+      if (underHealth.isMisReplicated()) {
+        report.incrementAndSample(
+            ReplicationManagerReport.HealthState.MIS_REPLICATED,
+            container.containerID());
+      }
+      // TODO - if it is unrecoverable, should we return false to other
+      //        handlers can be tried?
+      if (!underHealth.isUnrecoverable() &&
+          (underHealth.isMisReplicatedAfterPending() ||
+              !underHealth.isSufficientlyReplicatedAfterPending())) {
+        request.getReplicationQueue().enqueue(underHealth);
+      }
+      return true;
+    }
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.OVER_REPLICATED) {
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.OVER_REPLICATED,
+          container.containerID());
+      ContainerHealthResult.OverReplicatedHealthResult overHealth
+          = ((ContainerHealthResult.OverReplicatedHealthResult) health);
+      if (!overHealth.isSufficientlyReplicatedAfterPending()) {
+        request.getReplicationQueue().enqueue(overHealth);
+      }
+      return true;
+    }
+    return false;
+  }
+
+  public ContainerHealthResult checkHealth(ContainerCheckRequest request) {
+    ContainerInfo container = request.getContainerInfo();
+    Set<ContainerReplica> replicas = request.getContainerReplicas();
+    List<ContainerReplicaOp> replicaPendingOps = request.getPendingOps();
+    // Note that this setting is minReplicasForMaintenance. For EC the variable
+    // is defined as remainingRedundancy which is subtly different.
+    int minReplicasForMaintenance = request.getMaintenanceRedundancy();
+    int pendingAdd = 0;
+    int pendingDelete = 0;
+    for (ContainerReplicaOp op : replicaPendingOps) {
+      if (op.getOpType() == ContainerReplicaOp.PendingOpType.ADD) {
+        pendingAdd++;
+      } else if (op.getOpType() == ContainerReplicaOp.PendingOpType.DELETE) {
+        pendingDelete++;
+      }
+    }
+    int requiredNodes = container.getReplicationConfig().getRequiredNodes();
+
+    // RatisContainerReplicaCount uses the minReplicasForMaintenance rather
+    // than remainingRedundancy which ECContainerReplicaCount uses.
+    RatisContainerReplicaCount replicaCount =
+        new RatisContainerReplicaCount(container, replicas, pendingAdd,
+            pendingDelete, requiredNodes, minReplicasForMaintenance);
+
+    ContainerPlacementStatus placementStatus =
+        getPlacementStatus(replicas, requiredNodes, Collections.EMPTY_LIST);

Review Comment:
   I fixed this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] sodonnel merged pull request #3802: HDDS-7058. EC: ReplicationManager - Implement ratis container replication check handler

Posted by GitBox <gi...@apache.org>.
sodonnel merged PR #3802:
URL: https://github.com/apache/ozone/pull/3802


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] siddhantsangwan commented on a diff in pull request #3802: HDDS-7058. EC: ReplicationManager - Implement ratis container replication check handler

Posted by GitBox <gi...@apache.org>.
siddhantsangwan commented on code in PR #3802:
URL: https://github.com/apache/ozone/pull/3802#discussion_r998096952


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestRatisReplicationCheckHandler.java:
##########
@@ -0,0 +1,560 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.health;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementStatusDefault;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult.HealthState;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult.OverReplicatedHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult.UnderReplicatedHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationQueue;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONING;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_MAINTENANCE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_SERVICE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
+import static org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp.PendingOpType.ADD;
+import static org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp.PendingOpType.DELETE;
+import static org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil.createContainerInfo;
+import static org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil.createReplicas;
+
+/**
+ * Tests for the RatisReplicationCheckHandler class.
+ */
+public class TestRatisReplicationCheckHandler {
+
+  private RatisReplicationCheckHandler healthCheck;
+  private ReplicationConfig repConfig;
+  private PlacementPolicy containerPlacementPolicy;
+  private ReplicationQueue repQueue;
+  private ContainerCheckRequest.Builder requestBuilder;
+  private ReplicationManagerReport report;
+  private int maintenanceRedundancy = 2;
+
+  @Before
+  public void setup() throws IOException {
+    containerPlacementPolicy = Mockito.mock(PlacementPolicy.class);
+    Mockito.when(containerPlacementPolicy.validateContainerPlacement(
+        Mockito.any(),
+        Mockito.anyInt()
+    )).thenAnswer(invocation ->
+        new ContainerPlacementStatusDefault(2, 2, 3));
+    healthCheck = new RatisReplicationCheckHandler(containerPlacementPolicy);
+    repConfig = RatisReplicationConfig.getInstance(THREE);
+    repQueue = new ReplicationQueue();
+    report = new ReplicationManagerReport();
+    requestBuilder = new ContainerCheckRequest.Builder()
+        .setReplicationQueue(repQueue)
+        .setMaintenanceRedundancy(maintenanceRedundancy)
+        .setPendingOps(Collections.emptyList())
+        .setReport(report);
+  }
+
+  @Test
+  public void testReturnFalseForNonRatis() {
+    ContainerInfo container =
+        createContainerInfo(new ECReplicationConfig(3, 2));
+    Set<ContainerReplica> replicas =
+        createReplicas(container.containerID(), 1, 2, 3, 4);
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    Assert.assertFalse(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+  }
+
+  @Test
+  public void testHealthyContainerIsHealthy() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas
+        = createReplicas(container.containerID(), 0, 0, 0);
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    ContainerHealthResult result =
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.HEALTHY, result.getHealthState());
+
+    Assert.assertFalse(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+  }
+
+  @Test
+  public void testUnderReplicatedContainerIsUnderReplicated() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas
+        = createReplicas(container.containerID(), 0, 0);
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getRemainingRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertFalse(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(1, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedContainerDueToPendingDelete() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas
+        = createReplicas(container.containerID(), 0, 0, 0);
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        DELETE, MockDatanodeDetails.randomDatanodeDetails(), 0));
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container)
+        .setPendingOps(pending);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getRemainingRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertFalse(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(1, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedContainerFixedWithPending() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas
+        = createReplicas(container.containerID(), 0, 0);
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        ADD, MockDatanodeDetails.randomDatanodeDetails(), 0));
+    requestBuilder.setContainerReplicas(replicas)
+        .setPendingOps(pending)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getRemainingRedundancy());
+    Assert.assertTrue(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertFalse(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    // Fixed with pending, so nothing added to the queue
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    // Still under replicated until the pending complete
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedDueToDecommission() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas = createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(DECOMMISSIONING, 0),
+        Pair.of(DECOMMISSIONED, 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(2, result.getRemainingRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertTrue(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(1, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedDueToDecommissionFixedWithPending() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas = createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(DECOMMISSIONED, 0));
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        ADD, MockDatanodeDetails.randomDatanodeDetails(), 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setPendingOps(pending)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(2, result.getRemainingRedundancy());
+    Assert.assertTrue(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertTrue(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    // Nothing queued as inflight replicas will fix it.
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    // Still under replicated in the report until pending complete
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedDueToDecommissionAndMissing() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas = createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(DECOMMISSIONED, 0));
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        ADD, MockDatanodeDetails.randomDatanodeDetails(), 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setPendingOps(pending)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getRemainingRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertFalse(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(1, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedAndUnrecoverable() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas = Collections.EMPTY_SET;
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(0, result.getRemainingRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertFalse(result.underReplicatedDueToDecommission());
+    Assert.assertTrue(result.isUnrecoverable());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    // Unrecoverable, so not added to the queue.
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.MISSING));
+  }
+
+  @Test
+  public void testOverReplicatedContainer() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas =  createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0));
+
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        DELETE, MockDatanodeDetails.randomDatanodeDetails(), 0));
+    pending.add(ContainerReplicaOp.create(
+        DELETE, MockDatanodeDetails.randomDatanodeDetails(), 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setPendingOps(pending)
+        .setContainerInfo(container);
+    OverReplicatedHealthResult result = (OverReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.OVER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(4, result.getExcessRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(1, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.OVER_REPLICATED));
+  }
+
+  @Test
+  public void testOverReplicatedContainerFixedByPending() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas =  createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0));
+
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        DELETE, MockDatanodeDetails.randomDatanodeDetails(), 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setPendingOps(pending)
+        .setContainerInfo(container);
+    OverReplicatedHealthResult result = (OverReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.OVER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getExcessRedundancy());
+    Assert.assertTrue(result.isSufficientlyReplicatedAfterPending());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    // Fixed by pending so nothing queued.
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    // Still over replicated, so the report should contain it
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.OVER_REPLICATED));
+  }
+
+  @Test
+  public void testOverReplicatedContainerWithMaintenance() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas =  createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_MAINTENANCE, 0), Pair.of(DECOMMISSIONED, 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    OverReplicatedHealthResult result = (OverReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.OVER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getExcessRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(1, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.OVER_REPLICATED));
+  }
+
+  @Test
+  public void testOverReplicatedContainerDueToMaintenance() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas = createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_MAINTENANCE, 0),
+        Pair.of(IN_MAINTENANCE, 2));

Review Comment:
   The replica index should be 0 instead of 2 I think. `(IN_MAINTENANCE, 0)`



##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestRatisReplicationCheckHandler.java:
##########
@@ -0,0 +1,560 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.health;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementStatusDefault;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult.HealthState;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult.OverReplicatedHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerHealthResult.UnderReplicatedHealthResult;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationQueue;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONING;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_MAINTENANCE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_SERVICE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
+import static org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp.PendingOpType.ADD;
+import static org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp.PendingOpType.DELETE;
+import static org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil.createContainerInfo;
+import static org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil.createReplicas;
+
+/**
+ * Tests for the RatisReplicationCheckHandler class.
+ */
+public class TestRatisReplicationCheckHandler {
+
+  private RatisReplicationCheckHandler healthCheck;
+  private ReplicationConfig repConfig;
+  private PlacementPolicy containerPlacementPolicy;
+  private ReplicationQueue repQueue;
+  private ContainerCheckRequest.Builder requestBuilder;
+  private ReplicationManagerReport report;
+  private int maintenanceRedundancy = 2;
+
+  @Before
+  public void setup() throws IOException {
+    containerPlacementPolicy = Mockito.mock(PlacementPolicy.class);
+    Mockito.when(containerPlacementPolicy.validateContainerPlacement(
+        Mockito.any(),
+        Mockito.anyInt()
+    )).thenAnswer(invocation ->
+        new ContainerPlacementStatusDefault(2, 2, 3));
+    healthCheck = new RatisReplicationCheckHandler(containerPlacementPolicy);
+    repConfig = RatisReplicationConfig.getInstance(THREE);
+    repQueue = new ReplicationQueue();
+    report = new ReplicationManagerReport();
+    requestBuilder = new ContainerCheckRequest.Builder()
+        .setReplicationQueue(repQueue)
+        .setMaintenanceRedundancy(maintenanceRedundancy)
+        .setPendingOps(Collections.emptyList())
+        .setReport(report);
+  }
+
+  @Test
+  public void testReturnFalseForNonRatis() {
+    ContainerInfo container =
+        createContainerInfo(new ECReplicationConfig(3, 2));
+    Set<ContainerReplica> replicas =
+        createReplicas(container.containerID(), 1, 2, 3, 4);
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    Assert.assertFalse(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+  }
+
+  @Test
+  public void testHealthyContainerIsHealthy() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas
+        = createReplicas(container.containerID(), 0, 0, 0);
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    ContainerHealthResult result =
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.HEALTHY, result.getHealthState());
+
+    Assert.assertFalse(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+  }
+
+  @Test
+  public void testUnderReplicatedContainerIsUnderReplicated() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas
+        = createReplicas(container.containerID(), 0, 0);
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getRemainingRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertFalse(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(1, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedContainerDueToPendingDelete() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas
+        = createReplicas(container.containerID(), 0, 0, 0);
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        DELETE, MockDatanodeDetails.randomDatanodeDetails(), 0));
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container)
+        .setPendingOps(pending);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getRemainingRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertFalse(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(1, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedContainerFixedWithPending() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas
+        = createReplicas(container.containerID(), 0, 0);
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        ADD, MockDatanodeDetails.randomDatanodeDetails(), 0));
+    requestBuilder.setContainerReplicas(replicas)
+        .setPendingOps(pending)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getRemainingRedundancy());
+    Assert.assertTrue(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertFalse(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    // Fixed with pending, so nothing added to the queue
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    // Still under replicated until the pending complete
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedDueToDecommission() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas = createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(DECOMMISSIONING, 0),
+        Pair.of(DECOMMISSIONED, 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(2, result.getRemainingRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertTrue(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(1, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedDueToDecommissionFixedWithPending() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas = createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(DECOMMISSIONED, 0));
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        ADD, MockDatanodeDetails.randomDatanodeDetails(), 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setPendingOps(pending)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(2, result.getRemainingRedundancy());
+    Assert.assertTrue(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertTrue(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    // Nothing queued as inflight replicas will fix it.
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    // Still under replicated in the report until pending complete
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedDueToDecommissionAndMissing() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas = createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(DECOMMISSIONED, 0));
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        ADD, MockDatanodeDetails.randomDatanodeDetails(), 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setPendingOps(pending)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getRemainingRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertFalse(result.underReplicatedDueToDecommission());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(1, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+  }
+
+  @Test
+  public void testUnderReplicatedAndUnrecoverable() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas = Collections.EMPTY_SET;
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    UnderReplicatedHealthResult result = (UnderReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.UNDER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(0, result.getRemainingRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+    Assert.assertFalse(result.underReplicatedDueToDecommission());
+    Assert.assertTrue(result.isUnrecoverable());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    // Unrecoverable, so not added to the queue.
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.UNDER_REPLICATED));
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.MISSING));
+  }
+
+  @Test
+  public void testOverReplicatedContainer() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas =  createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0));
+
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        DELETE, MockDatanodeDetails.randomDatanodeDetails(), 0));
+    pending.add(ContainerReplicaOp.create(
+        DELETE, MockDatanodeDetails.randomDatanodeDetails(), 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setPendingOps(pending)
+        .setContainerInfo(container);
+    OverReplicatedHealthResult result = (OverReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.OVER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(4, result.getExcessRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(1, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.OVER_REPLICATED));
+  }
+
+  @Test
+  public void testOverReplicatedContainerFixedByPending() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas =  createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0));
+
+    List<ContainerReplicaOp> pending = new ArrayList<>();
+    pending.add(ContainerReplicaOp.create(
+        DELETE, MockDatanodeDetails.randomDatanodeDetails(), 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setPendingOps(pending)
+        .setContainerInfo(container);
+    OverReplicatedHealthResult result = (OverReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.OVER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getExcessRedundancy());
+    Assert.assertTrue(result.isSufficientlyReplicatedAfterPending());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    // Fixed by pending so nothing queued.
+    Assert.assertEquals(0, repQueue.overReplicatedQueueSize());
+    // Still over replicated, so the report should contain it
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.OVER_REPLICATED));
+  }
+
+  @Test
+  public void testOverReplicatedContainerWithMaintenance() {
+    ContainerInfo container = createContainerInfo(repConfig);
+    Set<ContainerReplica> replicas =  createReplicas(container.containerID(),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_SERVICE, 0), Pair.of(IN_SERVICE, 0),
+        Pair.of(IN_MAINTENANCE, 0), Pair.of(DECOMMISSIONED, 0));
+
+    requestBuilder.setContainerReplicas(replicas)
+        .setContainerInfo(container);
+    OverReplicatedHealthResult result = (OverReplicatedHealthResult)
+        healthCheck.checkHealth(requestBuilder.build());
+    Assert.assertEquals(HealthState.OVER_REPLICATED, result.getHealthState());
+    Assert.assertEquals(1, result.getExcessRedundancy());
+    Assert.assertFalse(result.isSufficientlyReplicatedAfterPending());
+
+    Assert.assertTrue(healthCheck.handle(requestBuilder.build()));
+    Assert.assertEquals(0, repQueue.underReplicatedQueueSize());
+    Assert.assertEquals(1, repQueue.overReplicatedQueueSize());
+    Assert.assertEquals(1, report.getStat(
+        ReplicationManagerReport.HealthState.OVER_REPLICATED));
+  }
+
+  @Test
+  public void testOverReplicatedContainerDueToMaintenance() {

Review Comment:
   Since we're testing the HEALTHY case, let's change the test's name?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org