Explorar el Código

HADOOP-12348. MetricsSystemImpl creates MetricsSourceAdapter with wrong time unit parameter. (zxu via rkanter)

(cherry picked from commit 764f8baa99de35a0ced297b19b0cbe090c615a3b)
(cherry picked from commit d83395031761cb34ee50da5228c9404f0c6e9093)
Robert Kanter hace 9 años
padre
commit
f3863cdbd4

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

@@ -57,6 +57,9 @@ Release 2.6.5 - UNRELEASED
     HADOOP-13192. org.apache.hadoop.util.LineReader cannot handle multibyte
     delimiters correctly. (binde via aajisaka)
 
+    HADOOP-12348. MetricsSystemImpl creates MetricsSourceAdapter with wrong
+    time unit parameter. (zxu via rkanter)
+
 Release 2.6.4 - 2016-02-11
 
   INCOMPATIBLE CHANGES

+ 8 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSourceAdapter.java

@@ -61,7 +61,7 @@ class MetricsSourceAdapter implements DynamicMBean {
 
   private Iterable<MetricsRecordImpl> lastRecs;
   private long jmxCacheTS = 0;
-  private int jmxCacheTTL;
+  private long jmxCacheTTL;
   private MBeanInfo infoCache;
   private ObjectName mbeanName;
   private final boolean startMBeans;
@@ -69,7 +69,7 @@ class MetricsSourceAdapter implements DynamicMBean {
   MetricsSourceAdapter(String prefix, String name, String description,
                        MetricsSource source, Iterable<MetricsTag> injectedTags,
                        MetricsFilter recordFilter, MetricsFilter metricFilter,
-                       int jmxCacheTTL, boolean startMBeans) {
+                       long jmxCacheTTL, boolean startMBeans) {
     this.prefix = checkNotNull(prefix, "prefix");
     this.name = checkNotNull(name, "name");
     this.source = checkNotNull(source, "source");
@@ -84,7 +84,7 @@ class MetricsSourceAdapter implements DynamicMBean {
 
   MetricsSourceAdapter(String prefix, String name, String description,
                        MetricsSource source, Iterable<MetricsTag> injectedTags,
-                       int period, MetricsConfig conf) {
+                       long period, MetricsConfig conf) {
     this(prefix, name, description, source, injectedTags,
          conf.getFilter(RECORD_FILTER_KEY),
          conf.getFilter(METRIC_FILTER_KEY),
@@ -233,7 +233,11 @@ class MetricsSourceAdapter implements DynamicMBean {
     return mbeanName;
   }
 
-  
+  @VisibleForTesting
+  long getJmxCacheTTL() {
+    return jmxCacheTTL;
+  }
+
   private void updateInfoCache() {
     LOG.debug("Updating info cache...");
     infoCache = infoBuilder.reset(lastRecs).get();

+ 5 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java

@@ -260,11 +260,9 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
   void registerSource(String name, String desc, MetricsSource source) {
     checkNotNull(config, "config");
     MetricsConfig conf = sourceConfigs.get(name);
-    MetricsSourceAdapter sa = conf != null
-        ? new MetricsSourceAdapter(prefix, name, desc, source,
-                                   injectedTags, period, conf)
-        : new MetricsSourceAdapter(prefix, name, desc, source,
-          injectedTags, period, config.subset(SOURCE_KEY));
+    MetricsSourceAdapter sa = new MetricsSourceAdapter(prefix, name, desc,
+        source, injectedTags, period * 1000L, conf != null ? conf
+            : config.subset(SOURCE_KEY));
     sources.put(name, sa);
     sa.start();
     LOG.debug("Registered source "+ name);
@@ -361,7 +359,7 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
       return;
     }
     logicalTime = 0;
-    long millis = period * 1000;
+    long millis = period * 1000L;
     timer = new Timer("Timer for '"+ prefix +"' metrics system", true);
     timer.scheduleAtFixedRate(new TimerTask() {
           public void run() {
@@ -554,7 +552,7 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
   private void registerSystemSource() {
     MetricsConfig sysConf = sourceConfigs.get(MS_NAME);
     sysSource = new MetricsSourceAdapter(prefix, MS_STATS_NAME, MS_STATS_DESC,
-        MetricsAnnotations.makeSource(this), injectedTags, period,
+        MetricsAnnotations.makeSource(this), injectedTags, period * 1000L,
         sysConf == null ? config.subset(SOURCE_KEY) : sysConf);
     sysSource.start();
   }

+ 13 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestMetricsSystemImpl.java

@@ -494,6 +494,19 @@ public class TestMetricsSystemImpl {
     }
   }
 
+  @Test
+  public void testRegisterSourceJmxCacheTTL() {
+    MetricsSystem ms = new MetricsSystemImpl();
+    ms.init("TestMetricsSystem");
+    TestSource ts = new TestSource("ts");
+    ms.register(ts);
+    MetricsSourceAdapter sa = ((MetricsSystemImpl) ms)
+        .getSourceAdapter("TestSource");
+    assertEquals(MetricsConfig.PERIOD_DEFAULT * 1000 + 1,
+        sa.getJmxCacheTTL());
+    ms.shutdown();
+  }
+
   @Metrics(context="test")
   private static class TestSource {
     @Metric("C1 desc") MutableCounterLong c1;