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/11 07:24:22 UTC

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

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