You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2012/09/18 21:10:06 UTC

svn commit: r1387323 [2/2] - 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/metrics/ hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/rest/ hba...

Added: hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/rest/metrics/RESTMetricsSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/rest/metrics/RESTMetricsSourceImpl.java?rev=1387323&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/rest/metrics/RESTMetricsSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/rest/metrics/RESTMetricsSourceImpl.java Tue Sep 18 19:10:03 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.metrics;
+
+import org.apache.hadoop.hbase.metrics.BaseMetricsSourceImpl;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+
+/**
+ * Hadoop Two implementation of a metrics2 source that will export metrics from the Rest server to
+ * the hadoop metrics2 subsystem.
+ */
+public class RESTMetricsSourceImpl extends BaseMetricsSourceImpl implements RESTMetricsSource {
+
+  private MutableCounterLong request;
+  private MutableCounterLong sucGet;
+  private MutableCounterLong sucPut;
+  private MutableCounterLong sucDel;
+  private MutableCounterLong fGet;
+  private MutableCounterLong fPut;
+  private MutableCounterLong fDel;
+
+  public RESTMetricsSourceImpl() {
+    this(METRICS_NAME, METRICS_DESCRIPTION, CONTEXT, JMX_CONTEXT);
+  }
+
+  public RESTMetricsSourceImpl(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-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/thrift/metrics/ThriftServerMetricsSourceFactoryImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/thrift/metrics/ThriftServerMetricsSourceFactoryImpl.java?rev=1387323&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/thrift/metrics/ThriftServerMetricsSourceFactoryImpl.java (added)
+++ hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/thrift/metrics/ThriftServerMetricsSourceFactoryImpl.java Tue Sep 18 19:10:03 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.thrift.metrics;
+
+/**
+ *  Class used to create metrics sources for Thrift and Thrift2 servers.
+ */
+public class ThriftServerMetricsSourceFactoryImpl implements ThriftServerMetricsSourceFactory {
+
+  /**
+   * A singleton used to make sure that only one thrift metrics source per server type is ever
+   * created.
+   */
+  private static enum FactoryStorage {
+    INSTANCE;
+    ThriftServerMetricsSourceImpl thriftOne = new ThriftServerMetricsSourceImpl(METRICS_NAME,
+        METRICS_DESCRIPTION,
+        THRIFT_ONE_METRICS_CONTEXT,
+        THRIFT_ONE_JMX_CONTEXT);
+    ThriftServerMetricsSourceImpl thriftTwo = new ThriftServerMetricsSourceImpl(METRICS_NAME,
+        METRICS_DESCRIPTION,
+        THRIFT_TWO_METRICS_CONTEXT,
+        THRIFT_TWO_JMX_CONTEXT);
+  }
+
+  @Override
+  public ThriftServerMetricsSource createThriftOneSource() {
+    return FactoryStorage.INSTANCE.thriftOne;
+  }
+
+  @Override
+  public ThriftServerMetricsSource createThriftTwoSource() {
+    return FactoryStorage.INSTANCE.thriftTwo;
+  }
+}

Added: hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/thrift/metrics/ThriftServerMetricsSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/thrift/metrics/ThriftServerMetricsSourceImpl.java?rev=1387323&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/thrift/metrics/ThriftServerMetricsSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/thrift/metrics/ThriftServerMetricsSourceImpl.java Tue Sep 18 19:10:03 2012
@@ -0,0 +1,98 @@
+/**
+ * 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.metrics;
+
+import org.apache.hadoop.hbase.metrics.BaseMetricsSourceImpl;
+import org.apache.hadoop.hbase.thrift.metrics.ThriftServerMetricsSource;
+import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
+import org.apache.hadoop.metrics2.lib.MutableStat;
+
+/**
+ * Hadoop 2 version of ThriftServerMetricsSource{@link ThriftServerMetricsSource}
+ */
+public class ThriftServerMetricsSourceImpl extends BaseMetricsSourceImpl implements
+    ThriftServerMetricsSource {
+
+  private MutableStat batchGetStat;
+  private MutableStat batchMutateStat;
+  private MutableStat queueTimeStat;
+
+  private MutableStat thriftCallStat;
+  private MutableStat thriftSlowCallStat;
+
+  private MutableGaugeLong callQueueLenGauge;
+
+  public ThriftServerMetricsSourceImpl(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().newRate(TIME_IN_QUEUE_KEY) ;
+
+    thriftCallStat = getMetricsRegistry().newRate(THRIFT_CALL_KEY);
+    thriftSlowCallStat = getMetricsRegistry().newRate(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) {
+    MutableStat s = getMetricsRegistry().newRate(name);
+    s.add(time);
+  }
+
+  @Override
+  public void incCall(long time) {
+    thriftCallStat.add(time);
+  }
+
+  @Override
+  public void incSlowCall(long time) {
+    thriftSlowCallStat.add(time);
+  }
+
+}

Modified: 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=1387323&r1=1387322&r2=1387323&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java (original)
+++ hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java Tue Sep 18 19:10:03 2012
@@ -465,4 +465,8 @@ public class DynamicMetricsRegistry {
 
     return (T) metric;
   }
+
+  public void clearMetrics() {
+    metricsMap.clear();
+  }
 }

Added: hbase/trunk/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.rest.metrics.RESTMetricsSource
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.rest.metrics.RESTMetricsSource?rev=1387323&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.rest.metrics.RESTMetricsSource (added)
+++ hbase/trunk/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.rest.metrics.RESTMetricsSource Tue Sep 18 19:10:03 2012
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.rest.metrics.RESTMetricsSourceImpl
\ No newline at end of file

Added: hbase/trunk/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.thrift.metrics.ThriftServerMetricsSourceFactory
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.thrift.metrics.ThriftServerMetricsSourceFactory?rev=1387323&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.thrift.metrics.ThriftServerMetricsSourceFactory (added)
+++ hbase/trunk/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.thrift.metrics.ThriftServerMetricsSourceFactory Tue Sep 18 19:10:03 2012
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.thrift.metrics.ThriftServerMetricsSourceFactoryImpl
\ No newline at end of file

Added: hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/TestMasterMetricsSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/TestMasterMetricsSourceImpl.java?rev=1387323&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/TestMasterMetricsSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/master/metrics/TestMasterMetricsSourceImpl.java Tue Sep 18 19:10:03 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.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 TestMasterMetricsSourceImpl {
+
+  @Test
+  public void testGetInstance() throws Exception {
+    MasterMetricsSourceFactory masterMetricsSourceFactory = CompatibilitySingletonFactory
+        .getInstance(MasterMetricsSourceFactory.class);
+    MasterMetricsSource masterMetricsSource = masterMetricsSourceFactory.create(null);
+    assertTrue(masterMetricsSource instanceof MasterMetricsSourceImpl);
+    assertSame(masterMetricsSourceFactory, CompatibilitySingletonFactory.getInstance(MasterMetricsSourceFactory.class));
+  }
+
+}

Added: hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseMetricsSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseMetricsSourceImpl.java?rev=1387323&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseMetricsSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseMetricsSourceImpl.java Tue Sep 18 19:10:03 2012
@@ -0,0 +1,90 @@
+/**
+ * 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.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+/**
+ *  Test of default BaseMetricsSource for hadoop 2
+ */
+public class TestBaseMetricsSourceImpl {
+
+  private static BaseMetricsSourceImpl bmsi;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    bmsi = new BaseMetricsSourceImpl("TestName", "test description", "testcontext", "TestContext");
+  }
+
+  @Test
+  public void testSetGauge() throws Exception {
+    bmsi.setGauge("testset", 100);
+    assertEquals(100, ((MutableGaugeLong) bmsi.metricsRegistry.get("testset")).value());
+    bmsi.setGauge("testset", 300);
+    assertEquals(300, ((MutableGaugeLong) bmsi.metricsRegistry.get("testset")).value());
+
+  }
+
+  @Test
+  public void testIncGauge() throws Exception {
+    bmsi.incGauge("testincgauge", 100);
+    assertEquals(100, ((MutableGaugeLong) bmsi.metricsRegistry.get("testincgauge")).value());
+    bmsi.incGauge("testincgauge", 100);
+    assertEquals(200, ((MutableGaugeLong) bmsi.metricsRegistry.get("testincgauge")).value());
+
+  }
+
+  @Test
+  public void testDecGauge() throws Exception {
+    bmsi.decGauge("testdec", 100);
+    assertEquals(-100, ((MutableGaugeLong) bmsi.metricsRegistry.get("testdec")).value());
+    bmsi.decGauge("testdec", 100);
+    assertEquals(-200, ((MutableGaugeLong) bmsi.metricsRegistry.get("testdec")).value());
+
+  }
+
+  @Test
+  public void testIncCounters() throws Exception {
+    bmsi.incCounters("testinccounter", 100);
+    assertEquals(100, ((MutableCounterLong) bmsi.metricsRegistry.get("testinccounter")).value());
+    bmsi.incCounters("testinccounter", 100);
+    assertEquals(200, ((MutableCounterLong) bmsi.metricsRegistry.get("testinccounter")).value());
+
+  }
+
+  @Test
+  public void testRemoveGauge() throws Exception {
+    bmsi.setGauge("testrmgauge", 100);
+    bmsi.removeGauge("testrmgauge");
+    assertNull(bmsi.metricsRegistry.get("testrmgauge"));
+  }
+
+  @Test
+  public void testRemoveCounter() throws Exception {
+    bmsi.incCounters("testrmcounter", 100);
+    bmsi.removeCounter("testrmcounter");
+    assertNull(bmsi.metricsRegistry.get("testrmcounter"));
+  }
+}

Added: hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/metrics/TestReplicationMetricsSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/metrics/TestReplicationMetricsSourceImpl.java?rev=1387323&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/metrics/TestReplicationMetricsSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/replication/regionserver/metrics/TestReplicationMetricsSourceImpl.java Tue Sep 18 19:10:03 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.replication.regionserver.metrics;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+
+/** Test for ReplicationMetricsSourceImpl */
+public class TestReplicationMetricsSourceImpl {
+
+  @Test
+  public void testGetInstance() throws Exception {
+    ReplicationMetricsSource rms = CompatibilitySingletonFactory
+        .getInstance(ReplicationMetricsSource.class);
+    assertTrue(rms instanceof ReplicationMetricsSourceImpl);
+  }
+}

Added: hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/rest/metrics/TestRESTMetricsSourceImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/rest/metrics/TestRESTMetricsSourceImpl.java?rev=1387323&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/rest/metrics/TestRESTMetricsSourceImpl.java (added)
+++ hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/rest/metrics/TestRESTMetricsSourceImpl.java Tue Sep 18 19:10:03 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.rest.metrics;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.junit.Test;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ *   Test for hadoop 2's version of RESTMetricsSource
+ */
+public class TestRESTMetricsSourceImpl {
+
+  @Test
+  public void ensureCompatRegistered() throws Exception {
+    assertNotNull(CompatibilitySingletonFactory.getInstance(RESTMetricsSource.class));
+    assertTrue(CompatibilitySingletonFactory.getInstance(RESTMetricsSource.class) instanceof RESTMetricsSourceImpl);
+  }
+
+}

Added: hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java?rev=1387323&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java (added)
+++ hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java Tue Sep 18 19:10:03 2012
@@ -0,0 +1,233 @@
+/**
+ * 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.test;
+
+import org.apache.hadoop.hbase.metrics.BaseMetricsSource;
+import org.apache.hadoop.hbase.metrics.BaseMetricsSourceImpl;
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsInfo;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsTag;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.*;
+
+/**
+ *  A helper class that will allow tests to get into hadoop2's metrics2 values.
+ */
+public class MetricsAssertHelperImpl implements MetricsAssertHelper {
+
+  private Map<String, String> tags = new HashMap<String, String>();
+  private Map<String, Number> gauges = new HashMap<String, Number>();
+  private Map<String, Long> counters = new HashMap<String, Long>();
+
+  public class MockMetricsBuilder implements MetricsCollector {
+
+    @Override
+    public MetricsRecordBuilder addRecord(String s) {
+      return new MockRecordBuilder(this);
+    }
+
+    @Override
+    public MetricsRecordBuilder addRecord(MetricsInfo metricsInfo) {
+      return new MockRecordBuilder(this);
+    }
+  }
+
+  public class MockRecordBuilder extends MetricsRecordBuilder {
+
+    private final MetricsCollector mockMetricsBuilder;
+
+    public MockRecordBuilder(MetricsCollector mockMetricsBuilder) {
+
+      this.mockMetricsBuilder = mockMetricsBuilder;
+    }
+
+    @Override
+    public MetricsRecordBuilder tag(MetricsInfo metricsInfo, String s) {
+
+      tags.put(metricsInfo.name(), s);
+      return this;
+    }
+
+    @Override
+    public MetricsRecordBuilder add(MetricsTag metricsTag) {
+      tags.put(canonicalizeMetricName(metricsTag.name()), metricsTag.value());
+      return this;
+    }
+
+    @Override
+    public MetricsRecordBuilder add(AbstractMetric abstractMetric) {
+      gauges.put(canonicalizeMetricName(abstractMetric.name()), abstractMetric.value());
+      return this;
+    }
+
+    @Override
+    public MetricsRecordBuilder setContext(String s) {
+      return this;
+    }
+
+    @Override
+    public MetricsRecordBuilder addCounter(MetricsInfo metricsInfo, int i) {
+      counters.put(canonicalizeMetricName(metricsInfo.name()), Long.valueOf(i));
+      return this;
+    }
+
+    @Override
+    public MetricsRecordBuilder addCounter(MetricsInfo metricsInfo, long l) {
+      counters.put(canonicalizeMetricName(metricsInfo.name()), Long.valueOf(l));
+      return this;
+    }
+
+    @Override
+    public MetricsRecordBuilder addGauge(MetricsInfo metricsInfo, int i) {
+      gauges.put(canonicalizeMetricName(metricsInfo.name()), Long.valueOf(i));
+      return this;
+    }
+
+    @Override
+    public MetricsRecordBuilder addGauge(MetricsInfo metricsInfo, long l) {
+      gauges.put(canonicalizeMetricName(metricsInfo.name()), Long.valueOf(l));
+      return this;
+    }
+
+    @Override
+    public MetricsRecordBuilder addGauge(MetricsInfo metricsInfo, float v) {
+      gauges.put(canonicalizeMetricName(metricsInfo.name()), Double.valueOf(v));
+      return this;
+    }
+
+    @Override
+    public MetricsRecordBuilder addGauge(MetricsInfo metricsInfo, double v) {
+      gauges.put(canonicalizeMetricName(metricsInfo.name()), Double.valueOf(v));
+      return this;
+    }
+
+    @Override
+    public MetricsCollector parent() {
+      return mockMetricsBuilder;
+    }
+  }
+
+  @Override
+  public void assertTag(String name, String expected, BaseMetricsSource source) {
+    getMetrics(source);
+    String cName = canonicalizeMetricName(name);
+    assertEquals("Tags should be equal", expected, tags.get(cName));
+  }
+
+  @Override
+  public void assertGauge(String name, long expected, BaseMetricsSource source) {
+    getMetrics(source);
+    String cName = canonicalizeMetricName(name);
+    assertEquals("Metrics Should be equal", Long.valueOf(expected), gauges.get(cName));
+  }
+
+  @Override
+  public void assertGaugeGt(String name, long expected, BaseMetricsSource source) {
+    getMetrics(source);
+    String cName = canonicalizeMetricName(name);
+    assertNotNull(gauges.get(cName));
+    long found = gauges.get(cName).longValue();
+    assertTrue(name + " (" + found + ") should be greater than " + expected, found > expected);
+  }
+
+  @Override
+  public void assertGaugeLt(String name, long expected, BaseMetricsSource source) {
+    getMetrics(source);
+    String cName = canonicalizeMetricName(name);
+    assertNotNull(gauges.get(cName));
+    long found = gauges.get(cName).longValue();
+    assertTrue(name + "(" + found + ") should be less than " + expected, found < expected);
+  }
+
+  @Override
+  public void assertGauge(String name, double expected, BaseMetricsSource source) {
+    getMetrics(source);
+    String cName = canonicalizeMetricName(name);
+    assertEquals("Metrics Should be equal", Double.valueOf(expected), gauges.get(cName));
+  }
+
+  @Override
+  public void assertGaugeGt(String name, double expected, BaseMetricsSource source) {
+    getMetrics(source);
+    String cName = canonicalizeMetricName(name);
+    assertNotNull(gauges.get(cName));
+    double found = gauges.get(cName).doubleValue();
+    assertTrue(name + "(" + found + ") should be greater than " + expected, found > expected);
+  }
+
+  @Override
+  public void assertGaugeLt(String name, double expected, BaseMetricsSource source) {
+    getMetrics(source);
+    String cName = canonicalizeMetricName(name);
+    assertNotNull(gauges.get(cName));
+    double found = gauges.get(cName).doubleValue();
+    assertTrue(name + "(" + found + ") should be less than " + expected, found < expected);
+  }
+
+  @Override
+  public void assertCounter(String name, long expected, BaseMetricsSource source) {
+    getMetrics(source);
+    String cName = canonicalizeMetricName(name);
+    assertEquals("Metrics Counters should be equal", Long.valueOf(expected), counters.get(cName));
+  }
+
+  @Override
+  public void assertCounterGt(String name, long expected, BaseMetricsSource source) {
+    getMetrics(source);
+    String cName = canonicalizeMetricName(name);
+    assertNotNull(counters.get(cName));
+    long found = gauges.get(cName).longValue();
+    assertTrue(name + " (" + found + ") should be greater than " + expected, found > expected);
+  }
+
+  @Override
+  public void assertCounterLt(String name, long expected, BaseMetricsSource source) {
+    getMetrics(source);
+    String cName = canonicalizeMetricName(name);
+    assertNotNull(counters.get(cName));
+    long found = gauges.get(cName).longValue();
+    assertTrue(name + "(" + found + ") should be less than " + expected, found < expected);
+  }
+
+  private void reset() {
+    tags.clear();
+    gauges.clear();
+    counters.clear();
+  }
+
+  private void getMetrics(BaseMetricsSource source) {
+    reset();
+    if (!(source instanceof BaseMetricsSourceImpl)) {
+      assertTrue(false);
+    }
+    BaseMetricsSourceImpl impl = (BaseMetricsSourceImpl) source;
+
+    impl.getMetrics(new MockMetricsBuilder(), true);
+
+  }
+
+  private String canonicalizeMetricName(String in) {
+    return in.toLowerCase().replaceAll("[^A-Za-z0-9 ]", "");
+  }
+}

Added: hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/thrift/metrics/TestThriftServerMetricsSourceFactoryImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/thrift/metrics/TestThriftServerMetricsSourceFactoryImpl.java?rev=1387323&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/thrift/metrics/TestThriftServerMetricsSourceFactoryImpl.java (added)
+++ hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/thrift/metrics/TestThriftServerMetricsSourceFactoryImpl.java Tue Sep 18 19:10:03 2012
@@ -0,0 +1,53 @@
+/**
+ * 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.metrics;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.junit.Test;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+/**
+ *   Test for hadoop 2's version of ThriftServerMetricsSourceFactory
+ */
+public class TestThriftServerMetricsSourceFactoryImpl {
+
+  @Test
+  public void testCompatabilityRegistered() throws Exception {
+    assertNotNull(CompatibilitySingletonFactory.getInstance(ThriftServerMetricsSourceFactory.class));
+    assertTrue(CompatibilitySingletonFactory.getInstance(ThriftServerMetricsSourceFactory.class) instanceof ThriftServerMetricsSourceFactoryImpl);
+  }
+
+  @Test
+  public void testCreateThriftOneSource() throws Exception {
+    //Make sure that the factory gives back a singleton.
+    assertSame(new ThriftServerMetricsSourceFactoryImpl().createThriftOneSource(),
+        new ThriftServerMetricsSourceFactoryImpl().createThriftOneSource());
+
+  }
+
+  @Test
+  public void testCreateThriftTwoSource() throws Exception {
+    //Make sure that the factory gives back a singleton.
+    assertSame(new ThriftServerMetricsSourceFactoryImpl().createThriftTwoSource(),
+        new ThriftServerMetricsSourceFactoryImpl().createThriftTwoSource());
+  }
+}

Added: hbase/trunk/hbase-hadoop2-compat/src/test/resources/META-INF/services/org.apache.hadoop.hbase.test.MetricsAssertHelper
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/test/resources/META-INF/services/org.apache.hadoop.hbase.test.MetricsAssertHelper?rev=1387323&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/test/resources/META-INF/services/org.apache.hadoop.hbase.test.MetricsAssertHelper (added)
+++ hbase/trunk/hbase-hadoop2-compat/src/test/resources/META-INF/services/org.apache.hadoop.hbase.test.MetricsAssertHelper Tue Sep 18 19:10:03 2012
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.test.MetricsAssertHelperImpl

Modified: hbase/trunk/hbase-server/pom.xml
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/pom.xml?rev=1387323&r1=1387322&r2=1387323&view=diff
==============================================================================
--- hbase/trunk/hbase-server/pom.xml (original)
+++ hbase/trunk/hbase-server/pom.xml Tue Sep 18 19:10:03 2012
@@ -269,16 +269,19 @@
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-hadoop-compat</artifactId>
+      <version>${project.version}</version>
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>${compat.module}</artifactId>
+      <version>${project.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>${compat.module}</artifactId>
+      <version>${project.version}</version>
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/metrics/RESTMetrics.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/metrics/RESTMetrics.java?rev=1387323&r1=1387322&r2=1387323&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/metrics/RESTMetrics.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/metrics/RESTMetrics.java Tue Sep 18 19:10:03 2012
@@ -20,6 +20,7 @@
 package org.apache.hadoop.hbase.rest.metrics;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
 import org.apache.hadoop.hbase.metrics.MetricsRate;
 
 import org.apache.hadoop.metrics.MetricsContext;
@@ -30,162 +31,65 @@ import org.apache.hadoop.metrics.jvm.Jvm
 import org.apache.hadoop.metrics.util.MetricsRegistry;
 
 @InterfaceAudience.Private
-public class RESTMetrics implements Updater {
-  private final MetricsRecord metricsRecord;
-  private final MetricsRegistry registry = new MetricsRegistry();
-  private final RESTStatistics restStatistics;
-
-  private MetricsRate requests = new MetricsRate("requests", registry);
-  private MetricsRate sucessfulGetCount =
-      new MetricsRate("sucessful.get.count", registry);
-  private MetricsRate sucessfulPutCount =
-      new MetricsRate("sucessful.put.count", registry);
-  private MetricsRate sucessfulDeleteCount =
-      new MetricsRate("sucessful.delete.count", registry);
-  
-  private MetricsRate failedGetCount =
-      new MetricsRate("failed.get.count", registry);
-  private MetricsRate failedPutCount =
-      new MetricsRate("failed.put.count", registry);
-  private MetricsRate failedDeleteCount =
-      new MetricsRate("failed.delete.count", registry);
-
-  public RESTMetrics() {
-    MetricsContext context = MetricsUtil.getContext("rest");
-    metricsRecord = MetricsUtil.createRecord(context, "rest");
-    String name = Thread.currentThread().getName();
-    metricsRecord.setTag("REST", name);
-    context.registerUpdater(this);
-    JvmMetrics.init("rest", name);
-    // expose the MBean for metrics
-    restStatistics = new RESTStatistics(registry);
+public class RESTMetrics {
 
+  public RESTMetricsSource getSource() {
+    return source;
   }
 
-  public void shutdown() {
-    if (restStatistics != null) {
-      restStatistics.shutdown();
-    }
-  }
-
-  /**
-   * Since this object is a registered updater, this method will be called
-   * periodically, e.g. every 5 seconds.
-   * @param unused 
-   */
-  public void doUpdates(MetricsContext unused) {
-    synchronized (this) {
-      requests.pushMetric(metricsRecord);
-      sucessfulGetCount.pushMetric(metricsRecord);
-      sucessfulPutCount.pushMetric(metricsRecord);
-      sucessfulDeleteCount.pushMetric(metricsRecord);
-      failedGetCount.pushMetric(metricsRecord);
-      failedPutCount.pushMetric(metricsRecord);
-      failedDeleteCount.pushMetric(metricsRecord);
-    }
-    this.metricsRecord.update();
-  }
-  
-  public void resetAllMinMax() {
-    // Nothing to do
-  }
+  private RESTMetricsSource source;
 
-  /**
-   * @return Count of requests.
-   */
-  public float getRequests() {
-    return requests.getPreviousIntervalValue();
+  public RESTMetrics() {
+     source = CompatibilitySingletonFactory.getInstance(RESTMetricsSource.class);
   }
   
   /**
    * @param inc How much to add to requests.
    */
   public void incrementRequests(final int inc) {
-    requests.inc(inc);
-  }
-  
-  /**
-   * @return Count of sucessfulGetCount.
-   */
-  public float getSucessfulGetCount() {
-    return sucessfulGetCount.getPreviousIntervalValue();
+    source.incrementRequests(inc);
   }
   
   /**
    * @param inc How much to add to sucessfulGetCount.
    */
   public void incrementSucessfulGetRequests(final int inc) {
-    sucessfulGetCount.inc(inc);
-  }
-  
-  /**
-   * @return Count of sucessfulGetCount.
-   */
-  public float getSucessfulPutCount() {
-    return sucessfulPutCount.getPreviousIntervalValue();
+    source.incrementSucessfulGetRequests(inc);
   }
   
   /**
    * @param inc How much to add to sucessfulPutCount.
    */
   public void incrementSucessfulPutRequests(final int inc) {
-    sucessfulPutCount.inc(inc);
+    source.incrementSucessfulPutRequests(inc);
   }
-  
-  /**
-   * @return Count of failedPutCount.
-   */
-  public float getFailedPutCount() {
-    return failedPutCount.getPreviousIntervalValue();
-  }
-  
+
   /**
    * @param inc How much to add to failedPutCount.
    */
   public void incrementFailedPutRequests(final int inc) {
-    failedPutCount.inc(inc);
-  }
-  
-  /**
-   * @return Count of failedGetCount.
-   */
-  public float getFailedGetCount() {
-    return failedGetCount.getPreviousIntervalValue();
+    source.incrementFailedPutRequests(inc);
   }
   
   /**
    * @param inc How much to add to failedGetCount.
    */
   public void incrementFailedGetRequests(final int inc) {
-    failedGetCount.inc(inc);
+    source.incrementFailedGetRequests(inc);
   }
-  
-  /**
-   * @return Count of sucessfulGetCount.
-   */
-  public float getSucessfulDeleteCount() {
-    return sucessfulDeleteCount.getPreviousIntervalValue();
-  }
-  
+
   /**
    * @param inc How much to add to sucessfulDeleteCount.
    */
   public void incrementSucessfulDeleteRequests(final int inc) {
-    sucessfulDeleteCount.inc(inc);
-  }
-
-  /**
-   * @return Count of failedDeleteCount.
-   */
-  public float getFailedDeleteCount() {
-    return failedDeleteCount.getPreviousIntervalValue();
+    source.incrementSucessfulDeleteRequests(inc);
   }
   
   /**
    * @param inc How much to add to failedDeleteCount.
    */
   public void incrementFailedDeleteRequests(final int inc) {
-    failedDeleteCount.inc(inc);
+    source.incrementFailedDeleteRequests(inc);
   }
   
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/ThriftMetrics.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/ThriftMetrics.java?rev=1387323&r1=1387322&r2=1387323&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/ThriftMetrics.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/ThriftMetrics.java Tue Sep 18 19:10:03 2012
@@ -19,128 +19,74 @@
 
 package org.apache.hadoop.hbase.thrift;
 
-import java.lang.reflect.Method;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.thrift.generated.Hbase;
-import org.apache.hadoop.metrics.MetricsContext;
-import org.apache.hadoop.metrics.MetricsRecord;
-import org.apache.hadoop.metrics.MetricsUtil;
-import org.apache.hadoop.metrics.Updater;
-import org.apache.hadoop.metrics.util.MetricsBase;
-import org.apache.hadoop.metrics.util.MetricsIntValue;
-import org.apache.hadoop.metrics.util.MetricsRegistry;
-import org.apache.hadoop.metrics.util.MetricsTimeVaryingInt;
-import org.apache.hadoop.metrics.util.MetricsTimeVaryingLong;
-import org.apache.hadoop.metrics.util.MetricsTimeVaryingRate;
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.hbase.thrift.metrics.ThriftServerMetricsSource;
+import org.apache.hadoop.hbase.thrift.metrics.ThriftServerMetricsSourceFactory;
 
 /**
  * This class is for maintaining the various statistics of thrift server
  * and publishing them through the metrics interfaces.
  */
 @InterfaceAudience.Private
-public class ThriftMetrics implements Updater {
-  public final static Log LOG = LogFactory.getLog(ThriftMetrics.class);
-  public final static String CONTEXT_NAME = "thriftserver";
-
-  private final MetricsContext context;
-  private final MetricsRecord metricsRecord;
-  private final MetricsRegistry registry = new MetricsRegistry();
+public class ThriftMetrics  {
+
+
+  public enum ThriftServerType {
+    ONE,
+    TWO
+  }
+
+  public ThriftServerMetricsSource getSource() {
+    return source;
+  }
+
+  public void setSource(ThriftServerMetricsSource source) {
+    this.source = source;
+  }
+
+  private ThriftServerMetricsSource source;
   private final long slowResponseTime;
   public static final String SLOW_RESPONSE_NANO_SEC =
     "hbase.thrift.slow.response.nano.second";
   public static final long DEFAULT_SLOW_RESPONSE_NANO_SEC = 10 * 1000 * 1000;
 
-  private final MetricsIntValue callQueueLen =
-      new MetricsIntValue("callQueueLen", registry);
-  private final MetricsTimeVaryingRate numRowKeysInBatchGet =
-      new MetricsTimeVaryingRate("numRowKeysInBatchGet", registry);
-  private final MetricsTimeVaryingRate numRowKeysInBatchMutate =
-      new MetricsTimeVaryingRate("numRowKeysInBatchMutate", registry);
-  private final MetricsTimeVaryingRate timeInQueue =
-      new MetricsTimeVaryingRate("timeInQueue", registry);
-  private MetricsTimeVaryingRate thriftCall =
-      new MetricsTimeVaryingRate("thriftCall", registry);
-  private MetricsTimeVaryingRate slowThriftCall =
-      new MetricsTimeVaryingRate("slowThriftCall", registry);
-
-  public ThriftMetrics(int port, Configuration conf, Class<?> iface) {
-    slowResponseTime = conf.getLong(
-        SLOW_RESPONSE_NANO_SEC, DEFAULT_SLOW_RESPONSE_NANO_SEC);
-    context = MetricsUtil.getContext(CONTEXT_NAME);
-    metricsRecord = MetricsUtil.createRecord(context, CONTEXT_NAME);
 
-    metricsRecord.setTag("port", port + "");
+  public ThriftMetrics(Configuration conf, ThriftServerType t) {
+    slowResponseTime = conf.getLong( SLOW_RESPONSE_NANO_SEC, DEFAULT_SLOW_RESPONSE_NANO_SEC);
 
-    LOG.info("Initializing RPC Metrics with port=" + port);
-
-    context.registerUpdater(this);
+    if (t == ThriftServerType.ONE) {
+      source = CompatibilitySingletonFactory.getInstance(ThriftServerMetricsSourceFactory.class).createThriftOneSource();
+    } else if (t == ThriftServerType.TWO) {
+      source = CompatibilitySingletonFactory.getInstance(ThriftServerMetricsSourceFactory.class).createThriftTwoSource();
+    }
 
-    createMetricsForMethods(iface);
   }
 
   public void incTimeInQueue(long time) {
-    timeInQueue.inc(time);
+    source.incTimeInQueue(time);
   }
 
   public void setCallQueueLen(int len) {
-    callQueueLen.set(len);
+    source.setCallQueueLen(len);
   }
 
   public void incNumRowKeysInBatchGet(int diff) {
-    numRowKeysInBatchGet.inc(diff);
+    source.incNumRowKeysInBatchGet(diff);
   }
 
   public void incNumRowKeysInBatchMutate(int diff) {
-    numRowKeysInBatchMutate.inc(diff);
+    source.incNumRowKeysInBatchMutate(diff);
   }
 
   public void incMethodTime(String name, long time) {
-    MetricsTimeVaryingRate methodTimeMetric = getMethodTimeMetrics(name);
-    if (methodTimeMetric == null) {
-      LOG.warn(
-          "Got incMethodTime() request for method that doesnt exist: " + name);
-      return; // ignore methods that dont exist.
-    }
-
-    // inc method specific processTime
-    methodTimeMetric.inc(time);
-
+    source.incMethodTime(name, time);
     // inc general processTime
-    thriftCall.inc(time);
+    source.incCall(time);
     if (time > slowResponseTime) {
-      slowThriftCall.inc(time);
-    }
-  }
-
-  private void createMetricsForMethods(Class<?> iface) {
-    LOG.debug("Creating metrics for interface " + iface.toString());
-    for (Method m : iface.getDeclaredMethods()) {
-      if (getMethodTimeMetrics(m.getName()) == null)
-        LOG.debug("Creating metrics for method:" + m.getName());
-        createMethodTimeMetrics(m.getName());
+      source.incSlowCall(time);
     }
   }
 
-  private MetricsTimeVaryingRate getMethodTimeMetrics(String key) {
-    return (MetricsTimeVaryingRate) registry.get(key);
-  }
-
-  private MetricsTimeVaryingRate createMethodTimeMetrics(String key) {
-    return new MetricsTimeVaryingRate(key, this.registry);
-  }
-
-  /**
-   * Push the metrics to the monitoring subsystem on doUpdate() call.
-   */
-  public void doUpdates(final MetricsContext context) {
-    // getMetricsList() and pushMetric() are thread safe methods
-    for (MetricsBase m : registry.getMetricsList()) {
-      m.pushMetric(metricsRecord);
-    }
-    metricsRecord.update();
-  }
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java?rev=1387323&r1=1387322&r2=1387323&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java Tue Sep 18 19:10:03 2012
@@ -235,7 +235,7 @@ public class ThriftServerRunner implemen
   public ThriftServerRunner(Configuration conf, HBaseHandler handler) {
     this.conf = HBaseConfiguration.create(conf);
     this.listenPort = conf.getInt(PORT_CONF_KEY, DEFAULT_LISTEN_PORT);
-    this.metrics = new ThriftMetrics(listenPort, conf, Hbase.Iface.class);
+    this.metrics = new ThriftMetrics(conf, ThriftMetrics.ThriftServerType.ONE);
     handler.initMetrics(metrics);
     this.handler = HbaseHandlerMetricsProxy.newInstance(handler, metrics, conf);
   }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java?rev=1387323&r1=1387322&r2=1387323&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java Tue Sep 18 19:10:03 2012
@@ -223,8 +223,7 @@ public class ThriftServer {
       boolean hsha = cmd.hasOption("hsha");
 
       Configuration conf = HBaseConfiguration.create();
-      ThriftMetrics metrics = new ThriftMetrics(
-          listenPort, conf, THBaseService.Iface.class);
+      ThriftMetrics metrics = new ThriftMetrics(conf, ThriftMetrics.ThriftServerType.TWO);
 
       // Construct correct ProtocolFactory
       TProtocolFactory protocolFactory = getTProtocolFactory(cmd.hasOption("compact"));

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java?rev=1387323&r1=1387322&r2=1387323&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCheckTestClasses.java Tue Sep 18 19:10:03 2012
@@ -30,6 +30,7 @@ import java.net.URL;
 import java.util.ArrayList;
 import java.util.Enumeration;
 import java.util.List;
+import java.util.regex.Pattern;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -159,9 +160,13 @@ public class TestCheckTestClasses {
 
     File[] files = baseDirectory.listFiles(TEST_CLASS_FILE_FILTER);
     assertNotNull(files);
-
+    Pattern p = Pattern.compile("hbase-hadoop\\d?-compat");
     for (File file : files) {
       final String fileName = file.getName();
+      if (p.matcher(file.getAbsolutePath()).find()) {
+        continue;
+      }
+
       if (file.isDirectory()) {
         classes.addAll(findTestClasses(file, packageName + "." + fileName));
       } else {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestRowResource.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestRowResource.java?rev=1387323&r1=1387322&r2=1387323&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestRowResource.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestRowResource.java Tue Sep 18 19:10:03 2012
@@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.rest.clie
 import org.apache.hadoop.hbase.rest.model.CellModel;
 import org.apache.hadoop.hbase.rest.model.CellSetModel;
 import org.apache.hadoop.hbase.rest.model.RowModel;
+import org.apache.hadoop.hbase.test.MetricsAssertHelper;
 import org.apache.hadoop.hbase.util.Bytes;
 
 import static org.junit.Assert.*;
@@ -67,6 +68,8 @@ public class TestRowResource {
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static final HBaseRESTTestingUtility REST_TEST_UTIL = 
     new HBaseRESTTestingUtility();
+  private static final MetricsAssertHelper METRICS_ASSERT =
+      CompatibilityFactory.getInstance(MetricsAssertHelper.class);
   private static Client client;
   private static JAXBContext context;
   private static Marshaller marshaller;
@@ -526,6 +529,35 @@ public class TestRowResource {
   }
 
   @Test
+  public void testMetrics() throws IOException, JAXBException {
+    final String path = "/" + TABLE + "/" + ROW_4 + "/" + COLUMN_1;
+    Response response = client.put(path, Constants.MIMETYPE_BINARY,
+        Bytes.toBytes(VALUE_4));
+    assertEquals(response.getCode(), 200);
+    Thread.yield();
+    response = client.get(path, Constants.MIMETYPE_JSON);
+    assertEquals(response.getCode(), 200);
+    response = deleteRow(TABLE, ROW_4);
+    assertEquals(response.getCode(), 200);
+
+    METRICS_ASSERT.assertCounterGt("requests",
+                                    2l,
+                                    RESTServlet.getInstance(conf).getMetrics().getSource());
+
+    METRICS_ASSERT.assertCounterGt("successfulGet",
+                                   0l,
+                                   RESTServlet.getInstance(conf).getMetrics().getSource());
+
+    METRICS_ASSERT.assertCounterGt("successfulPut",
+                                    0l,
+                                    RESTServlet.getInstance(conf).getMetrics().getSource());
+
+    METRICS_ASSERT.assertCounterGt("successfulDelete",
+                                    0l,
+                                    RESTServlet.getInstance(conf).getMetrics().getSource());
+  }
+
+  @Test
   public void testURLEncodedKey() throws IOException, JAXBException {
     String urlKey = "http://example.com/foo";
     StringBuilder path = new StringBuilder();

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift/TestCallQueue.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift/TestCallQueue.java?rev=1387323&r1=1387322&r2=1387323&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift/TestCallQueue.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift/TestCallQueue.java Tue Sep 18 19:10:03 2012
@@ -28,15 +28,11 @@ import java.util.concurrent.LinkedBlocki
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.test.MetricsAssertHelper;
 import org.apache.hadoop.hbase.thrift.CallQueue.Call;
-import org.apache.hadoop.hbase.thrift.generated.Hbase;
-import org.apache.hadoop.metrics.ContextFactory;
-import org.apache.hadoop.metrics.MetricsContext;
-import org.apache.hadoop.metrics.MetricsUtil;
-import org.apache.hadoop.metrics.spi.NoEmitMetricsContext;
-import org.apache.hadoop.metrics.spi.OutputRecord;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -54,6 +50,9 @@ public class TestCallQueue {
   public static final Log LOG = LogFactory.getLog(TestCallQueue.class);
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
+  private static final MetricsAssertHelper metricsHelper =
+      CompatibilitySingletonFactory.getInstance(MetricsAssertHelper.class);
+
   private int elementsAdded;
   private int elementsRemoved;
 
@@ -74,6 +73,7 @@ public class TestCallQueue {
     this.elementsRemoved = elementsRemoved;
     LOG.debug("elementsAdded:" + elementsAdded +
               " elementsRemoved:" + elementsRemoved);
+
   }
 
   @Test(timeout=3000)
@@ -105,28 +105,16 @@ public class TestCallQueue {
   }
 
   private static ThriftMetrics createMetrics() throws Exception {
-    setupMetricsContext();
     Configuration conf = UTIL.getConfiguration();
-    return new ThriftMetrics(
-        ThriftServerRunner.DEFAULT_LISTEN_PORT, conf, Hbase.Iface.class);
+    ThriftMetrics m = new ThriftMetrics(conf, ThriftMetrics.ThriftServerType.ONE);
+    m.getSource().init();
+    return m;
   }
 
-  private static void setupMetricsContext() throws Exception {
-    ContextFactory factory = ContextFactory.getFactory();
-    factory.setAttribute(ThriftMetrics.CONTEXT_NAME + ".class",
-        NoEmitMetricsContext.class.getName());
-    MetricsUtil.getContext(ThriftMetrics.CONTEXT_NAME)
-               .createRecord(ThriftMetrics.CONTEXT_NAME).remove();
-  }
 
   private static void verifyMetrics(ThriftMetrics metrics, String name, int expectValue)
       throws Exception { 
-    MetricsContext context = MetricsUtil.getContext( 
-        ThriftMetrics.CONTEXT_NAME); 
-    metrics.doUpdates(context); 
-    OutputRecord record = context.getAllRecords().get( 
-        ThriftMetrics.CONTEXT_NAME).iterator().next(); 
-    assertEquals(expectValue, record.getMetric(name).intValue()); 
+      metricsHelper.assertCounter(name, expectValue, metrics.getSource());
   }
 
   private static Runnable createDummyRunnable() {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java?rev=1387323&r1=1387322&r2=1387323&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java Tue Sep 18 19:10:03 2012
@@ -31,11 +31,13 @@ import java.util.Map;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CompatibilityFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.filter.ParseFilter;
+import org.apache.hadoop.hbase.test.MetricsAssertHelper;
 import org.apache.hadoop.hbase.thrift.ThriftServerRunner.HBaseHandler;
 import org.apache.hadoop.hbase.thrift.generated.BatchMutation;
 import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
@@ -66,6 +68,8 @@ import org.junit.experimental.categories
 public class TestThriftServer {
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
   private static final Log LOG = LogFactory.getLog(TestThriftServer.class);
+  private static final MetricsAssertHelper metricsHelper = CompatibilityFactory
+      .getInstance(MetricsAssertHelper.class);
   protected static final int MAXVERSIONS = 3;
 
   private static ByteBuffer asByteBuffer(String i) {
@@ -164,14 +168,14 @@ public class TestThriftServer {
     Hbase.Iface handler = getHandlerForMetricsTest(metrics, conf);
     createTestTables(handler);
     dropTestTables(handler);
-    verifyMetrics(metrics, "createTable_num_ops", 2);
-    verifyMetrics(metrics, "deleteTable_num_ops", 2);
-    verifyMetrics(metrics, "disableTable_num_ops", 2);
+    metricsHelper.assertCounter("createTable_num_ops", 2, metrics.getSource());
+    metricsHelper.assertCounter("deleteTable_num_ops", 2, metrics.getSource());
+    metricsHelper.assertCounter("disableTable_num_ops", 2, metrics.getSource());
     handler.getTableNames(); // This will have an artificial delay.
 
-    // 3 to 6 seconds (to account for potential slowness), measured in nanoseconds.
-    verifyMetricRange(metrics, "getTableNames_avg_time", 3L * 1000 * 1000 * 1000,
-        6L * 1000 * 1000 * 1000);
+    // 3 to 6 seconds (to account for potential slowness), measured in nanoseconds
+   metricsHelper.assertGaugeGt("getTableNames_avg_time", 3L * 1000 * 1000 * 1000, metrics.getSource());
+   metricsHelper.assertGaugeLt("getTableNames_avg_time",6L * 1000 * 1000 * 1000, metrics.getSource());
   }
 
   private static Hbase.Iface getHandlerForMetricsTest(ThriftMetrics metrics, Configuration conf)
@@ -181,17 +185,9 @@ public class TestThriftServer {
   }
 
   private static ThriftMetrics getMetrics(Configuration conf) throws Exception {
-    setupMetricsContext();
-    return new ThriftMetrics(ThriftServerRunner.DEFAULT_LISTEN_PORT, conf, Hbase.Iface.class);
+    return new ThriftMetrics( conf, ThriftMetrics.ThriftServerType.ONE);
   }
 
-  private static void setupMetricsContext() throws IOException {
-    ContextFactory factory = ContextFactory.getFactory();
-    factory.setAttribute(ThriftMetrics.CONTEXT_NAME + ".class",
-        NoEmitMetricsContext.class.getName());
-    MetricsUtil.getContext(ThriftMetrics.CONTEXT_NAME)
-               .createRecord(ThriftMetrics.CONTEXT_NAME).remove();
-  }
 
   public static void createTestTables(Hbase.Iface handler) throws Exception {
     // Create/enable/disable/delete tables, ensure methods act correctly
@@ -224,31 +220,6 @@ public class TestThriftServer {
     assertEquals(handler.getTableNames().size(), 0);
   }
 
-  private static void verifyMetrics(ThriftMetrics metrics, String name, long expectValue)
-      throws Exception {
-    long metricVal = getMetricValue(metrics, name);
-    assertEquals(expectValue, metricVal);
-  }
-
-  private static void verifyMetricRange(ThriftMetrics metrics, String name,
-      long minValue, long maxValue)
-      throws Exception {
-    long metricVal = getMetricValue(metrics, name);
-    if (metricVal < minValue || metricVal > maxValue) {
-      throw new AssertionError("Value of metric " + name + " is outside of the expected " +
-          "range [" +  minValue + ", " + maxValue + "]: " + metricVal);
-    }
-  }
-
-  private static long getMetricValue(ThriftMetrics metrics, String name) {
-    MetricsContext context = MetricsUtil.getContext(
-        ThriftMetrics.CONTEXT_NAME);
-    metrics.doUpdates(context);
-    OutputRecord record = context.getAllRecords().get(
-        ThriftMetrics.CONTEXT_NAME).iterator().next();
-    return record.getMetric(name).longValue();
-  }
-
   public void doTestIncrements() throws Exception {
     ThriftServerRunner.HBaseHandler handler =
         new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration());

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java?rev=1387323&r1=1387322&r2=1387323&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java Tue Sep 18 19:10:03 2012
@@ -18,28 +18,16 @@
  */
 package org.apache.hadoop.hbase.thrift2;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CompatibilityFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.test.MetricsAssertHelper;
 import org.apache.hadoop.hbase.thrift.ThriftMetrics;
 import org.apache.hadoop.hbase.thrift2.generated.TColumn;
 import org.apache.hadoop.hbase.thrift2.generated.TColumnIncrement;
@@ -55,11 +43,6 @@ import org.apache.hadoop.hbase.thrift2.g
 import org.apache.hadoop.hbase.thrift2.generated.TResult;
 import org.apache.hadoop.hbase.thrift2.generated.TScan;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.metrics.ContextFactory;
-import org.apache.hadoop.metrics.MetricsContext;
-import org.apache.hadoop.metrics.MetricsUtil;
-import org.apache.hadoop.metrics.spi.NoEmitMetricsContext;
-import org.apache.hadoop.metrics.spi.OutputRecord;
 import org.apache.thrift.TException;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -67,6 +50,14 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import static org.junit.Assert.*;
+
 /**
  * Unit testing for ThriftServer.HBaseHandler, a part of the org.apache.hadoop.hbase.thrift2 package.
  */
@@ -90,6 +81,11 @@ public class TestThriftHBaseServiceHandl
           .setMaxVersions(2)
   };
 
+
+  private static final MetricsAssertHelper metricsHelper =
+      CompatibilityFactory.getInstance(MetricsAssertHelper.class);
+
+
   public void assertTColumnValuesEqual(List<TColumnValue> columnValuesA, List<TColumnValue> columnValuesB) {
     assertEquals(columnValuesA.size(), columnValuesB.size());
     Comparator<TColumnValue> comparator = new Comparator<TColumnValue>() {
@@ -557,50 +553,14 @@ public class TestThriftHBaseServiceHandl
     handler.put(table, put);
 
     assertTrue(handler.exists(table, get));
-    logMetrics(metrics);
-    verifyMetrics(metrics, "put_num_ops", 1);
-    verifyMetrics(metrics, "exists_num_ops", 2);
-  }
- 
-  private static ThriftMetrics getMetrics(Configuration conf) throws Exception {
-    setupMetricsContext();
-    return new ThriftMetrics(Integer.parseInt(ThriftServer.DEFAULT_LISTEN_PORT),
-        conf, THBaseService.Iface.class);
-  }
- 
-  private static void setupMetricsContext() throws IOException {
-    ContextFactory factory = ContextFactory.getFactory();
-    factory.setAttribute(ThriftMetrics.CONTEXT_NAME + ".class",
-        NoEmitMetricsContext.class.getName());
-    MetricsUtil.getContext(ThriftMetrics.CONTEXT_NAME)
-               .createRecord(ThriftMetrics.CONTEXT_NAME).remove();
-  }
- 
-  private static void logMetrics(ThriftMetrics metrics) throws Exception {
-    if (LOG.isDebugEnabled()) {
-      return;
-    }
-    MetricsContext context = MetricsUtil.getContext( 
-        ThriftMetrics.CONTEXT_NAME); 
-    metrics.doUpdates(context); 
-    for (String key : context.getAllRecords().keySet()) {
-      for (OutputRecord record : context.getAllRecords().get(key)) {
-        for (String name : record.getMetricNames()) {
-          LOG.debug("metrics:" + name + " value:" +
-              record.getMetric(name).intValue());
-        }
-      }
-    }
+    metricsHelper.assertCounter("put_num_ops", 1, metrics.getSource());
+    metricsHelper.assertCounter( "exists_num_ops", 2, metrics.getSource());
   }
 
-  private static void verifyMetrics(ThriftMetrics metrics, String name, int expectValue)
-      throws Exception { 
-    MetricsContext context = MetricsUtil.getContext( 
-        ThriftMetrics.CONTEXT_NAME); 
-    metrics.doUpdates(context); 
-    OutputRecord record = context.getAllRecords().get( 
-        ThriftMetrics.CONTEXT_NAME).iterator().next(); 
-    assertEquals(expectValue, record.getMetric(name).intValue()); 
+  private static ThriftMetrics getMetrics(Configuration conf) throws Exception {
+    ThriftMetrics m = new ThriftMetrics(conf, ThriftMetrics.ThriftServerType.TWO);
+    m.getSource().init(); //Clear all the metrics
+    return m;
   }
 
   @org.junit.Rule

Modified: hbase/trunk/src/docbkx/developer.xml
URL: http://svn.apache.org/viewvc/hbase/trunk/src/docbkx/developer.xml?rev=1387323&r1=1387322&r2=1387323&view=diff
==============================================================================
--- hbase/trunk/src/docbkx/developer.xml (original)
+++ hbase/trunk/src/docbkx/developer.xml Tue Sep 18 19:10:03 2012
@@ -128,13 +128,13 @@ Access restriction: The method getLong(O
        <title>Building HBase</title>
       <section xml:id="build.basic">
        <title>Basic Compile</title>
-       <para>Thanks to maven, building HBase is easy. You can read about the various maven commands in <xref linkend="maven.build.commands"/>, but the simplest command to compile HBase from its java source code is:
+       <para>Thanks to maven, building HBase is pretty easy. You can read about the various maven commands in <xref linkend="maven.build.commands"/>, but the simplest command to compile HBase from its java source code is:
        <programlisting>
-mvn compile
+mvn package -DskipTests
        </programlisting>
        Or, to clean up before compiling:
        <programlisting>
-mvn clean compile
+mvn clean package -DskipTests
        </programlisting>
        With Eclipse set up as explained above in <xref linkend="eclipse"/>, you can also simply use the build command in Eclipse. To create the full installable HBase package takes a little bit more work, so read on. 
        </para>