You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by "errose28 (via GitHub)" <gi...@apache.org> on 2023/02/08 01:49:55 UTC

[GitHub] [ozone] errose28 commented on a diff in pull request #4227: HDDS-7847. Handle Replication of Unhealthy Replicas in RM

errose28 commented on code in PR #4227:
URL: https://github.com/apache/ozone/pull/4227#discussion_r1099303927


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisOverReplicationHandler.java:
##########
@@ -95,12 +96,11 @@ public Set<Pair<DatanodeDetails, SCMCommand<?>>> processAndCreateCommands(
     Set<ContainerReplica> healthyReplicas = replicas.stream()
         .filter(r -> ReplicationManager.getNodeStatus(
             r.getDatanodeDetails(), nodeManager).isHealthy()
-        )
-        .collect(Collectors.toSet());
+        ).collect(Collectors.toSet());
 
     RatisContainerReplicaCount replicaCount =
         new RatisContainerReplicaCount(containerInfo, healthyReplicas,
-            pendingOps, minHealthyForMaintenance);
+            pendingOps, minHealthyForMaintenance, true);

Review Comment:
   Can you point me to the part of the new RM that makes sure containers are only put in the over rep queue if they have enough matching replicas?



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/RatisUnhealthyReplicationCheckHandler.java:
##########
@@ -0,0 +1,193 @@
+/**
+ * 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 com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+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.RatisContainerReplicaCount;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
+
+/**
+ * This class handles UNHEALTHY RATIS replicas. Its responsibilities include:
+ * <ul>
+ *   <li>If only unhealthy replicas exist, check if there are replication
+ *   factor number of replicas and queue them for under/over replication if
+ *   needed.
+ *   </li>
+ *   <li>If there is perfect replication of healthy replicas, remove any
+ *   excess unhealthy replicas.
+ *   </li>
+ * </ul>
+ */
+public class RatisUnhealthyReplicationCheckHandler extends AbstractCheck {
+  public static final Logger LOG = LoggerFactory.getLogger(
+      RatisUnhealthyReplicationCheckHandler.class);
+
+  @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();
+
+    /*
+    First, verify there's perfect replication without considering UNHEALTHY
+    replicas.
+     */
+    if (!verifyPerfectReplication(request)) {
+      return false;
+    }
+
+    // Now, consider UNHEALTHY replicas when calculating replication status
+    RatisContainerReplicaCount replicaCount =
+        new RatisContainerReplicaCount(container,
+            request.getContainerReplicas(), request.getPendingOps(),
+            request.getMaintenanceRedundancy(), true);
+    if (replicaCount.getUnhealthyReplicaCount() == 0) {
+      LOG.info("No UNHEALTHY replicas are present for container {} with " +
+          "replicas [{}].", container, request.getContainerReplicas());
+      return false;
+    } else {
+      LOG.debug("Container {} has UNHEALTHY replicas. Checking its " +
+          "replication status.", container);
+      report.incrementAndSample(ReplicationManagerReport.HealthState.UNHEALTHY,
+          container.containerID());
+    }
+
+    ContainerHealthResult health = checkReplication(request);
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.UNDER_REPLICATED) {
+      ContainerHealthResult.UnderReplicatedHealthResult underHealth
+          = ((ContainerHealthResult.UnderReplicatedHealthResult) health);
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.UNDER_REPLICATED,
+          container.containerID());
+      LOG.debug("Container {} is Under Replicated. isReplicatedOkAfterPending" +
+              " is [{}]. isUnrecoverable is [{}]. hasHealthyReplicas is [{}].",
+          container,
+          underHealth.isReplicatedOkAfterPending(),
+          underHealth.isUnrecoverable(), underHealth.hasHealthyReplicas());
+
+      if (!underHealth.isReplicatedOkAfterPending()) {
+        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);
+      LOG.debug("Container {} is Over Replicated. isReplicatedOkAfterPending" +
+              " is [{}]. hasMismatchedReplicas is [{}]", container,
+          overHealth.isReplicatedOkAfterPending(),
+          overHealth.hasMismatchedReplicas());
+
+      if (!overHealth.isReplicatedOkAfterPending()) {
+        request.getReplicationQueue().enqueue(overHealth);
+      }
+      return true;
+    }
+
+    return false;
+  }
+
+  /**
+   * Verify there's no under or over replication if only healthy replicas are
+   * considered, or that there are no healthy replicas.
+   * @return true if there's no under/over replication considering healthy
+   * replicas
+   */
+  private boolean verifyPerfectReplication(ContainerCheckRequest request) {
+    RatisContainerReplicaCount replicaCountWithoutUnhealthy =
+        new RatisContainerReplicaCount(request.getContainerInfo(),
+            request.getContainerReplicas(), request.getPendingOps(),
+            request.getMaintenanceRedundancy(), false);
+
+    if (replicaCountWithoutUnhealthy.getHealthyReplicaCount() == 0) {
+      return true;
+    }
+    if (!replicaCountWithoutUnhealthy.isSufficientlyReplicated() ||
+        replicaCountWithoutUnhealthy.isOverReplicated()) {
+      LOG.info("Checking replication for container {} without considering " +
+              "UNHEALTHY replicas. isSufficientlyReplicated is [{}]. " +
+              "isOverReplicated is [{}]. Returning false because there should" +
+              " be perfect replication for this handler to work.",

Review Comment:
   This looks more like a debug log.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisUnderReplicationHandler.java:
##########
@@ -133,24 +138,32 @@ public Set<Pair<DatanodeDetails, SCMCommand<?>>> processAndCreateCommands(
    * container is sufficiently replicated or unrecoverable.
    */
   private boolean verifyUnderReplication(
-      RatisContainerReplicaCount replicaCount) {
-    if (replicaCount.isSufficientlyReplicated()) {
+      RatisContainerReplicaCount replicaCount,
+      RatisContainerReplicaCount withoutUnhealthy) {

Review Comment:
   This might be easier to follow in the method:
   ```suggestion
         RatisContainerReplicaCount withUnhealthy,
         RatisContainerReplicaCount withoutUnhealthy) {
   ```
   
   Same for the caller `processAndCreateCommands`



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisUnderReplicationHandler.java:
##########
@@ -175,14 +189,21 @@ private List<DatanodeDetails> getSources(
       }
     }
 
+    Predicate<ContainerReplica> predicate;
+    if (replicaCount.getHealthyReplicaCount() == 0) {
+      predicate = replica -> replica.getState() == State.UNHEALTHY;
+    } else {
+      predicate = replica -> replica.getState() == State.CLOSED ||
+          replica.getState() == State.QUASI_CLOSED;
+    }
+
     /*
-     * Return healthy datanodes that have closed/quasi-closed replicas and
-     * are not pending replica deletion. Sorted in descending order of
-     * sequence id.
+     * Return healthy datanodes which have a replica that satisfies the
+     * predicate and is not pending replica deletion. Sorted in descending
+     * order of sequence id.

Review Comment:
   For the pull model, the datanodes are shuffling the sources. For the push model, it looks like SCM is doing a shuffle in `RatisUnderReplicationHandler#createReplicationCommands`. So I don't think the sorting has any effect right now. Is this being changed in a follow up Jira? 



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/RatisReplicationCheckHandler.java:
##########
@@ -205,6 +207,24 @@ public ContainerHealthResult checkHealth(ContainerCheckRequest request) {
     return new ContainerHealthResult.HealthyResult(container);
   }
 
+  private boolean hasSufficientMatchingReplicas(ContainerCheckRequest request) {

Review Comment:
   This method is currently unused.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/RatisUnhealthyReplicationCheckHandler.java:
##########
@@ -0,0 +1,193 @@
+/**
+ * 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 com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+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.RatisContainerReplicaCount;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
+
+/**
+ * This class handles UNHEALTHY RATIS replicas. Its responsibilities include:
+ * <ul>
+ *   <li>If only unhealthy replicas exist, check if there are replication
+ *   factor number of replicas and queue them for under/over replication if
+ *   needed.
+ *   </li>
+ *   <li>If there is perfect replication of healthy replicas, remove any
+ *   excess unhealthy replicas.
+ *   </li>
+ * </ul>
+ */
+public class RatisUnhealthyReplicationCheckHandler extends AbstractCheck {
+  public static final Logger LOG = LoggerFactory.getLogger(
+      RatisUnhealthyReplicationCheckHandler.class);
+
+  @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();
+
+    /*
+    First, verify there's perfect replication without considering UNHEALTHY
+    replicas.
+     */
+    if (!verifyPerfectReplication(request)) {
+      return false;
+    }
+
+    // Now, consider UNHEALTHY replicas when calculating replication status
+    RatisContainerReplicaCount replicaCount =
+        new RatisContainerReplicaCount(container,
+            request.getContainerReplicas(), request.getPendingOps(),
+            request.getMaintenanceRedundancy(), true);
+    if (replicaCount.getUnhealthyReplicaCount() == 0) {
+      LOG.info("No UNHEALTHY replicas are present for container {} with " +
+          "replicas [{}].", container, request.getContainerReplicas());

Review Comment:
   This should probably be a debug log. Looks like it would get printed for every perfectly replicated container without unhealthy replicas.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/RatisUnhealthyReplicationCheckHandler.java:
##########
@@ -0,0 +1,193 @@
+/**
+ * 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 com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+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.RatisContainerReplicaCount;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
+
+/**
+ * This class handles UNHEALTHY RATIS replicas. Its responsibilities include:
+ * <ul>
+ *   <li>If only unhealthy replicas exist, check if there are replication
+ *   factor number of replicas and queue them for under/over replication if
+ *   needed.
+ *   </li>
+ *   <li>If there is perfect replication of healthy replicas, remove any
+ *   excess unhealthy replicas.
+ *   </li>
+ * </ul>
+ */
+public class RatisUnhealthyReplicationCheckHandler extends AbstractCheck {
+  public static final Logger LOG = LoggerFactory.getLogger(
+      RatisUnhealthyReplicationCheckHandler.class);
+
+  @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();
+
+    /*
+    First, verify there's perfect replication without considering UNHEALTHY
+    replicas.
+     */
+    if (!verifyPerfectReplication(request)) {
+      return false;
+    }
+
+    // Now, consider UNHEALTHY replicas when calculating replication status
+    RatisContainerReplicaCount replicaCount =
+        new RatisContainerReplicaCount(container,
+            request.getContainerReplicas(), request.getPendingOps(),
+            request.getMaintenanceRedundancy(), true);
+    if (replicaCount.getUnhealthyReplicaCount() == 0) {
+      LOG.info("No UNHEALTHY replicas are present for container {} with " +
+          "replicas [{}].", container, request.getContainerReplicas());
+      return false;
+    } else {
+      LOG.debug("Container {} has UNHEALTHY replicas. Checking its " +
+          "replication status.", container);
+      report.incrementAndSample(ReplicationManagerReport.HealthState.UNHEALTHY,
+          container.containerID());
+    }
+
+    ContainerHealthResult health = checkReplication(request);
+    if (health.getHealthState()
+        == ContainerHealthResult.HealthState.UNDER_REPLICATED) {
+      ContainerHealthResult.UnderReplicatedHealthResult underHealth
+          = ((ContainerHealthResult.UnderReplicatedHealthResult) health);
+      report.incrementAndSample(
+          ReplicationManagerReport.HealthState.UNDER_REPLICATED,
+          container.containerID());
+      LOG.debug("Container {} is Under Replicated. isReplicatedOkAfterPending" +
+              " is [{}]. isUnrecoverable is [{}]. hasHealthyReplicas is [{}].",
+          container,
+          underHealth.isReplicatedOkAfterPending(),
+          underHealth.isUnrecoverable(), underHealth.hasHealthyReplicas());
+
+      if (!underHealth.isReplicatedOkAfterPending()) {
+        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);
+      LOG.debug("Container {} is Over Replicated. isReplicatedOkAfterPending" +
+              " is [{}]. hasMismatchedReplicas is [{}]", container,
+          overHealth.isReplicatedOkAfterPending(),
+          overHealth.hasMismatchedReplicas());
+
+      if (!overHealth.isReplicatedOkAfterPending()) {
+        request.getReplicationQueue().enqueue(overHealth);
+      }
+      return true;
+    }
+
+    return false;
+  }
+
+  /**
+   * Verify there's no under or over replication if only healthy replicas are
+   * considered, or that there are no healthy replicas.
+   * @return true if there's no under/over replication considering healthy
+   * replicas
+   */
+  private boolean verifyPerfectReplication(ContainerCheckRequest request) {
+    RatisContainerReplicaCount replicaCountWithoutUnhealthy =
+        new RatisContainerReplicaCount(request.getContainerInfo(),
+            request.getContainerReplicas(), request.getPendingOps(),
+            request.getMaintenanceRedundancy(), false);
+
+    if (replicaCountWithoutUnhealthy.getHealthyReplicaCount() == 0) {
+      return true;
+    }
+    if (!replicaCountWithoutUnhealthy.isSufficientlyReplicated() ||
+        replicaCountWithoutUnhealthy.isOverReplicated()) {

Review Comment:
   This check looks correct although it is confusing. Sufficiently replicated means there are >= 0 excess replicas, and over replicated means there are > 0 excess replicas. So this says `if !(excess >= 0) || excess > 0` which is `if excess < 0 || excess > 0`. So this does check that there is not perfect replication, but in a roundabout way. The `isSufficientlyReplicated` method is confusing to me in general. It seems it could be replaced by a more explicit `isUnderReplicated` method and negating the result when needed. This check would then be more literal:
   ```
   if (replicaCountWithoutUnhealthy.isUnderReplicated() ||
           replicaCountWithoutUnhealthy.isOverReplicated()) {
   ```



-- 
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