Browse Source

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 10 năm trước cách đây
mục cha
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;