Browse Source

HDFS-6403. Add metrics for log warnings reported by JVM pauses. Contributed by Yongjun Zhang.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1604074 13f79535-47bb-0310-9956-ffa450edef68
Aaron Myers 11 years ago
parent
commit
af6c91a80c

+ 15 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/source/JvmMetrics.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.lib.Interns;
 import org.apache.hadoop.metrics2.lib.Interns;
 import static org.apache.hadoop.metrics2.source.JvmMetricsInfo.*;
 import static org.apache.hadoop.metrics2.source.JvmMetricsInfo.*;
 import static org.apache.hadoop.metrics2.impl.MsInfo.*;
 import static org.apache.hadoop.metrics2.impl.MsInfo.*;
+import org.apache.hadoop.util.JvmPauseMonitor;
 
 
 /**
 /**
  * JVM and logging related metrics.
  * JVM and logging related metrics.
@@ -65,6 +66,7 @@ public class JvmMetrics implements MetricsSource {
       ManagementFactory.getGarbageCollectorMXBeans();
       ManagementFactory.getGarbageCollectorMXBeans();
   final ThreadMXBean threadMXBean = ManagementFactory.getThreadMXBean();
   final ThreadMXBean threadMXBean = ManagementFactory.getThreadMXBean();
   final String processName, sessionId;
   final String processName, sessionId;
+  private JvmPauseMonitor pauseMonitor = null;
   final ConcurrentHashMap<String, MetricsInfo[]> gcInfoCache =
   final ConcurrentHashMap<String, MetricsInfo[]> gcInfoCache =
       new ConcurrentHashMap<String, MetricsInfo[]>();
       new ConcurrentHashMap<String, MetricsInfo[]>();
 
 
@@ -73,6 +75,10 @@ public class JvmMetrics implements MetricsSource {
     this.sessionId = sessionId;
     this.sessionId = sessionId;
   }
   }
 
 
+  public void setPauseMonitor(final JvmPauseMonitor pauseMonitor) {
+    this.pauseMonitor = pauseMonitor;
+  }
+
   public static JvmMetrics create(String processName, String sessionId,
   public static JvmMetrics create(String processName, String sessionId,
                                   MetricsSystem ms) {
                                   MetricsSystem ms) {
     return ms.register(JvmMetrics.name(), JvmMetrics.description(),
     return ms.register(JvmMetrics.name(), JvmMetrics.description(),
@@ -120,6 +126,15 @@ public class JvmMetrics implements MetricsSource {
     }
     }
     rb.addCounter(GcCount, count)
     rb.addCounter(GcCount, count)
       .addCounter(GcTimeMillis, timeMillis);
       .addCounter(GcTimeMillis, timeMillis);
+    
+    if (pauseMonitor != null) {
+      rb.addCounter(GcNumWarnThresholdExceeded,
+          pauseMonitor.getNumGcWarnThreadholdExceeded());
+      rb.addCounter(GcNumInfoThresholdExceeded,
+          pauseMonitor.getNumGcInfoThresholdExceeded());
+      rb.addCounter(GcTotalExtraSleepTime,
+          pauseMonitor.getTotalGcExtraSleepTime());
+    }
   }
   }
 
 
   private MetricsInfo[] getGcInfo(String gcName) {
   private MetricsInfo[] getGcInfo(String gcName) {

+ 4 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/source/JvmMetricsInfo.java

@@ -48,7 +48,10 @@ public enum JvmMetricsInfo implements MetricsInfo {
   LogFatal("Total number of fatal log events"),
   LogFatal("Total number of fatal log events"),
   LogError("Total number of error log events"),
   LogError("Total number of error log events"),
   LogWarn("Total number of warning log events"),
   LogWarn("Total number of warning log events"),
-  LogInfo("Total number of info log events");
+  LogInfo("Total number of info log events"),
+  GcNumWarnThresholdExceeded("Number of times that the GC warn threshold is exceeded"),
+  GcNumInfoThresholdExceeded("Number of times that the GC info threshold is exceeded"),
+  GcTotalExtraSleepTime("Total GC extra sleep time in milliseconds");
 
 
   private final String desc;
   private final String desc;
 
 

+ 24 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JvmPauseMonitor.java

@@ -62,10 +62,13 @@ public class JvmPauseMonitor {
       "jvm.pause.info-threshold.ms";
       "jvm.pause.info-threshold.ms";
   private static final long INFO_THRESHOLD_DEFAULT = 1000;
   private static final long INFO_THRESHOLD_DEFAULT = 1000;
 
 
-  
+  private long numGcWarnThresholdExceeded = 0;
+  private long numGcInfoThresholdExceeded = 0;
+  private long totalGcExtraSleepTime = 0;
+   
   private Thread monitorThread;
   private Thread monitorThread;
   private volatile boolean shouldRun = true;
   private volatile boolean shouldRun = true;
-  
+
   public JvmPauseMonitor(Configuration conf) {
   public JvmPauseMonitor(Configuration conf) {
     this.warnThresholdMs = conf.getLong(WARN_THRESHOLD_KEY, WARN_THRESHOLD_DEFAULT);
     this.warnThresholdMs = conf.getLong(WARN_THRESHOLD_KEY, WARN_THRESHOLD_DEFAULT);
     this.infoThresholdMs = conf.getLong(INFO_THRESHOLD_KEY, INFO_THRESHOLD_DEFAULT);
     this.infoThresholdMs = conf.getLong(INFO_THRESHOLD_KEY, INFO_THRESHOLD_DEFAULT);
@@ -87,6 +90,22 @@ public class JvmPauseMonitor {
       Thread.currentThread().interrupt();
       Thread.currentThread().interrupt();
     }
     }
   }
   }
+
+  public boolean isStarted() {
+    return monitorThread != null;
+  }
+  
+  public long getNumGcWarnThreadholdExceeded() {
+    return numGcWarnThresholdExceeded;
+  }
+  
+  public long getNumGcInfoThresholdExceeded() {
+    return numGcInfoThresholdExceeded;
+  }
+  
+  public long getTotalGcExtraSleepTime() {
+    return totalGcExtraSleepTime;
+  }
   
   
   private String formatMessage(long extraSleepTime,
   private String formatMessage(long extraSleepTime,
       Map<String, GcTimes> gcTimesAfterSleep,
       Map<String, GcTimes> gcTimesAfterSleep,
@@ -166,13 +185,15 @@ public class JvmPauseMonitor {
         Map<String, GcTimes> gcTimesAfterSleep = getGcTimes();
         Map<String, GcTimes> gcTimesAfterSleep = getGcTimes();
 
 
         if (extraSleepTime > warnThresholdMs) {
         if (extraSleepTime > warnThresholdMs) {
+          ++numGcWarnThresholdExceeded;
           LOG.warn(formatMessage(
           LOG.warn(formatMessage(
               extraSleepTime, gcTimesAfterSleep, gcTimesBeforeSleep));
               extraSleepTime, gcTimesAfterSleep, gcTimesBeforeSleep));
         } else if (extraSleepTime > infoThresholdMs) {
         } else if (extraSleepTime > infoThresholdMs) {
+          ++numGcInfoThresholdExceeded;
           LOG.info(formatMessage(
           LOG.info(formatMessage(
               extraSleepTime, gcTimesAfterSleep, gcTimesBeforeSleep));
               extraSleepTime, gcTimesAfterSleep, gcTimesBeforeSleep));
         }
         }
-        
+        totalGcExtraSleepTime += extraSleepTime;
         gcTimesBeforeSleep = gcTimesAfterSleep;
         gcTimesBeforeSleep = gcTimesAfterSleep;
       }
       }
     }
     }

+ 8 - 0
hadoop-common-project/hadoop-common/src/site/apt/Metrics.apt.vm

@@ -86,6 +86,14 @@ jvm context
 *-------------------------------------+--------------------------------------+
 *-------------------------------------+--------------------------------------+
 |<<<LogInfo>>> | Total number of INFO logs
 |<<<LogInfo>>> | Total number of INFO logs
 *-------------------------------------+--------------------------------------+
 *-------------------------------------+--------------------------------------+
+|<<<GcNumWarnThresholdExceeded>>> | Number of times that the GC warn
+                                  | threshold is exceeded
+*-------------------------------------+--------------------------------------+
+|<<<GcNumInfoThresholdExceeded>>> | Number of times that the GC info
+                                  | threshold is exceeded
+*-------------------------------------+--------------------------------------+
+|<<<GcTotalExtraSleepTime>>> | Total GC extra sleep time in msec
+*-------------------------------------+--------------------------------------+
 
 
 rpc context
 rpc context
 
 

+ 8 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/source/TestJvmMetrics.java

@@ -19,18 +19,25 @@
 package org.apache.hadoop.metrics2.source;
 package org.apache.hadoop.metrics2.source;
 
 
 import org.junit.Test;
 import org.junit.Test;
+
 import static org.mockito.Mockito.*;
 import static org.mockito.Mockito.*;
 import static org.apache.hadoop.test.MetricsAsserts.*;
 import static org.apache.hadoop.test.MetricsAsserts.*;
 
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.metrics2.MetricsCollector;
 import org.apache.hadoop.metrics2.MetricsCollector;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.util.JvmPauseMonitor;
+
 import static org.apache.hadoop.metrics2.source.JvmMetricsInfo.*;
 import static org.apache.hadoop.metrics2.source.JvmMetricsInfo.*;
 import static org.apache.hadoop.metrics2.impl.MsInfo.*;
 import static org.apache.hadoop.metrics2.impl.MsInfo.*;
 
 
 public class TestJvmMetrics {
 public class TestJvmMetrics {
 
 
   @Test public void testPresence() {
   @Test public void testPresence() {
-    MetricsRecordBuilder rb = getMetrics(new JvmMetrics("test", "test"));
+    JvmPauseMonitor pauseMonitor = new JvmPauseMonitor(new Configuration());
+    JvmMetrics jvmMetrics = new JvmMetrics("test", "test");
+    jvmMetrics.setPauseMonitor(pauseMonitor);
+    MetricsRecordBuilder rb = getMetrics(jvmMetrics);
     MetricsCollector mc = rb.parent();
     MetricsCollector mc = rb.parent();
 
 
     verify(mc).addRecord(JvmMetrics);
     verify(mc).addRecord(JvmMetrics);

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -453,6 +453,9 @@ Release 2.5.0 - UNRELEASED
 
 
     HDFS-6480. Move waitForReady() from FSDirectory to FSNamesystem. (wheat9)
     HDFS-6480. Move waitForReady() from FSDirectory to FSNamesystem. (wheat9)
 
 
+    HDFS-6403. Add metrics for log warnings reported by JVM pauses. (Yongjun
+    Zhang via atm)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-6214. Webhdfs has poor throughput for files >2GB (daryn)
     HDFS-6214. Webhdfs has poor throughput for files >2GB (daryn)

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -778,7 +778,8 @@ public class DataNode extends Configured
     initIpcServer(conf);
     initIpcServer(conf);
 
 
     metrics = DataNodeMetrics.create(conf, getDisplayName());
     metrics = DataNodeMetrics.create(conf, getDisplayName());
-
+    metrics.getJvmMetrics().setPauseMonitor(pauseMonitor);
+    
     blockPoolManager = new BlockPoolManager(this);
     blockPoolManager = new BlockPoolManager(this);
     blockPoolManager.refreshNamenodes(conf);
     blockPoolManager.refreshNamenodes(conf);
 
 

+ 11 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java

@@ -90,13 +90,15 @@ public class DataNodeMetrics {
   final MutableQuantiles[] sendDataPacketBlockedOnNetworkNanosQuantiles;
   final MutableQuantiles[] sendDataPacketBlockedOnNetworkNanosQuantiles;
   @Metric MutableRate sendDataPacketTransferNanos;
   @Metric MutableRate sendDataPacketTransferNanos;
   final MutableQuantiles[] sendDataPacketTransferNanosQuantiles;
   final MutableQuantiles[] sendDataPacketTransferNanosQuantiles;
-  
 
 
   final MetricsRegistry registry = new MetricsRegistry("datanode");
   final MetricsRegistry registry = new MetricsRegistry("datanode");
   final String name;
   final String name;
-
-  public DataNodeMetrics(String name, String sessionId, int[] intervals) {
+  JvmMetrics jvmMetrics = null;
+  
+  public DataNodeMetrics(String name, String sessionId, int[] intervals,
+      final JvmMetrics jvmMetrics) {
     this.name = name;
     this.name = name;
+    this.jvmMetrics = jvmMetrics;    
     registry.tag(SessionId, sessionId);
     registry.tag(SessionId, sessionId);
     
     
     final int len = intervals.length;
     final int len = intervals.length;
@@ -131,7 +133,7 @@ public class DataNodeMetrics {
   public static DataNodeMetrics create(Configuration conf, String dnName) {
   public static DataNodeMetrics create(Configuration conf, String dnName) {
     String sessionId = conf.get(DFSConfigKeys.DFS_METRICS_SESSION_ID_KEY);
     String sessionId = conf.get(DFSConfigKeys.DFS_METRICS_SESSION_ID_KEY);
     MetricsSystem ms = DefaultMetricsSystem.instance();
     MetricsSystem ms = DefaultMetricsSystem.instance();
-    JvmMetrics.create("DataNode", sessionId, ms);
+    JvmMetrics jm = JvmMetrics.create("DataNode", sessionId, ms);
     String name = "DataNodeActivity-"+ (dnName.isEmpty()
     String name = "DataNodeActivity-"+ (dnName.isEmpty()
         ? "UndefinedDataNodeName"+ DFSUtil.getRandom().nextInt() 
         ? "UndefinedDataNodeName"+ DFSUtil.getRandom().nextInt() 
             : dnName.replace(':', '-'));
             : dnName.replace(':', '-'));
@@ -141,11 +143,15 @@ public class DataNodeMetrics {
         conf.getInts(DFSConfigKeys.DFS_METRICS_PERCENTILES_INTERVALS_KEY);
         conf.getInts(DFSConfigKeys.DFS_METRICS_PERCENTILES_INTERVALS_KEY);
     
     
     return ms.register(name, null, new DataNodeMetrics(name, sessionId,
     return ms.register(name, null, new DataNodeMetrics(name, sessionId,
-        intervals));
+        intervals, jm));
   }
   }
 
 
   public String name() { return name; }
   public String name() { return name; }
 
 
+  public JvmMetrics getJvmMetrics() {
+    return jvmMetrics;
+  }
+  
   public void addHeartbeat(long latency) {
   public void addHeartbeat(long latency) {
     heartbeats.add(latency);
     heartbeats.add(latency);
   }
   }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -598,7 +598,8 @@ public class NameNode implements NameNodeStatusMXBean {
     
     
     pauseMonitor = new JvmPauseMonitor(conf);
     pauseMonitor = new JvmPauseMonitor(conf);
     pauseMonitor.start();
     pauseMonitor.start();
-
+    metrics.getJvmMetrics().setPauseMonitor(pauseMonitor);
+    
     startCommonServices(conf);
     startCommonServices(conf);
   }
   }
   
   

+ 12 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java

@@ -98,7 +98,11 @@ public class NameNodeMetrics {
   @Metric("GetImageServlet putImage")
   @Metric("GetImageServlet putImage")
   MutableRate putImage;
   MutableRate putImage;
 
 
-  NameNodeMetrics(String processName, String sessionId, int[] intervals) {
+  JvmMetrics jvmMetrics = null;
+  
+  NameNodeMetrics(String processName, String sessionId, int[] intervals,
+      final JvmMetrics jvmMetrics) {
+    this.jvmMetrics = jvmMetrics;
     registry.tag(ProcessName, processName).tag(SessionId, sessionId);
     registry.tag(ProcessName, processName).tag(SessionId, sessionId);
     
     
     final int len = intervals.length;
     final int len = intervals.length;
@@ -124,14 +128,19 @@ public class NameNodeMetrics {
     String sessionId = conf.get(DFSConfigKeys.DFS_METRICS_SESSION_ID_KEY);
     String sessionId = conf.get(DFSConfigKeys.DFS_METRICS_SESSION_ID_KEY);
     String processName = r.toString();
     String processName = r.toString();
     MetricsSystem ms = DefaultMetricsSystem.instance();
     MetricsSystem ms = DefaultMetricsSystem.instance();
-    JvmMetrics.create(processName, sessionId, ms);
+    JvmMetrics jm = JvmMetrics.create(processName, sessionId, ms);
     
     
     // Percentile measurement is off by default, by watching no intervals
     // Percentile measurement is off by default, by watching no intervals
     int[] intervals = 
     int[] intervals = 
         conf.getInts(DFSConfigKeys.DFS_METRICS_PERCENTILES_INTERVALS_KEY);
         conf.getInts(DFSConfigKeys.DFS_METRICS_PERCENTILES_INTERVALS_KEY);
-    return ms.register(new NameNodeMetrics(processName, sessionId, intervals));
+    return ms.register(new NameNodeMetrics(processName, sessionId,
+        intervals, jm));
   }
   }
 
 
+  public JvmMetrics getJvmMetrics() {
+    return jvmMetrics;
+  }
+  
   public void shutdown() {
   public void shutdown() {
     DefaultMetricsSystem.shutdown();
     DefaultMetricsSystem.shutdown();
   }
   }