This is an automated email from the ASF dual-hosted git repository.

adoroszlai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new cba23a5f66 HDDS-9478. Add more details to Safemode verbose output. 
(#5760)
cba23a5f66 is described below

commit cba23a5f66a4fb7345db801cb1c8e2538146d4c4
Author: Sadanand Shenoy <[email protected]>
AuthorDate: Fri Dec 15 21:04:24 2023 +0530

    HDDS-9478. Add more details to Safemode verbose output. (#5760)
---
 .../hdds/scm/safemode/ContainerSafeModeRule.java   | 22 ++++++++++---
 .../scm/safemode/HealthyPipelineSafeModeRule.java  | 36 ++++++++++++++++++----
 .../safemode/OneReplicaPipelineSafeModeRule.java   | 27 +++++++++++-----
 .../hadoop/hdds/scm/safemode/SafeModeExitRule.java |  2 ++
 4 files changed, 69 insertions(+), 18 deletions(-)

diff --git 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ContainerSafeModeRule.java
 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ContainerSafeModeRule.java
index b82cae9ffa..ae645858a3 100644
--- 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ContainerSafeModeRule.java
+++ 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ContainerSafeModeRule.java
@@ -22,6 +22,7 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
 
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
@@ -139,11 +140,22 @@ public class ContainerSafeModeRule extends
 
   @Override
   public String getStatusText() {
-    return String
-        .format(
-            "%% of containers with at least one reported replica (=%1.2f) >= "
-                + "safeModeCutoff (=%1.2f)",
-            getCurrentContainerThreshold(), this.safeModeCutoff);
+    List<Long> sampleContainers = containerMap.keySet()
+        .stream()
+        .limit(SAMPLE_CONTAINER_DISPLAY_LIMIT)
+        .collect(Collectors.toList());
+
+    String status = String.format("%% of containers with at least one reported"
+            + " replica (=%1.2f) >= safeModeCutoff (=%1.2f)",
+        getCurrentContainerThreshold(), this.safeModeCutoff);
+
+    if (!sampleContainers.isEmpty()) {
+      String sampleContainerText =
+          "Sample containers not satisfying the criteria : " + 
sampleContainers;
+      status = status.concat("\n").concat(sampleContainerText);
+    }
+
+    return status;
   }
 
 
diff --git 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/HealthyPipelineSafeModeRule.java
 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/HealthyPipelineSafeModeRule.java
index 02e5fdc430..e9f25f3a94 100644
--- 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/HealthyPipelineSafeModeRule.java
+++ 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/HealthyPipelineSafeModeRule.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdds.scm.safemode;
 
 import java.util.HashSet;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.client.RatisReplicationConfig;
@@ -56,6 +57,7 @@ public class HealthyPipelineSafeModeRule extends 
SafeModeExitRule<Pipeline> {
   private final PipelineManager pipelineManager;
   private final int minHealthyPipelines;
   private final SCMContext scmContext;
+  private final Set<PipelineID> unProcessedPipelineSet = new HashSet<>();
 
   HealthyPipelineSafeModeRule(String ruleName, EventQueue eventQueue,
       PipelineManager pipelineManager, SCMSafeModeManager manager,
@@ -131,6 +133,7 @@ public class HealthyPipelineSafeModeRule extends 
SafeModeExitRule<Pipeline> {
       getSafeModeMetrics().incCurrentHealthyPipelinesCount();
       currentHealthyPipelineCount++;
       processedPipelineIDs.add(pipeline.getId());
+      unProcessedPipelineSet.remove(pipeline.getId());
     }
 
     if (scmInSafeMode()) {
@@ -154,9 +157,13 @@ public class HealthyPipelineSafeModeRule extends 
SafeModeExitRule<Pipeline> {
   }
 
   private synchronized void initializeRule(boolean refresh) {
-    int pipelineCount = pipelineManager.getPipelines(
-        RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE),
-        Pipeline.PipelineState.OPEN).size();
+    unProcessedPipelineSet.addAll(pipelineManager.getPipelines(
+            RatisReplicationConfig.getInstance(
+                HddsProtos.ReplicationFactor.THREE),
+            Pipeline.PipelineState.OPEN).stream().map(Pipeline::getId)
+        .collect(Collectors.toSet()));
+
+    int pipelineCount = unProcessedPipelineSet.size();
 
     healthyPipelineThresholdCount = Math.max(minHealthyPipelines,
         (int) Math.ceil(healthyPipelinesPercent * pipelineCount));
@@ -179,6 +186,7 @@ public class HealthyPipelineSafeModeRule extends 
SafeModeExitRule<Pipeline> {
   @Override
   protected synchronized void cleanup() {
     processedPipelineIDs.clear();
+    unProcessedPipelineSet.clear();
   }
 
   @VisibleForTesting
@@ -193,9 +201,25 @@ public class HealthyPipelineSafeModeRule extends 
SafeModeExitRule<Pipeline> {
 
   @Override
   public String getStatusText() {
-    return String.format("healthy Ratis/THREE pipelines (=%d) >= "
-            + "healthyPipelineThresholdCount (=%d)",
-        getCurrentHealthyPipelineCount(),
+    String status = String.format(
+        "healthy Ratis/THREE pipelines (=%d) >= healthyPipelineThresholdCount" 
+
+            " (=%d)", getCurrentHealthyPipelineCount(),
         getHealthyPipelineThresholdCount());
+    status = updateStatusTextWithSamplePipelines(status);
+    return status;
+  }
+
+  private synchronized String updateStatusTextWithSamplePipelines(
+      String status) {
+    Set<PipelineID> samplePipelines =
+        unProcessedPipelineSet.stream().limit(SAMPLE_PIPELINE_DISPLAY_LIMIT)
+            .collect(Collectors.toSet());
+
+    if (!samplePipelines.isEmpty()) {
+      String samplePipelineText =
+          "Sample pipelines not satisfying the criteria : " + samplePipelines;
+      status = status.concat("\n").concat(samplePipelineText);
+    }
+    return status;
   }
 }
diff --git 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/OneReplicaPipelineSafeModeRule.java
 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/OneReplicaPipelineSafeModeRule.java
index 72af1cce5f..08fe4d59d6 100644
--- 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/OneReplicaPipelineSafeModeRule.java
+++ 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/OneReplicaPipelineSafeModeRule.java
@@ -52,7 +52,7 @@ public class OneReplicaPipelineSafeModeRule extends
       LoggerFactory.getLogger(OneReplicaPipelineSafeModeRule.class);
 
   private int thresholdCount;
-  private Set<PipelineID> reportedPipelineIDSet = new HashSet<>();
+  private final Set<PipelineID> reportedPipelineIDSet = new HashSet<>();
   private Set<PipelineID> oldPipelineIDSet;
   private int currentReportedPipelineCount = 0;
   private PipelineManager pipelineManager;
@@ -142,12 +142,25 @@ public class OneReplicaPipelineSafeModeRule extends
 
   @Override
   public String getStatusText() {
-    return String
-        .format(
-            "reported Ratis/THREE pipelines with at least one datanode (=%d) "
-                + ">= threshold (=%d)",
-            getCurrentReportedPipelineCount(),
-            getThresholdCount());
+    String status = String.format(
+        "reported Ratis/THREE pipelines with at least one datanode (=%d) "
+            + ">= threshold (=%d)", getCurrentReportedPipelineCount(),
+        getThresholdCount());
+    status = updateStatusTextWithSamplePipelines(status);
+    return status;
+  }
+
+  private synchronized String updateStatusTextWithSamplePipelines(
+      String status) {
+    Set<PipelineID> samplePipelines = oldPipelineIDSet.stream()
+        .filter(element -> !reportedPipelineIDSet.contains(element))
+        .limit(SAMPLE_PIPELINE_DISPLAY_LIMIT).collect(Collectors.toSet());
+    if (!samplePipelines.isEmpty()) {
+      String samplePipelineText =
+          "Sample pipelines not satisfying the criteria : " + samplePipelines;
+      status = status.concat("\n").concat(samplePipelineText);
+    }
+    return status;
   }
 
   @Override
diff --git 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeExitRule.java
 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeExitRule.java
index 0d68cf624e..69c1a86ac3 100644
--- 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeExitRule.java
+++ 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeExitRule.java
@@ -38,6 +38,8 @@ public abstract class SafeModeExitRule<T> implements 
EventHandler<T> {
 
   private final SCMSafeModeManager safeModeManager;
   private final String ruleName;
+  protected static final int SAMPLE_CONTAINER_DISPLAY_LIMIT = 5;
+  protected static final int SAMPLE_PIPELINE_DISPLAY_LIMIT = 5;
 
   public SafeModeExitRule(SCMSafeModeManager safeModeManager,
       String ruleName, EventQueue eventQueue) {


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

Reply via email to