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


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ContainerSafeModeRule.java:
##########
@@ -71,83 +84,166 @@ public ContainerSafeModeRule(String ruleName, EventQueue 
eventQueue,
         HddsConfigKeys.HDDS_SCM_SAFEMODE_THRESHOLD_PCT  +
             " value should be >= 0.0 and <= 1.0");
 
-    containerMap = new ConcurrentHashMap<>();
+    ratisContainerMap = new ConcurrentHashMap<>();
+    ratisContainerDNsMap = new ConcurrentHashMap<>();
+    ecContainerMap = new ConcurrentHashMap<>();
+    ecContainerDNsMap = 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));
+      LifeCycleState containerState = container.getState();
+      ReplicationConfig replicationConfig = container.getReplicationConfig();
+
+      if (checkContainerState(containerState) && container.getNumberOfKeys() > 
0) {
+        if (replicationConfig instanceof RatisReplicationConfig) {
+          ratisContainerMap.put(container.getContainerID(), container);
+        }
+        if (replicationConfig instanceof ECReplicationConfig) {
+          ecContainerMap.put(container.getContainerID(), container);
+        }
+      }
     });
-    maxContainer = containerMap.size();
-    long cutOff = (long) Math.ceil(maxContainer * safeModeCutoff);
-    
getSafeModeMetrics().setNumContainerWithOneReplicaReportedThreshold(cutOff);
 
-    LOG.info("containers with one replica threshold count {}", cutOff);
+    ratisMaxContainer = ratisContainerMap.size();
+    ecMaxContainer = ecContainerMap.size();
+
+    long ratisCutOff = (long) Math.ceil(ratisMaxContainer * safeModeCutoff);
+    long ecCutOff = (long) Math.ceil(ecMaxContainer * safeModeCutoff);
+
+    
getSafeModeMetrics().setNumContainerWithOneReplicaReportedThreshold(ratisCutOff);
+
+    LOG.info("Containers with one replica threshold count {}, with ec n 
replica threshold count {}.",
+        ratisCutOff, ecCutOff);
   }
 
 
   @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 (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 (containerWithMinReplicas.doubleValue() / maxContainer);
+    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();
+
+    report.getReportsList().forEach(c -> {
+      long containerID = c.getContainerID();
 
-    reportsProto.getReport().getReportsList().forEach(c -> {
-      if (containerMap.containsKey(c.getContainerID())) {
-        if (containerMap.remove(c.getContainerID()) != null) {
-          containerWithMinReplicas.getAndAdd(1);
-          getSafeModeMetrics()
-              .incCurrentContainersWithOneReplicaReportedCount();
+      if (ratisContainerMap.containsKey(containerID)) {
+        ratisContainerDNsMap.computeIfAbsent(containerID, key -> 
Sets.newHashSet());
+        ratisContainerDNsMap.get(containerID).add(datanodeUUID);
+        if (!reportedConatinerIDSet.contains(containerID)) {
+          Set<UUID> uuids = ratisContainerDNsMap.get(containerID);
+          if (uuids != null && uuids.size() >= 1) {
+            ratisContainerWithMinReplicas.getAndAdd(1);
+            reportedConatinerIDSet.add(containerID);
+            getSafeModeMetrics()
+                .incCurrentContainersWithOneReplicaReportedCount();
+          }
+        }
+      }

Review Comment:
   Thank you for the question!
   
   > I didn't really understand this change. It seems correct, but is there any 
reason this logic isn't the same as before? 
   
   The previous logic was correct. I made this modification for two reasons:
   
   - To align with the EC's logic and improve code readability.
   - To facilitate the retrieval of additional data in future pr. For example, 
this will allow users not only to understand the progress but also to identify 
which `containers` have not reported and which `DataNodes ` are included in the 
reported containers.
   
   > Why do we need to track Datanodes in a set for Ratis containers? Is it 
because ratisContainerDNsMap and reportedConatinerIDSet are going to be used 
somewhere else as well?
   
   The type of `ratisContainerDNsMap` is `Map<Long, Set<UUID>>`, where the key 
is the `ContainerId`. The reason for using a `Set` as the value is to avoid 
retaining duplicate DN information, as we may encounter the same DN registering 
multiple times. 
   
   > Or is it done this way just so it's similar to the EC logic?
   
   Here's one reason; it has already been explained in the previous comment. 
   
   Can we modify it this way? The original code contains some insufficient 
information.
   



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