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

zhangduo pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2 by this push:
     new d07ed70ac3c HBASE-29431 Update the 'ExcludeDNs' information with the 
cause in RS UI (#7128)
d07ed70ac3c is described below

commit d07ed70ac3cc4d6cd37cec12b88cb32955dbfea6
Author: Sreenivasulu <[email protected]>
AuthorDate: Sun Aug 31 18:43:35 2025 +0530

    HBASE-29431 Update the 'ExcludeDNs' information with the cause in RS UI 
(#7128)
    
    Signed-off-by: Duo Zhang <[email protected]>
    Signed-off-by: Pankaj Kumar <[email protected]>
    Signed-off-by: Chandra Kambham <[email protected]>
---
 .../FanOutOneBlockAsyncDFSOutputHelper.java        |  3 ++-
 .../io/asyncfs/monitor/ExcludeDatanodeManager.java | 31 +++++++++++++++++++---
 .../io/asyncfs/monitor/StreamSlowMonitor.java      |  3 ++-
 .../MetricsRegionServerWrapperImpl.java            |  5 ++--
 4 files changed, 35 insertions(+), 7 deletions(-)

diff --git 
a/hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java
 
b/hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java
index b93768ae084..81716182f68 100644
--- 
a/hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java
+++ 
b/hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java
@@ -647,7 +647,8 @@ public final class FanOutOneBlockAsyncDFSOutputHelper {
           } catch (Exception e) {
             // exclude the broken DN next time
             toExcludeNodes.add(datanodeInfo);
-            excludeDatanodeManager.tryAddExcludeDN(datanodeInfo, "connect 
error");
+            excludeDatanodeManager.tryAddExcludeDN(datanodeInfo,
+              ExcludeDatanodeManager.ExcludeCause.CONNECT_ERROR.getCause());
             throw e;
           }
         }
diff --git 
a/hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/monitor/ExcludeDatanodeManager.java
 
b/hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/monitor/ExcludeDatanodeManager.java
index 61f75582a1c..7bed67a94be 100644
--- 
a/hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/monitor/ExcludeDatanodeManager.java
+++ 
b/hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/monitor/ExcludeDatanodeManager.java
@@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -52,7 +53,7 @@ public class ExcludeDatanodeManager implements 
ConfigurationObserver {
     "hbase.regionserver.async.wal.exclude.datanode.info.ttl.hour";
   public static final int DEFAULT_WAL_EXCLUDE_DATANODE_TTL = 6; // 6 hours
 
-  private volatile Cache<DatanodeInfo, Long> excludeDNsCache;
+  private volatile Cache<DatanodeInfo, Pair<String, Long>> excludeDNsCache;
   private final int maxExcludeDNCount;
   private final Configuration conf;
   // This is a map of providerId->StreamSlowMonitor
@@ -78,7 +79,7 @@ public class ExcludeDatanodeManager implements 
ConfigurationObserver {
   public boolean tryAddExcludeDN(DatanodeInfo datanodeInfo, String cause) {
     boolean alreadyMarkedSlow = getExcludeDNs().containsKey(datanodeInfo);
     if (!alreadyMarkedSlow) {
-      excludeDNsCache.put(datanodeInfo, EnvironmentEdgeManager.currentTime());
+      excludeDNsCache.put(datanodeInfo, new Pair<>(cause, 
EnvironmentEdgeManager.currentTime()));
       LOG.info(
         "Added datanode: {} to exclude cache by [{}] success, current 
excludeDNsCache size={}",
         datanodeInfo, cause, excludeDNsCache.size());
@@ -95,7 +96,31 @@ public class ExcludeDatanodeManager implements 
ConfigurationObserver {
     return streamSlowMonitors.computeIfAbsent(key, k -> new 
StreamSlowMonitor(conf, key, this));
   }
 
-  public Map<DatanodeInfo, Long> getExcludeDNs() {
+  /**
+   * Enumerates the reasons for excluding a datanode from certain operations. 
Each enum constant
+   * represents a specific cause leading to exclusion.
+   */
+  public enum ExcludeCause {
+    CONNECT_ERROR("connect error"),
+    SLOW_PACKET_ACK("slow packet ack");
+
+    private final String cause;
+
+    ExcludeCause(String cause) {
+      this.cause = cause;
+    }
+
+    public String getCause() {
+      return cause;
+    }
+
+    @Override
+    public String toString() {
+      return cause;
+    }
+  }
+
+  public Map<DatanodeInfo, Pair<String, Long>> getExcludeDNs() {
     return excludeDNsCache.asMap();
   }
 
diff --git 
a/hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/monitor/StreamSlowMonitor.java
 
b/hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/monitor/StreamSlowMonitor.java
index c415706aa6a..a4b80fc6456 100644
--- 
a/hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/monitor/StreamSlowMonitor.java
+++ 
b/hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/monitor/StreamSlowMonitor.java
@@ -156,7 +156,8 @@ public class StreamSlowMonitor implements 
ConfigurationObserver {
             + "lastAckTimestamp={}, monitor name: {}",
           datanodeInfo, packetDataLen, processTimeMs, unfinished, 
lastAckTimestamp, this.name);
         if (addSlowAckData(datanodeInfo, packetDataLen, processTimeMs)) {
-          excludeDatanodeManager.tryAddExcludeDN(datanodeInfo, "slow packet 
ack");
+          excludeDatanodeManager.tryAddExcludeDN(datanodeInfo,
+            ExcludeDatanodeManager.ExcludeCause.SLOW_PACKET_ACK.getCause());
         }
       }
     }
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
index 2bd396242a1..d52509e0808 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
@@ -433,8 +433,9 @@ class MetricsRegionServerWrapperImpl implements 
MetricsRegionServerWrapper {
     if (excludeDatanodeManager == null) {
       return Collections.emptyList();
     }
-    return excludeDatanodeManager.getExcludeDNs().entrySet().stream()
-      .map(e -> e.getKey().toString() + ", " + 
e.getValue()).collect(Collectors.toList());
+    return excludeDatanodeManager.getExcludeDNs().entrySet().stream().map(e -> 
e.getKey().toString()
+      + " - " + e.getValue().getSecond() + " - " + e.getValue().getFirst())
+      .collect(Collectors.toList());
   }
 
   @Override

Reply via email to