You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ec...@apache.org on 2012/11/07 00:22:09 UTC

svn commit: r1406396 [2/6] - in /hbase/trunk: dev-support/ 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/hbase/master/met...

Modified: hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelper.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelper.java (original)
+++ hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelper.java Tue Nov  6 23:22:01 2012
@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.hbase.test;
 
-import org.apache.hadoop.hbase.metrics.BaseMetricsSource;
+import org.apache.hadoop.hbase.metrics.BaseSource;
 
 /** Interface of a class to make assertions about metrics values. */
 public interface MetricsAssertHelper {
@@ -28,128 +28,128 @@ public interface MetricsAssertHelper {
    *
    * @param name     The name of the tag.
    * @param expected The expected value
-   * @param source   The BaseMetricsSource{@link BaseMetricsSource} that will provide the tags,
+   * @param source   The BaseSource{@link BaseSource} that will provide the tags,
    *                 gauges, and counters.
    */
-  public void assertTag(String name, String expected, BaseMetricsSource source);
+  public void assertTag(String name, String expected, BaseSource source);
 
   /**
    * Assert that a gauge exists and that it's value is equal to the expected value.
    *
    * @param name     The name of the gauge
    * @param expected The expected value of the gauge.
-   * @param source   The BaseMetricsSource{@link BaseMetricsSource} that will provide the tags,
+   * @param source   The BaseSource{@link BaseSource} that will provide the tags,
    *                 gauges, and counters.
    */
-  public void assertGauge(String name, long expected, BaseMetricsSource source);
+  public void assertGauge(String name, long expected, BaseSource source);
 
   /**
    * Assert that a gauge exists and it's value is greater than a given value
    *
    * @param name     The name of the gauge
    * @param expected Value that the gauge is expected to be greater than
-   * @param source   The BaseMetricsSource{@link BaseMetricsSource} that will provide the tags,
+   * @param source   The BaseSource{@link BaseSource} that will provide the tags,
    *                 gauges, and counters.
    */
-  public void assertGaugeGt(String name, long expected, BaseMetricsSource source);
+  public void assertGaugeGt(String name, long expected, BaseSource source);
 
   /**
    * Assert that a gauge exists and it's value is less than a given value
    *
    * @param name     The name of the gauge
    * @param expected Value that the gauge is expected to be less than
-   * @param source   The BaseMetricsSource{@link BaseMetricsSource} that will provide the tags,
+   * @param source   The BaseSource{@link BaseSource} that will provide the tags,
    *                 gauges, and counters.
    */
-  public void assertGaugeLt(String name, long expected, BaseMetricsSource source);
+  public void assertGaugeLt(String name, long expected, BaseSource source);
 
   /**
    * Assert that a gauge exists and that it's value is equal to the expected value.
    *
    * @param name     The name of the gauge
    * @param expected The expected value of the gauge.
-   * @param source   The BaseMetricsSource{@link BaseMetricsSource} that will provide the tags,
+   * @param source   The BaseSource{@link BaseSource} that will provide the tags,
    *                 gauges, and counters.
    */
-  public void assertGauge(String name, double expected, BaseMetricsSource source);
+  public void assertGauge(String name, double expected, BaseSource source);
 
   /**
    * Assert that a gauge exists and it's value is greater than a given value
    *
    * @param name     The name of the gauge
    * @param expected Value that the gauge is expected to be greater than
-   * @param source   The BaseMetricsSource{@link BaseMetricsSource} that will provide the tags,
+   * @param source   The BaseSource{@link BaseSource} that will provide the tags,
    *                 gauges, and counters.
    */
-  public void assertGaugeGt(String name, double expected, BaseMetricsSource source);
+  public void assertGaugeGt(String name, double expected, BaseSource source);
 
   /**
    * Assert that a gauge exists and it's value is less than a given value
    *
    * @param name     The name of the gauge
    * @param expected Value that the gauge is expected to be less than
-   * @param source   The BaseMetricsSource{@link BaseMetricsSource} that will provide the tags,
+   * @param source   The BaseSource{@link BaseSource} that will provide the tags,
    *                 gauges, and counters.
    */
-  public void assertGaugeLt(String name, double expected, BaseMetricsSource source);
+  public void assertGaugeLt(String name, double expected, BaseSource source);
 
   /**
    * Assert that a counter exists and that it's value is equal to the expected value.
    *
    * @param name     The name of the counter.
    * @param expected The expected value
-   * @param source   The BaseMetricsSource{@link BaseMetricsSource} that will provide the tags,
+   * @param source   The BaseSource{@link BaseSource} that will provide the tags,
    *                 gauges, and counters.
    */
-  public void assertCounter(String name, long expected, BaseMetricsSource source);
+  public void assertCounter(String name, long expected, BaseSource source);
 
   /**
    * Assert that a counter exists and that it's value is greater than the given value.
    *
    * @param name     The name of the counter.
    * @param expected The value the counter is expected to be greater than.
-   * @param source   The BaseMetricsSource{@link BaseMetricsSource} that will provide the tags,
+   * @param source   The BaseSource{@link BaseSource} that will provide the tags,
    *                 gauges, and counters.
    */
-  public void assertCounterGt(String name, long expected, BaseMetricsSource source);
+  public void assertCounterGt(String name, long expected, BaseSource source);
 
   /**
    * Assert that a counter exists and that it's value is less than the given value.
    *
    * @param name     The name of the counter.
    * @param expected The value the counter is expected to be less than.
-   * @param source   The BaseMetricsSource{@link BaseMetricsSource} that will provide the tags,
+   * @param source   The BaseSource{@link BaseSource} that will provide the tags,
    *                 gauges, and counters.
    */
-  public void assertCounterLt(String name, long expected, BaseMetricsSource source);
+  public void assertCounterLt(String name, long expected, BaseSource source);
 
   /**
    * Get the value of a counter.
    *
    * @param name   name of the counter.
-   * @param source The BaseMetricsSource{@link BaseMetricsSource} that will provide the tags,
+   * @param source The BaseSource{@link BaseSource} that will provide the tags,
    *               gauges, and counters.
    * @return long value of the counter.
    */
-  public long getCounter(String name, BaseMetricsSource source);
+  public long getCounter(String name, BaseSource source);
 
   /**
    * Get the value of a gauge as a double.
    *
    * @param name   name of the gauge.
-   * @param source The BaseMetricsSource{@link BaseMetricsSource} that will provide the tags,
+   * @param source The BaseSource{@link BaseSource} that will provide the tags,
    *               gauges, and counters.
    * @return double value of the gauge.
    */
-  public double getGaugeDouble(String name, BaseMetricsSource source);
+  public double getGaugeDouble(String name, BaseSource source);
 
   /**
    * Get the value of a gauge as a long.
    *
    * @param name   name of the gauge.
-   * @param source The BaseMetricsSource{@link BaseMetricsSource} that will provide the tags,
+   * @param source The BaseSource{@link BaseSource} that will provide the tags,
    *               gauges, and counters.
    * @return long value of the gauge.
    */
-  public long getGaugeLong(String name, BaseMetricsSource source);
+  public long getGaugeLong(String name, BaseSource source);
 }

Added: hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/thrift/TestMetricsThriftServerSourceFactory.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/thrift/TestMetricsThriftServerSourceFactory.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/thrift/TestMetricsThriftServerSourceFactory.java (added)
+++ hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/thrift/TestMetricsThriftServerSourceFactory.java Tue Nov  6 23:22:01 2012
@@ -0,0 +1,37 @@
+/**
+ * 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.thrift;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.hbase.thrift.MetricsThriftServerSourceFactory;
+import org.junit.Test;
+
+/**
+ * Test for the interface of MetricsThriftServerSourceFactory
+ */
+public class TestMetricsThriftServerSourceFactory {
+
+
+  @Test(expected=RuntimeException.class)
+  public void testGetInstanceNoHadoopCompat() throws RuntimeException {
+    //This should throw an exception because there is no compat lib on the class path.
+    CompatibilitySingletonFactory.getInstance(MetricsThriftServerSourceFactory.class);
+  }
+
+}

Modified: hbase/trunk/hbase-hadoop1-compat/pom.xml
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/pom.xml?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/pom.xml (original)
+++ hbase/trunk/hbase-hadoop1-compat/pom.xml Tue Nov  6 23:22:01 2012
@@ -98,6 +98,10 @@ limitations under the License.
       <artifactId>metrics-core</artifactId>
     </dependency>
     <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-test</artifactId>
       <version>${hadoop-one.version}</version><!--$NO-MVN-MAN-VER$-->

Added: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceFactoryImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceFactoryImpl.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceFactoryImpl.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceFactoryImpl.java Tue Nov  6 23:22:01 2012
@@ -0,0 +1,37 @@
+/**
+ * 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;
+
+/**
+ * Factory to create MetricsMasterSource when given a  MetricsMasterWrapper
+ */
+public class MetricsMasterSourceFactoryImpl implements MetricsMasterSourceFactory {
+  private static enum FactoryStorage {
+    INSTANCE;
+    MetricsMasterSource masterSource;
+  }
+
+  @Override
+  public synchronized MetricsMasterSource create(MetricsMasterWrapper masterWrapper) {
+    if (FactoryStorage.INSTANCE.masterSource == null) {
+      FactoryStorage.INSTANCE.masterSource = new MetricsMasterSourceImpl(masterWrapper);
+    }
+    return FactoryStorage.INSTANCE.masterSource;
+  }
+}

Added: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java Tue Nov  6 23:22:01 2012
@@ -0,0 +1,132 @@
+/**
+ * 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;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.metrics2.MetricsBuilder;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.lib.MetricMutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MetricMutableGaugeLong;
+import org.apache.hadoop.metrics2.lib.MetricMutableHistogram;
+
+/**
+ * Hadoop1 implementation of MetricsMasterSource.
+ */
+public class MetricsMasterSourceImpl
+    extends BaseSourceImpl implements MetricsMasterSource {
+
+  private static final Log LOG = LogFactory.getLog(MetricsMasterSourceImpl.class.getName());
+
+  private final MetricsMasterWrapper masterWrapper;
+  private MetricMutableCounterLong clusterRequestsCounter;
+  private MetricMutableGaugeLong ritGauge;
+  private MetricMutableGaugeLong ritCountOverThresholdGauge;
+  private MetricMutableGaugeLong ritOldestAgeGauge;
+  private MetricMutableHistogram splitTimeHisto;
+  private MetricMutableHistogram splitSizeHisto;
+
+  public MetricsMasterSourceImpl(MetricsMasterWrapper masterWrapper) {
+    this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT, masterWrapper);
+  }
+
+  public MetricsMasterSourceImpl(String metricsName,
+                                 String metricsDescription,
+                                 String metricsContext,
+                                 String metricsJmxContext,
+                                 MetricsMasterWrapper masterWrapper) {
+    super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
+    this.masterWrapper = masterWrapper;
+  }
+
+  @Override
+  public void init() {
+    super.init();
+    clusterRequestsCounter = metricsRegistry.newCounter(CLUSTER_REQUESTS_NAME, "", 0l);
+    ritGauge = metricsRegistry.newGauge(RIT_COUNT_NAME, "", 0l);
+    ritCountOverThresholdGauge = metricsRegistry.newGauge(RIT_COUNT_OVER_THRESHOLD_NAME, "", 0l);
+    ritOldestAgeGauge = metricsRegistry.newGauge(RIT_OLDEST_AGE_NAME, "", 0l);
+    splitTimeHisto = metricsRegistry.newHistogram(SPLIT_SIZE_NAME, SPLIT_SIZE_DESC);
+    splitSizeHisto = metricsRegistry.newHistogram(SPLIT_TIME_NAME, SPLIT_TIME_DESC);
+  }
+
+  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) {
+    ritOldestAgeGauge.set(ritCount);
+  }
+
+  @Override
+  public void updateSplitTime(long time) {
+    splitTimeHisto.add(time);
+  }
+
+  @Override
+  public void updateSplitSize(long size) {
+    splitSizeHisto.add(size);
+  }
+
+  /**
+   * Method to export all the metrics.
+   *
+   * @param metricsBuilder Builder to accept metrics
+   * @param all            push all or only changed?
+   */
+  @Override
+  public void getMetrics(MetricsBuilder metricsBuilder, boolean all) {
+
+    MetricsRecordBuilder metricsRecordBuilder = metricsBuilder.addRecord(metricsName)
+        .setContext(metricsContext);
+
+    // masterWrapper can be null because this function is called inside of init.
+    if (masterWrapper != null) {
+      metricsRecordBuilder
+          .addGauge(MASTER_ACTIVE_TIME_NAME,
+              MASTER_ACTIVE_TIME_DESC, masterWrapper.getActiveTime())
+          .addGauge(MASTER_START_TIME_NAME,
+              MASTER_START_TIME_DESC, masterWrapper.getStartTime())
+          .addGauge(AVERAGE_LOAD_NAME, AVERAGE_LOAD_DESC, masterWrapper.getAverageLoad())
+          .addGauge(NUM_REGION_SERVERS_NAME,
+              NUMBER_OF_REGION_SERVERS_DESC, masterWrapper.getRegionServers())
+          .addGauge(NUM_DEAD_REGION_SERVERS_NAME,
+              NUMBER_OF_DEAD_REGION_SERVERS_DESC,
+              masterWrapper.getDeadRegionServers())
+          .tag(ZOOKEEPER_QUORUM_NAME, ZOOKEEPER_QUORUM_DESC, masterWrapper.getZookeeperQuorum())
+          .tag(SERVER_NAME_NAME, SERVER_NAME_DESC, masterWrapper.getServerName())
+          .tag(CLUSTER_ID_NAME, CLUSTER_ID_DESC, masterWrapper.getClusterId())
+          .tag(IS_ACTIVE_MASTER_NAME,
+              IS_ACTIVE_MASTER_DESC,
+              String.valueOf(masterWrapper.getIsActiveMaster()));
+    }
+
+    metricsRegistry.snapshot(metricsRecordBuilder, all);
+  }
+
+}

Added: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSourceImpl.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSourceImpl.java Tue Nov  6 23:22:01 2012
@@ -0,0 +1,165 @@
+/**
+ * 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.MetricsBuilder;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.impl.JmxCacheBuster;
+import org.apache.hadoop.metrics2.lib.*;
+import org.apache.hadoop.metrics2.source.JvmMetricsSource;
+
+/**
+ * Hadoop 1 implementation of BaseSource (using metrics2 framework)
+ */
+public class BaseSourceImpl implements BaseSource, MetricsSource {
+
+  private static enum DefaultMetricsSystemInitializer {
+    INSTANCE;
+    private boolean inited = false;
+    private JvmMetricsSource jvmMetricsSource;
+
+    synchronized void init(String name) {
+      if (inited) return;
+      inited = true;
+      DefaultMetricsSystem.initialize(HBASE_METRICS_SYSTEM_NAME);
+      jvmMetricsSource = JvmMetricsSource.create(name, "");
+
+    }
+  }
+
+  protected final DynamicMetricsRegistry metricsRegistry;
+  protected final String metricsName;
+  protected final String metricsDescription;
+  protected final String metricsContext;
+  protected final String metricsJmxContext;
+
+  public BaseSourceImpl(
+      String metricsName,
+      String metricsDescription,
+      String metricsContext,
+      String metricsJmxContext) {
+
+    this.metricsName = metricsName;
+    this.metricsDescription = metricsDescription;
+    this.metricsContext = metricsContext;
+    this.metricsJmxContext = metricsJmxContext;
+
+    metricsRegistry = new DynamicMetricsRegistry(metricsName).setContext(metricsContext);
+    DefaultMetricsSystemInitializer.INSTANCE.init(metricsName);
+
+    //Register this instance.
+    DefaultMetricsSystem.INSTANCE.registerSource(metricsJmxContext, metricsDescription, this);
+    init();
+  }
+
+  public void init() {
+    this.metricsRegistry.clearMetrics();
+  }
+
+
+  /**
+   * Set a single gauge to a value.
+   *
+   * @param gaugeName gauge name
+   * @param value     the new value of the gauge.
+   */
+  public void setGauge(String gaugeName, long value) {
+    MetricMutableGaugeLong gaugeInt = metricsRegistry.getLongGauge(gaugeName, value);
+    gaugeInt.set(value);
+  }
+
+  /**
+   * Add some amount to a gauge.
+   *
+   * @param gaugeName The name of the gauge to increment.
+   * @param delta     The amount to increment the gauge by.
+   */
+  public void incGauge(String gaugeName, long delta) {
+    MetricMutableGaugeLong gaugeInt = metricsRegistry.getLongGauge(gaugeName, 0l);
+    gaugeInt.incr(delta);
+  }
+
+  /**
+   * Decrease the value of a named gauge.
+   *
+   * @param gaugeName The name of the gauge.
+   * @param delta     the ammount to subtract from a gauge value.
+   */
+  public void decGauge(String gaugeName, long delta) {
+    MetricMutableGaugeLong gaugeInt = metricsRegistry.getLongGauge(gaugeName, 0l);
+    gaugeInt.decr(delta);
+  }
+
+  /**
+   * Increment a named counter by some value.
+   *
+   * @param key   the name of the counter
+   * @param delta the ammount to increment
+   */
+  public void incCounters(String key, long delta) {
+    MetricMutableCounterLong counter = metricsRegistry.getLongCounter(key, 0l);
+    counter.incr(delta);
+
+  }
+
+  @Override
+  public void updateHistogram(String name, long value) {
+    MetricMutableHistogram histo = metricsRegistry.getHistogram(name);
+    histo.add(value);
+  }
+
+  @Override
+  public void updateQuantile(String name, long value) {
+    MetricMutableQuantiles histo = metricsRegistry.getQuantile(name);
+    histo.add(value);
+  }
+
+  /**
+   * Remove a named metric.
+   *
+   * @param key
+   */
+  public void removeMetric(String key) {
+    metricsRegistry.removeMetric(key);
+    JmxCacheBuster.clearJmxCache();
+  }
+
+
+  /**
+   * Method to export all the metrics.
+   *
+   * @param metricsBuilder Builder to accept metrics
+   * @param all            push all or only changed?
+   */
+  @Override
+  public void getMetrics(MetricsBuilder metricsBuilder, boolean all) {
+    MetricsRecordBuilder mrb = metricsBuilder.addRecord(metricsName)
+                                             .setContext(metricsContext);
+    metricsRegistry.snapshot(mrb, all);
+  }
+
+  /**
+   * Used to get at the DynamicMetricsRegistry.
+   * @return DynamicMetricsRegistry
+   */
+  public DynamicMetricsRegistry getMetricsRegistry() {
+    return metricsRegistry;
+  }
+}

Added: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionAggregateSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionAggregateSourceImpl.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionAggregateSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionAggregateSourceImpl.java Tue Nov  6 23:22:01 2012
@@ -0,0 +1,82 @@
+/**
+ * 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.regionserver;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.metrics2.MetricsBuilder;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+
+import java.util.TreeSet;
+
+public class MetricsRegionAggregateSourceImpl extends BaseSourceImpl
+    implements MetricsRegionAggregateSource {
+  private final Log LOG = LogFactory.getLog(this.getClass());
+
+  private final TreeSet<MetricsRegionSourceImpl> regionSources =
+      new TreeSet<MetricsRegionSourceImpl>();
+
+  public MetricsRegionAggregateSourceImpl() {
+    this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);
+  }
+
+
+  public MetricsRegionAggregateSourceImpl(String metricsName,
+                                          String metricsDescription,
+                                          String metricsContext,
+                                          String metricsJmxContext) {
+    super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
+  }
+
+  @Override
+  public void register(MetricsRegionSource source) {
+    regionSources.add((MetricsRegionSourceImpl) source);
+  }
+
+  @Override
+  public void deregister(MetricsRegionSource source) {
+    regionSources.remove(source);
+  }
+
+  /**
+   * Yes this is a get function that doesn't return anything.  Thanks Hadoop for breaking all
+   * expectations of java programmers.  Instead of returning anything Hadoop metrics expects
+   * getMetrics to push the metrics into the metricsBuilder.
+   *
+   * @param metricsBuilder Builder to accept metrics
+   * @param all            push all or only changed?
+   */
+  @Override
+  public void getMetrics(MetricsBuilder metricsBuilder, boolean all) {
+
+
+    MetricsRecordBuilder mrb = metricsBuilder.addRecord(metricsName)
+        .setContext(metricsContext);
+
+    if (regionSources != null) {
+      for (MetricsRegionSourceImpl regionMetricSource : regionSources) {
+        regionMetricSource.snapshot(mrb, all);
+      }
+    }
+
+
+    metricsRegistry.snapshot(mrb, all);
+  }
+}

Added: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceFactoryImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceFactoryImpl.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceFactoryImpl.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceFactoryImpl.java Tue Nov  6 23:22:01 2012
@@ -0,0 +1,52 @@
+/**
+ * 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.regionserver;
+
+/**
+ * Factory to create MetricsRegionServerSource when given a  MetricsRegionServerWrapper
+ */
+public class MetricsRegionServerSourceFactoryImpl implements MetricsRegionServerSourceFactory {
+  private static enum FactoryStorage {
+    INSTANCE;
+    private MetricsRegionServerSource serverSource;
+    private MetricsRegionAggregateSourceImpl aggImpl;
+  }
+
+  private synchronized MetricsRegionAggregateSourceImpl getAggregate() {
+    if (FactoryStorage.INSTANCE.aggImpl == null) {
+      FactoryStorage.INSTANCE.aggImpl = new MetricsRegionAggregateSourceImpl();
+    }
+    return FactoryStorage.INSTANCE.aggImpl;
+  }
+
+
+  @Override
+  public synchronized MetricsRegionServerSource createServer(MetricsRegionServerWrapper regionServerWrapper) {
+    if (FactoryStorage.INSTANCE.serverSource == null) {
+      FactoryStorage.INSTANCE.serverSource = new MetricsRegionServerSourceImpl(
+          regionServerWrapper);
+    }
+    return FactoryStorage.INSTANCE.serverSource;
+  }
+
+  @Override
+  public MetricsRegionSource createRegion(MetricsRegionWrapper wrapper) {
+    return new MetricsRegionSourceImpl(wrapper, getAggregate());
+  }
+}

Added: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java Tue Nov  6 23:22:01 2012
@@ -0,0 +1,161 @@
+/**
+ * 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.regionserver;
+
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.metrics2.MetricHistogram;
+import org.apache.hadoop.metrics2.MetricsBuilder;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+
+/**
+ * Hadoop1 implementation of MetricsRegionServerSource.
+ */
+public class MetricsRegionServerSourceImpl
+    extends BaseSourceImpl implements MetricsRegionServerSource {
+
+  final MetricsRegionServerWrapper rsWrap;
+  private final MetricHistogram putHisto;
+  private final MetricHistogram deleteHisto;
+  private final MetricHistogram getHisto;
+  private final MetricHistogram incrementHisto;
+  private final MetricHistogram appendHisto;
+
+  public MetricsRegionServerSourceImpl(MetricsRegionServerWrapper rsWrap) {
+    this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT, rsWrap);
+  }
+
+  public MetricsRegionServerSourceImpl(String metricsName,
+                                       String metricsDescription,
+                                       String metricsContext,
+                                       String metricsJmxContext,
+                                       MetricsRegionServerWrapper rsWrap) {
+    super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
+    this.rsWrap = rsWrap;
+
+    putHisto = getMetricsRegistry().getHistogram(PUT_KEY);
+    deleteHisto = getMetricsRegistry().getHistogram(DELETE_KEY);
+    getHisto = getMetricsRegistry().getHistogram(GET_KEY);
+    incrementHisto = getMetricsRegistry().getHistogram(INCREMENT_KEY);
+    appendHisto = getMetricsRegistry().getHistogram(APPEND_KEY);
+  }
+
+  @Override
+  public void init() {
+    super.init();
+  }
+
+  @Override
+  public void updatePut(long t) {
+    putHisto.add(t);
+  }
+
+  @Override
+  public void updateDelete(long t) {
+    deleteHisto.add(t);
+  }
+
+  @Override
+  public void updateGet(long t) {
+    getHisto.add(t);
+  }
+
+  @Override
+  public void updateIncrement(long t) {
+    incrementHisto.add(t);
+  }
+
+  @Override
+  public void updateAppend(long t) {
+    appendHisto.add(t);
+  }
+
+  /**
+   * Yes this is a get function that doesn't return anything.  Thanks Hadoop for breaking all
+   * expectations of java programmers.  Instead of returning anything Hadoop metrics expects
+   * getMetrics to push the metrics into the metricsBuilder.
+   *
+   * @param metricsBuilder Builder to accept metrics
+   * @param all            push all or only changed?
+   */
+  @Override
+  public void getMetrics(MetricsBuilder metricsBuilder, boolean all) {
+
+    MetricsRecordBuilder mrb = metricsBuilder.addRecord(metricsName)
+        .setContext(metricsContext);
+
+    // rsWrap can be null because this function is called inside of init.
+    if (rsWrap != null) {
+      mrb.addGauge(REGION_COUNT, REGION_COUNT_DESC, rsWrap.getNumOnlineRegions())
+          .addGauge(STORE_COUNT, STORE_COUNT_DESC, rsWrap.getNumStores())
+          .addGauge(STOREFILE_COUNT, STOREFILE_COUNT_DESC, rsWrap.getNumStoreFiles())
+          .addGauge(MEMSTORE_SIZE, MEMSTORE_SIZE_DESC, rsWrap.getMemstoreSize())
+          .addGauge(STOREFILE_SIZE, STOREFILE_SIZE_DESC, rsWrap.getStoreFileSize())
+          .addGauge(RS_START_TIME_NAME, RS_START_TIME_DESC, rsWrap.getStartCode())
+          .addCounter(TOTAL_REQUEST_COUNT, TOTAL_REQUEST_COUNT_DESC, rsWrap.getTotalRequestCount())
+          .addCounter(READ_REQUEST_COUNT, READ_REQUEST_COUNT_DESC, rsWrap.getReadRequestsCount())
+          .addCounter(WRITE_REQUEST_COUNT, WRITE_REQUEST_COUNT_DESC, rsWrap.getWriteRequestsCount())
+          .addCounter(CHECK_MUTATE_FAILED_COUNT,
+              CHECK_MUTATE_FAILED_COUNT_DESC,
+              rsWrap.getCheckAndMutateChecksFailed())
+          .addCounter(CHECK_MUTATE_PASSED_COUNT,
+              CHECK_MUTATE_PASSED_COUNT_DESC,
+              rsWrap.getCheckAndMutateChecksPassed())
+          .addGauge(STOREFILE_INDEX_SIZE, STOREFILE_INDEX_SIZE_DESC, rsWrap.getStoreFileIndexSize())
+          .addGauge(STATIC_INDEX_SIZE, STATIC_INDEX_SIZE_DESC, rsWrap.getTotalStaticIndexSize())
+          .addGauge(STATIC_BLOOM_SIZE, STATIC_BLOOM_SIZE_DESC, rsWrap.getTotalStaticBloomSize())
+          .addGauge(NUMBER_OF_PUTS_WITHOUT_WAL,
+              NUMBER_OF_PUTS_WITHOUT_WAL_DESC,
+              rsWrap.getNumPutsWithoutWAL())
+          .addGauge(DATA_SIZE_WITHOUT_WAL,
+              DATA_SIZE_WITHOUT_WAL_DESC,
+              rsWrap.getDataInMemoryWithoutWAL())
+          .addGauge(PERCENT_FILES_LOCAL, PERCENT_FILES_LOCAL_DESC, rsWrap.getPercentFileLocal())
+          .addGauge(COMPACTION_QUEUE_LENGTH,
+              COMPACTION_QUEUE_LENGTH_DESC,
+              rsWrap.getCompactionQueueSize())
+          .addGauge(FLUSH_QUEUE_LENGTH, FLUSH_QUEUE_LENGTH_DESC, rsWrap.getFlushQueueSize())
+          .addGauge(BLOCK_CACHE_FREE_SIZE, BLOCK_CACHE_FREE_DESC, rsWrap.getBlockCacheFreeSize())
+          .addGauge(BLOCK_CACHE_COUNT, BLOCK_CACHE_COUNT_DESC, rsWrap.getBlockCacheCount())
+          .addGauge(BLOCK_CACHE_SIZE, BLOCK_CACHE_SIZE_DESC, rsWrap.getBlockCacheSize())
+          .addCounter(BLOCK_CACHE_HIT_COUNT,
+              BLOCK_CACHE_HIT_COUNT_DESC,
+              rsWrap.getBlockCacheHitCount())
+          .addCounter(BLOCK_CACHE_MISS_COUNT,
+              BLOCK_COUNT_MISS_COUNT_DESC,
+              rsWrap.getBlockCacheMissCount())
+          .addCounter(BLOCK_CACHE_EVICTION_COUNT,
+              BLOCK_CACHE_EVICTION_COUNT_DESC,
+              rsWrap.getBlockCacheEvictedCount())
+          .addGauge(BLOCK_CACHE_HIT_PERCENT,
+              BLOCK_CACHE_HIT_PERCENT_DESC,
+              rsWrap.getBlockCacheHitPercent())
+          .addGauge(BLOCK_CACHE_EXPRESS_HIT_PERCENT,
+              BLOCK_CACHE_EXPRESS_HIT_PERCENT_DESC,
+              rsWrap.getBlockCacheHitCachingPercent())
+          .addCounter(UPDATES_BLOCKED_TIME, UPDATES_BLOCKED_DESC, rsWrap.getUpdatesBlockedTime())
+          .tag(ZOOKEEPER_QUORUM_NAME, ZOOKEEPER_QUORUM_DESC, rsWrap.getZookeeperQuorum())
+          .tag(SERVER_NAME_NAME, SERVER_NAME_DESC, rsWrap.getServerName())
+          .tag(CLUSTER_ID_NAME, CLUSTER_ID_DESC, rsWrap.getClusterId());
+    }
+
+    metricsRegistry.snapshot(mrb, all);
+  }
+
+
+}

Added: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java Tue Nov  6 23:22:01 2012
@@ -0,0 +1,163 @@
+/**
+ * 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.regionserver;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.impl.JmxCacheBuster;
+import org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MetricMutableCounterLong;
+
+public class MetricsRegionSourceImpl implements MetricsRegionSource {
+
+  private final MetricsRegionWrapper regionWrapper;
+  private boolean closed = false;
+  private MetricsRegionAggregateSourceImpl agg;
+  private DynamicMetricsRegistry registry;
+  private static final Log LOG = LogFactory.getLog(MetricsRegionSourceImpl.class);
+
+  private String regionNamePrefix;
+  private String regionPutKey;
+  private String regionDeleteKey;
+  private String regionGetKey;
+  private String regionIncrementKey;
+  private String regionAppendKey;
+  private MetricMutableCounterLong regionPut;
+  private MetricMutableCounterLong regionDelete;
+  private MetricMutableCounterLong regionGet;
+  private MetricMutableCounterLong regionIncrement;
+  private MetricMutableCounterLong regionAppend;
+
+  public MetricsRegionSourceImpl(MetricsRegionWrapper regionWrapper,
+                                 MetricsRegionAggregateSourceImpl aggregate) {
+    this.regionWrapper = regionWrapper;
+    agg = aggregate;
+    agg.register(this);
+
+    LOG.debug("Creating new MetricsRegionSourceImpl for table " +
+        regionWrapper.getTableName() +
+        " " +
+        regionWrapper.getRegionName());
+
+    registry = agg.getMetricsRegistry();
+
+    regionNamePrefix = "table." + regionWrapper.getTableName() + "."
+        + "region." + regionWrapper.getRegionName() + ".";
+
+    String suffix = "Count";
+
+
+    regionPutKey = regionNamePrefix + MetricsRegionServerSource.PUT_KEY + suffix;
+    regionPut = registry.getLongCounter(regionPutKey, 0l);
+
+    regionDeleteKey = regionNamePrefix + MetricsRegionServerSource.DELETE_KEY + suffix;
+    regionDelete = registry.getLongCounter(regionDeleteKey, 0l);
+
+    regionGetKey = regionNamePrefix + MetricsRegionServerSource.GET_KEY + suffix;
+    regionGet = registry.getLongCounter(regionGetKey, 0l);
+
+    regionIncrementKey = regionNamePrefix + MetricsRegionServerSource.INCREMENT_KEY + suffix;
+    regionIncrement = registry.getLongCounter(regionIncrementKey, 0l);
+
+    regionAppendKey = regionNamePrefix + MetricsRegionServerSource.APPEND_KEY + suffix;
+    regionAppend = registry.getLongCounter(regionAppendKey, 0l);
+  }
+
+  @Override
+  public void close() {
+    closed = true;
+    agg.deregister(this);
+
+    LOG.trace("Removing region Metrics: " + regionWrapper.getRegionName());
+    registry.removeMetric(regionPutKey);
+    registry.removeMetric(regionDeleteKey);
+    registry.removeMetric(regionGetKey);
+    registry.removeMetric(regionIncrementKey);
+
+    registry.removeMetric(regionAppendKey);
+
+    JmxCacheBuster.clearJmxCache();
+  }
+
+  @Override
+  public void updatePut() {
+    regionPut.incr();
+  }
+
+  @Override
+  public void updateDelete() {
+    regionDelete.incr();
+  }
+
+  @Override
+  public void updateGet() {
+    regionGet.incr();
+  }
+
+  @Override
+  public void updateIncrement() {
+    regionIncrement.incr();
+  }
+
+  @Override
+  public void updateAppend() {
+    regionAppend.incr();
+  }
+
+  @Override
+  public MetricsRegionAggregateSource getAggregateSource() {
+    return agg;
+  }
+
+  @Override
+  public int compareTo(MetricsRegionSource source) {
+
+    if (!(source instanceof MetricsRegionSourceImpl))
+      return -1;
+
+    MetricsRegionSourceImpl impl = (MetricsRegionSourceImpl) source;
+    return this.regionWrapper.getRegionName()
+        .compareTo(impl.regionWrapper.getRegionName());
+  }
+
+  void snapshot(MetricsRecordBuilder mrb, boolean ignored) {
+    if (closed) return;
+
+    mrb.addGauge(regionNamePrefix + MetricsRegionServerSource.STORE_COUNT,
+        MetricsRegionServerSource.STORE_COUNT_DESC,
+        this.regionWrapper.getNumStores());
+    mrb.addGauge(regionNamePrefix + MetricsRegionServerSource.STOREFILE_COUNT,
+        MetricsRegionServerSource.STOREFILE_COUNT_DESC,
+        this.regionWrapper.getNumStoreFiles());
+    mrb.addGauge(regionNamePrefix + MetricsRegionServerSource.MEMSTORE_SIZE,
+        MetricsRegionServerSource.MEMSTORE_SIZE_DESC,
+        this.regionWrapper.getMemstoreSize());
+    mrb.addGauge(regionNamePrefix + MetricsRegionServerSource.STOREFILE_SIZE,
+        MetricsRegionServerSource.STOREFILE_SIZE_DESC,
+        this.regionWrapper.getStoreFileSize());
+    mrb.addCounter(regionNamePrefix + MetricsRegionServerSource.READ_REQUEST_COUNT,
+        MetricsRegionServerSource.READ_REQUEST_COUNT_DESC,
+        this.regionWrapper.getReadRequestCount());
+    mrb.addCounter(regionNamePrefix + MetricsRegionServerSource.WRITE_REQUEST_COUNT,
+        MetricsRegionServerSource.WRITE_REQUEST_COUNT_DESC,
+        this.regionWrapper.getWriteRequestCount());
+
+  }
+}

Added: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceImpl.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceImpl.java Tue Nov  6 23:22:01 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.replication.regionserver;
+
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+
+/**
+ * Hadoop1 implementation of MetricsReplicationSource. This provides access to metrics gauges and
+ * counters.
+ */
+public class MetricsReplicationSourceImpl extends BaseSourceImpl implements
+    MetricsReplicationSource {
+
+  public MetricsReplicationSourceImpl() {
+    this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);
+  }
+
+  MetricsReplicationSourceImpl(String metricsName,
+                               String metricsDescription,
+                               String metricsContext,
+                               String metricsJmxContext) {
+    super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
+  }
+}

Added: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/rest/MetricsRESTSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/rest/MetricsRESTSourceImpl.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/rest/MetricsRESTSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/rest/MetricsRESTSourceImpl.java Tue Nov  6 23:22:01 2012
@@ -0,0 +1,97 @@
+/**
+ * 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.rest;
+
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.metrics2.lib.MetricMutableCounterLong;
+
+/**
+ * Hadoop One implementation of a metrics2 source that will export metrics from the Rest server to
+ * the hadoop metrics2 subsystem.
+ */
+public class MetricsRESTSourceImpl extends BaseSourceImpl implements MetricsRESTSource {
+
+  private MetricMutableCounterLong request;
+  private MetricMutableCounterLong sucGet;
+  private MetricMutableCounterLong sucPut;
+  private MetricMutableCounterLong sucDel;
+  private MetricMutableCounterLong fGet;
+  private MetricMutableCounterLong fPut;
+  private MetricMutableCounterLong fDel;
+
+  public MetricsRESTSourceImpl() {
+    this(METRICS_NAME, METRICS_DESCRIPTION, CONTEXT, JMX_CONTEXT);
+  }
+
+  public MetricsRESTSourceImpl(String metricsName,
+                               String metricsDescription,
+                               String metricsContext,
+                               String metricsJmxContext) {
+    super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
+  }
+
+  @Override
+  public void init() {
+    super.init();
+    request = getMetricsRegistry().getLongCounter(REQUEST_KEY, 0l);
+
+    sucGet = getMetricsRegistry().getLongCounter(SUCCESSFUL_GET_KEY, 0l);
+    sucPut = getMetricsRegistry().getLongCounter(SUCCESSFUL_PUT_KEY, 0l);
+    sucDel = getMetricsRegistry().getLongCounter(SUCCESSFUL_DELETE_KEY, 0l);
+
+    fGet = getMetricsRegistry().getLongCounter(FAILED_GET_KEY, 0l);
+    fPut = getMetricsRegistry().getLongCounter(FAILED_PUT_KEY, 0l);
+    fDel = getMetricsRegistry().getLongCounter(FAILED_DELETE_KEY, 0l);
+  }
+
+  @Override
+  public void incrementRequests(int inc) {
+    request.incr(inc);
+  }
+
+  @Override
+  public void incrementSucessfulGetRequests(int inc) {
+    sucGet.incr(inc);
+  }
+
+  @Override
+  public void incrementSucessfulPutRequests(int inc) {
+    sucPut.incr(inc);
+  }
+
+  @Override
+  public void incrementSucessfulDeleteRequests(int inc) {
+    sucDel.incr(inc);
+  }
+
+  @Override
+  public void incrementFailedGetRequests(int inc) {
+    fGet.incr(inc);
+  }
+
+  @Override
+  public void incrementFailedPutRequests(int inc) {
+    fPut.incr(inc);
+  }
+
+  @Override
+  public void incrementFailedDeleteRequests(int inc) {
+    fDel.incr(inc);
+  }
+}

Added: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/thrift/MetricsThriftServerSourceFactoryImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/thrift/MetricsThriftServerSourceFactoryImpl.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/thrift/MetricsThriftServerSourceFactoryImpl.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/thrift/MetricsThriftServerSourceFactoryImpl.java Tue Nov  6 23:22:01 2012
@@ -0,0 +1,52 @@
+/**
+ * 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.thrift;
+
+/**
+ * Class used to create metrics sources for Thrift and Thrift2 servers in hadoop 1's compat
+ * library.
+ */
+public class MetricsThriftServerSourceFactoryImpl implements MetricsThriftServerSourceFactory {
+
+  /**
+   * A singleton used to make sure that only one thrift metrics source per server type is ever
+   * created.
+   */
+  private static enum FactoryStorage {
+    INSTANCE;
+    MetricsThriftServerSourceImpl thriftOne = new MetricsThriftServerSourceImpl(METRICS_NAME,
+        METRICS_DESCRIPTION,
+        THRIFT_ONE_METRICS_CONTEXT,
+        THRIFT_ONE_JMX_CONTEXT);
+    MetricsThriftServerSourceImpl thriftTwo = new MetricsThriftServerSourceImpl(METRICS_NAME,
+        METRICS_DESCRIPTION,
+        THRIFT_TWO_METRICS_CONTEXT,
+        THRIFT_TWO_JMX_CONTEXT);
+  }
+
+  @Override
+  public MetricsThriftServerSource createThriftOneSource() {
+    return FactoryStorage.INSTANCE.thriftOne;
+  }
+
+  @Override
+  public MetricsThriftServerSource createThriftTwoSource() {
+    return FactoryStorage.INSTANCE.thriftTwo;
+  }
+}

Added: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/thrift/MetricsThriftServerSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/thrift/MetricsThriftServerSourceImpl.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/thrift/MetricsThriftServerSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/thrift/MetricsThriftServerSourceImpl.java Tue Nov  6 23:22:01 2012
@@ -0,0 +1,96 @@
+/**
+ * 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.thrift;
+
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.metrics2.lib.MetricMutableGaugeLong;
+import org.apache.hadoop.metrics2.lib.MetricMutableStat;
+
+/**
+ * Hadoop 1 version of MetricsThriftServerSource{@link MetricsThriftServerSource}
+ */
+public class MetricsThriftServerSourceImpl extends BaseSourceImpl implements
+    MetricsThriftServerSource {
+
+
+  private MetricMutableStat batchGetStat;
+  private MetricMutableStat batchMutateStat;
+  private MetricMutableStat queueTimeStat;
+
+  private MetricMutableStat thriftCallStat;
+  private MetricMutableStat thriftSlowCallStat;
+
+  private MetricMutableGaugeLong callQueueLenGauge;
+
+  public MetricsThriftServerSourceImpl(String metricsName,
+                                       String metricsDescription,
+                                       String metricsContext,
+                                       String metricsJmxContext) {
+    super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
+  }
+
+
+  @Override
+  public void init() {
+    super.init();
+    batchGetStat = getMetricsRegistry().newStat(BATCH_GET_KEY, "", "Keys", "Ops");
+    batchMutateStat = getMetricsRegistry().newStat(BATCH_MUTATE_KEY, "", "Keys", "Ops");
+    queueTimeStat = getMetricsRegistry().newStat(TIME_IN_QUEUE_KEY);
+    thriftCallStat = getMetricsRegistry().newStat(THRIFT_CALL_KEY);
+    thriftSlowCallStat = getMetricsRegistry().newStat(SLOW_THRIFT_CALL_KEY);
+    callQueueLenGauge = getMetricsRegistry().getLongGauge(CALL_QUEUE_LEN_KEY, 0);
+  }
+
+  @Override
+  public void incTimeInQueue(long time) {
+    queueTimeStat.add(time);
+  }
+
+  @Override
+  public void setCallQueueLen(int len) {
+    callQueueLenGauge.set(len);
+  }
+
+  @Override
+  public void incNumRowKeysInBatchGet(int diff) {
+    batchGetStat.add(diff);
+  }
+
+  @Override
+  public void incNumRowKeysInBatchMutate(int diff) {
+    batchMutateStat.add(diff);
+  }
+
+  @Override
+  public void incMethodTime(String name, long time) {
+    MetricMutableStat s = getMetricsRegistry().newStat(name);
+    s.add(time);
+  }
+
+  @Override
+  public void incCall(long time) {
+    thriftCallStat.add(time);
+  }
+
+  @Override
+  public void incSlowCall(long time) {
+    thriftSlowCallStat.add(time);
+  }
+
+}

Added: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java Tue Nov  6 23:22:01 2012
@@ -0,0 +1,52 @@
+/**
+ * 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.impl;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+
+/**
+ * JMX caches the beans that have been exported; even after the values are removed from hadoop's
+ * metrics system the keys and old values will still remain.  This class stops and restarts the
+ * Hadoop metrics system, forcing JMX to clear the cache of exported metrics.
+ *
+ * This class need to be in the o.a.h.metrics2.impl namespace as many of the variables/calls used
+ * are package private.
+ */
+public class JmxCacheBuster {
+  private static final Log LOG = LogFactory.getLog(JmxCacheBuster.class);
+
+  /**
+   * For JMX to forget about all previously exported metrics.
+   */
+  public static void clearJmxCache() {
+    LOG.trace("Clearing JMX mbean cache.");
+
+    // This is pretty extreme but it's the best way that
+    // I could find to get metrics to be removed.
+
+    try {
+      DefaultMetricsSystem.INSTANCE.stop();
+      DefaultMetricsSystem.INSTANCE.start();
+    }  catch (Exception exception )  {
+      LOG.debug("error clearing the jmx it appears the metrics system hasn't been started", exception);
+    }
+  }
+}

Modified: 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=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java (original)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java Tue Nov  6 23:22:01 2012
@@ -23,6 +23,8 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.metrics2.MetricsException;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.MetricsTag;
@@ -39,6 +41,8 @@ import org.apache.hadoop.metrics2.Metric
  */
 public class DynamicMetricsRegistry {
 
+  private final Log LOG = LogFactory.getLog(this.getClass());
+
   /** key for the context tag */
   public static final String CONTEXT_KEY = "context";
   /** description for the context tag */
@@ -284,6 +288,7 @@ public class DynamicMetricsRegistry {
    * @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());
     }

Modified: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/lib/MetricMutableHistogram.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/lib/MetricMutableHistogram.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/lib/MetricMutableHistogram.java (original)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/lib/MetricMutableHistogram.java Tue Nov  6 23:22:01 2012
@@ -21,9 +21,8 @@ package org.apache.hadoop.metrics2.lib;
 import com.yammer.metrics.stats.ExponentiallyDecayingSample;
 import com.yammer.metrics.stats.Sample;
 import com.yammer.metrics.stats.Snapshot;
-import org.apache.hadoop.metrics.MetricHistogram;
+import org.apache.hadoop.metrics2.MetricHistogram;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
-import org.apache.hadoop.metrics2.lib.MetricMutable;
 
 import java.util.concurrent.atomic.AtomicLong;
 

Modified: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/lib/MetricMutableQuantiles.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/lib/MetricMutableQuantiles.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/lib/MetricMutableQuantiles.java (original)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/lib/MetricMutableQuantiles.java Tue Nov  6 23:22:01 2012
@@ -20,8 +20,8 @@ package org.apache.hadoop.metrics2.lib;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.metrics.MetricHistogram;
-import org.apache.hadoop.metrics.MetricsExecutor;
+import org.apache.hadoop.metrics2.MetricHistogram;
+import org.apache.hadoop.metrics2.MetricsExecutor;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.util.MetricQuantile;
 import org.apache.hadoop.metrics2.util.MetricSampleQuantiles;

Modified: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/lib/MetricsExecutorImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/lib/MetricsExecutorImpl.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/lib/MetricsExecutorImpl.java (original)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/lib/MetricsExecutorImpl.java Tue Nov  6 23:22:01 2012
@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.metrics2.lib;
 
-import org.apache.hadoop.metrics.MetricsExecutor;
+import org.apache.hadoop.metrics2.MetricsExecutor;
 
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledThreadPoolExecutor;

Added: hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterSourceFactory
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterSourceFactory?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterSourceFactory (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.master.MetricsMasterSourceFactory Tue Nov  6 23:22:01 2012
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.master.MetricsMasterSourceFactoryImpl
\ No newline at end of file

Added: hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.regionserver.MetricsRegionServerSourceFactory
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.regionserver.MetricsRegionServerSourceFactory?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.regionserver.MetricsRegionServerSourceFactory (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.regionserver.MetricsRegionServerSourceFactory Tue Nov  6 23:22:01 2012
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.regionserver.MetricsRegionServerSourceFactoryImpl

Added: hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationSource
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationSource?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationSource (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationSource Tue Nov  6 23:22:01 2012
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationSourceImpl
\ No newline at end of file

Added: hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.rest.MetricsRESTSource
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.rest.MetricsRESTSource?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.rest.MetricsRESTSource (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.rest.MetricsRESTSource Tue Nov  6 23:22:01 2012
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.rest.MetricsRESTSourceImpl
\ No newline at end of file

Added: hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.thrift.MetricsThriftServerSourceFactory
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.thrift.MetricsThriftServerSourceFactory?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.thrift.MetricsThriftServerSourceFactory (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.thrift.MetricsThriftServerSourceFactory Tue Nov  6 23:22:01 2012
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.thrift.MetricsThriftServerSourceFactoryImpl
\ No newline at end of file

Added: hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.metrics2.MetricsExecutor
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.metrics2.MetricsExecutor?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.metrics2.MetricsExecutor (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.metrics2.MetricsExecutor Tue Nov  6 23:22:01 2012
@@ -0,0 +1 @@
+org.apache.hadoop.metrics2.lib.MetricsExecutorImpl

Added: hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/master/TestMetricsMasterSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/master/TestMetricsMasterSourceImpl.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/master/TestMetricsMasterSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/master/TestMetricsMasterSourceImpl.java Tue Nov  6 23:22:01 2012
@@ -0,0 +1,44 @@
+/**
+ * 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;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.hbase.master.MetricsMasterSource;
+import org.apache.hadoop.hbase.master.MetricsMasterSourceFactory;
+import org.apache.hadoop.hbase.master.MetricsMasterSourceImpl;
+import org.junit.Test;
+
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+/**
+ *  Test for MetricsMasterSourceImpl
+ */
+public class TestMetricsMasterSourceImpl {
+
+  @Test
+  public void testGetInstance() throws Exception {
+    MetricsMasterSourceFactory metricsMasterSourceFactory = CompatibilitySingletonFactory
+        .getInstance(MetricsMasterSourceFactory.class);
+    MetricsMasterSource masterSource = metricsMasterSourceFactory.create(null);
+    assertTrue(masterSource instanceof MetricsMasterSourceImpl);
+    assertSame(metricsMasterSourceFactory, CompatibilitySingletonFactory.getInstance(MetricsMasterSourceFactory.class));
+  }
+
+}

Added: hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseSourceImpl.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseSourceImpl.java Tue Nov  6 23:22:01 2012
@@ -0,0 +1,91 @@
+/**
+ * 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.lib.MetricMutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MetricMutableGaugeLong;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
+
+/**
+ * Test of the default BaseSource implementation for hadoop 1
+ */
+public class TestBaseSourceImpl {
+
+  private static BaseSourceImpl bmsi;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    bmsi = new BaseSourceImpl("TestName", "test description", "testcontext", "TestContext");
+  }
+
+  @Test
+  public void testSetGauge() throws Exception {
+    String key = "testset";
+    bmsi.setGauge(key, 100);
+    MetricMutableGaugeLong g = (MetricMutableGaugeLong) bmsi.metricsRegistry.get(key);
+    assertEquals(key, g.name);
+    bmsi.setGauge(key, 110);
+    assertSame(g, bmsi.metricsRegistry.get(key));
+
+  }
+
+  @Test
+  public void testIncGauge() throws Exception {
+    String key = "testincgauge";
+    bmsi.incGauge(key, 100);
+    MetricMutableGaugeLong g = (MetricMutableGaugeLong) bmsi.metricsRegistry.get(key);
+    assertEquals(key, g.name);
+    bmsi.incGauge(key, 10);
+    assertSame(g, bmsi.metricsRegistry.get(key));
+  }
+
+  @Test
+  public void testDecGauge() throws Exception {
+    String key = "testdec";
+    bmsi.decGauge(key, 100);
+    MetricMutableGaugeLong g = (MetricMutableGaugeLong) bmsi.metricsRegistry.get(key);
+    assertEquals(key, g.name);
+    bmsi.decGauge(key, 100);
+    assertSame(g, bmsi.metricsRegistry.get(key));
+  }
+
+  @Test
+  public void testIncCounters() throws Exception {
+    String key = "testinccounter";
+    bmsi.incCounters(key, 100);
+    MetricMutableCounterLong c = (MetricMutableCounterLong) bmsi.metricsRegistry.get(key);
+    assertEquals(key, c.name);
+    bmsi.incCounters(key, 100);
+    assertSame(c, bmsi.metricsRegistry.get(key));
+  }
+
+  @Test
+  public void testRemoveMetric() throws Exception {
+    bmsi.setGauge("testrm", 100);
+    bmsi.removeMetric("testrm");
+    assertNull(bmsi.metricsRegistry.get("testrm"));
+
+  }
+
+}

Added: hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServerSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServerSourceImpl.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServerSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServerSourceImpl.java Tue Nov  6 23:22:01 2012
@@ -0,0 +1,51 @@
+/**
+ * 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.regionserver;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.junit.Test;
+
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test for MetricsRegionServerSourceImpl
+ */
+public class TestMetricsRegionServerSourceImpl {
+
+  @Test
+  public void testGetInstance() throws Exception {
+    MetricsRegionServerSourceFactory metricsRegionServerSourceFactory =
+        CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceFactory.class);
+    MetricsRegionServerSource serverSource =
+        metricsRegionServerSourceFactory.createServer(null);
+    assertTrue(serverSource instanceof MetricsRegionServerSourceImpl);
+    assertSame(metricsRegionServerSourceFactory,
+        CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceFactory.class));
+  }
+
+
+  @Test(expected = RuntimeException.class)
+  public void testNoGetRegionServerMetricsSourceImpl() throws Exception {
+    // This should throw an exception because MetricsRegionServerSourceImpl should only
+    // be created by a factory.
+    CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceImpl.class);
+  }
+
+}

Added: hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java Tue Nov  6 23:22:01 2012
@@ -0,0 +1,101 @@
+/**
+ * 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.regionserver;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestMetricsRegionSourceImpl {
+
+  @Test
+  public void testCompareTo() throws Exception {
+    MetricsRegionServerSourceFactory fact = CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceFactory.class);
+
+    MetricsRegionSource one = fact.createRegion(new RegionWrapperStub("TEST"));
+    MetricsRegionSource oneClone = fact.createRegion(new RegionWrapperStub("TEST"));
+    MetricsRegionSource two = fact.createRegion(new RegionWrapperStub("TWO"));
+
+    assertEquals(0, one.compareTo(oneClone));
+
+    assertTrue( one.compareTo(two) < 0);
+    assertTrue( two.compareTo(one) > 0);
+  }
+
+
+  @Test(expected = RuntimeException.class)
+  public void testNoGetRegionServerMetricsSourceImpl() throws Exception {
+    // This should throw an exception because MetricsRegionSourceImpl should only
+    // be created by a factory.
+    CompatibilitySingletonFactory.getInstance(MetricsRegionSource.class);
+  }
+
+  class RegionWrapperStub implements MetricsRegionWrapper {
+
+    private String regionName;
+
+    public RegionWrapperStub(String regionName) {
+
+
+      this.regionName = regionName;
+    }
+
+    @Override
+    public String getTableName() {
+      return null;  //To change body of implemented methods use File | Settings | File Templates.
+    }
+
+    @Override
+    public String getRegionName() {
+      return this.regionName;
+    }
+
+    @Override
+    public long getNumStores() {
+      return 0;  //To change body of implemented methods use File | Settings | File Templates.
+    }
+
+    @Override
+    public long getNumStoreFiles() {
+      return 0;  //To change body of implemented methods use File | Settings | File Templates.
+    }
+
+    @Override
+    public long getMemstoreSize() {
+      return 0;  //To change body of implemented methods use File | Settings | File Templates.
+    }
+
+    @Override
+    public long getStoreFileSize() {
+      return 0;  //To change body of implemented methods use File | Settings | File Templates.
+    }
+
+    @Override
+    public long getReadRequestCount() {
+      return 0;  //To change body of implemented methods use File | Settings | File Templates.
+    }
+
+    @Override
+    public long getWriteRequestCount() {
+      return 0;  //To change body of implemented methods use File | Settings | File Templates.
+    }
+  }
+}

Added: hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationMetricsSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationMetricsSourceImpl.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationMetricsSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationMetricsSourceImpl.java Tue Nov  6 23:22:01 2012
@@ -0,0 +1,39 @@
+/**
+ * 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.replication.regionserver;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationSource;
+import org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationSourceImpl;
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ *  Test to make sure that MetricsReplicationSourceImpl is hooked up to ServiceLoader
+ */
+public class TestReplicationMetricsSourceImpl {
+
+  @Test
+  public void testGetInstance() throws Exception {
+    MetricsReplicationSource rms = CompatibilitySingletonFactory
+        .getInstance(MetricsReplicationSource.class);
+    assertTrue(rms instanceof MetricsReplicationSourceImpl);
+  }
+}

Added: hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/rest/TestRESTMetricsSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/rest/TestRESTMetricsSourceImpl.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/rest/TestRESTMetricsSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/rest/TestRESTMetricsSourceImpl.java Tue Nov  6 23:22:01 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.rest;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.hbase.rest.MetricsRESTSource;
+import org.apache.hadoop.hbase.rest.MetricsRESTSourceImpl;
+import org.junit.Test;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ *  Test for hadoop1's version of MetricsRESTSource
+ */
+public class TestRESTMetricsSourceImpl {
+
+  @Test
+  public void ensureCompatRegistered() throws Exception {
+    assertNotNull(CompatibilitySingletonFactory.getInstance(MetricsRESTSource.class));
+    assertTrue(CompatibilitySingletonFactory.getInstance(MetricsRESTSource.class) instanceof MetricsRESTSourceImpl);
+  }
+
+}