Prechádzať zdrojové kódy

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 rokov pred
rodič
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 static org.apache.hadoop.metrics2.source.JvmMetricsInfo.*;
 import static org.apache.hadoop.metrics2.impl.MsInfo.*;
+import org.apache.hadoop.util.JvmPauseMonitor;
 
 /**
  * JVM and logging related metrics.
@@ -65,6 +66,7 @@ public class JvmMetrics implements MetricsSource {
       ManagementFactory.getGarbageCollectorMXBeans();
   final ThreadMXBean threadMXBean = ManagementFactory.getThreadMXBean();
   final String processName, sessionId;
+  private JvmPauseMonitor pauseMonitor = null;
   final ConcurrentHashMap<String, MetricsInfo[]> gcInfoCache =
       new ConcurrentHashMap<String, MetricsInfo[]>();
 
@@ -73,6 +75,10 @@ public class JvmMetrics implements MetricsSource {
     this.sessionId = sessionId;
   }
 
+  public void setPauseMonitor(final JvmPauseMonitor pauseMonitor) {
+    this.pauseMonitor = pauseMonitor;
+  }
+
   public static JvmMetrics create(String processName, String sessionId,
                                   MetricsSystem ms) {
     return ms.register(JvmMetrics.name(), JvmMetrics.description(),
@@ -120,6 +126,15 @@ public class JvmMetrics implements MetricsSource {
     }
     rb.addCounter(GcCount, count)
       .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) {

+ 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"),
   LogError("Total number of error 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;
 

+ 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";
   private static final long INFO_THRESHOLD_DEFAULT = 1000;
 
-  
+  private long numGcWarnThresholdExceeded = 0;
+  private long numGcInfoThresholdExceeded = 0;
+  private long totalGcExtraSleepTime = 0;
+   
   private Thread monitorThread;
   private volatile boolean shouldRun = true;
-  
+
   public JvmPauseMonitor(Configuration conf) {
     this.warnThresholdMs = conf.getLong(WARN_THRESHOLD_KEY, WARN_THRESHOLD_DEFAULT);
     this.infoThresholdMs = conf.getLong(INFO_THRESHOLD_KEY, INFO_THRESHOLD_DEFAULT);
@@ -87,6 +90,22 @@ public class JvmPauseMonitor {
       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,
       Map<String, GcTimes> gcTimesAfterSleep,
@@ -166,13 +185,15 @@ public class JvmPauseMonitor {
         Map<String, GcTimes> gcTimesAfterSleep = getGcTimes();
 
         if (extraSleepTime > warnThresholdMs) {
+          ++numGcWarnThresholdExceeded;
           LOG.warn(formatMessage(
               extraSleepTime, gcTimesAfterSleep, gcTimesBeforeSleep));
         } else if (extraSleepTime > infoThresholdMs) {
+          ++numGcInfoThresholdExceeded;
           LOG.info(formatMessage(
               extraSleepTime, gcTimesAfterSleep, gcTimesBeforeSleep));
         }
-        
+        totalGcExtraSleepTime += extraSleepTime;
         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
 *-------------------------------------+--------------------------------------+
+|<<<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
 

+ 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;
 
 import org.junit.Test;
+
 import static org.mockito.Mockito.*;
 import static org.apache.hadoop.test.MetricsAsserts.*;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.metrics2.MetricsCollector;
 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.impl.MsInfo.*;
 
 public class TestJvmMetrics {
 
   @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();
 
     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-6403. Add metrics for log warnings reported by JVM pauses. (Yongjun
+    Zhang via atm)
+
   OPTIMIZATIONS
 
     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);
 
     metrics = DataNodeMetrics.create(conf, getDisplayName());
-
+    metrics.getJvmMetrics().setPauseMonitor(pauseMonitor);
+    
     blockPoolManager = new BlockPoolManager(this);
     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;
   @Metric MutableRate sendDataPacketTransferNanos;
   final MutableQuantiles[] sendDataPacketTransferNanosQuantiles;
-  
 
   final MetricsRegistry registry = new MetricsRegistry("datanode");
   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.jvmMetrics = jvmMetrics;    
     registry.tag(SessionId, sessionId);
     
     final int len = intervals.length;
@@ -131,7 +133,7 @@ public class DataNodeMetrics {
   public static DataNodeMetrics create(Configuration conf, String dnName) {
     String sessionId = conf.get(DFSConfigKeys.DFS_METRICS_SESSION_ID_KEY);
     MetricsSystem ms = DefaultMetricsSystem.instance();
-    JvmMetrics.create("DataNode", sessionId, ms);
+    JvmMetrics jm = JvmMetrics.create("DataNode", sessionId, ms);
     String name = "DataNodeActivity-"+ (dnName.isEmpty()
         ? "UndefinedDataNodeName"+ DFSUtil.getRandom().nextInt() 
             : dnName.replace(':', '-'));
@@ -141,11 +143,15 @@ public class DataNodeMetrics {
         conf.getInts(DFSConfigKeys.DFS_METRICS_PERCENTILES_INTERVALS_KEY);
     
     return ms.register(name, null, new DataNodeMetrics(name, sessionId,
-        intervals));
+        intervals, jm));
   }
 
   public String name() { return name; }
 
+  public JvmMetrics getJvmMetrics() {
+    return jvmMetrics;
+  }
+  
   public void addHeartbeat(long 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.start();
-
+    metrics.getJvmMetrics().setPauseMonitor(pauseMonitor);
+    
     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")
   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);
     
     final int len = intervals.length;
@@ -124,14 +128,19 @@ public class NameNodeMetrics {
     String sessionId = conf.get(DFSConfigKeys.DFS_METRICS_SESSION_ID_KEY);
     String processName = r.toString();
     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
     int[] intervals = 
         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() {
     DefaultMetricsSystem.shutdown();
   }