You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2010/01/04 22:00:57 UTC

svn commit: r895780 - in /hadoop/hbase/trunk: ./ src/java/org/apache/hadoop/hbase/master/metrics/ src/java/org/apache/hadoop/hbase/metrics/ src/java/org/apache/hadoop/hbase/regionserver/metrics/ src/test/org/apache/hadoop/hbase/metrics/

Author: stack
Date: Mon Jan  4 21:00:57 2010
New Revision: 895780

URL: http://svn.apache.org/viewvc?rev=895780&view=rev
Log:
HBASE-2068 MetricsRate is missing registry parameter

Added:
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/metrics/MetricsMBeanBase.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/metrics/
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/metrics/TestMetricsMBeanBase.java
Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/metrics/MasterStatistics.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerStatistics.java

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=895780&r1=895779&r2=895780&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Mon Jan  4 21:00:57 2010
@@ -148,6 +148,8 @@
                (Lars George via JD)
    HBASE-2082  TableInputFormat is ignoring input scan's stop row setting
                (Scott Wang via Andrew Purtell)
+   HBASE-2068  MetricsRate is missing "registry" parameter
+               (Lars George and Gary Helmling via Stack)
 
   IMPROVEMENTS
    HBASE-1760  Cleanup TODOs in HTable

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/metrics/MasterStatistics.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/metrics/MasterStatistics.java?rev=895780&r1=895779&r2=895780&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/metrics/MasterStatistics.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/metrics/MasterStatistics.java Mon Jan  4 21:00:57 2010
@@ -19,15 +19,15 @@
 
 import javax.management.ObjectName;
 
+import org.apache.hadoop.hbase.metrics.MetricsMBeanBase;
 import org.apache.hadoop.metrics.util.MBeanUtil;
-import org.apache.hadoop.metrics.util.MetricsDynamicMBeanBase;
 import org.apache.hadoop.metrics.util.MetricsRegistry;
 
 /**
  * Exports the {@link MasterMetrics} statistics as an MBean
  * for JMX.
  */
-public class MasterStatistics extends MetricsDynamicMBeanBase {
+public class MasterStatistics extends MetricsMBeanBase {
   private final ObjectName mbeanName;
 
   public MasterStatistics(MetricsRegistry registry) {

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/metrics/MetricsMBeanBase.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/metrics/MetricsMBeanBase.java?rev=895780&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/metrics/MetricsMBeanBase.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/metrics/MetricsMBeanBase.java Mon Jan  4 21:00:57 2010
@@ -0,0 +1,145 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.metrics;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import javax.management.AttributeNotFoundException;
+import javax.management.MBeanAttributeInfo;
+import javax.management.MBeanException;
+import javax.management.MBeanInfo;
+import javax.management.ReflectionException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.metrics.util.MetricsBase;
+import org.apache.hadoop.metrics.util.MetricsDynamicMBeanBase;
+import org.apache.hadoop.metrics.util.MetricsRegistry;
+
+/**
+ * Extends the Hadoop MetricsDynamicMBeanBase class to provide JMX support for
+ * custom HBase MetricsBase implementations.  MetricsDynamicMBeanBase ignores 
+ * registered MetricsBase instance that are not instances of one of the 
+ * org.apache.hadoop.metrics.util implementations.
+ *
+ */
+public class MetricsMBeanBase extends MetricsDynamicMBeanBase {
+
+  private static final Log LOG = LogFactory.getLog("org.apache.hadoop.hbase.metrics");
+
+  protected final MetricsRegistry registry;
+  protected final String description;
+  protected int registryLength;
+  /** HBase MetricsBase implementations that MetricsDynamicMBeanBase does 
+   * not understand 
+   */
+  protected Map<String,MetricsBase> extendedAttributes = 
+      new HashMap<String,MetricsBase>();
+  protected MBeanInfo extendedInfo;
+  
+  protected MetricsMBeanBase( MetricsRegistry mr, String description ) {
+    super(mr, description);
+    this.registry = mr;
+    this.description = description;
+    this.init();
+  }
+
+  protected void init() {
+    List<MBeanAttributeInfo> attributes = new ArrayList<MBeanAttributeInfo>();
+    MBeanInfo parentInfo = super.getMBeanInfo();
+    List<String> parentAttributes = new ArrayList<String>();
+    for (MBeanAttributeInfo attr : parentInfo.getAttributes()) {
+      attributes.add(attr);
+      parentAttributes.add(attr.getName());
+    }
+    
+    this.registryLength = this.registry.getMetricsList().size();
+    
+    for (MetricsBase metric : this.registry.getMetricsList()) {
+      if (metric.getName() == null || parentAttributes.contains(metric.getName()))
+        continue;
+      
+      // add on custom HBase metric types
+      if (metric instanceof MetricsRate) {
+        attributes.add( new MBeanAttributeInfo(metric.getName(), 
+            "java.lang.Float", metric.getDescription(), true, false, false) );
+        extendedAttributes.put(metric.getName(), metric);
+      }  else {
+        LOG.error("unknown metrics instance: "+metric.getClass().getName());
+      }      
+    }
+
+    this.extendedInfo = new MBeanInfo( this.getClass().getName(), 
+        this.description, attributes.toArray( new MBeanAttributeInfo[0] ), 
+        parentInfo.getConstructors(), parentInfo.getOperations(), 
+        parentInfo.getNotifications() );
+  }
+
+  private void checkAndUpdateAttributes() {
+    if (this.registryLength != this.registry.getMetricsList().size()) 
+      this.init();
+  }
+  
+  @Override
+  public Object getAttribute( String name )
+      throws AttributeNotFoundException, MBeanException,
+      ReflectionException {
+    
+    if (name == null) {
+      throw new IllegalArgumentException("Attribute name is NULL");
+    }
+
+    /*
+     * Ugly.  Since MetricsDynamicMBeanBase implementation is private,
+     * we need to first check the parent class for the attribute.  
+     * In case that the MetricsRegistry contents have changed, this will
+     * allow the parent to update it's internal structures (which we rely on
+     * to update our own.
+     */
+    try {
+      return super.getAttribute(name);
+    } catch (AttributeNotFoundException ex) {
+      
+      checkAndUpdateAttributes();
+      
+      MetricsBase metric = this.extendedAttributes.get(name);
+      if (metric != null) {
+        if (metric instanceof MetricsRate) {
+          return ((MetricsRate) metric).getPreviousIntervalValue();
+        } else {
+          LOG.warn( String.format("unknown metrics type %s for attribute %s",
+                        metric.getClass().getName(), name) );
+        }
+      }
+    }
+    
+    throw new AttributeNotFoundException();
+  }
+
+  @Override
+  public MBeanInfo getMBeanInfo() {
+    return this.extendedInfo;
+  }
+
+}

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerStatistics.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerStatistics.java?rev=895780&r1=895779&r2=895780&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerStatistics.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerStatistics.java Mon Jan  4 21:00:57 2010
@@ -19,6 +19,7 @@
 
 import javax.management.ObjectName;
 
+import org.apache.hadoop.hbase.metrics.MetricsMBeanBase;
 import org.apache.hadoop.metrics.util.MBeanUtil;
 import org.apache.hadoop.metrics.util.MetricsDynamicMBeanBase;
 import org.apache.hadoop.metrics.util.MetricsRegistry;
@@ -27,7 +28,7 @@
  * Exports metrics recorded by {@link RegionServerMetrics} as an MBean
  * for JMX monitoring.
  */
-public class RegionServerStatistics extends MetricsDynamicMBeanBase {
+public class RegionServerStatistics extends MetricsMBeanBase {
 
   private final ObjectName mbeanName;
 

Added: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/metrics/TestMetricsMBeanBase.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/metrics/TestMetricsMBeanBase.java?rev=895780&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/metrics/TestMetricsMBeanBase.java (added)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/metrics/TestMetricsMBeanBase.java Mon Jan  4 21:00:57 2010
@@ -0,0 +1,121 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.metrics;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import javax.management.MBeanAttributeInfo;
+import javax.management.MBeanInfo;
+
+import org.apache.hadoop.metrics.MetricsContext;
+import org.apache.hadoop.metrics.MetricsRecord;
+import org.apache.hadoop.metrics.MetricsUtil;
+import org.apache.hadoop.metrics.util.MetricsIntValue;
+import org.apache.hadoop.metrics.util.MetricsRegistry;
+import org.apache.hadoop.metrics.util.MetricsTimeVaryingRate;
+
+import junit.framework.TestCase;
+
+public class TestMetricsMBeanBase extends TestCase {
+
+  private class TestStatistics extends MetricsMBeanBase {
+    public TestStatistics(MetricsRegistry registry) {
+      super(registry, "TestStatistics");
+    }
+  }
+  
+  private MetricsRegistry registry;
+  private MetricsRecord metricsRecord;
+  private TestStatistics stats;
+  private MetricsRate metricsRate;
+  private MetricsIntValue intValue;
+  private MetricsTimeVaryingRate varyRate;
+  
+  public void setUp() {
+    this.registry = new MetricsRegistry();
+    this.metricsRate = new MetricsRate("metricsRate", registry, "test");
+    this.intValue = new MetricsIntValue("intValue", registry, "test");
+    this.varyRate = new MetricsTimeVaryingRate("varyRate", registry, "test");
+    this.stats = new TestStatistics(registry);
+    MetricsContext context = MetricsUtil.getContext("hbase");
+    this.metricsRecord = MetricsUtil.createRecord(context, "test");
+    this.metricsRecord.setTag("TestStatistics", "test");
+    //context.registerUpdater(this);
+    
+  }
+  
+  public void tearDown() {
+    
+  }
+  
+  public void testGetAttribute() throws Exception {
+    this.metricsRate.inc(2);
+    this.metricsRate.pushMetric(this.metricsRecord);
+    this.intValue.set(5);
+    this.intValue.pushMetric(this.metricsRecord);
+    this.varyRate.inc(10);
+    this.varyRate.inc(50);
+    this.varyRate.pushMetric(this.metricsRecord);
+    
+    
+    assertEquals( 2.0, (Float)this.stats.getAttribute("metricsRate"), 0.001 );
+    assertEquals( 5, this.stats.getAttribute("intValue") );
+    assertEquals( 10L, this.stats.getAttribute("varyRateMinTime") );
+    assertEquals( 50L, this.stats.getAttribute("varyRateMaxTime") );
+    assertEquals( 30L, this.stats.getAttribute("varyRateAvgTime") );
+    assertEquals( 2, this.stats.getAttribute("varyRateNumOps") );
+  }
+  
+  public void testGetMBeanInfo() {
+    MBeanInfo info = this.stats.getMBeanInfo();
+    MBeanAttributeInfo[] attributes = info.getAttributes();
+    assertEquals( 6, attributes.length );
+    
+    Map<String,MBeanAttributeInfo> attributeByName = 
+        new HashMap<String,MBeanAttributeInfo>(attributes.length);
+    for (MBeanAttributeInfo attr : attributes)
+      attributeByName.put(attr.getName(), attr);
+    
+    assertAttribute( attributeByName.get("metricsRate"),
+        "metricsRate", "java.lang.Float", "test");
+    assertAttribute( attributeByName.get("intValue"),
+        "intValue", "java.lang.Integer", "test");
+    assertAttribute( attributeByName.get("varyRateMinTime"),
+        "varyRateMinTime", "java.lang.Long", "test");
+    assertAttribute( attributeByName.get("varyRateMaxTime"),
+        "varyRateMaxTime", "java.lang.Long", "test");
+    assertAttribute( attributeByName.get("varyRateAvgTime"),
+        "varyRateAvgTime", "java.lang.Long", "test");
+    assertAttribute( attributeByName.get("varyRateNumOps"),
+        "varyRateNumOps", "java.lang.Integer", "test");
+  }
+  
+  protected void assertAttribute(MBeanAttributeInfo attr, String name,
+      String type, String description) {
+    
+    assertEquals(attr.getName(), name);
+    assertEquals(attr.getType(), type);
+    assertEquals(attr.getDescription(), description);
+  }
+
+}