slfan1989 commented on code in PR #7008:
URL: https://github.com/apache/ozone/pull/7008#discussion_r1832428552


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ContainerSafeModeRule.java:
##########
@@ -71,127 +84,262 @@ public ContainerSafeModeRule(String ruleName, EventQueue 
eventQueue,
         HddsConfigKeys.HDDS_SCM_SAFEMODE_THRESHOLD_PCT  +
             " value should be >= 0.0 and <= 1.0");
 
-    containerMap = new ConcurrentHashMap<>();
-    containers.forEach(container -> {
-      // There can be containers in OPEN/CLOSING state which were never
-      // created by the client. We are not considering these containers for
-      // now. These containers can be handled by tracking pipelines.
-
-      Optional.ofNullable(container.getState())
-          .filter(state -> (state == HddsProtos.LifeCycleState.QUASI_CLOSED ||
-              state == HddsProtos.LifeCycleState.CLOSED)
-              && container.getNumberOfKeys() > 0)
-          .ifPresent(s -> containerMap.put(container.getContainerID(),
-              container));
-    });
-    maxContainer = containerMap.size();
-    long cutOff = (long) Math.ceil(maxContainer * safeModeCutoff);
-    
getSafeModeMetrics().setNumContainerWithOneReplicaReportedThreshold(cutOff);
+    ratisContainerMap = new ConcurrentHashMap<>();
+    ratisContainerDNsMap = new ConcurrentHashMap<>();
+    ecContainerMap = new ConcurrentHashMap<>();
+    ecContainerDNsMap = new ConcurrentHashMap<>();
 
-    LOG.info("containers with one replica threshold count {}", cutOff);
+    initializeRule(containers);
   }
 
 
   @Override
   protected TypedEvent<NodeRegistrationContainerReport> getEventType() {
-    return SCMEvents.NODE_REGISTRATION_CONT_REPORT;
+    return SCMEvents.CONTAINER_REGISTRATION_REPORT;
   }
 
 
   @Override
   protected synchronized boolean validate() {
-    return getCurrentContainerThreshold() >= safeModeCutoff;
+    return (getCurrentContainerThreshold() >= safeModeCutoff) &&
+        (getCurrentECContainerThreshold() >= safeModeCutoff);
   }
 
   @VisibleForTesting
   public synchronized double getCurrentContainerThreshold() {
-    if (maxContainer == 0) {
+    if (ratisMaxContainer == 0) {
       return 1;
     }
-    return (containerWithMinReplicas.doubleValue() / maxContainer);
+    return (ratisContainerWithMinReplicas.doubleValue() / ratisMaxContainer);
+  }
+
+  @VisibleForTesting
+  public synchronized double getCurrentECContainerThreshold() {
+    if (ecMaxContainer == 0) {
+      return 1;
+    }
+    return (ecContainerWithMinReplicas.doubleValue() / ecMaxContainer);
+  }
+
+  public synchronized double getEcMaxContainer() {
+    if (ecMaxContainer == 0) {
+      return 1;
+    }
+    return ecMaxContainer;
+  }
+
+  private synchronized double getRatisMaxContainer() {
+    if (ratisMaxContainer == 0) {
+      return 1;
+    }
+    return ratisMaxContainer;
   }
 
   @Override
   protected synchronized void process(
       NodeRegistrationContainerReport reportsProto) {
+    DatanodeDetails datanodeDetails = reportsProto.getDatanodeDetails();
+    UUID datanodeUUID = datanodeDetails.getUuid();
+    StorageContainerDatanodeProtocolProtos.ContainerReportsProto report = 
reportsProto.getReport();
 
-    reportsProto.getReport().getReportsList().forEach(c -> {
-      if (containerMap.containsKey(c.getContainerID())) {
-        if (containerMap.remove(c.getContainerID()) != null) {
-          containerWithMinReplicas.getAndAdd(1);
-          getSafeModeMetrics()
-              .incCurrentContainersWithOneReplicaReportedCount();
-        }
+    report.getReportsList().forEach(c -> {
+      long containerID = c.getContainerID();
+
+      // If it is a Ratis container.
+      if (ratisContainerMap.containsKey(containerID)) {
+        initContainerDNsMap(containerID, ratisContainerDNsMap, datanodeUUID);
+        recordReportedContainer(containerID, Boolean.FALSE);
+      }
+
+      // If it is a EC container.
+      if (ecContainerMap.containsKey(containerID)) {
+        initContainerDNsMap(containerID, ecContainerDNsMap, datanodeUUID);
+        recordReportedContainer(containerID, Boolean.TRUE);
       }
     });
 
     if (scmInSafeMode()) {
       SCMSafeModeManager.getLogger().info(
-          "SCM in safe mode. {} % containers have at least one"
-              + " reported replica.",
-          (containerWithMinReplicas.doubleValue() / maxContainer) * 100);
+          "SCM in safe mode. {} % containers [Ratis] have at least one"
+          + " reported replica, {} % containers [EC] have at N reported 
replica.",
+          ((ratisContainerWithMinReplicas.doubleValue() / 
getRatisMaxContainer()) * 100),
+          ((ecContainerWithMinReplicas.doubleValue() / getEcMaxContainer()) * 
100)
+      );
+    }
+  }
+
+  /**
+   * Record the reported Container.
+   *
+   * We will differentiate and count according to the type of Container.
+   *
+   * @param containerID containerID
+   * @param isEcContainer true, means ECContainer, false, means not 
ECContainer.
+   */
+  private void recordReportedContainer(long containerID, boolean 
isEcContainer) {
+    if (!reportedContainerIDSet.contains(containerID)) {

Review Comment:
   Thank you for your suggestion! We can indeed remove reportedContainerIDSet, 
and I will improve the code.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ContainerSafeModeRule.java:
##########
@@ -71,127 +84,262 @@ public ContainerSafeModeRule(String ruleName, EventQueue 
eventQueue,
         HddsConfigKeys.HDDS_SCM_SAFEMODE_THRESHOLD_PCT  +
             " value should be >= 0.0 and <= 1.0");
 
-    containerMap = new ConcurrentHashMap<>();
-    containers.forEach(container -> {
-      // There can be containers in OPEN/CLOSING state which were never
-      // created by the client. We are not considering these containers for
-      // now. These containers can be handled by tracking pipelines.
-
-      Optional.ofNullable(container.getState())
-          .filter(state -> (state == HddsProtos.LifeCycleState.QUASI_CLOSED ||
-              state == HddsProtos.LifeCycleState.CLOSED)
-              && container.getNumberOfKeys() > 0)
-          .ifPresent(s -> containerMap.put(container.getContainerID(),
-              container));
-    });
-    maxContainer = containerMap.size();
-    long cutOff = (long) Math.ceil(maxContainer * safeModeCutoff);
-    
getSafeModeMetrics().setNumContainerWithOneReplicaReportedThreshold(cutOff);
+    ratisContainerMap = new ConcurrentHashMap<>();
+    ratisContainerDNsMap = new ConcurrentHashMap<>();
+    ecContainerMap = new ConcurrentHashMap<>();
+    ecContainerDNsMap = new ConcurrentHashMap<>();
 
-    LOG.info("containers with one replica threshold count {}", cutOff);
+    initializeRule(containers);
   }
 
 
   @Override
   protected TypedEvent<NodeRegistrationContainerReport> getEventType() {
-    return SCMEvents.NODE_REGISTRATION_CONT_REPORT;
+    return SCMEvents.CONTAINER_REGISTRATION_REPORT;
   }
 
 
   @Override
   protected synchronized boolean validate() {
-    return getCurrentContainerThreshold() >= safeModeCutoff;
+    return (getCurrentContainerThreshold() >= safeModeCutoff) &&
+        (getCurrentECContainerThreshold() >= safeModeCutoff);
   }
 
   @VisibleForTesting
   public synchronized double getCurrentContainerThreshold() {
-    if (maxContainer == 0) {
+    if (ratisMaxContainer == 0) {
       return 1;
     }
-    return (containerWithMinReplicas.doubleValue() / maxContainer);
+    return (ratisContainerWithMinReplicas.doubleValue() / ratisMaxContainer);
+  }
+
+  @VisibleForTesting
+  public synchronized double getCurrentECContainerThreshold() {
+    if (ecMaxContainer == 0) {
+      return 1;
+    }
+    return (ecContainerWithMinReplicas.doubleValue() / ecMaxContainer);
+  }
+
+  public synchronized double getEcMaxContainer() {
+    if (ecMaxContainer == 0) {
+      return 1;
+    }
+    return ecMaxContainer;
+  }
+
+  private synchronized double getRatisMaxContainer() {
+    if (ratisMaxContainer == 0) {
+      return 1;
+    }
+    return ratisMaxContainer;
   }
 
   @Override
   protected synchronized void process(
       NodeRegistrationContainerReport reportsProto) {
+    DatanodeDetails datanodeDetails = reportsProto.getDatanodeDetails();
+    UUID datanodeUUID = datanodeDetails.getUuid();
+    StorageContainerDatanodeProtocolProtos.ContainerReportsProto report = 
reportsProto.getReport();
 
-    reportsProto.getReport().getReportsList().forEach(c -> {
-      if (containerMap.containsKey(c.getContainerID())) {
-        if (containerMap.remove(c.getContainerID()) != null) {
-          containerWithMinReplicas.getAndAdd(1);
-          getSafeModeMetrics()
-              .incCurrentContainersWithOneReplicaReportedCount();
-        }
+    report.getReportsList().forEach(c -> {
+      long containerID = c.getContainerID();
+
+      // If it is a Ratis container.
+      if (ratisContainerMap.containsKey(containerID)) {
+        initContainerDNsMap(containerID, ratisContainerDNsMap, datanodeUUID);
+        recordReportedContainer(containerID, Boolean.FALSE);
+      }
+
+      // If it is a EC container.
+      if (ecContainerMap.containsKey(containerID)) {
+        initContainerDNsMap(containerID, ecContainerDNsMap, datanodeUUID);
+        recordReportedContainer(containerID, Boolean.TRUE);
       }
     });
 
     if (scmInSafeMode()) {
       SCMSafeModeManager.getLogger().info(
-          "SCM in safe mode. {} % containers have at least one"
-              + " reported replica.",
-          (containerWithMinReplicas.doubleValue() / maxContainer) * 100);
+          "SCM in safe mode. {} % containers [Ratis] have at least one"
+          + " reported replica, {} % containers [EC] have at N reported 
replica.",
+          ((ratisContainerWithMinReplicas.doubleValue() / 
getRatisMaxContainer()) * 100),
+          ((ecContainerWithMinReplicas.doubleValue() / getEcMaxContainer()) * 
100)
+      );
+    }
+  }
+
+  /**
+   * Record the reported Container.
+   *
+   * We will differentiate and count according to the type of Container.
+   *
+   * @param containerID containerID
+   * @param isEcContainer true, means ECContainer, false, means not 
ECContainer.
+   */
+  private void recordReportedContainer(long containerID, boolean 
isEcContainer) {
+    if (!reportedContainerIDSet.contains(containerID)) {
+      Set<UUID> uuids = isEcContainer ? ecContainerDNsMap.get(containerID) :
+          ratisContainerDNsMap.get(containerID);
+      int minReplica = getMinReplica(containerID, isEcContainer);
+      if (uuids != null && uuids.size() >= minReplica) {
+        reportedContainerIDSet.add(containerID);
+        if (isEcContainer) {
+          getSafeModeMetrics()
+              .incCurrentContainersWithECDataReplicaReportedCount();
+          ecContainerWithMinReplicas.getAndAdd(1);
+        } else {
+          ratisContainerWithMinReplicas.getAndAdd(1);
+          getSafeModeMetrics()
+              .incCurrentContainersWithOneReplicaReportedCount();
+        }
+      }
     }
   }
 
+  /**
+   * Get the minimum replica.
+   *
+   * If it is a Ratis Contianer, the minimum copy is 1.
+   * If it is an EC Container, the minimum copy will be the number of Data in 
replicationConfig.
+   *
+   * @param containerID containerID
+   * @param isEcContainer true, means ECContainer, false, means not 
ECContainer.
+   * @return MinReplica.
+   */
+  private int getMinReplica(long containerID, boolean isEcContainer) {
+    if (isEcContainer) {
+      ContainerInfo containerInfo = ecContainerMap.get(containerID);
+      if (containerInfo != null) {
+        ReplicationConfig replicationConfig = 
containerInfo.getReplicationConfig();
+        if (replicationConfig != null && replicationConfig instanceof 
ECReplicationConfig) {
+          ECReplicationConfig ecReplicationConfig = (ECReplicationConfig) 
replicationConfig;
+          return ecReplicationConfig.getData();
+        }
+      }
+    }
+    return 1;
+  }
+
+  private void initContainerDNsMap(long containerID, Map<Long, Set<UUID>> 
containerDNsMap,

Review Comment:
   I will improve the code.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to