You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2012/08/02 18:58:37 UTC

svn commit: r1368598 [1/2] - in /hbase/trunk: conf/ hbase-hadoop-compat/ hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/ hbase-hadoop-compat/src/main/java/org/apache/hadoop/hb...

Author: tedyu
Date: Thu Aug  2 16:58:35 2012
New Revision: 1368598

URL: http://svn.apache.org/viewvc?rev=1368598&view=rev
Log:
HBASE-6411 Move Master Metrics to metrics 2 (Alex Baranau)


Added:
    hbase/trunk/conf/hadoop-metrics2.properties
    hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/CompatibilitySingletonFactory.java
    hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/
    hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/metrics/
    hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSource.java
    hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/MBeanSource.java
    hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/master/
    hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/
    hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceFactoryTest.java
    hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/
    hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/metrics/
    hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceImpl.java
    hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/metrics/MBeanSourceImpl.java
    hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/
    hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/lib/
    hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java
    hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.metrics.MasterMetricsSource
    hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.metrics.MBeanSource
    hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/master/
    hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/
    hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceImplTest.java
    hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/
    hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/metrics/
    hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceImpl.java
    hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/MBeanSourceImpl.java
    hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java
    hbase/trunk/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.metrics.MasterMetricsSource
    hbase/trunk/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.metrics.MBeanSource
    hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/master/
    hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/
    hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceImplTest.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/metrics/MXBean.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/metrics/MXBeanImpl.java
Removed:
    hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceFactory.java
    hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/HBaseMetricsFactory.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MXBean.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MXBeanImpl.java
Modified:
    hbase/trunk/hbase-hadoop-compat/pom.xml
    hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSource.java
    hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceFactoryTest.java
    hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseMetricsSourceImpl.java
    hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceImpl.java
    hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/metrics/BaseMetricsSourceImplTest.java
    hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceImplTest.java
    hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseMetricsSourceImpl.java
    hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceImpl.java
    hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/BaseMetricsSourceImplTest.java
    hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceImplTest.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterMetrics.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationSinkMetrics.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationSourceMetrics.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMXBean.java

Added: hbase/trunk/conf/hadoop-metrics2.properties
URL: http://svn.apache.org/viewvc/hbase/trunk/conf/hadoop-metrics2.properties?rev=1368598&view=auto
==============================================================================
--- hbase/trunk/conf/hadoop-metrics2.properties (added)
+++ hbase/trunk/conf/hadoop-metrics2.properties Thu Aug  2 16:58:35 2012
@@ -0,0 +1,11 @@
+# syntax: [prefix].[source|sink].[instance].[options]
+# See javadoc of package-info.java for org.apache.hadoop.metrics2 for details
+
+*.sink.file.class=org.apache.hadoop.metrics2.sink.FileSink
+# default sampling period
+*.period=10
+# syntax: [prefix].[source|sink|jmx].[instance].[options]
+# See package.html for org.apache.hadoop.metrics2 for details
+
+*.sink.file.class=org.apache.hadoop.metrics2.sink.FileSink
+

Modified: hbase/trunk/hbase-hadoop-compat/pom.xml
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop-compat/pom.xml?rev=1368598&r1=1368597&r2=1368598&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop-compat/pom.xml (original)
+++ hbase/trunk/hbase-hadoop-compat/pom.xml Thu Aug  2 16:58:35 2012
@@ -58,6 +58,11 @@
     </build>
 
     <dependencies>
+      <!-- General dependencies -->
+      <dependency>
+        <groupId>commons-logging</groupId>
+        <artifactId>commons-logging</artifactId>
+      </dependency>
     </dependencies>
 
 </project>

Added: hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/CompatibilitySingletonFactory.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/CompatibilitySingletonFactory.java?rev=1368598&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/CompatibilitySingletonFactory.java (added)
+++ hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/CompatibilitySingletonFactory.java Thu Aug  2 16:58:35 2012
@@ -0,0 +1,83 @@
+/**
+ * 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;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.master.metrics.MasterMetricsSource;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+/**
+ *  Factory for classes supplied by hadoop compatibility modules.
+ */
+public class CompatibilitySingletonFactory {
+  private static final Log LOG = LogFactory.getLog(CompatibilitySingletonFactory.class);
+  public static final String EXCEPTION_START = "Could not create  ";
+  public static final String EXCEPTION_END = " Is the hadoop compatibility jar on the classpath?";
+
+  private static final Map<Class, Object> instances = new HashMap<Class, Object>();
+
+  /**
+   * Get the singleton instance of Any classes defined by compatibiliy jar's
+   *
+   * @return the singleton
+   */
+  public static synchronized <T> T getInstance(Class<T> klass) {
+    T instance = (T) instances.get(klass);
+    if (instance == null) {
+      try {
+        ServiceLoader<T> loader = ServiceLoader.load(klass);
+        Iterator<T> it = loader.iterator();
+        instance = it.next();
+        if (it.hasNext()) {
+          StringBuilder msg = new StringBuilder();
+          msg.append("ServiceLoader provided more than one implementation for class: ")
+                  .append(klass)
+                  .append(", using implementation: ").append(instance.getClass())
+                  .append(", other implementations: {");
+          while (it.hasNext()) {
+            msg.append(it.next()).append(" ");
+          }
+          msg.append("}");
+          LOG.warn(msg);
+        }
+      } catch (Exception e) {
+        throw new RuntimeException(createExceptionString(klass), e);
+      } catch (Error e) {
+        throw new RuntimeException(createExceptionString(klass), e);
+      }
+
+      // If there was nothing returned and no exception then throw an exception.
+      if (instance == null) {
+        throw new RuntimeException(createExceptionString(klass));
+      }
+      instances.put(klass, instance);
+    }
+    return instance;
+  }
+
+  private static String createExceptionString(Class klass) {
+     return EXCEPTION_START + klass.toString() + EXCEPTION_END;
+  }
+
+}

Added: hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSource.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSource.java?rev=1368598&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSource.java (added)
+++ hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSource.java Thu Aug  2 16:58:35 2012
@@ -0,0 +1,67 @@
+/**
+ * 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.master.metrics;
+
+import org.apache.hadoop.hbase.metrics.BaseMetricsSource;
+
+/**
+ * Interface that classes that expose metrics about the master will implement.
+ */
+public interface MasterMetricsSource extends BaseMetricsSource {
+
+  /**
+   * The name of the metrics
+   */
+  public static final String METRICS_NAME = "HMaster";
+
+  /**
+   * The name of the metrics context that metrics will be under.
+   */
+  public static final String METRICS_CONTEXT = "HMaster,sub=Dynamic";
+
+  /**
+   * Description
+   */
+  public static final String METRICS_DESCRIPTION = "Metrics about HBase master server";
+
+  /**
+   * Increment the number of requests the cluster has seen.
+   * @param inc Ammount to increment the total by.
+   */
+  public void incRequests(final int inc);
+
+  /**
+   * Set the number of regions in transition.
+   * @param ritCount count of the regions in transition.
+   */
+  public void setRIT(int ritCount);
+
+  /**
+   * Set the count of the number of regions that have been in transition over the threshold time.
+   * @param ritCountOverThreshold number of regions in transition for longer than threshold.
+   */
+  public void setRITCountOverThreshold(int ritCountOverThreshold);
+
+  /**
+   * Set the oldest region in transition.
+   * @param age age of the oldest RIT.
+   */
+  public void setRITOldestAge(long age);
+
+}

Added: hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/MBeanSource.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/MBeanSource.java?rev=1368598&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/MBeanSource.java (added)
+++ hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/MBeanSource.java Thu Aug  2 16:58:35 2012
@@ -0,0 +1,38 @@
+/**
+ * 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 javax.management.ObjectName;
+
+/**
+ *  Object that will register an mbean with the underlying metrics implementation.
+ */
+public interface MBeanSource  {
+
+  /**
+   * Register an mbean with the underlying metrics system
+   * @param serviceName Metrics service/system name
+   * @param metricsName name of the metrics object to expose
+   * @param theMbean the actual MBean
+   * @return ObjectName from jmx
+   */
+  public ObjectName register(String serviceName, String metricsName,
+                             Object theMbean);
+
+}

Modified: hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSource.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSource.java?rev=1368598&r1=1368597&r2=1368598&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSource.java (original)
+++ hbase/trunk/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSource.java Thu Aug  2 16:58:35 2012
@@ -25,6 +25,19 @@ import org.apache.hadoop.hbase.metrics.B
  * hadoop2's metrics2 classes and publishing.
  */
 public interface ReplicationMetricsSource extends BaseMetricsSource {
-  //Empty interface so that ServiceLoader can find the right implementation.
+  /**
+   * The name of the metrics
+   */
+  public static final String METRICS_NAME = "ReplicationMetrics";
+
+  /**
+   * The name of the metrics context that metrics will be under.
+   */
+  public static final String METRICS_CONTEXT = "replicationmetrics";
+
+  /**
+   * A description.
+   */
+  public static final String METRICS_DESCRIPTION = "Metrics about HBase replication";
 
 }

Added: hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceFactoryTest.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceFactoryTest.java?rev=1368598&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceFactoryTest.java (added)
+++ hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceFactoryTest.java Thu Aug  2 16:58:35 2012
@@ -0,0 +1,35 @@
+/**
+ * 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.master.metrics;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.junit.Test;
+
+/**
+ *  Test for the CompatibilitySingletonFactory and building MasterMetricsSource
+ */
+public class MasterMetricsSourceFactoryTest {
+
+  @Test(expected=RuntimeException.class)
+  public void testGetInstanceNoHadoopCompat() throws Exception {
+    //This should throw an exception because there is no compat lib on the class path.
+    CompatibilitySingletonFactory.getInstance(MasterMetricsSource.class);
+
+  }
+}

Modified: hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceFactoryTest.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceFactoryTest.java?rev=1368598&r1=1368597&r2=1368598&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceFactoryTest.java (original)
+++ hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceFactoryTest.java Thu Aug  2 16:58:35 2012
@@ -18,17 +18,17 @@
 
 package org.apache.hadoop.hbase.replication.regionserver.metrics;
 
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
 import org.junit.Test;
 
 /**
- *  Test for the ReplicationMetricsSourceFactory
+ *  Test for the CompatibilitySingletonFactory and building ReplicationMetricsSource
  */
 public class ReplicationMetricsSourceFactoryTest {
 
   @Test(expected=RuntimeException.class)
   public void testGetInstanceNoHadoopCompat() throws Exception {
     //This should throw an exception because there is no compat lib on the class path.
-    ReplicationMetricsSourceFactory.getInstance();
-
+    CompatibilitySingletonFactory.getInstance(ReplicationMetricsSource.class);
   }
 }

Added: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceImpl.java?rev=1368598&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceImpl.java Thu Aug  2 16:58:35 2012
@@ -0,0 +1,67 @@
+/**
+ * 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.master.metrics;
+
+import org.apache.hadoop.hbase.metrics.BaseMetricsSourceImpl;
+import org.apache.hadoop.metrics2.lib.MetricMutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MetricMutableGaugeLong;
+
+/**
+ * Hadoop1 implementation of MasterMetricsSource.
+ */
+public class MasterMetricsSourceImpl
+        extends BaseMetricsSourceImpl implements MasterMetricsSource {
+
+  MetricMutableCounterLong clusterRequestsCounter;
+  MetricMutableGaugeLong ritGauge;
+  MetricMutableGaugeLong ritCountOverThresholdGauge;
+  MetricMutableGaugeLong ritOldestAgeGauge;
+
+
+  public MasterMetricsSourceImpl() {
+    this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT);
+  }
+
+  public MasterMetricsSourceImpl(String metricsName,
+                                 String metricsDescription,
+                                 String metricsContext) {
+    super(metricsName, metricsDescription, metricsContext);
+
+    clusterRequestsCounter = getLongCounter("cluster_requests", 0);
+    ritGauge = getLongGauge("ritCount", 0);
+    ritCountOverThresholdGauge = getLongGauge("ritCountOverThreshold", 0);
+    ritOldestAgeGauge = getLongGauge("ritOldestAge", 0);
+  }
+
+  public void incRequests(final int inc) {
+    this.clusterRequestsCounter.incr(inc);
+  }
+
+  public void setRIT(int ritCount) {
+    ritGauge.set(ritCount);
+  }
+
+  public void setRITCountOverThreshold(int ritCount) {
+    ritCountOverThresholdGauge.set(ritCount);
+  }
+
+  public void setRITOldestAge(long ritCount) {
+    ritCountOverThresholdGauge.set(ritCount);
+  }
+}

Modified: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseMetricsSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseMetricsSourceImpl.java?rev=1368598&r1=1368597&r2=1368598&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseMetricsSourceImpl.java (original)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseMetricsSourceImpl.java Thu Aug  2 16:58:35 2012
@@ -19,49 +19,73 @@
 package org.apache.hadoop.hbase.metrics;
 
 import org.apache.hadoop.metrics2.MetricsBuilder;
+import org.apache.hadoop.metrics2.MetricsException;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.MetricsSource;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MetricMutable;
 import org.apache.hadoop.metrics2.lib.MetricMutableCounterLong;
 import org.apache.hadoop.metrics2.lib.MetricMutableGaugeLong;
+import org.apache.hadoop.metrics2.source.JvmMetricsSource;
 
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
 /**
- * Hadoop 1 implementation of BaseMetricsSource
+ * Hadoop 1 implementation of BaseMetricsSource (using metrics2 framework)
  */
 public class BaseMetricsSourceImpl implements BaseMetricsSource, MetricsSource {
 
   private static boolean defaultMetricsSystemInited = false;
   public static final String HBASE_METRICS_SYSTEM_NAME = "hbase";
 
-  public ConcurrentMap<String, MetricMutableGaugeLong>
-      gauges = new ConcurrentHashMap<String, MetricMutableGaugeLong>();
-  public ConcurrentMap<String, MetricMutableCounterLong> counters =
-      new ConcurrentHashMap<String, MetricMutableCounterLong>();
-
-  protected String metricsContext;
-  protected String metricsName;
-  protected String metricsDescription;
+  final DynamicMetricsRegistry metricsRegistry;
+
+  private JvmMetricsSource jvmMetricsSource;
 
   public BaseMetricsSourceImpl(
       String metricsName,
       String metricsDescription,
       String metricsContext) {
-    this.metricsContext = metricsContext;
-    this.metricsName = metricsName;
-    this.metricsDescription = metricsDescription;
+
+    metricsRegistry = new DynamicMetricsRegistry(metricsName).setContext(metricsContext);
 
     if (!defaultMetricsSystemInited) {
       //Not too worried about mutli-threaded here as all it does is spam the logs.
       defaultMetricsSystemInited = true;
       DefaultMetricsSystem.initialize(HBASE_METRICS_SYSTEM_NAME);
+
+      //If this is the first time through register a jvm source.
+      jvmMetricsSource = JvmMetricsSource.create(metricsName, "");
     }
 
     //Register this instance.
-    DefaultMetricsSystem.registerSource(this.metricsContext, this.metricsDescription, this);
+    DefaultMetricsSystem.INSTANCE.registerSource(metricsContext, metricsDescription, this);
+  }
+
+  /**
+   * Get a MetricMutableGaugeLong from the storage.  If it is not there atomically put it.
+   *
+   * @param gaugeName              name of the gauge to create or get.
+   * @param potentialStartingValue value of the new gauge if we have to create it.
+   * @return a metric object
+   */
+  protected MetricMutableGaugeLong getLongGauge(String gaugeName, long potentialStartingValue) {
+    return metricsRegistry.getLongGauge(gaugeName, potentialStartingValue);
+  }
+
+  /**
+   * Get a MetricMutableCounterLong from the storage.  If it is not there atomically put it.
+   *
+   * @param counterName            Name of the counter to get
+   * @param potentialStartingValue starting value if we have to create a new counter
+   * @return a metric object
+   */
+  protected MetricMutableCounterLong getLongCounter(String counterName,
+                                                    long potentialStartingValue) {
+    return metricsRegistry.getLongCounter(counterName, potentialStartingValue);
   }
 
   /**
@@ -71,7 +95,7 @@ public class BaseMetricsSourceImpl imple
    * @param value     the new value of the gauge.
    */
   public void setGauge(String gaugeName, long value) {
-    MetricMutableGaugeLong gaugeInt = getLongGauge(gaugeName, value);
+    MetricMutableGaugeLong gaugeInt = metricsRegistry.getLongGauge(gaugeName, value);
     gaugeInt.set(value);
   }
 
@@ -82,7 +106,7 @@ public class BaseMetricsSourceImpl imple
    * @param delta     The amount to increment the gauge by.
    */
   public void incGauge(String gaugeName, long delta) {
-    MetricMutableGaugeLong gaugeInt = getLongGauge(gaugeName, 0l);
+    MetricMutableGaugeLong gaugeInt = metricsRegistry.getLongGauge(gaugeName, 0l);
     gaugeInt.incr(delta);
   }
 
@@ -93,7 +117,7 @@ public class BaseMetricsSourceImpl imple
    * @param delta     the ammount to subtract from a gauge value.
    */
   public void decGauge(String gaugeName, long delta) {
-    MetricMutableGaugeLong gaugeInt = getLongGauge(gaugeName, 0l);
+    MetricMutableGaugeLong gaugeInt = metricsRegistry.getLongGauge(gaugeName, 0l);
     gaugeInt.decr(delta);
   }
 
@@ -104,7 +128,7 @@ public class BaseMetricsSourceImpl imple
    * @param delta the ammount to increment
    */
   public void incCounters(String key, long delta) {
-    MetricMutableCounterLong counter = getLongCounter(key, 0l);
+    MetricMutableCounterLong counter = metricsRegistry.getLongCounter(key, 0l);
     counter.incr(delta);
 
   }
@@ -115,7 +139,7 @@ public class BaseMetricsSourceImpl imple
    * @param key
    */
   public void removeGauge(String key) {
-    gauges.remove(key);
+    metricsRegistry.removeMetric(key);
   }
 
   /**
@@ -124,7 +148,7 @@ public class BaseMetricsSourceImpl imple
    * @param key
    */
   public void removeCounter(String key) {
-    counters.remove(key);
+    metricsRegistry.removeMetric(key);
   }
 
   /**
@@ -135,67 +159,6 @@ public class BaseMetricsSourceImpl imple
    */
   @Override
   public void getMetrics(MetricsBuilder metricsBuilder, boolean all) {
-
-    MetricsRecordBuilder rb = metricsBuilder.addRecord(metricsName).setContext(metricsContext);
-
-    for (Map.Entry<String, MetricMutableCounterLong> entry : counters.entrySet()) {
-      entry.getValue().snapshot(rb, all);
-    }
-    for (Map.Entry<String, MetricMutableGaugeLong> entry : gauges.entrySet()) {
-      entry.getValue().snapshot(rb, all);
-    }
-
-  }
-
-  /**
-   * Get a MetricMutableGaugeLong from the storage.  If it is not there atomically put it.
-   *
-   * @param gaugeName              name of the gauge to create or get.
-   * @param potentialStartingValue value of the new counter if we have to create it.
-   * @return
-   */
-  private MetricMutableGaugeLong getLongGauge(String gaugeName, long potentialStartingValue) {
-    //Try and get the guage.
-    MetricMutableGaugeLong gauge = gauges.get(gaugeName);
-
-    //If it's not there then try and put a new one in the storage.
-    if (gauge == null) {
-
-      //Create the potential new gauge.
-      MetricMutableGaugeLong newGauge = new MetricMutableGaugeLong(gaugeName, "",
-          potentialStartingValue);
-
-      // Try and put the gauge in.  This is atomic.
-      gauge = gauges.putIfAbsent(gaugeName, newGauge);
-
-      //If the value we get back is null then the put was successful and we will return that.
-      //otherwise gaugeLong should contain the thing that was in before the put could be completed.
-      if (gauge == null) {
-        gauge = newGauge;
-      }
-    }
-    return gauge;
+    metricsRegistry.snapshot(metricsBuilder.addRecord(metricsRegistry.name()), all);
   }
-
-  /**
-   * Get a MetricMutableCounterLong from the storage.  If it is not there atomically put it.
-   *
-   * @param counterName            Name of the counter to get
-   * @param potentialStartingValue starting value if we have to create a new counter
-   * @return
-   */
-  private MetricMutableCounterLong getLongCounter(String counterName, long potentialStartingValue) {
-    //See getLongGauge for description on how this works.
-    MetricMutableCounterLong counter = counters.get(counterName);
-    if (counter == null) {
-      MetricMutableCounterLong newCounter =
-          new MetricMutableCounterLong(counterName, "", potentialStartingValue);
-      counter = counters.putIfAbsent(counterName, newCounter);
-      if (counter == null) {
-        counter = newCounter;
-      }
-    }
-    return counter;
-  }
-
 }

Added: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/metrics/MBeanSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/metrics/MBeanSourceImpl.java?rev=1368598&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/metrics/MBeanSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/metrics/MBeanSourceImpl.java Thu Aug  2 16:58:35 2012
@@ -0,0 +1,41 @@
+/**
+ * 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 org.apache.hadoop.metrics2.util.MBeans;
+
+import javax.management.ObjectName;
+
+/**
+ * Hadoop1 metrics2 implementation of an object that registers MBeans.
+ */
+public class MBeanSourceImpl implements MBeanSource {
+
+  /**
+   * Register an mbean with the underlying metrics system
+   * @param serviceName Metrics service/system name
+   * @param metricsName name of the metrics obejct to expose
+   * @param theMbean the actual MBean
+   * @return ObjectName from jmx
+   */
+  @Override
+  public ObjectName register(String serviceName, String metricsName, Object theMbean) {
+    return MBeans.register(serviceName, metricsName, theMbean);
+  }
+}

Modified: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceImpl.java?rev=1368598&r1=1368597&r2=1368598&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceImpl.java (original)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceImpl.java Thu Aug  2 16:58:35 2012
@@ -28,9 +28,7 @@ import org.apache.hadoop.metrics2.Metric
 public class ReplicationMetricsSourceImpl extends BaseMetricsSourceImpl implements
     ReplicationMetricsSource {
 
-  public static final String METRICS_NAME = "ReplicationMetrics";
-  public static final String METRICS_CONTEXT = "replicationmetrics";
-  public static final String METRICS_DESCRIPTION = "Metrics about HBase replication";
+
 
   public ReplicationMetricsSourceImpl() {
     this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT);

Added: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java?rev=1368598&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java Thu Aug  2 16:58:35 2012
@@ -0,0 +1,355 @@
+/**
+ * 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.metrics2.lib;
+
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.hadoop.metrics2.MetricsException;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsTag;
+
+/**
+ * An optional metrics registry class for creating and maintaining a
+ * collection of MetricsMutables, making writing metrics source easier.
+ * NOTE: this is a copy of org.apache.hadoop.metrics2.lib.MetricsRegistry with added one
+ *       feature: metrics can be removed. When HADOOP-8313 is fixed, usages of this class
+ *       should be substituted with org.apache.hadoop.metrics2.lib.MetricsRegistry.
+ *       This implementation also provides handy methods for creating metrics dynamically.
+ *       Another difference is that metricsMap & tagsMap implementation is substituted with
+ *       concurrent map, as we allow dynamic metrics additions/removals.
+ */
+public class DynamicMetricsRegistry {
+
+  /** key for the context tag */
+  public static final String CONTEXT_KEY = "context";
+  /** description for the context tag */
+  public static final String CONTEXT_DESC = "Metrics context";
+
+  private final ConcurrentMap<String, MetricMutable> metricsMap =
+      new ConcurrentHashMap<String, MetricMutable>();
+  private final ConcurrentMap<String, MetricsTag> tagsMap =
+      new ConcurrentHashMap<String, MetricsTag>();
+  private final String name;
+  private final MetricMutableFactory mf;
+
+  /**
+   * Construct the registry with a record name
+   * @param name  of the record of the metrics
+   */
+  public DynamicMetricsRegistry(String name) {
+    this.name = name;
+    this.mf = new MetricMutableFactory();
+  }
+
+  /**
+   * Construct the registry with a name and a metric factory
+   * @param name  of the record of the metrics
+   * @param factory for creating new mutable metrics
+   */
+  public DynamicMetricsRegistry(String name, MetricMutableFactory factory) {
+    this.name = name;
+    this.mf = factory;
+  }
+
+  /**
+   * @return  the name of the metrics registry
+   */
+  public String name() {
+    return name;
+  }
+
+  /**
+   * Get a metric by name
+   * @param name  of the metric
+   * @return  the metric object
+   */
+  public MetricMutable get(String name) {
+    return metricsMap.get(name);
+  }
+
+  /**
+   * Create a mutable integer counter
+   * @param name  of the metric
+   * @param description of the metric
+   * @param initValue of the metric
+   * @return  a new counter object
+   */
+  public MetricMutableCounterInt
+  newCounter(String name, String description, int initValue) {
+    MetricMutableCounterInt ret = mf.newCounter(name, description, initValue);
+    return addNewMetricIfAbsent(name, ret, MetricMutableCounterInt.class);
+  }
+
+  /**
+   * Create a mutable long integer counter
+   * @param name  of the metric
+   * @param description of the metric
+   * @param initValue of the metric
+   * @return  a new counter object
+   */
+  public MetricMutableCounterLong
+  newCounter(String name, String description, long initValue) {
+    MetricMutableCounterLong ret = mf.newCounter(name, description, initValue);
+    return addNewMetricIfAbsent(name, ret, MetricMutableCounterLong.class);
+  }
+
+  /**
+   * Create a mutable integer gauge
+   * @param name  of the metric
+   * @param description of the metric
+   * @param initValue of the metric
+   * @return  a new gauge object
+   */
+  public MetricMutableGaugeInt
+  newGauge(String name, String description, int initValue) {
+    MetricMutableGaugeInt ret = mf.newGauge(name, description, initValue);
+    return addNewMetricIfAbsent(name, ret, MetricMutableGaugeInt.class);
+  }
+
+  /**
+   * Create a mutable long integer gauge
+   * @param name  of the metric
+   * @param description of the metric
+   * @param initValue of the metric
+   * @return  a new gauge object
+   */
+  public MetricMutableGaugeLong
+  newGauge(String name, String description, long initValue) {
+    MetricMutableGaugeLong ret = mf.newGauge(name, description, initValue);
+    return addNewMetricIfAbsent(name, ret, MetricMutableGaugeLong.class);
+  }
+
+  /**
+   * Create a mutable metric with stats
+   * @param name  of the metric
+   * @param description of the metric
+   * @param sampleName  of the metric (e.g., "ops")
+   * @param valueName   of the metric (e.g., "time" or "latency")
+   * @param extended    produce extended stat (stdev, min/max etc.) if true.
+   * @return  a new metric object
+   */
+  public MetricMutableStat newStat(String name, String description,
+                                   String sampleName, String valueName,
+                                   boolean extended) {
+    MetricMutableStat ret =
+        mf.newStat(name, description, sampleName, valueName, extended);
+    return addNewMetricIfAbsent(name, ret, MetricMutableStat.class);
+  }
+
+  /**
+   * Create a mutable metric with stats
+   * @param name  of the metric
+   * @param description of the metric
+   * @param sampleName  of the metric (e.g., "ops")
+   * @param valueName   of the metric (e.g., "time" or "latency")
+   * @return  a new metric object
+   */
+  public MetricMutableStat newStat(String name, String description,
+                                   String sampleName, String valueName) {
+    return newStat(name, description, sampleName, valueName, false);
+  }
+
+  /**
+   * Create a mutable metric with stats using the name only
+   * @param name  of the metric
+   * @return a new metric object
+   */
+  public MetricMutableStat newStat(String name) {
+    return newStat(name, "", "ops", "time", false);
+  }
+
+  /**
+   * Set the metrics context tag
+   * @param name of the context
+   * @return the registry itself as a convenience
+   */
+  public DynamicMetricsRegistry setContext(String name) {
+    return tag(CONTEXT_KEY, CONTEXT_DESC, name);
+  }
+
+  /**
+   * Add a tag to the metrics
+   * @param name  of the tag
+   * @param description of the tag
+   * @param value of the tag
+   * @return  the registry (for keep adding tags)
+   */
+  public DynamicMetricsRegistry tag(String name, String description, String value) {
+    return tag(name, description, value, false);
+  }
+
+  /**
+   * Add a tag to the metrics
+   * @param name  of the tag
+   * @param description of the tag
+   * @param value of the tag
+   * @param override  existing tag if true
+   * @return  the registry (for keep adding tags)
+   */
+  public DynamicMetricsRegistry tag(String name, String description, String value,
+                             boolean override) {
+    MetricsTag tag = new MetricsTag(name, description, value);
+
+    if (!override) {
+      MetricsTag existing = tagsMap.putIfAbsent(name, tag);
+      if (existing != null) {
+        throw new MetricsException("Tag "+ name +" already exists!");
+      }
+      return this;
+    }
+
+    tagsMap.put(name, tag);
+
+    return this;
+  }
+
+  /**
+   * Get the tags
+   * @return  the tags set
+   */
+  public Set<Entry<String, MetricsTag>> tags() {
+    return tagsMap.entrySet();
+  }
+
+  /**
+   * Get the metrics
+   * @return  the metrics set
+   */
+  public Set<Entry<String, MetricMutable>> metrics() {
+    return metricsMap.entrySet();
+  }
+
+  /**
+   * Sample all the mutable metrics and put the snapshot in the builder
+   * @param builder to contain the metrics snapshot
+   * @param all get all the metrics even if the values are not changed.
+   */
+  public void snapshot(MetricsRecordBuilder builder, boolean all) {
+    for (Entry<String, MetricsTag> entry : tags()) {
+      builder.add(entry.getValue());
+    }
+    for (Entry<String, MetricMutable> entry : metrics()) {
+      entry.getValue().snapshot(builder, all);
+    }
+  }
+
+  /**
+   * Removes metric by name
+   * @param name name of the metric to remove
+   */
+  public void removeMetric(String name) {
+    metricsMap.remove(name);
+  }
+
+  /**
+   * Get a MetricMutableGaugeLong from the storage.  If it is not there
+   * atomically put it.
+   *
+   * @param gaugeName              name of the gauge to create or get.
+   * @param potentialStartingValue value of the new counter if we have to create it.
+   * @return a metric object
+   */
+  public MetricMutableGaugeLong getLongGauge(String gaugeName,
+                                             long potentialStartingValue) {
+    //Try and get the guage.
+    MetricMutable metric = metricsMap.get(gaugeName);
+
+    //If it's not there then try and put a new one in the storage.
+    if (metric == null) {
+
+      //Create the potential new gauge.
+      MetricMutableGaugeLong newGauge = new MetricMutableGaugeLong(gaugeName, "",
+              potentialStartingValue);
+
+        // Try and put the gauge in.  This is atomic.
+      metric = metricsMap.putIfAbsent(gaugeName, newGauge);
+
+      //If the value we get back is null then the put was successful and we will
+      // return that. Otherwise gaugeLong should contain the thing that was in
+      // before the put could be completed.
+      if (metric == null) {
+        return newGauge;
+      }
+    }
+
+    if (!(metric instanceof MetricMutableGaugeLong)) {
+      throw new MetricsException("Metric already exists in registry for metric name: " +
+              name + " and not of type MetricMutableGaugeLong");
+    }
+
+    return (MetricMutableGaugeLong) metric;
+  }
+
+  /**
+   * Get a MetricMutableCounterLong from the storage.  If it is not there
+   * atomically put it.
+   *
+   * @param counterName            Name of the counter to get
+   * @param potentialStartingValue starting value if we have to create a new counter
+   * @return a metric object
+   */
+  public MetricMutableCounterLong getLongCounter(String counterName,
+                                                 long potentialStartingValue) {
+    //See getLongGauge for description on how this works.
+    MetricMutable counter = metricsMap.get(counterName);
+    if (counter == null) {
+      MetricMutableCounterLong newCounter =
+              new MetricMutableCounterLong(counterName, "", potentialStartingValue);
+      counter = metricsMap.putIfAbsent(counterName, newCounter);
+      if (counter == null) {
+        return newCounter;
+      }
+    }
+
+    if (!(counter instanceof MetricMutableCounterLong)) {
+      throw new MetricsException("Metric already exists in registry for metric name: " +
+              name + "and not of type MetricMutableCounterLong");
+    }
+
+    return (MetricMutableCounterLong) counter;
+  }
+
+  private<T extends MetricMutable> T
+  addNewMetricIfAbsent(String name,
+                       T ret,
+                       Class<T> metricClass) {
+    //If the value we get back is null then the put was successful and we will
+    // return that. Otherwise metric should contain the thing that was in
+    // before the put could be completed.
+    MetricMutable metric = metricsMap.putIfAbsent(name, ret);
+    if (metric == null) {
+      return ret;
+    }
+
+    return returnExistingWithCast(metric, metricClass, name);
+  }
+
+  private<T> T returnExistingWithCast(MetricMutable metric,
+                                      Class<T> metricClass, String name) {
+    if (!metricClass.isAssignableFrom(metric.getClass())) {
+      throw new MetricsException("Metric already exists in registry for metric name: " +
+              name + " and not of type " + metricClass);
+    }
+
+    return (T) metric;
+  }
+}

Added: hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.metrics.MasterMetricsSource
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.metrics.MasterMetricsSource?rev=1368598&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.metrics.MasterMetricsSource (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.metrics.MasterMetricsSource Thu Aug  2 16:58:35 2012
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.master.metrics.MasterMetricsSourceImpl

Added: hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.metrics.MBeanSource
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.metrics.MBeanSource?rev=1368598&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.metrics.MBeanSource (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.metrics.MBeanSource Thu Aug  2 16:58:35 2012
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.metrics.MBeanSourceImpl

Added: hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceImplTest.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceImplTest.java?rev=1368598&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceImplTest.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceImplTest.java Thu Aug  2 16:58:35 2012
@@ -0,0 +1,40 @@
+/**
+ * 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.master.metrics;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.junit.Test;
+
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+/**
+ *  Test for MasterMetricsSourceImpl
+ */
+public class MasterMetricsSourceImplTest {
+
+  @Test
+  public void testGetInstance() throws Exception {
+    MasterMetricsSource rms = CompatibilitySingletonFactory
+        .getInstance(MasterMetricsSource.class);
+    assertTrue(rms instanceof MasterMetricsSourceImpl);
+    assertSame(rms, CompatibilitySingletonFactory.getInstance(MasterMetricsSource.class));
+  }
+
+}

Modified: hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/metrics/BaseMetricsSourceImplTest.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/metrics/BaseMetricsSourceImplTest.java?rev=1368598&r1=1368597&r2=1368598&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/metrics/BaseMetricsSourceImplTest.java (original)
+++ hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/metrics/BaseMetricsSourceImplTest.java Thu Aug  2 16:58:35 2012
@@ -43,10 +43,10 @@ public class BaseMetricsSourceImplTest {
   public void testSetGauge() throws Exception {
     String key = "testset";
     bmsi.setGauge(key, 100);
-    MetricMutableGaugeLong g = bmsi.gauges.get(key);
+    MetricMutableGaugeLong g = (MetricMutableGaugeLong) bmsi.metricsRegistry.get(key);
     assertEquals(key, g.name);
     bmsi.setGauge(key, 110);
-    assertSame(g, bmsi.gauges.get(key));
+    assertSame(g, bmsi.metricsRegistry.get(key));
 
   }
 
@@ -54,37 +54,37 @@ public class BaseMetricsSourceImplTest {
   public void testIncGauge() throws Exception {
     String key = "testincgauge";
     bmsi.incGauge(key, 100);
-    MetricMutableGaugeLong g = bmsi.gauges.get(key);
+    MetricMutableGaugeLong g = (MetricMutableGaugeLong) bmsi.metricsRegistry.get(key);
     assertEquals(key, g.name);
     bmsi.incGauge(key, 10);
-    assertSame(g, bmsi.gauges.get(key));
+    assertSame(g, bmsi.metricsRegistry.get(key));
   }
 
   @Test
   public void testDecGauge() throws Exception {
     String key = "testdec";
     bmsi.decGauge(key, 100);
-    MetricMutableGaugeLong g = bmsi.gauges.get(key);
+    MetricMutableGaugeLong g = (MetricMutableGaugeLong) bmsi.metricsRegistry.get(key);
     assertEquals(key, g.name);
     bmsi.decGauge(key, 100);
-    assertSame(g, bmsi.gauges.get(key));
+    assertSame(g, bmsi.metricsRegistry.get(key));
   }
 
   @Test
   public void testIncCounters() throws Exception {
     String key = "testinccounter";
     bmsi.incCounters(key, 100);
-    MetricMutableCounterLong c = bmsi.counters.get(key);
+    MetricMutableCounterLong c = (MetricMutableCounterLong) bmsi.metricsRegistry.get(key);
     assertEquals(key, c.name);
     bmsi.incCounters(key, 100);
-    assertSame(c, bmsi.counters.get(key));
+    assertSame(c, bmsi.metricsRegistry.get(key));
   }
 
   @Test
   public void testRemoveGauge() throws Exception {
     bmsi.setGauge("testrm", 100);
     bmsi.removeGauge("testrm");
-    assertNull(bmsi.gauges.get("testrm"));
+    assertNull(bmsi.metricsRegistry.get("testrm"));
 
   }
 
@@ -92,6 +92,6 @@ public class BaseMetricsSourceImplTest {
   public void testRemoveCounter() throws Exception {
     bmsi.incCounters("testrm", 100);
     bmsi.removeCounter("testrm");
-    assertNull(bmsi.counters.get("testrm"));
+    assertNull(bmsi.metricsRegistry.get("testrm"));
   }
 }

Modified: hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceImplTest.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceImplTest.java?rev=1368598&r1=1368597&r2=1368598&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceImplTest.java (original)
+++ hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceImplTest.java Thu Aug  2 16:58:35 2012
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hbase.replication.regionserver.metrics;
 
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
 import org.junit.Test;
 
 import static org.junit.Assert.assertTrue;
@@ -29,7 +30,8 @@ public class ReplicationMetricsSourceImp
 
   @Test
   public void testGetInstance() throws Exception {
-    ReplicationMetricsSource rms = ReplicationMetricsSourceFactory.getInstance();
+    ReplicationMetricsSource rms = CompatibilitySingletonFactory
+        .getInstance(ReplicationMetricsSource.class);
     assertTrue(rms instanceof ReplicationMetricsSourceImpl);
   }
 }

Added: hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceImpl.java?rev=1368598&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceImpl.java Thu Aug  2 16:58:35 2012
@@ -0,0 +1,66 @@
+/**
+ * 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.master.metrics;
+
+import org.apache.hadoop.hbase.metrics.BaseMetricsSourceImpl;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
+
+/**
+ * Hadoop2 implementation of MasterMetricsSource.
+ */
+public class MasterMetricsSourceImpl
+        extends BaseMetricsSourceImpl implements MasterMetricsSource {
+
+  MutableCounterLong clusterRequestsCounter;
+  MutableGaugeLong ritGauge;
+  MutableGaugeLong ritCountOverThresholdGauge;
+  MutableGaugeLong ritOldestAgeGauge;
+
+  public MasterMetricsSourceImpl() {
+    this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT);
+  }
+
+  public MasterMetricsSourceImpl(String metricsName,
+                                 String metricsDescription,
+                                 String metricsContext) {
+    super(metricsName, metricsDescription, metricsContext);
+
+    clusterRequestsCounter = getLongCounter("cluster_requests", 0);
+    ritGauge = getLongGauge("ritCount", 0);
+    ritCountOverThresholdGauge = getLongGauge("ritCountOverThreshold", 0);
+    ritOldestAgeGauge = getLongGauge("ritOldestAge", 0);
+  }
+
+  public void incRequests(final int inc) {
+    this.clusterRequestsCounter.incr(inc);
+  }
+
+  public void setRIT(int ritCount) {
+    ritGauge.set(ritCount);
+  }
+
+  public void setRITCountOverThreshold(int ritCount) {
+    ritCountOverThresholdGauge.set(ritCount);
+  }
+
+  public void setRITOldestAge(long ritCount) {
+    ritCountOverThresholdGauge.set(ritCount);
+  }
+}

Modified: hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseMetricsSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseMetricsSourceImpl.java?rev=1368598&r1=1368597&r2=1368598&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseMetricsSourceImpl.java (original)
+++ hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseMetricsSourceImpl.java Thu Aug  2 16:58:35 2012
@@ -19,45 +19,37 @@
 package org.apache.hadoop.hbase.metrics;
 
 import org.apache.hadoop.metrics2.MetricsCollector;
-import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.MetricsSource;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
-import org.apache.hadoop.metrics2.lib.HBaseMetricsFactory;
+import org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry;
 import org.apache.hadoop.metrics2.lib.MutableCounterLong;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
+import org.apache.hadoop.metrics2.source.JvmMetrics;
 
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-/** Hadoop 2 implementation of BaseMetricsSource for */
+/**
+ * Hadoop 2 implementation of BaseMetricsSource (using metrics2 framework)
+ */
 public class BaseMetricsSourceImpl implements BaseMetricsSource, MetricsSource {
 
   private static boolean defaultMetricsSystemInited = false;
   public static final String HBASE_METRICS_SYSTEM_NAME = "hbase";
 
-  public ConcurrentMap<String, MutableGaugeLong>
-      gauges = new ConcurrentHashMap<String, MutableGaugeLong>();
-  public ConcurrentMap<String, MutableCounterLong> counters =
-      new ConcurrentHashMap<String, MutableCounterLong>();
-
-  protected String metricsContext;
-  protected String metricsName;
-  protected String metricsDescription;
+  final DynamicMetricsRegistry metricsRegistry;
+
+  private JvmMetrics jvmMetricsSource;
 
   public BaseMetricsSourceImpl(String metricsName,
                                String metricsDescription,
                                String metricsContext) {
-    this.metricsContext = metricsContext;
-    this.metricsName = metricsName;
-    this.metricsDescription = metricsDescription;
+    metricsRegistry = new DynamicMetricsRegistry(metricsName).setContext(metricsContext);
 
     if (!defaultMetricsSystemInited) {
       //Not too worried about mutlithread here as all it does is spam the logs.
       defaultMetricsSystemInited = true;
       DefaultMetricsSystem.initialize(HBASE_METRICS_SYSTEM_NAME);
+      jvmMetricsSource = JvmMetrics.create(metricsName, "", DefaultMetricsSystem.instance());
     }
-    DefaultMetricsSystem.instance().register(this.metricsContext, this.metricsDescription, this);
+    DefaultMetricsSystem.instance().register(metricsContext, metricsDescription, this);
 
   }
 
@@ -112,7 +104,7 @@ public class BaseMetricsSourceImpl imple
    * @param key
    */
   public void removeGauge(String key) {
-    gauges.remove(key);
+    metricsRegistry.removeMetric(key);
   }
 
   /**
@@ -121,21 +113,12 @@ public class BaseMetricsSourceImpl imple
    * @param key
    */
   public void removeCounter(String key) {
-    counters.remove(key);
+    metricsRegistry.removeMetric(key);
   }
 
   @Override
   public void getMetrics(MetricsCollector metricsCollector, boolean all) {
-    MetricsRecordBuilder rb =
-        metricsCollector.addRecord(this.metricsName).setContext(metricsContext);
-
-    for (Map.Entry<String, MutableCounterLong> entry : counters.entrySet()) {
-      entry.getValue().snapshot(rb, all);
-    }
-    for (Map.Entry<String, MutableGaugeLong> entry : gauges.entrySet()) {
-      entry.getValue().snapshot(rb, all);
-    }
-
+    metricsRegistry.snapshot(metricsCollector.addRecord(metricsRegistry.info()), all);
   }
 
   /**
@@ -145,28 +128,8 @@ public class BaseMetricsSourceImpl imple
    * @param potentialStartingValue value of the new counter if we have to create it.
    * @return
    */
-  private MutableGaugeLong getLongGauge(String gaugeName, long potentialStartingValue) {
-    //Try and get the guage.
-    MutableGaugeLong gaugeInt = gauges.get(gaugeName);
-
-    //If it's not there then try and put a new one in the storage.
-    if (gaugeInt == null) {
-
-      //Create the potential new gauge.
-      MutableGaugeLong newGauge = HBaseMetricsFactory.newGauge(gaugeName,
-          "",
-          potentialStartingValue);
-
-      // Try and put the gauge in.  This is atomic.
-      gaugeInt = gauges.putIfAbsent(gaugeName, newGauge);
-
-      //If the value we get back is null then the put was successful and we will return that.
-      //otherwise gaugeInt should contain the thing that was in before the put could be completed.
-      if (gaugeInt == null) {
-        gaugeInt = newGauge;
-      }
-    }
-    return gaugeInt;
+  protected MutableGaugeLong getLongGauge(String gaugeName, long potentialStartingValue) {
+    return metricsRegistry.getLongGauge(gaugeName, potentialStartingValue);
   }
 
   /**
@@ -176,18 +139,7 @@ public class BaseMetricsSourceImpl imple
    * @param potentialStartingValue starting value if we have to create a new counter
    * @return
    */
-  private MutableCounterLong getLongCounter(String counterName, long potentialStartingValue) {
-    //See getLongGauge for description on how this works.
-    MutableCounterLong counter = counters.get(counterName);
-    if (counter == null) {
-      MutableCounterLong newCounter =
-          HBaseMetricsFactory.newCounter(counterName, "", potentialStartingValue);
-      counter = counters.putIfAbsent(counterName, newCounter);
-      if (counter == null) {
-        counter = newCounter;
-      }
-    }
-    return counter;
+  protected MutableCounterLong getLongCounter(String counterName, long potentialStartingValue) {
+    return metricsRegistry.getLongCounter(counterName, potentialStartingValue);
   }
-
 }

Added: hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/MBeanSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/MBeanSourceImpl.java?rev=1368598&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/MBeanSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/MBeanSourceImpl.java Thu Aug  2 16:58:35 2012
@@ -0,0 +1,41 @@
+/**
+ * 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 org.apache.hadoop.metrics2.util.MBeans;
+
+import javax.management.ObjectName;
+
+/**
+ * Hadoop2 metrics2 implementation of an object that registers MBeans.
+ */
+public class MBeanSourceImpl implements MBeanSource {
+
+  /**
+   * Register an mbean with the underlying metrics system
+   * @param serviceName Metrics service/system name
+   * @param metricsName name of the metrics obejct to expose
+   * @param theMbean the actual MBean
+   * @return ObjectName from jmx
+   */
+  @Override
+  public ObjectName register(String serviceName, String metricsName, Object theMbean) {
+    return MBeans.register(serviceName, metricsName, theMbean);
+  }
+}

Modified: hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceImpl.java?rev=1368598&r1=1368597&r2=1368598&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceImpl.java (original)
+++ hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationMetricsSourceImpl.java Thu Aug  2 16:58:35 2012
@@ -28,10 +28,6 @@ import org.apache.hadoop.metrics2.Metric
 public class ReplicationMetricsSourceImpl extends BaseMetricsSourceImpl implements
     ReplicationMetricsSource {
 
-  public static final String METRICS_NAME = "ReplicationMetrics";
-  public static final String METRICS_CONTEXT = "replicationmetrics";
-  public static final String METRICS_DESCRIPTION = "Metrics about HBase replication";
-
   public ReplicationMetricsSourceImpl() {
     this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT);
   }

Added: hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java?rev=1368598&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java (added)
+++ hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java Thu Aug  2 16:58:35 2012
@@ -0,0 +1,468 @@
+/**
+ * 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.metrics2.lib;
+
+import java.util.Collection;
+import java.util.concurrent.ConcurrentMap;
+
+import com.google.common.base.Objects;
+import com.google.common.collect.Maps;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.metrics2.MetricsException;
+import org.apache.hadoop.metrics2.MetricsInfo;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsTag;
+import org.apache.hadoop.metrics2.impl.MsInfo;
+
+/**
+ * An optional metrics registry class for creating and maintaining a
+ * collection of MetricsMutables, making writing metrics source easier.
+ * NOTE: this is a copy of org.apache.hadoop.metrics2.lib.MetricsRegistry with added one
+ *       feature: metrics can be removed. When HADOOP-8313 is fixed, usages of this class
+ *       should be substituted with org.apache.hadoop.metrics2.lib.MetricsRegistry.
+ *       This implementation also provides handy methods for creating metrics
+ *       dynamically.
+ *       Another difference is that metricsMap implementation is substituted with
+ *       thread-safe map, as we allow dynamic metrics additions/removals.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class DynamicMetricsRegistry {
+  private final ConcurrentMap<String, MutableMetric> metricsMap =
+          Maps.newConcurrentMap();
+  private final ConcurrentMap<String, MetricsTag> tagsMap =
+          Maps.newConcurrentMap();
+  private final MetricsInfo metricsInfo;
+
+  /**
+   * Construct the registry with a record name
+   * @param name  of the record of the metrics
+   */
+  public DynamicMetricsRegistry(String name) {
+    metricsInfo = Interns.info(name, name);
+  }
+
+  /**
+   * Construct the registry with a metadata object
+   * @param info  the info object for the metrics record/group
+   */
+  public DynamicMetricsRegistry(MetricsInfo info) {
+    metricsInfo = info;
+  }
+
+  /**
+   * @return the info object of the metrics registry
+   */
+  public MetricsInfo info() {
+    return metricsInfo;
+  }
+
+  /**
+   * Get a metric by name
+   * @param name  of the metric
+   * @return the metric object
+   */
+  public MutableMetric get(String name) {
+    return metricsMap.get(name);
+  }
+
+  /**
+   * Get a tag by name
+   * @param name  of the tag
+   * @return the tag object
+   */
+  public MetricsTag getTag(String name) {
+    return tagsMap.get(name);
+  }
+
+  /**
+   * Create a mutable integer counter
+   * @param name  of the metric
+   * @param desc  metric description
+   * @param iVal  initial value
+   * @return a new counter object
+   */
+  public MutableCounterInt newCounter(String name, String desc, int iVal) {
+    return newCounter(Interns.info(name, desc), iVal);
+  }
+
+  /**
+   * Create a mutable integer counter
+   * @param info  metadata of the metric
+   * @param iVal  initial value
+   * @return a new counter object
+   */
+  public MutableCounterInt newCounter(MetricsInfo info, int iVal) {
+    MutableCounterInt ret = new MutableCounterInt(info, iVal);
+    return addNewMetricIfAbsent(info.name(), ret, MutableCounterInt.class);
+  }
+
+  /**
+   * Create a mutable long integer counter
+   * @param name  of the metric
+   * @param desc  metric description
+   * @param iVal  initial value
+   * @return a new counter object
+   */
+  public MutableCounterLong newCounter(String name, String desc, long iVal) {
+    return newCounter(Interns.info(name, desc), iVal);
+  }
+
+  /**
+   * Create a mutable long integer counter
+   * @param info  metadata of the metric
+   * @param iVal  initial value
+   * @return a new counter object
+   */
+  public MutableCounterLong newCounter(MetricsInfo info, long iVal) {
+    MutableCounterLong ret = new MutableCounterLong(info, iVal);
+    return addNewMetricIfAbsent(info.name(), ret, MutableCounterLong.class);
+  }
+
+  /**
+   * Create a mutable integer gauge
+   * @param name  of the metric
+   * @param desc  metric description
+   * @param iVal  initial value
+   * @return a new gauge object
+   */
+  public MutableGaugeInt newGauge(String name, String desc, int iVal) {
+    return newGauge(Interns.info(name, desc), iVal);
+  }
+  /**
+   * Create a mutable integer gauge
+   * @param info  metadata of the metric
+   * @param iVal  initial value
+   * @return a new gauge object
+   */
+  public MutableGaugeInt newGauge(MetricsInfo info, int iVal) {
+    MutableGaugeInt ret = new MutableGaugeInt(info, iVal);
+    return addNewMetricIfAbsent(info.name(), ret, MutableGaugeInt.class);
+  }
+
+  /**
+   * Create a mutable long integer gauge
+   * @param name  of the metric
+   * @param desc  metric description
+   * @param iVal  initial value
+   * @return a new gauge object
+   */
+  public MutableGaugeLong newGauge(String name, String desc, long iVal) {
+    return newGauge(Interns.info(name, desc), iVal);
+  }
+
+  /**
+   * Create a mutable long integer gauge
+   * @param info  metadata of the metric
+   * @param iVal  initial value
+   * @return a new gauge object
+   */
+  public MutableGaugeLong newGauge(MetricsInfo info, long iVal) {
+    MutableGaugeLong ret = new MutableGaugeLong(info, iVal);
+    return addNewMetricIfAbsent(info.name(), ret, MutableGaugeLong.class);
+  }
+
+  /**
+   * Create a mutable metric with stats
+   * @param name  of the metric
+   * @param desc  metric description
+   * @param sampleName  of the metric (e.g., "Ops")
+   * @param valueName   of the metric (e.g., "Time" or "Latency")
+   * @param extended    produce extended stat (stdev, min/max etc.) if true.
+   * @return a new mutable stat metric object
+   */
+  public MutableStat newStat(String name, String desc,
+      String sampleName, String valueName, boolean extended) {
+    MutableStat ret =
+        new MutableStat(name, desc, sampleName, valueName, extended);
+    return addNewMetricIfAbsent(name, ret, MutableStat.class);
+  }
+
+  /**
+   * Create a mutable metric with stats
+   * @param name  of the metric
+   * @param desc  metric description
+   * @param sampleName  of the metric (e.g., "Ops")
+   * @param valueName   of the metric (e.g., "Time" or "Latency")
+   * @return a new mutable metric object
+   */
+  public MutableStat newStat(String name, String desc,
+                             String sampleName, String valueName) {
+    return newStat(name, desc, sampleName, valueName, false);
+  }
+
+  /**
+   * Create a mutable rate metric
+   * @param name  of the metric
+   * @return a new mutable metric object
+   */
+  public MutableRate newRate(String name) {
+    return newRate(name, name, false);
+  }
+
+  /**
+   * Create a mutable rate metric
+   * @param name  of the metric
+   * @param description of the metric
+   * @return a new mutable rate metric object
+   */
+  public MutableRate newRate(String name, String description) {
+    return newRate(name, description, false);
+  }
+
+  /**
+   * Create a mutable rate metric (for throughput measurement)
+   * @param name  of the metric
+   * @param desc  description
+   * @param extended  produce extended stat (stdev/min/max etc.) if true
+   * @return a new mutable rate metric object
+   */
+  public MutableRate newRate(String name, String desc, boolean extended) {
+    return newRate(name, desc, extended, true);
+  }
+
+  @InterfaceAudience.Private
+  public MutableRate newRate(String name, String desc,
+      boolean extended, boolean returnExisting) {
+    if (returnExisting) {
+      MutableMetric rate = metricsMap.get(name);
+      if (rate != null) {
+        if (rate instanceof MutableRate) return (MutableRate) rate;
+        throw new MetricsException("Unexpected metrics type "+ rate.getClass()
+                                   +" for "+ name);
+      }
+    }
+    MutableRate ret = new MutableRate(name, desc, extended);
+    metricsMap.put(name, ret);
+    return ret;
+  }
+
+  synchronized void add(String name, MutableMetric metric) {
+    addNewMetricIfAbsent(name, metric, MutableMetric.class);
+  }
+
+  /**
+   * Add sample to a stat metric by name.
+   * @param name  of the metric
+   * @param value of the snapshot to add
+   */
+  public void add(String name, long value) {
+    MutableMetric m = metricsMap.get(name);
+
+    if (m != null) {
+      if (m instanceof MutableStat) {
+        ((MutableStat) m).add(value);
+      }
+      else {
+        throw new MetricsException("Unsupported add(value) for metric "+ name);
+      }
+    }
+    else {
+      metricsMap.put(name, newRate(name)); // default is a rate metric
+      add(name, value);
+    }
+  }
+
+  /**
+   * Set the metrics context tag
+   * @param name of the context
+   * @return the registry itself as a convenience
+   */
+  public DynamicMetricsRegistry setContext(String name) {
+    return tag(MsInfo.Context, name, true);
+  }
+
+  /**
+   * Add a tag to the metrics
+   * @param name  of the tag
+   * @param description of the tag
+   * @param value of the tag
+   * @return the registry (for keep adding tags)
+   */
+  public DynamicMetricsRegistry tag(String name, String description, String value) {
+    return tag(name, description, value, false);
+  }
+
+  /**
+   * Add a tag to the metrics
+   * @param name  of the tag
+   * @param description of the tag
+   * @param value of the tag
+   * @param override  existing tag if true
+   * @return the registry (for keep adding tags)
+   */
+  public DynamicMetricsRegistry tag(String name, String description, String value,
+                             boolean override) {
+    return tag(Interns.info(name, description), value, override);
+  }
+
+  /**
+   * Add a tag to the metrics
+   * @param info  metadata of the tag
+   * @param value of the tag
+   * @param override existing tag if true
+   * @return the registry (for keep adding tags etc.)
+   */
+  public DynamicMetricsRegistry tag(MetricsInfo info, String value, boolean override) {
+    MetricsTag tag = Interns.tag(info, value);
+
+    if (!override) {
+      MetricsTag existing = tagsMap.putIfAbsent(info.name(), tag);
+      if (existing != null) {
+        throw new MetricsException("Tag "+ info.name() +" already exists!");
+      }
+      return this;
+    }
+
+    tagsMap.put(info.name(), tag);
+
+    return this;
+  }
+
+  public DynamicMetricsRegistry tag(MetricsInfo info, String value) {
+    return tag(info, value, false);
+  }
+
+  Collection<MetricsTag> tags() {
+    return tagsMap.values();
+  }
+
+  Collection<MutableMetric> metrics() {
+    return metricsMap.values();
+  }
+
+  /**
+   * Sample all the mutable metrics and put the snapshot in the builder
+   * @param builder to contain the metrics snapshot
+   * @param all get all the metrics even if the values are not changed.
+   */
+  public void snapshot(MetricsRecordBuilder builder, boolean all) {
+    for (MetricsTag tag : tags()) {
+      builder.add(tag);
+    }
+    for (MutableMetric metric : metrics()) {
+      metric.snapshot(builder, all);
+    }
+  }
+
+  @Override public String toString() {
+    return Objects.toStringHelper(this)
+        .add("info", metricsInfo).add("tags", tags()).add("metrics", metrics())
+        .toString();
+  }
+
+  /**
+   * Removes metric by name
+   * @param name name of the metric to remove
+   */
+  public void removeMetric(String name) {
+    metricsMap.remove(name);
+  }
+
+  /**
+   * Get a MetricMutableGaugeLong from the storage.  If it is not there atomically put it.
+   *
+   * @param gaugeName              name of the gauge to create or get.
+   * @param potentialStartingValue value of the new gauge if we have to create it.
+   * @return
+   */
+  public MutableGaugeLong getLongGauge(String gaugeName, long potentialStartingValue) {
+    //Try and get the guage.
+    MutableMetric metric = metricsMap.get(gaugeName);
+
+    //If it's not there then try and put a new one in the storage.
+    if (metric == null) {
+
+      //Create the potential new gauge.
+      MutableGaugeLong newGauge = new MutableGaugeLong(Interns.info(gaugeName, ""),
+              potentialStartingValue);
+
+      // Try and put the gauge in.  This is atomic.
+      metric = metricsMap.putIfAbsent(gaugeName, newGauge);
+
+      //If the value we get back is null then the put was successful and we will return that.
+      //otherwise gaugeLong should contain the thing that was in before the put could be completed.
+      if (metric == null) {
+        return newGauge;
+      }
+    }
+
+    if (!(metric instanceof MutableGaugeLong)) {
+      throw new MetricsException("Metric already exists in registry for metric name: " + gaugeName +
+              " and not of type MetricMutableGaugeLong");
+    }
+
+    return (MutableGaugeLong) metric;
+  }
+
+  /**
+   * Get a MetricMutableCounterLong from the storage.  If it is not there atomically put it.
+   *
+   * @param counterName            Name of the counter to get
+   * @param potentialStartingValue starting value if we have to create a new counter
+   * @return
+   */
+  public MutableCounterLong getLongCounter(String counterName, long potentialStartingValue) {
+    //See getLongGauge for description on how this works.
+    MutableMetric counter = metricsMap.get(counterName);
+    if (counter == null) {
+      MutableCounterLong newCounter =
+              new MutableCounterLong(Interns.info(counterName, ""), potentialStartingValue);
+      counter = metricsMap.putIfAbsent(counterName, newCounter);
+      if (counter == null) {
+        return newCounter;
+      }
+    }
+
+
+    if (!(counter instanceof MutableCounterLong)) {
+      throw new MetricsException("Metric already exists in registry for metric name: " +
+              counterName + " and not of type MetricMutableCounterLong");
+    }
+
+    return (MutableCounterLong) counter;
+  }
+
+  private<T extends MutableMetric> T
+  addNewMetricIfAbsent(String name,
+                       T ret,
+                       Class<T> metricClass) {
+    //If the value we get back is null then the put was successful and we will
+    // return that. Otherwise metric should contain the thing that was in
+    // before the put could be completed.
+    MutableMetric metric = metricsMap.putIfAbsent(name, ret);
+    if (metric == null) {
+      return ret;
+    }
+
+    return returnExistingWithCast(metric, metricClass, name);
+  }
+
+  private<T> T returnExistingWithCast(MutableMetric metric,
+                                      Class<T> metricClass, String name) {
+    if (!metricClass.isAssignableFrom(metric.getClass())) {
+      throw new MetricsException("Metric already exists in registry for metric name: " +
+              name + " and not of type " + metricClass +
+              " but instead of type " + metric.getClass());
+    }
+
+    return (T) metric;
+  }
+}

Added: hbase/trunk/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.metrics.MasterMetricsSource
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.metrics.MasterMetricsSource?rev=1368598&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.metrics.MasterMetricsSource (added)
+++ hbase/trunk/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.metrics.MasterMetricsSource Thu Aug  2 16:58:35 2012
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.master.metrics.MasterMetricsSourceImpl

Added: hbase/trunk/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.metrics.MBeanSource
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.metrics.MBeanSource?rev=1368598&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.metrics.MBeanSource (added)
+++ hbase/trunk/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.metrics.MBeanSource Thu Aug  2 16:58:35 2012
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.metrics.MBeanSourceImpl

Added: hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceImplTest.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceImplTest.java?rev=1368598&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceImplTest.java (added)
+++ hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/MasterMetricsSourceImplTest.java Thu Aug  2 16:58:35 2012
@@ -0,0 +1,40 @@
+/**
+ * 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.master.metrics;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.junit.Test;
+
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+/**
+ *  Test for MasterMetricsSourceImpl
+ */
+public class MasterMetricsSourceImplTest {
+
+  @Test
+  public void testGetInstance() throws Exception {
+    MasterMetricsSource rms = CompatibilitySingletonFactory
+        .getInstance(MasterMetricsSource.class);
+    assertTrue(rms instanceof MasterMetricsSourceImpl);
+    assertSame(rms, CompatibilitySingletonFactory.getInstance(MasterMetricsSource.class));
+  }
+
+}