You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2017/05/24 23:53:27 UTC
[1/4] hbase git commit: HBASE-18060 Backport to branch-1 HBASE-9774
HBase native metrics and metric collection for coprocessors
Repository: hbase
Updated Branches:
refs/heads/branch-1 f2ba52ac4 -> a3c3f1012
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java
index 8bca6c5..4c28763 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java
@@ -17,10 +17,14 @@
*/
package org.apache.hadoop.hbase.regionserver;
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.hbase.metrics.MetricRegistries;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.Timer;
+
+import com.google.common.annotations.VisibleForTesting;
/**
* <p>
@@ -36,11 +40,20 @@ public class MetricsRegionServer {
private MetricsRegionServerSource serverSource;
private MetricsRegionServerWrapper regionServerWrapper;
+ private MetricRegistry metricRegistry;
+ private Timer bulkLoadTimer;
+
public MetricsRegionServer(MetricsRegionServerWrapper regionServerWrapper) {
this(regionServerWrapper,
CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceFactory.class)
.createServer(regionServerWrapper));
+ // Create hbase-metrics module based metrics. The registry should already be registered by the
+ // MetricsRegionServerSource
+ metricRegistry = MetricRegistries.global().get(serverSource.getMetricRegistryInfo()).get();
+
+ // create and use metrics from the new hbase-metrics based registry.
+ bulkLoadTimer = metricRegistry.timer("Bulkload");
}
MetricsRegionServer(MetricsRegionServerWrapper regionServerWrapper,
@@ -131,4 +144,8 @@ public class MetricsRegionServer {
serverSource.updateCompactionInputSize(isMajor, inputBytes);
serverSource.updateCompactionOutputSize(isMajor, outputBytes);
}
+
+ public void updateBulkLoad(long millis) {
+ this.bulkLoadTimer.updateMillis(millis);
+ }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 9b270c6..3f23d2b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -197,6 +197,8 @@ import org.apache.hadoop.hbase.wal.WALSplitter;
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
import org.apache.zookeeper.KeeperException;
+import com.google.common.annotations.VisibleForTesting;
+
/**
* Implements the regionserver RPC services.
*/
@@ -2061,6 +2063,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
@Override
public BulkLoadHFileResponse bulkLoadHFile(final RpcController controller,
final BulkLoadHFileRequest request) throws ServiceException {
+ long start = EnvironmentEdgeManager.currentTime();
try {
checkOpen();
requestCount.increment();
@@ -2089,6 +2092,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
return builder.build();
} catch (IOException ie) {
throw new ServiceException(ie);
+ } finally {
+ if (regionServer.metricsRegionServer != null) {
+ regionServer.metricsRegionServer.updateBulkLoad(
+ EnvironmentEdgeManager.currentTime() - start);
+ }
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
index 74ecbc8..3ecd970 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
@@ -60,6 +60,7 @@ import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
import org.apache.hadoop.hbase.coprocessor.EndpointObserver;
+import org.apache.hadoop.hbase.coprocessor.MetricsCoprocessor;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.RegionObserver;
@@ -70,6 +71,8 @@ import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.ipc.RpcServer;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
import org.apache.hadoop.hbase.regionserver.Region.Operation;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker;
@@ -85,6 +88,11 @@ import com.google.common.collect.Lists;
import com.google.protobuf.Message;
import com.google.protobuf.Service;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.protobuf.Message;
+import com.google.protobuf.Service;
+
/**
* Implements the coprocessor environment and runtime support for coprocessors
* loaded within a {@link Region}.
@@ -103,7 +111,7 @@ public class RegionCoprocessorHost
private final boolean hasCustomPostScannerFilterRow;
/**
- *
+ *
* Encapsulation of the environment of each coprocessor
*/
static class RegionEnvironment extends CoprocessorHost.Environment
@@ -114,6 +122,7 @@ public class RegionCoprocessorHost
ConcurrentMap<String, Object> sharedData;
private final boolean useLegacyPre;
private final boolean useLegacyPost;
+ private final MetricRegistry metricRegistry;
/**
* Constructor
@@ -135,6 +144,8 @@ public class RegionCoprocessorHost
HRegionInfo.class, WALKey.class, WALEdit.class);
useLegacyPost = useLegacyMethod(impl.getClass(), "postWALRestore", ObserverContext.class,
HRegionInfo.class, WALKey.class, WALEdit.class);
+ this.metricRegistry =
+ MetricsCoprocessor.createRegistryForRegionCoprocessor(impl.getClass().getName());
}
/** @return the region */
@@ -151,6 +162,7 @@ public class RegionCoprocessorHost
public void shutdown() {
super.shutdown();
+ MetricsCoprocessor.removeRegistry(this.metricRegistry);
}
@Override
@@ -163,6 +175,10 @@ public class RegionCoprocessorHost
return region.getRegionInfo();
}
+ @Override
+ public MetricRegistry getMetricRegistryForRegionServer() {
+ return metricRegistry;
+ }
}
static class TableCoprocessorAttribute {
@@ -358,7 +374,7 @@ public class RegionCoprocessorHost
// scan the table attributes for coprocessor load specifications
// initialize the coprocessors
List<RegionEnvironment> configured = new ArrayList<RegionEnvironment>();
- for (TableCoprocessorAttribute attr: getTableCoprocessorAttrsFromSchema(conf,
+ for (TableCoprocessorAttribute attr: getTableCoprocessorAttrsFromSchema(conf,
region.getTableDesc())) {
// Load encompasses classloading and coprocessor initialization
try {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
index bb27dd2..1b64ab8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
@@ -25,22 +25,27 @@ import java.util.List;
import org.apache.commons.lang.ClassUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.MetaMutationAnnotation;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.MetricsCoprocessor;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.RegionServerObserver;
import org.apache.hadoop.hbase.coprocessor.SingletonCoprocessorService;
-import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
+
+import org.apache.hadoop.hbase.ipc.RpcServer;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving
@@ -324,8 +329,8 @@ public class RegionServerCoprocessorHost extends
*/
static class RegionServerEnvironment extends CoprocessorHost.Environment
implements RegionServerCoprocessorEnvironment {
-
- private RegionServerServices regionServerServices;
+ private final RegionServerServices regionServerServices;
+ private final MetricRegistry metricRegistry;
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="BC_UNCONFIRMED_CAST",
justification="Intentional; FB has trouble detecting isAssignableFrom")
@@ -342,12 +347,25 @@ public class RegionServerCoprocessorHost extends
break;
}
}
+ this.metricRegistry =
+ MetricsCoprocessor.createRegistryForRSCoprocessor(implClass.getName());
}
@Override
public RegionServerServices getRegionServerServices() {
return regionServerServices;
}
+
+ @Override
+ public MetricRegistry getMetricRegistryForRegionServer() {
+ return metricRegistry;
+ }
+
+ @Override
+ protected void shutdown() {
+ super.shutdown();
+ MetricsCoprocessor.removeRegistry(metricRegistry);
+ }
}
/**
@@ -356,6 +374,7 @@ public class RegionServerCoprocessorHost extends
*/
static class EnvironmentPriorityComparator implements
Comparator<CoprocessorEnvironment> {
+ @Override
public int compare(final CoprocessorEnvironment env1,
final CoprocessorEnvironment env2) {
if (env1.getPriority() < env2.getPriority()) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
index 87019e8..c726001 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
@@ -29,6 +29,12 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.coprocessor.*;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.MetricsCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.WALCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.WALObserver;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALKey;
@@ -50,6 +56,7 @@ public class WALCoprocessorHost
final boolean useLegacyPre;
final boolean useLegacyPost;
+ private final MetricRegistry metricRegistry;
@Override
public WAL getWAL() {
@@ -78,6 +85,18 @@ public class WALCoprocessorHost
HRegionInfo.class, WALKey.class, WALEdit.class);
useLegacyPost = useLegacyMethod(impl.getClass(), "postWALWrite", ObserverContext.class,
HRegionInfo.class, WALKey.class, WALEdit.class);
+ this.metricRegistry = MetricsCoprocessor.createRegistryForWALCoprocessor(implClass.getName());
+ }
+
+ @Override
+ public MetricRegistry getMetricRegistryForRegionServer() {
+ return metricRegistry;
+ }
+
+ @Override
+ protected void shutdown() {
+ super.shutdown();
+ MetricsCoprocessor.removeRegistry(this.metricRegistry);
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
index a647d03..e808570 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSource;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
@@ -338,4 +339,10 @@ public class MetricsSource implements BaseSource {
public String getMetricsName() {
return globalSourceSource.getMetricsName();
}
+
+ @Override
+ public MetricRegistryInfo getMetricRegistryInfo() {
+ return new MetricRegistryInfo(getMetricsName(), getMetricsDescription(),
+ getMetricsContext(), getMetricsJmxContext(), true);
+ }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
index 706d8e7..af89d3c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
@@ -660,6 +660,16 @@ public class MiniHBaseCluster extends HBaseCluster {
return hbaseCluster.getRegionServer(serverNumber);
}
+ public HRegionServer getRegionServer(ServerName serverName) {
+ for (RegionServerThread t : hbaseCluster.getRegionServers()) {
+ HRegionServer r = t.getRegionServer();
+ if (r.getServerName().equals(serverName)) {
+ return r;
+ }
+ }
+ return null;
+ }
+
public List<HRegion> getRegions(byte[] tableName) {
return getRegions(TableName.valueOf(tableName));
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
new file mode 100644
index 0000000..6c4562e
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
@@ -0,0 +1,549 @@
+/**
+ * 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.coprocessor;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+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.Cell;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter.Predicate;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.metrics.Metric;
+import org.apache.hadoop.hbase.metrics.MetricRegistries;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+import org.apache.hadoop.hbase.metrics.Timer;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService;
+import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsResponse;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+/**
+ * Testing of coprocessor metrics end-to-end.
+ */
+@Category({CoprocessorTests.class, MediumTests.class})
+public class TestCoprocessorMetrics {
+
+ private static final Log LOG = LogFactory.getLog(TestCoprocessorMetrics.class);
+ private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+ private static final byte[] foo = Bytes.toBytes("foo");
+ private static final byte[] bar = Bytes.toBytes("bar");
+ /**
+ * MasterObserver that has a Timer metric for create table operation.
+ */
+ public static class CustomMasterObserver extends BaseMasterObserver {
+ private Timer createTableTimer;
+ private long start = Long.MIN_VALUE;
+
+ @Override
+ public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
+ HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
+ super.preCreateTable(ctx, desc, regions);
+
+ // we rely on the fact that there is only 1 instance of our MasterObserver
+ this.start = System.currentTimeMillis();
+ }
+
+ @Override
+ public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
+ HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
+ super.postCreateTable(ctx, desc, regions);
+ if (this.start > 0) {
+ long time = System.currentTimeMillis() - start;
+ LOG.info("Create table took: " + time);
+ createTableTimer.updateMillis(time);
+ }
+ }
+
+ @Override
+ public void start(CoprocessorEnvironment env) throws IOException {
+ super.start(env);
+ if (env instanceof MasterCoprocessorEnvironment) {
+ MetricRegistry registry =
+ ((MasterCoprocessorEnvironment) env).getMetricRegistryForMaster();
+
+ createTableTimer = registry.timer("CreateTable");
+ }
+ }
+ }
+
+ /**
+ * RegionServerObserver that has a Counter for rollWAL requests.
+ */
+ public static class CustomRegionServerObserver extends BaseRegionServerObserver {
+ /** This is the Counter metric object to keep track of the current count across invocations */
+ private Counter rollWALCounter;
+ @Override
+ public void postRollWALWriterRequest(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
+ throws IOException {
+ // Increment the Counter whenever the coprocessor is called
+ rollWALCounter.increment();
+ super.postRollWALWriterRequest(ctx);
+ }
+
+ @Override
+ public void start(CoprocessorEnvironment env) throws IOException {
+ super.start(env);
+ if (env instanceof RegionServerCoprocessorEnvironment) {
+ MetricRegistry registry =
+ ((RegionServerCoprocessorEnvironment) env).getMetricRegistryForRegionServer();
+
+ if (rollWALCounter == null) {
+ rollWALCounter = registry.counter("rollWALRequests");
+ }
+ }
+ }
+ }
+
+ /**
+ * WALObserver that has a Counter for walEdits written.
+ */
+ public static class CustomWALObserver extends BaseWALObserver {
+ private Counter walEditsCount;
+
+ @Override
+ public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
+ HRegionInfo info, org.apache.hadoop.hbase.wal.WALKey logKey,
+ WALEdit logEdit) throws IOException {
+ super.postWALWrite(ctx, info, logKey, logEdit);
+ walEditsCount.increment();
+ }
+
+ @Override
+ public void start(CoprocessorEnvironment env) throws IOException {
+ super.start(env);
+ if (env instanceof WALCoprocessorEnvironment) {
+ MetricRegistry registry =
+ ((WALCoprocessorEnvironment) env).getMetricRegistryForRegionServer();
+
+ if (walEditsCount == null) {
+ walEditsCount = registry.counter("walEditsCount");
+ }
+ }
+ }
+ }
+
+ /**
+ * RegionObserver that has a Counter for preGet()
+ */
+ public static class CustomRegionObserver extends BaseRegionObserver {
+ private Counter preGetCounter;
+
+ @Override
+ public void preGetOp(ObserverContext<RegionCoprocessorEnvironment> e, Get get,
+ List<Cell> results) throws IOException {
+ super.preGetOp(e, get, results);
+ preGetCounter.increment();
+ }
+
+ @Override
+ public void start(CoprocessorEnvironment env) throws IOException {
+ super.start(env);
+
+ if (env instanceof RegionCoprocessorEnvironment) {
+ MetricRegistry registry =
+ ((RegionCoprocessorEnvironment) env).getMetricRegistryForRegionServer();
+
+ if (preGetCounter == null) {
+ preGetCounter = registry.counter("preGetRequests");
+ }
+ }
+ }
+ }
+
+ public static class CustomRegionObserver2 extends CustomRegionObserver {
+ }
+
+ /**
+ * RegionEndpoint to test metrics from endpoint calls
+ */
+ public static class CustomRegionEndpoint extends MultiRowMutationEndpoint {
+
+ private Timer endpointExecution;
+
+ @Override
+ public void mutateRows(RpcController controller, MutateRowsRequest request,
+ RpcCallback<MutateRowsResponse> done) {
+ long start = System.nanoTime();
+ super.mutateRows(controller, request, done);
+ endpointExecution.updateNanos(System.nanoTime() - start);
+ }
+
+ @Override
+ public void start(CoprocessorEnvironment env) throws IOException {
+ super.start(env);
+
+ if (env instanceof RegionCoprocessorEnvironment) {
+ MetricRegistry registry =
+ ((RegionCoprocessorEnvironment) env).getMetricRegistryForRegionServer();
+
+ if (endpointExecution == null) {
+ endpointExecution = registry.timer("EndpointExecution");
+ }
+ }
+ }
+ }
+
+ @BeforeClass
+ public static void setupBeforeClass() throws Exception {
+ Configuration conf = UTIL.getConfiguration();
+ // inject master, regionserver and WAL coprocessors
+ conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
+ CustomMasterObserver.class.getName());
+ conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY,
+ CustomRegionServerObserver.class.getName());
+ conf.set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY,
+ CustomWALObserver.class.getName());
+ conf.setBoolean(CoprocessorHost.ABORT_ON_ERROR_KEY, true);
+ UTIL.startMiniCluster();
+ }
+
+ @AfterClass
+ public static void teardownAfterClass() throws Exception {
+ UTIL.shutdownMiniCluster();
+ }
+
+ @Before
+ public void setup() throws IOException {
+ try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
+ Admin admin = connection.getAdmin()) {
+ for (HTableDescriptor htd : admin.listTables()) {
+ UTIL.deleteTable(htd.getTableName());
+ }
+ }
+ }
+
+ @Test
+ public void testMasterObserver() throws IOException {
+ // Find out the MetricRegistry used by the CP using the global registries
+ MetricRegistryInfo info = MetricsCoprocessor.createRegistryInfoForMasterCoprocessor(
+ CustomMasterObserver.class.getName());
+ Optional<MetricRegistry> registry = MetricRegistries.global().get(info);
+ assertTrue(registry.isPresent());
+
+ Optional<Metric> metric = registry.get().get("CreateTable");
+ assertTrue(metric.isPresent());
+
+ try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
+ Admin admin = connection.getAdmin()) {
+
+ Timer createTableTimer = (Timer)metric.get();
+ long prevCount = createTableTimer.getHistogram().getCount();
+ LOG.info("Creating table");
+ admin.createTable(
+ new HTableDescriptor("testMasterObserver")
+ .addFamily(new HColumnDescriptor("foo")));
+
+ assertEquals(1, createTableTimer.getHistogram().getCount() - prevCount);
+ }
+ }
+
+ @Test
+ public void testRegionServerObserver() throws IOException {
+ try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
+ Admin admin = connection.getAdmin()) {
+ LOG.info("Rolling WALs");
+ admin.rollWALWriter(UTIL.getMiniHBaseCluster().getServerHoldingMeta());
+ }
+
+ // Find out the MetricRegistry used by the CP using the global registries
+ MetricRegistryInfo info = MetricsCoprocessor.createRegistryInfoForRSCoprocessor(
+ CustomRegionServerObserver.class.getName());
+
+ Optional<MetricRegistry> registry = MetricRegistries.global().get(info);
+ assertTrue(registry.isPresent());
+
+ Optional<Metric> metric = registry.get().get("rollWALRequests");
+ assertTrue(metric.isPresent());
+
+ Counter rollWalRequests = (Counter)metric.get();
+ assertEquals(1, rollWalRequests.getCount());
+ }
+
+ @Test
+ public void testWALObserver() throws IOException {
+ // Find out the MetricRegistry used by the CP using the global registries
+ MetricRegistryInfo info = MetricsCoprocessor.createRegistryInfoForWALCoprocessor(
+ CustomWALObserver.class.getName());
+
+ Optional<MetricRegistry> registry = MetricRegistries.global().get(info);
+ assertTrue(registry.isPresent());
+
+ Optional<Metric> metric = registry.get().get("walEditsCount");
+ assertTrue(metric.isPresent());
+
+ try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
+ Admin admin = connection.getAdmin()) {
+ admin.createTable(
+ new HTableDescriptor("testWALObserver")
+ .addFamily(new HColumnDescriptor("foo")));
+
+ Counter rollWalRequests = (Counter)metric.get();
+ long prevCount = rollWalRequests.getCount();
+ assertTrue(prevCount > 0);
+
+ try (Table table = connection.getTable(TableName.valueOf("testWALObserver"))) {
+ table.put(new Put(foo).addColumn(foo, foo, foo));
+ }
+
+ assertEquals(1, rollWalRequests.getCount() - prevCount);
+ }
+ }
+
+ /**
+ * Helper for below tests
+ */
+ private void assertPreGetRequestsCounter(Class<?> coprocClass) {
+ // Find out the MetricRegistry used by the CP using the global registries
+ MetricRegistryInfo info = MetricsCoprocessor.createRegistryInfoForRegionCoprocessor(
+ coprocClass.getName());
+
+ Optional<MetricRegistry> registry = MetricRegistries.global().get(info);
+ assertTrue(registry.isPresent());
+
+ Optional<Metric> metric = registry.get().get("preGetRequests");
+ assertTrue(metric.isPresent());
+
+ Counter preGetRequests = (Counter)metric.get();
+ assertEquals(2, preGetRequests.getCount());
+ }
+
+ @Test
+ public void testRegionObserverSingleRegion() throws IOException {
+ TableName tableName = TableName.valueOf("testRegionObserverSingleRegion");
+ try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
+ Admin admin = connection.getAdmin()) {
+ admin.createTable(
+ new HTableDescriptor(tableName)
+ .addFamily(new HColumnDescriptor(foo))
+ // add the coprocessor for the region
+ .addCoprocessor(CustomRegionObserver.class.getName()));
+ try (Table table = connection.getTable(tableName)) {
+ table.get(new Get(foo));
+ table.get(new Get(foo)); // 2 gets
+ }
+ }
+
+ assertPreGetRequestsCounter(CustomRegionObserver.class);
+ }
+
+ @Test
+ public void testRegionObserverMultiRegion() throws IOException {
+ TableName tableName = TableName.valueOf("testRegionObserverMultiRegion");
+ try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
+ Admin admin = connection.getAdmin()) {
+ admin.createTable(
+ new HTableDescriptor(tableName)
+ .addFamily(new HColumnDescriptor(foo))
+ // add the coprocessor for the region
+ .addCoprocessor(CustomRegionObserver.class.getName())
+ , new byte[][]{foo}); // create with 2 regions
+ try (Table table = connection.getTable(tableName);
+ RegionLocator locator = connection.getRegionLocator(tableName)) {
+ table.get(new Get(bar));
+ table.get(new Get(foo)); // 2 gets to 2 separate regions
+ assertEquals(2, locator.getAllRegionLocations().size());
+ assertNotEquals(locator.getRegionLocation(bar).getRegionInfo(),
+ locator.getRegionLocation(foo).getRegionInfo());
+ }
+ }
+
+ assertPreGetRequestsCounter(CustomRegionObserver.class);
+ }
+
+ @Test
+ public void testRegionObserverMultiTable() throws IOException {
+ TableName tableName1 = TableName.valueOf("testRegionObserverMultiTable1");
+ TableName tableName2 = TableName.valueOf("testRegionObserverMultiTable2");
+ try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
+ Admin admin = connection.getAdmin()) {
+ admin.createTable(
+ new HTableDescriptor(tableName1)
+ .addFamily(new HColumnDescriptor(foo))
+ // add the coprocessor for the region
+ .addCoprocessor(CustomRegionObserver.class.getName()));
+ admin.createTable(
+ new HTableDescriptor(tableName2)
+ .addFamily(new HColumnDescriptor(foo))
+ // add the coprocessor for the region
+ .addCoprocessor(CustomRegionObserver.class.getName()));
+ try (Table table1 = connection.getTable(tableName1);
+ Table table2 = connection.getTable(tableName2);) {
+ table1.get(new Get(bar));
+ table2.get(new Get(foo)); // 2 gets to 2 separate tables
+ }
+ }
+ assertPreGetRequestsCounter(CustomRegionObserver.class);
+ }
+
+ @Test
+ public void testRegionObserverMultiCoprocessor() throws IOException {
+ TableName tableName = TableName.valueOf("testRegionObserverMultiCoprocessor");
+ try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
+ Admin admin = connection.getAdmin()) {
+ admin.createTable(
+ new HTableDescriptor(tableName)
+ .addFamily(new HColumnDescriptor(foo))
+ // add the coprocessor for the region. We add two different coprocessors
+ .addCoprocessor(CustomRegionObserver.class.getName())
+ .addCoprocessor(CustomRegionObserver2.class.getName()));
+ try (Table table = connection.getTable(tableName)) {
+ table.get(new Get(foo));
+ table.get(new Get(foo)); // 2 gets
+ }
+ }
+
+ // we will have two counters coming from two coprocs, in two different MetricRegistries
+ assertPreGetRequestsCounter(CustomRegionObserver.class);
+ assertPreGetRequestsCounter(CustomRegionObserver2.class);
+ }
+
+ @Test
+ public void testRegionObserverAfterRegionClosed() throws IOException {
+ TableName tableName = TableName.valueOf("testRegionObserverAfterRegionClosed");
+ try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
+ Admin admin = connection.getAdmin()) {
+ admin.createTable(
+ new HTableDescriptor(tableName)
+ .addFamily(new HColumnDescriptor(foo))
+ // add the coprocessor for the region
+ .addCoprocessor(CustomRegionObserver.class.getName())
+ , new byte[][]{foo}); // create with 2 regions
+ try (Table table = connection.getTable(tableName)) {
+ table.get(new Get(foo));
+ table.get(new Get(foo)); // 2 gets
+ }
+
+ assertPreGetRequestsCounter(CustomRegionObserver.class);
+
+ // close one of the regions
+ try (RegionLocator locator = connection.getRegionLocator(tableName)) {
+ final HRegionLocation loc = locator.getRegionLocation(foo);
+ admin.closeRegion(loc.getServerName(), loc.getRegionInfo());
+
+ final HRegionServer server = UTIL.getMiniHBaseCluster().getRegionServer(loc.getServerName());
+ UTIL.waitFor(30000,new Predicate<IOException>() {
+ @Override
+ public boolean evaluate() throws IOException {
+ return server.getOnlineRegion(loc.getRegionInfo().getRegionName()) == null;
+ }
+ });
+ assertNull(server.getOnlineRegion(loc.getRegionInfo().getRegionName()));
+ }
+
+ // with only 1 region remaining, we should still be able to find the Counter
+ assertPreGetRequestsCounter(CustomRegionObserver.class);
+
+ // close the table
+ admin.disableTable(tableName);
+
+ MetricRegistryInfo info = MetricsCoprocessor.createRegistryInfoForRegionCoprocessor(
+ CustomRegionObserver.class.getName());
+
+ // ensure that MetricRegistry is deleted
+ Optional<MetricRegistry> registry = MetricRegistries.global().get(info);
+ assertFalse(registry.isPresent());
+ }
+ }
+
+ @Test
+ public void testRegionObserverEndpoint() throws IOException, ServiceException {
+ TableName tableName = TableName.valueOf("testRegionObserverEndpoint");
+ try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
+ Admin admin = connection.getAdmin()) {
+ admin.createTable(
+ new HTableDescriptor(tableName)
+ .addFamily(new HColumnDescriptor(foo))
+ // add the coprocessor for the region
+ .addCoprocessor(CustomRegionEndpoint.class.getName()));
+
+ try (Table table = connection.getTable(tableName)) {
+ List<Put> mutations = Lists.newArrayList(new Put(foo), new Put(bar));
+ MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder();
+
+ for (Mutation mutation : mutations) {
+ mrmBuilder.addMutationRequest(ProtobufUtil.toMutation(
+ ClientProtos.MutationProto.MutationType.PUT, mutation));
+ }
+
+ CoprocessorRpcChannel channel = table.coprocessorService(bar);
+ MultiRowMutationService.BlockingInterface service =
+ MultiRowMutationService.newBlockingStub(channel);
+ MutateRowsRequest mrm = mrmBuilder.build();
+ service.mutateRows(null, mrm);
+ }
+ }
+
+ // Find out the MetricRegistry used by the CP using the global registries
+ MetricRegistryInfo info = MetricsCoprocessor.createRegistryInfoForRegionCoprocessor(
+ CustomRegionEndpoint.class.getName());
+
+ Optional<MetricRegistry> registry = MetricRegistries.global().get(info);
+ assertTrue(registry.isPresent());
+
+ Optional<Metric> metric = registry.get().get("EndpointExecution");
+ assertTrue(metric.isPresent());
+
+ Timer endpointExecutions = (Timer)metric.get();
+ assertEquals(1, endpointExecutions.getHistogram().getCount());
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
index 50bc589..6a659cb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
@@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@@ -217,6 +218,11 @@ public class TestTokenAuthentication {
public ConcurrentMap<String, Object> getSharedData() { return null; }
@Override
+ public MetricRegistry getMetricRegistryForRegionServer() {
+ return null;
+ }
+
+ @Override
public int getVersion() { return 0; }
@Override
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a1639fb..1e45249 100644
--- a/pom.xml
+++ b/pom.xml
@@ -81,6 +81,8 @@
<module>hbase-external-blockcache</module>
<module>hbase-shaded</module>
<module>hbase-archetypes</module>
+ <module>hbase-metrics-api</module>
+ <module>hbase-metrics</module>
</modules>
<!--Add apache snapshots in case we want to use unreleased versions of plugins:
e.g. surefire 2.18-SNAPSHOT-->
@@ -1430,6 +1432,30 @@
<groupId>org.apache.hbase</groupId>
<version>${project.version}</version>
</dependency>
+ <dependency>
+ <artifactId>hbase-metrics-api</artifactId>
+ <groupId>org.apache.hbase</groupId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <artifactId>hbase-metrics-api</artifactId>
+ <groupId>org.apache.hbase</groupId>
+ <version>${project.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <artifactId>hbase-metrics</artifactId>
+ <groupId>org.apache.hbase</groupId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <artifactId>hbase-metrics</artifactId>
+ <groupId>org.apache.hbase</groupId>
+ <version>${project.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
<!-- General dependencies -->
<dependency>
<groupId>com.github.stephenc.findbugs</groupId>
@@ -1460,6 +1486,11 @@
<version>${metrics-core.version}</version>
</dependency>
<dependency>
+ <groupId>io.dropwizard.metrics</groupId>
+ <artifactId>metrics-core</artifactId>
+ <version>3.1.2</version>
+ </dependency>
+ <dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
<version>${guava.version}</version>
[4/4] hbase git commit: HBASE-18060 Backport to branch-1 HBASE-9774
HBase native metrics and metric collection for coprocessors
Posted by ap...@apache.org.
HBASE-18060 Backport to branch-1 HBASE-9774 HBase native metrics and metric collection for coprocessors
Signed-off-by: Andrew Purtell <ap...@apache.org>
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/a3c3f101
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/a3c3f101
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/a3c3f101
Branch: refs/heads/branch-1
Commit: a3c3f1012d0eae531c8f38ac5c7e7b81402d8a84
Parents: f2ba52a
Author: Vincent <vi...@gmail.com>
Authored: Thu May 18 15:39:29 2017 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed May 24 13:20:44 2017 -0700
----------------------------------------------------------------------
hbase-assembly/pom.xml | 10 +
.../hadoop/hbase/util/FastLongHistogram.java | 318 -----------
.../org/apache/hadoop/hbase/util/LongAdder.java | 224 ++++++++
.../org/apache/hadoop/hbase/util/Striped64.java | 356 ++++++++++++
.../hbase/util/TestFastLongHistogram.java | 132 -----
.../ExampleMasterObserverWithMetrics.java | 148 +++++
.../ExampleRegionObserverWithMetrics.java | 121 ++++
hbase-hadoop-compat/pom.xml | 4 +
.../apache/hadoop/hbase/metrics/BaseSource.java | 2 +
hbase-hadoop2-compat/pom.xml | 4 +
.../hadoop/hbase/metrics/BaseSourceImpl.java | 52 ++
.../impl/GlobalMetricRegistriesAdapter.java | 233 ++++++++
.../impl/HBaseMetrics2HadoopMetricsAdapter.java | 169 ++++++
.../MetricsRegionServerSourceImpl.java | 6 +
.../MetricsReplicationGlobalSourceSource.java | 7 +
.../MetricsReplicationSourceSourceImpl.java | 7 +
.../lib/DefaultMetricsSystemHelper.java | 50 +-
.../hadoop/metrics2/lib/MutableHistogram.java | 111 ++--
.../metrics2/lib/MutableRangeHistogram.java | 14 +-
.../impl/TestGlobalMetricRegistriesAdapter.java | 86 +++
hbase-metrics-api/README.txt | 78 +++
hbase-metrics-api/pom.xml | 112 ++++
.../apache/hadoop/hbase/metrics/Counter.java | 60 ++
.../org/apache/hadoop/hbase/metrics/Gauge.java | 35 ++
.../apache/hadoop/hbase/metrics/Histogram.java | 58 ++
.../org/apache/hadoop/hbase/metrics/Meter.java | 90 +++
.../org/apache/hadoop/hbase/metrics/Metric.java | 30 +
.../hadoop/hbase/metrics/MetricRegistries.java | 90 +++
.../hbase/metrics/MetricRegistriesLoader.java | 96 ++++
.../hadoop/hbase/metrics/MetricRegistry.java | 112 ++++
.../hbase/metrics/MetricRegistryFactory.java | 36 ++
.../hbase/metrics/MetricRegistryInfo.java | 112 ++++
.../apache/hadoop/hbase/metrics/MetricSet.java | 41 ++
.../hadoop/hbase/metrics/PackageMarker.java | 38 ++
.../apache/hadoop/hbase/metrics/Snapshot.java | 136 +++++
.../org/apache/hadoop/hbase/metrics/Timer.java | 62 +++
.../hadoop/hbase/metrics/package-info.java | 25 +
.../metrics/TestMetricRegistriesLoader.java | 56 ++
hbase-metrics/README.txt | 1 +
hbase-metrics/pom.xml | 136 +++++
.../hadoop/hbase/metrics/impl/CounterImpl.java | 60 ++
.../hbase/metrics/impl/DropwizardMeter.java | 74 +++
.../hbase/metrics/impl/FastLongHistogram.java | 406 ++++++++++++++
.../hbase/metrics/impl/HistogramImpl.java | 81 +++
.../metrics/impl/MetricRegistriesImpl.java | 82 +++
.../metrics/impl/MetricRegistryFactoryImpl.java | 34 ++
.../hbase/metrics/impl/MetricRegistryImpl.java | 183 +++++++
.../hbase/metrics/impl/RefCountingMap.java | 108 ++++
.../hadoop/hbase/metrics/impl/TimerImpl.java | 74 +++
.../hadoop/hbase/metrics/impl/package-info.java | 25 +
...apache.hadoop.hbase.metrics.MetricRegistries | 18 +
.../hbase/metrics/impl/TestCounterImpl.java | 59 ++
.../hbase/metrics/impl/TestDropwizardMeter.java | 51 ++
.../metrics/impl/TestFastLongHistogram.java | 132 +++++
.../hadoop/hbase/metrics/impl/TestGauge.java | 61 +++
.../hbase/metrics/impl/TestHistogramImpl.java | 103 ++++
.../metrics/impl/TestMetricRegistryImpl.java | 164 ++++++
.../hbase/metrics/impl/TestRefCountingMap.java | 284 ++++++++++
.../hbase/metrics/impl/TestTimerImpl.java | 53 ++
.../src/main/resources/supplemental-models.xml | 14 +
hbase-server/pom.xml | 8 +
.../MasterCoprocessorEnvironment.java | 10 +
.../hbase/coprocessor/MetricsCoprocessor.java | 136 +++++
.../RegionCoprocessorEnvironment.java | 24 +-
.../RegionServerCoprocessorEnvironment.java | 10 +
.../coprocessor/WALCoprocessorEnvironment.java | 10 +
.../hadoop/hbase/io/hfile/AgeSnapshot.java | 2 +-
.../hadoop/hbase/io/hfile/BlockCacheUtil.java | 2 +-
.../hadoop/hbase/io/hfile/CacheStats.java | 2 +-
.../hbase/master/MasterCoprocessorHost.java | 25 +-
.../hbase/regionserver/MetricsRegionServer.java | 21 +-
.../hbase/regionserver/RSRpcServices.java | 8 +
.../regionserver/RegionCoprocessorHost.java | 20 +-
.../RegionServerCoprocessorHost.java | 29 +-
.../regionserver/wal/WALCoprocessorHost.java | 19 +
.../replication/regionserver/MetricsSource.java | 7 +
.../apache/hadoop/hbase/MiniHBaseCluster.java | 10 +
.../coprocessor/TestCoprocessorMetrics.java | 549 +++++++++++++++++++
.../security/token/TestTokenAuthentication.java | 6 +
pom.xml | 31 ++
80 files changed, 5904 insertions(+), 549 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-assembly/pom.xml b/hbase-assembly/pom.xml
index 2c78516..a8ac79d 100644
--- a/hbase-assembly/pom.xml
+++ b/hbase-assembly/pom.xml
@@ -205,6 +205,16 @@
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-metrics-api</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-metrics</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
<artifactId>hbase-resource-bundle</artifactId>
<version>${project.version}</version>
<optional>true</optional>
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java
deleted file mode 100644
index 3c4eccc..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java
+++ /dev/null
@@ -1,318 +0,0 @@
-/**
- * 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.util;
-
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-
-/**
- * FastLongHistogram is a thread-safe class that estimate distribution of data and computes the
- * quantiles.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class FastLongHistogram {
-
- /**
- * Default number of bins.
- */
- public static final int DEFAULT_NBINS = 255;
-
- public static final double[] DEFAULT_QUANTILES =
- new double[]{0.25, 0.5, 0.75, 0.90, 0.95, 0.98, 0.99, 0.999};
-
- /**
- * Bins is a class containing a list of buckets(or bins) for estimation histogram of some data.
- */
- private static class Bins {
- private final Counter[] counts;
- // inclusive
- private final long binsMin;
- // exclusive
- private final long binsMax;
- private final long bins10XMax;
- private final AtomicLong min = new AtomicLong(Long.MAX_VALUE);
- private final AtomicLong max = new AtomicLong(0L);
-
- private final Counter count = new Counter(0);
- private final Counter total = new Counter(0);
-
- // set to true when any of data has been inserted to the Bins. It is set after the counts are
- // updated.
- private final AtomicBoolean hasData = new AtomicBoolean(false);
-
- /**
- * The constructor for creating a Bins without any prior data.
- */
- public Bins(int numBins) {
- counts = createCounters(numBins + 3);
- this.binsMin = 1L;
-
- // These two numbers are total guesses
- // and should be treated as highly suspect.
- this.binsMax = 1000;
- this.bins10XMax = binsMax * 10;
- }
-
- /**
- * The constructor for creating a Bins with last Bins.
- */
- public Bins(Bins last, int numOfBins, double minQ, double maxQ) {
- long[] values = last.getQuantiles(new double[] { minQ, maxQ });
- long wd = values[1] - values[0] + 1;
- // expand minQ and maxQ in two ends back assuming uniform distribution
- this.binsMin = Math.max(0L, (long) (values[0] - wd * minQ));
- long binsMax = (long) (values[1] + wd * (1 - maxQ)) + 1;
- // make sure each of bins is at least of width 1
- this.binsMax = Math.max(binsMax, this.binsMin + numOfBins);
- this.bins10XMax = Math.max((long) (values[1] + (binsMax - 1) * 9), this.binsMax + 1);
-
- this.counts = createCounters(numOfBins + 3);
- }
-
- private Counter[] createCounters(int num) {
- Counter[] counters = new Counter[num];
- for (int i = 0; i < num; i++) {
- counters[i] = new Counter();
- }
- return counters;
- }
-
- private int getIndex(long value) {
- if (value < this.binsMin) {
- return 0;
- } else if (value > this.bins10XMax) {
- return this.counts.length - 1;
- } else if (value >= this.binsMax) {
- return this.counts.length - 2;
- }
- // compute the position
- return 1 + (int) ((value - this.binsMin) * (this.counts.length - 3) /
- (this.binsMax - this.binsMin));
-
- }
-
- /**
- * Adds a value to the histogram.
- */
- public void add(long value, long count) {
- if (value < 0) {
- // The whole computation is completely thrown off if there are negative numbers
- //
- // Normally we would throw an IllegalArgumentException however this is the metrics
- // system and it should be completely safe at all times.
- // So silently throw it away.
- return;
- }
- AtomicUtils.updateMin(min, value);
- AtomicUtils.updateMax(max, value);
-
- this.count.add(count);
- this.total.add(value * count);
-
- int pos = getIndex(value);
- this.counts[pos].add(count);
-
- // hasData needs to be updated as last
- this.hasData.set(true);
- }
-
- /**
- * Computes the quantiles give the ratios.
- */
- public long[] getQuantiles(double[] quantiles) {
- if (!this.hasData.get()) {
- // No data yet.
- return new long[quantiles.length];
- }
-
- // Make a snapshot of lowerCounter, higherCounter and bins.counts to counts.
- // This is not synchronized, but since the counter are accumulating, the result is a good
- // estimation of a snapshot.
- long[] counts = new long[this.counts.length];
- long total = 0L;
- for (int i = 0; i < this.counts.length; i++) {
- counts[i] = this.counts[i].get();
- total += counts[i];
- }
-
- int rIndex = 0;
- double qCount = total * quantiles[0];
- long cum = 0L;
-
- long[] res = new long[quantiles.length];
- countsLoop: for (int i = 0; i < counts.length; i++) {
- // mn and mx define a value range
- long mn, mx;
- if (i == 0) {
- mn = this.min.get();
- mx = this.binsMin;
- } else if (i == counts.length - 1) {
- mn = this.bins10XMax;
- mx = this.max.get();
- } else if (i == counts.length - 2) {
- mn = this.binsMax;
- mx = this.bins10XMax;
- } else {
- mn = this.binsMin + (i - 1) * (this.binsMax - this.binsMin) / (this.counts.length - 3);
- mx = this.binsMin + i * (this.binsMax - this.binsMin) / (this.counts.length - 3);
- }
-
- if (mx < this.min.get()) {
- continue;
- }
- if (mn > this.max.get()) {
- break;
- }
- mn = Math.max(mn, this.min.get());
- mx = Math.min(mx, this.max.get());
-
- // lastCum/cum are the corresponding counts to mn/mx
- double lastCum = cum;
- cum += counts[i];
-
- // fill the results for qCount is within current range.
- while (qCount <= cum) {
- if (cum == lastCum) {
- res[rIndex] = mn;
- } else {
- res[rIndex] = (long) ((qCount - lastCum) * (mx - mn) / (cum - lastCum) + mn);
- }
-
- // move to next quantile
- rIndex++;
- if (rIndex >= quantiles.length) {
- break countsLoop;
- }
- qCount = total * quantiles[rIndex];
- }
- }
- // In case quantiles contains values >= 100%
- for (; rIndex < quantiles.length; rIndex++) {
- res[rIndex] = this.max.get();
- }
-
- return res;
- }
-
-
- long getNumAtOrBelow(long val) {
- final int targetIndex = getIndex(val);
- long totalToCurrentIndex = 0;
- for (int i = 0; i <= targetIndex; i++) {
- totalToCurrentIndex += this.counts[i].get();
- }
- return totalToCurrentIndex;
- }
- }
-
- // The bins counting values. It is replaced with a new one in calling of reset().
- private volatile Bins bins;
-
- /**
- * Constructor.
- */
- public FastLongHistogram() {
- this(DEFAULT_NBINS);
- }
-
- /**
- * Constructor.
- * @param numOfBins the number of bins for the histogram. A larger value results in more precise
- * results but with lower efficiency, and vice versus.
- */
- public FastLongHistogram(int numOfBins) {
- this.bins = new Bins(numOfBins);
- }
-
- /**
- * Constructor setting the bins assuming a uniform distribution within a range.
- * @param numOfBins the number of bins for the histogram. A larger value results in more precise
- * results but with lower efficiency, and vice versus.
- * @param min lower bound of the region, inclusive.
- * @param max higher bound of the region, inclusive.
- */
- public FastLongHistogram(int numOfBins, long min, long max) {
- this(numOfBins);
- Bins bins = new Bins(numOfBins);
- bins.add(min, 1);
- bins.add(max, 1);
- this.bins = new Bins(bins, numOfBins, 0.01, 0.999);
- }
-
- private FastLongHistogram(Bins bins) {
- this.bins = bins;
- }
-
- /**
- * Adds a value to the histogram.
- */
- public void add(long value, long count) {
- this.bins.add(value, count);
- }
-
- /**
- * Computes the quantiles give the ratios.
- */
- public long[] getQuantiles(double[] quantiles) {
- return this.bins.getQuantiles(quantiles);
- }
-
- public long[] getQuantiles() {
- return this.bins.getQuantiles(DEFAULT_QUANTILES);
- }
-
- public long getMin() {
- long min = this.bins.min.get();
- return min == Long.MAX_VALUE ? 0 : min; // in case it is not initialized
- }
-
- public long getMax() {
- return this.bins.max.get();
- }
-
- public long getCount() {
- return this.bins.count.get();
- }
-
- public long getMean() {
- Bins bins = this.bins;
- long count = bins.count.get();
- long total = bins.total.get();
- if (count == 0) {
- return 0;
- }
- return total / count;
- }
-
- public long getNumAtOrBelow(long value) {
- return this.bins.getNumAtOrBelow(value);
- }
-
- /**
- * Resets the histogram for new counting.
- */
- public FastLongHistogram reset() {
- Bins oldBins = this.bins;
- this.bins = new Bins(this.bins, this.bins.counts.length - 3, 0.01, 0.99);
- return new FastLongHistogram(oldBins);
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-common/src/main/java/org/apache/hadoop/hbase/util/LongAdder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/LongAdder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/LongAdder.java
new file mode 100644
index 0000000..9bdb829
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/LongAdder.java
@@ -0,0 +1,224 @@
+/**
+ *
+ * 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.
+ */
+/*
+ * Written by Doug Lea with assistance from members of JCP JSR-166
+ * Expert Group and released to the public domain, as explained at
+ * http://creativecommons.org/publicdomain/zero/1.0/
+ */
+
+package org.apache.hadoop.hbase.util;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * One or more variables that together maintain an initially zero
+ * {@code long} sum. When updates (method {@link #add}) are contended
+ * across threads, the set of variables may grow dynamically to reduce
+ * contention. Method {@link #sum} (or, equivalently, {@link
+ * #longValue}) returns the current total combined across the
+ * variables maintaining the sum.
+ *
+ * <p>This class is usually preferable to {@link AtomicLong} when
+ * multiple threads update a common sum that is used for purposes such
+ * as collecting statistics, not for fine-grained synchronization
+ * control. Under low update contention, the two classes have similar
+ * characteristics. But under high contention, expected throughput of
+ * this class is significantly higher, at the expense of higher space
+ * consumption.
+ *
+ * <p>This class extends {@link Number}, but does <em>not</em> define
+ * methods such as {@code equals}, {@code hashCode} and {@code
+ * compareTo} because instances are expected to be mutated, and so are
+ * not useful as collection keys.
+ *
+ * <p><em>jsr166e note: This class is targeted to be placed in
+ * java.util.concurrent.atomic.</em>
+ *
+ * @since 1.8
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class LongAdder extends Striped64 implements Serializable {
+ private static final long serialVersionUID = 7249069246863182397L;
+
+ /**
+ * Version of plus for use in retryUpdate
+ */
+ final long fn(long v, long x) { return v + x; }
+
+ /**
+ * Creates a new adder with initial sum of zero.
+ */
+ public LongAdder() {
+ }
+
+ /**
+ * Adds the given value.
+ *
+ * @param x the value to add
+ */
+ public void add(long x) {
+ Cell[] as; long b, v; int[] hc; Cell a; int n;
+ if ((as = cells) != null || !casBase(b = base, b + x)) {
+ boolean uncontended = true;
+ if ((hc = threadHashCode.get()) == null ||
+ as == null || (n = as.length) < 1 ||
+ (a = as[(n - 1) & hc[0]]) == null ||
+ !(uncontended = a.cas(v = a.value, v + x)))
+ retryUpdate(x, hc, uncontended);
+ }
+ }
+
+ /**
+ * Equivalent to {@code add(1)}.
+ */
+ public void increment() {
+ add(1L);
+ }
+
+ /**
+ * Equivalent to {@code add(-1)}.
+ */
+ public void decrement() {
+ add(-1L);
+ }
+
+ /**
+ * Returns the current sum. The returned value is <em>NOT</em> an
+ * atomic snapshot; invocation in the absence of concurrent
+ * updates returns an accurate result, but concurrent updates that
+ * occur while the sum is being calculated might not be
+ * incorporated.
+ *
+ * @return the sum
+ */
+ public long sum() {
+ long sum = base;
+ Cell[] as = cells;
+ if (as != null) {
+ int n = as.length;
+ for (int i = 0; i < n; ++i) {
+ Cell a = as[i];
+ if (a != null)
+ sum += a.value;
+ }
+ }
+ return sum;
+ }
+
+ /**
+ * Resets variables maintaining the sum to zero. This method may
+ * be a useful alternative to creating a new adder, but is only
+ * effective if there are no concurrent updates. Because this
+ * method is intrinsically racy, it should only be used when it is
+ * known that no threads are concurrently updating.
+ */
+ public void reset() {
+ internalReset(0L);
+ }
+
+ /**
+ * Equivalent in effect to {@link #sum} followed by {@link
+ * #reset}. This method may apply for example during quiescent
+ * points between multithreaded computations. If there are
+ * updates concurrent with this method, the returned value is
+ * <em>not</em> guaranteed to be the final value occurring before
+ * the reset.
+ *
+ * @return the sum
+ */
+ public long sumThenReset() {
+ long sum = base;
+ Cell[] as = cells;
+ base = 0L;
+ if (as != null) {
+ int n = as.length;
+ for (int i = 0; i < n; ++i) {
+ Cell a = as[i];
+ if (a != null) {
+ sum += a.value;
+ a.value = 0L;
+ }
+ }
+ }
+ return sum;
+ }
+
+ /**
+ * Returns the String representation of the {@link #sum}.
+ * @return the String representation of the {@link #sum}
+ */
+ public String toString() {
+ return Long.toString(sum());
+ }
+
+ /**
+ * Equivalent to {@link #sum}.
+ *
+ * @return the sum
+ */
+ public long longValue() {
+ return sum();
+ }
+
+ /**
+ * Returns the {@link #sum} as an {@code int} after a narrowing
+ * primitive conversion.
+ */
+ public int intValue() {
+ return (int)sum();
+ }
+
+ /**
+ * Returns the {@link #sum} as a {@code float}
+ * after a widening primitive conversion.
+ */
+ public float floatValue() {
+ return (float)sum();
+ }
+
+ /**
+ * Returns the {@link #sum} as a {@code double} after a widening
+ * primitive conversion.
+ */
+ public double doubleValue() {
+ return (double)sum();
+ }
+
+ private void writeObject(ObjectOutputStream s) throws IOException {
+ s.defaultWriteObject();
+ s.writeLong(sum());
+ }
+
+ private void readObject(ObjectInputStream s)
+ throws IOException, ClassNotFoundException {
+ s.defaultReadObject();
+ busy = 0;
+ cells = null;
+ base = s.readLong();
+ }
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Striped64.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Striped64.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Striped64.java
new file mode 100644
index 0000000..36f2fce
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Striped64.java
@@ -0,0 +1,356 @@
+/**
+ *
+ * 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.
+ */
+
+/*
+ * Written by Doug Lea with assistance from members of JCP JSR-166
+ * Expert Group and released to the public domain, as explained at
+ * http://creativecommons.org/publicdomain/zero/1.0/
+ */
+
+package org.apache.hadoop.hbase.util;
+
+import java.util.Random;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * A package-local class holding common representation and mechanics
+ * for classes supporting dynamic striping on 64bit values. The class
+ * extends Number so that concrete subclasses must publicly do so.
+ */
+@InterfaceAudience.Private
+abstract class Striped64 extends Number {
+ /*
+ * This class maintains a lazily-initialized table of atomically
+ * updated variables, plus an extra "base" field. The table size
+ * is a power of two. Indexing uses masked per-thread hash codes.
+ * Nearly all declarations in this class are package-private,
+ * accessed directly by subclasses.
+ *
+ * Table entries are of class Cell; a variant of AtomicLong padded
+ * to reduce cache contention on most processors. Padding is
+ * overkill for most Atomics because they are usually irregularly
+ * scattered in memory and thus don't interfere much with each
+ * other. But Atomic objects residing in arrays will tend to be
+ * placed adjacent to each other, and so will most often share
+ * cache lines (with a huge negative performance impact) without
+ * this precaution.
+ *
+ * In part because Cells are relatively large, we avoid creating
+ * them until they are needed. When there is no contention, all
+ * updates are made to the base field. Upon first contention (a
+ * failed CAS on base update), the table is initialized to size 2.
+ * The table size is doubled upon further contention until
+ * reaching the nearest power of two greater than or equal to the
+ * number of CPUS. Table slots remain empty (null) until they are
+ * needed.
+ *
+ * A single spinlock ("busy") is used for initializing and
+ * resizing the table, as well as populating slots with new Cells.
+ * There is no need for a blocking lock; when the lock is not
+ * available, threads try other slots (or the base). During these
+ * retries, there is increased contention and reduced locality,
+ * which is still better than alternatives.
+ *
+ * Per-thread hash codes are initialized to random values.
+ * Contention and/or table collisions are indicated by failed
+ * CASes when performing an update operation (see method
+ * retryUpdate). Upon a collision, if the table size is less than
+ * the capacity, it is doubled in size unless some other thread
+ * holds the lock. If a hashed slot is empty, and lock is
+ * available, a new Cell is created. Otherwise, if the slot
+ * exists, a CAS is tried. Retries proceed by "double hashing",
+ * using a secondary hash (Marsaglia XorShift) to try to find a
+ * free slot.
+ *
+ * The table size is capped because, when there are more threads
+ * than CPUs, supposing that each thread were bound to a CPU,
+ * there would exist a perfect hash function mapping threads to
+ * slots that eliminates collisions. When we reach capacity, we
+ * search for this mapping by randomly varying the hash codes of
+ * colliding threads. Because search is random, and collisions
+ * only become known via CAS failures, convergence can be slow,
+ * and because threads are typically not bound to CPUS forever,
+ * may not occur at all. However, despite these limitations,
+ * observed contention rates are typically low in these cases.
+ *
+ * It is possible for a Cell to become unused when threads that
+ * once hashed to it terminate, as well as in the case where
+ * doubling the table causes no thread to hash to it under
+ * expanded mask. We do not try to detect or remove such cells,
+ * under the assumption that for long-running instances, observed
+ * contention levels will recur, so the cells will eventually be
+ * needed again; and for short-lived ones, it does not matter.
+ */
+
+ /**
+ * Padded variant of AtomicLong supporting only raw accesses plus CAS.
+ * The value field is placed between pads, hoping that the JVM doesn't
+ * reorder them.
+ *
+ * JVM intrinsics note: It would be possible to use a release-only
+ * form of CAS here, if it were provided.
+ */
+ static final class Cell {
+ volatile long p0, p1, p2, p3, p4, p5, p6;
+ volatile long value;
+ volatile long q0, q1, q2, q3, q4, q5, q6;
+ Cell(long x) { value = x; }
+
+ final boolean cas(long cmp, long val) {
+ return UNSAFE.compareAndSwapLong(this, valueOffset, cmp, val);
+ }
+
+ // Unsafe mechanics
+ private static final sun.misc.Unsafe UNSAFE;
+ private static final long valueOffset;
+ static {
+ try {
+ UNSAFE = getUnsafe();
+ Class<?> ak = Cell.class;
+ valueOffset = UNSAFE.objectFieldOffset
+ (ak.getDeclaredField("value"));
+ } catch (Exception e) {
+ throw new Error(e);
+ }
+ }
+
+ }
+
+ /**
+ * ThreadLocal holding a single-slot int array holding hash code.
+ * Unlike the JDK8 version of this class, we use a suboptimal
+ * int[] representation to avoid introducing a new type that can
+ * impede class-unloading when ThreadLocals are not removed.
+ */
+ static final ThreadLocal<int[]> threadHashCode = new ThreadLocal<int[]>();
+
+ /**
+ * Generator of new random hash codes
+ */
+ static final Random rng = new Random();
+
+ /** Number of CPUS, to place bound on table size */
+ static final int NCPU = Runtime.getRuntime().availableProcessors();
+
+ /**
+ * Table of cells. When non-null, size is a power of 2.
+ */
+ transient volatile Cell[] cells;
+
+ /**
+ * Base value, used mainly when there is no contention, but also as
+ * a fallback during table initialization races. Updated via CAS.
+ */
+ transient volatile long base;
+
+ /**
+ * Spinlock (locked via CAS) used when resizing and/or creating Cells.
+ */
+ transient volatile int busy;
+
+ /**
+ * Package-private default constructor
+ */
+ Striped64() {
+ }
+
+ /**
+ * CASes the base field.
+ */
+ final boolean casBase(long cmp, long val) {
+ return UNSAFE.compareAndSwapLong(this, baseOffset, cmp, val);
+ }
+
+ /**
+ * CASes the busy field from 0 to 1 to acquire lock.
+ */
+ final boolean casBusy() {
+ return UNSAFE.compareAndSwapInt(this, busyOffset, 0, 1);
+ }
+
+ /**
+ * Computes the function of current and new value. Subclasses
+ * should open-code this update function for most uses, but the
+ * virtualized form is needed within retryUpdate.
+ *
+ * @param currentValue the current value (of either base or a cell)
+ * @param newValue the argument from a user update call
+ * @return result of the update function
+ */
+ abstract long fn(long currentValue, long newValue);
+
+ /**
+ * Handles cases of updates involving initialization, resizing,
+ * creating new Cells, and/or contention. See above for
+ * explanation. This method suffers the usual non-modularity
+ * problems of optimistic retry code, relying on rechecked sets of
+ * reads.
+ *
+ * @param x the value
+ * @param hc the hash code holder
+ * @param wasUncontended false if CAS failed before call
+ */
+ final void retryUpdate(long x, int[] hc, boolean wasUncontended) {
+ int h;
+ if (hc == null) {
+ threadHashCode.set(hc = new int[1]); // Initialize randomly
+ int r = rng.nextInt(); // Avoid zero to allow xorShift rehash
+ h = hc[0] = (r == 0) ? 1 : r;
+ }
+ else
+ h = hc[0];
+ boolean collide = false; // True if last slot nonempty
+ for (;;) {
+ Cell[] as; Cell a; int n; long v;
+ if ((as = cells) != null && (n = as.length) > 0) {
+ if ((a = as[(n - 1) & h]) == null) {
+ if (busy == 0) { // Try to attach new Cell
+ Cell r = new Cell(x); // Optimistically create
+ if (busy == 0 && casBusy()) {
+ boolean created = false;
+ try { // Recheck under lock
+ Cell[] rs; int m, j;
+ if ((rs = cells) != null &&
+ (m = rs.length) > 0 &&
+ rs[j = (m - 1) & h] == null) {
+ rs[j] = r;
+ created = true;
+ }
+ } finally {
+ busy = 0;
+ }
+ if (created)
+ break;
+ continue; // Slot is now non-empty
+ }
+ }
+ collide = false;
+ }
+ else if (!wasUncontended) // CAS already known to fail
+ wasUncontended = true; // Continue after rehash
+ else if (a.cas(v = a.value, fn(v, x)))
+ break;
+ else if (n >= NCPU || cells != as)
+ collide = false; // At max size or stale
+ else if (!collide)
+ collide = true;
+ else if (busy == 0 && casBusy()) {
+ try {
+ if (cells == as) { // Expand table unless stale
+ Cell[] rs = new Cell[n << 1];
+ for (int i = 0; i < n; ++i)
+ rs[i] = as[i];
+ cells = rs;
+ }
+ } finally {
+ busy = 0;
+ }
+ collide = false;
+ continue; // Retry with expanded table
+ }
+ h ^= h << 13; // Rehash
+ h ^= h >>> 17;
+ h ^= h << 5;
+ hc[0] = h; // Record index for next time
+ }
+ else if (busy == 0 && cells == as && casBusy()) {
+ boolean init = false;
+ try { // Initialize table
+ if (cells == as) {
+ Cell[] rs = new Cell[2];
+ rs[h & 1] = new Cell(x);
+ cells = rs;
+ init = true;
+ }
+ } finally {
+ busy = 0;
+ }
+ if (init)
+ break;
+ }
+ else if (casBase(v = base, fn(v, x)))
+ break; // Fall back on using base
+ }
+ }
+
+
+ /**
+ * Sets base and all cells to the given value.
+ */
+ final void internalReset(long initialValue) {
+ Cell[] as = cells;
+ base = initialValue;
+ if (as != null) {
+ int n = as.length;
+ for (int i = 0; i < n; ++i) {
+ Cell a = as[i];
+ if (a != null)
+ a.value = initialValue;
+ }
+ }
+ }
+
+ // Unsafe mechanics
+ private static final sun.misc.Unsafe UNSAFE;
+ private static final long baseOffset;
+ private static final long busyOffset;
+ static {
+ try {
+ UNSAFE = getUnsafe();
+ Class<?> sk = Striped64.class;
+ baseOffset = UNSAFE.objectFieldOffset
+ (sk.getDeclaredField("base"));
+ busyOffset = UNSAFE.objectFieldOffset
+ (sk.getDeclaredField("busy"));
+ } catch (Exception e) {
+ throw new Error(e);
+ }
+ }
+
+ /**
+ * Returns a sun.misc.Unsafe. Suitable for use in a 3rd party package.
+ * Replace with a simple call to Unsafe.getUnsafe when integrating
+ * into a jdk.
+ *
+ * @return a sun.misc.Unsafe
+ */
+ private static sun.misc.Unsafe getUnsafe() {
+ try {
+ return sun.misc.Unsafe.getUnsafe();
+ } catch (SecurityException tryReflectionInstead) {}
+ try {
+ return java.security.AccessController.doPrivileged
+ (new java.security.PrivilegedExceptionAction<sun.misc.Unsafe>() {
+ public sun.misc.Unsafe run() throws Exception {
+ Class<sun.misc.Unsafe> k = sun.misc.Unsafe.class;
+ for (java.lang.reflect.Field f : k.getDeclaredFields()) {
+ f.setAccessible(true);
+ Object x = f.get(null);
+ if (k.isInstance(x))
+ return k.cast(x);
+ }
+ throw new NoSuchFieldError("the Unsafe");
+ }});
+ } catch (java.security.PrivilegedActionException e) {
+ throw new RuntimeException("Could not initialize intrinsics",
+ e.getCause());
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestFastLongHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestFastLongHistogram.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestFastLongHistogram.java
deleted file mode 100644
index 7304e2d..0000000
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestFastLongHistogram.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/**
- * 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.util;
-
-import java.util.Arrays;
-import java.util.Random;
-
-
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Testcases for FastLongHistogram.
- */
-@Category(SmallTests.class)
-public class TestFastLongHistogram {
-
- private static void doTestUniform(FastLongHistogram hist) {
- long[] VALUES = { 0, 10, 20, 30, 40, 50 };
- double[] qs = new double[VALUES.length];
- for (int i = 0; i < qs.length; i++) {
- qs[i] = (double) VALUES[i] / VALUES[VALUES.length - 1];
- }
-
- for (int i = 0; i < 10; i++) {
- for (long v : VALUES) {
- hist.add(v, 1);
- }
- long[] vals = hist.getQuantiles(qs);
- System.out.println(Arrays.toString(vals));
- for (int j = 0; j < qs.length; j++) {
- Assert.assertTrue(j + "-th element org: " + VALUES[j] + ", act: " + vals[j],
- Math.abs(vals[j] - VALUES[j]) <= 10);
- }
- hist.reset();
- }
- }
-
- @Test
- public void testUniform() {
- FastLongHistogram hist = new FastLongHistogram(100, 0, 50);
- doTestUniform(hist);
- }
-
- @Test
- public void testAdaptionOfChange() {
- // assumes the uniform distribution
- FastLongHistogram hist = new FastLongHistogram(100, 0, 100);
-
- Random rand = new Random();
-
- for (int n = 0; n < 10; n++) {
- for (int i = 0; i < 900; i++) {
- hist.add(rand.nextInt(100), 1);
- }
-
- // add 10% outliers, this breaks the assumption, hope bin10xMax works
- for (int i = 0; i < 100; i++) {
- hist.add(1000 + rand.nextInt(100), 1);
- }
-
- long[] vals = hist.getQuantiles(new double[] { 0.25, 0.75, 0.95 });
- System.out.println(Arrays.toString(vals));
- if (n == 0) {
- Assert.assertTrue("Out of possible value", vals[0] >= 0 && vals[0] <= 50);
- Assert.assertTrue("Out of possible value", vals[1] >= 50 && vals[1] <= 100);
- Assert.assertTrue("Out of possible value", vals[2] >= 900 && vals[2] <= 1100);
- }
-
- hist.reset();
- }
- }
-
-
- @Test
- public void testGetNumAtOrBelow() {
- long[] VALUES = { 1, 10, 20, 30, 40, 50 };
-
- FastLongHistogram h = new FastLongHistogram();
- for (long v : VALUES) {
- for (int i = 0; i < 100; i++) {
- h.add(v, 1);
- }
- }
-
- h.add(Integer.MAX_VALUE, 1);
-
- h.reset();
-
- for (long v : VALUES) {
- for (int i = 0; i < 100; i++) {
- h.add(v, 1);
- }
- }
- // Add something way out there to make sure it doesn't throw off the counts.
- h.add(Integer.MAX_VALUE, 1);
-
- assertEquals(100, h.getNumAtOrBelow(1));
- assertEquals(200, h.getNumAtOrBelow(11));
- assertEquals(601, h.getNumAtOrBelow(Long.MAX_VALUE));
- }
-
-
- @Test
- public void testSameValues() {
- FastLongHistogram hist = new FastLongHistogram(100);
-
- hist.add(50, 100);
-
- hist.reset();
- doTestUniform(hist);
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.java
new file mode 100644
index 0000000..b86856a
--- /dev/null
+++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.java
@@ -0,0 +1,148 @@
+/**
+ * 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.coprocessor.example;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.coprocessor.BaseMasterObserver;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.metrics.Gauge;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.Timer;
+
+/**
+ * An example coprocessor that collects some metrics to demonstrate the usage of exporting custom
+ * metrics from the coprocessor.
+ *
+ * <p>
+ * These metrics will be available through the regular Hadoop metrics2 sinks (ganglia, opentsdb,
+ * etc) as well as JMX output. You can view a snapshot of the metrics by going to the http web UI
+ * of the master page, something like http://mymasterhost:16010/jmx
+ * </p>
+ * @see ExampleRegionObserverWithMetrics
+ */
+public class ExampleMasterObserverWithMetrics extends BaseMasterObserver {
+
+ private static final Log LOG = LogFactory.getLog(ExampleMasterObserverWithMetrics.class);
+
+ /** This is the Timer metric object to keep track of the current count across invocations */
+ private Timer createTableTimer;
+ private long createTableStartTime = Long.MIN_VALUE;
+
+ /** This is a Counter object to keep track of disableTable operations */
+ private Counter disableTableCounter;
+
+ /** Returns the total memory of the process. We will use this to define a gauge metric */
+ private long getTotalMemory() {
+ return Runtime.getRuntime().totalMemory();
+ }
+
+ /** Returns the max memory of the process. We will use this to define a gauge metric */
+ private long getMaxMemory() {
+ return Runtime.getRuntime().maxMemory();
+ }
+
+ @Override
+ public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
+ HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
+ super.preCreateTable(ctx, desc, regions);
+ // we rely on the fact that there is only 1 instance of our MasterObserver. We keep track of
+ // when the operation starts before the operation is executing.
+ this.createTableStartTime = System.currentTimeMillis();
+ }
+
+ @Override
+ public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
+ HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
+ super.postCreateTable(ctx, desc, regions);
+ if (this.createTableStartTime > 0) {
+ long time = System.currentTimeMillis() - this.createTableStartTime;
+ LOG.info("Create table took: " + time);
+
+ // Update the timer metric for the create table operation duration.
+ createTableTimer.updateMillis(time);
+ }
+ }
+
+ @Override
+ public void preDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException {
+ super.preDisableTable(ctx, tableName);
+
+ // Increment the Counter for disable table operations
+ this.disableTableCounter.increment();
+ }
+
+ @Override
+ public void start(CoprocessorEnvironment env) throws IOException {
+ super.start(env);
+
+ // start for the MasterObserver will be called only once in the lifetime of the
+ // server. We will construct and register all metrics that we will track across method
+ // invocations.
+
+ if (env instanceof MasterCoprocessorEnvironment) {
+ // Obtain the MetricRegistry for the Master. Metrics from this registry will be reported
+ // at the master level per-server.
+ MetricRegistry registry =
+ ((MasterCoprocessorEnvironment) env).getMetricRegistryForMaster();
+
+ if (createTableTimer == null) {
+ // Create a new Counter, or get the already registered counter.
+ // It is much better to only call this once and save the Counter as a class field instead
+ // of creating the counter every time a coprocessor method is invoked. This will negate
+ // any performance bottleneck coming from map lookups tracking metrics in the registry.
+ createTableTimer = registry.timer("CreateTable");
+
+ // on stop(), we can remove these registered metrics via calling registry.remove(). But
+ // it is not needed for coprocessors at the master level. If coprocessor is stopped,
+ // the server is stopping anyway, so there will not be any resource leaks.
+ }
+
+ if (disableTableCounter == null) {
+ disableTableCounter = registry.counter("DisableTable");
+ }
+
+ // Register a custom gauge. The Gauge object will be registered in the metrics registry and
+ // periodically the getValue() is invoked to obtain the snapshot.
+ registry.register("totalMemory", new Gauge<Long>() {
+ @Override
+ public Long getValue() {
+ return getTotalMemory();
+ }
+ });
+
+ // Register a custom gauge (Supplier converted into Gauge)
+ registry.register("maxMemory", new Gauge<Long>() {
+ @Override
+ public Long getValue() {
+ return getMaxMemory();
+ }
+
+ });
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleRegionObserverWithMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleRegionObserverWithMetrics.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleRegionObserverWithMetrics.java
new file mode 100644
index 0000000..7606b05
--- /dev/null
+++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleRegionObserverWithMetrics.java
@@ -0,0 +1,121 @@
+/**
+ *
+ * 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.coprocessor.example;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.Timer;
+
+/**
+ * An example coprocessor that collects some metrics to demonstrate the usage of exporting custom
+ * metrics from the coprocessor.
+ * <p>
+ * These metrics will be available through the regular Hadoop metrics2 sinks (ganglia, opentsdb,
+ * etc) as well as JMX output. You can view a snapshot of the metrics by going to the http web UI
+ * of the regionserver page, something like http://myregionserverhost:16030/jmx
+ * </p>
+ *
+ * @see ExampleMasterObserverWithMetrics
+ */
+public class ExampleRegionObserverWithMetrics extends BaseRegionObserver {
+
+ private Counter preGetCounter;
+ private Timer costlyOperationTimer;
+
+ @Override
+ public void preGetOp(ObserverContext<RegionCoprocessorEnvironment> e, Get get, List<Cell> results)
+ throws IOException {
+ super.preGetOp(e, get, results);
+
+ // Increment the Counter whenever the coprocessor is called
+ preGetCounter.increment();
+ }
+
+ @Override
+ public void postGetOp(ObserverContext<RegionCoprocessorEnvironment> e, Get get,
+ List<Cell> results) throws IOException {
+ super.postGetOp(e, get, results);
+
+ // do a costly (high latency) operation which we want to measure how long it takes by
+ // using a Timer (which is a Meter and a Histogram).
+ long start = System.nanoTime();
+ try {
+ performCostlyOperation();
+ } finally {
+ costlyOperationTimer.updateNanos(System.nanoTime() - start);
+ }
+ }
+
+ private void performCostlyOperation() {
+ try {
+ // simulate the operation by sleeping.
+ Thread.sleep(ThreadLocalRandom.current().nextLong(100));
+ } catch (InterruptedException ignore) {}
+ }
+
+ @Override
+ public void start(CoprocessorEnvironment env) throws IOException {
+ super.start(env);
+
+ // start for the RegionServerObserver will be called only once in the lifetime of the
+ // server. We will construct and register all metrics that we will track across method
+ // invocations.
+
+ if (env instanceof RegionCoprocessorEnvironment) {
+ // Obtain the MetricRegistry for the RegionServer. Metrics from this registry will be reported
+ // at the region server level per-regionserver.
+ MetricRegistry registry =
+ ((RegionCoprocessorEnvironment) env).getMetricRegistryForRegionServer();
+
+ if (preGetCounter == null) {
+ // Create a new Counter, or get the already registered counter.
+ // It is much better to only call this once and save the Counter as a class field instead
+ // of creating the counter every time a coprocessor method is invoked. This will negate
+ // any performance bottleneck coming from map lookups tracking metrics in the registry.
+ // Returned counter instance is shared by all coprocessors of the same class in the same
+ // region server.
+ preGetCounter = registry.counter("preGetRequests");
+ }
+
+ if (costlyOperationTimer == null) {
+ // Create a Timer to track execution times for the costly operation.
+ costlyOperationTimer = registry.timer("costlyOperation");
+ }
+ }
+ }
+
+ @Override
+ public void stop(CoprocessorEnvironment e) throws IOException {
+ // we should NOT remove / deregister the metrics in stop(). The whole registry will be
+ // removed when the last region of the table is closed.
+ super.stop(e);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-hadoop-compat/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/pom.xml b/hbase-hadoop-compat/pom.xml
index 102c8b6..bdb5d0a 100644
--- a/hbase-hadoop-compat/pom.xml
+++ b/hbase-hadoop-compat/pom.xml
@@ -122,6 +122,10 @@
<groupId>org.apache.commons</groupId>
<artifactId>commons-math</artifactId>
</dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-metrics-api</artifactId>
+ </dependency>
</dependencies>
<profiles>
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSource.java
index f79aa9f..0350bff 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSource.java
@@ -104,4 +104,6 @@ public interface BaseSource {
*/
String getMetricsName();
+ MetricRegistryInfo getMetricRegistryInfo();
+
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-hadoop2-compat/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/pom.xml b/hbase-hadoop2-compat/pom.xml
index 6462c61..54a31b6 100644
--- a/hbase-hadoop2-compat/pom.xml
+++ b/hbase-hadoop2-compat/pom.xml
@@ -166,6 +166,10 @@ limitations under the License.
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-metrics</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
<artifactId>hbase-hadoop-compat</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSourceImpl.java
index f843ec2..a112e9d 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSourceImpl.java
@@ -19,6 +19,9 @@
package org.apache.hadoop.hbase.metrics;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.metrics.impl.GlobalMetricRegistriesAdapter;
+import org.apache.hadoop.hbase.metrics.impl.HBaseMetrics2HadoopMetricsAdapter;
+import org.apache.hadoop.hbase.regionserver.MetricsRegionServerSourceImpl;
import org.apache.hadoop.metrics2.MetricsCollector;
import org.apache.hadoop.metrics2.MetricsSource;
import org.apache.hadoop.metrics2.impl.JmxCacheBuster;
@@ -47,15 +50,53 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
inited = true;
DefaultMetricsSystem.initialize(HBASE_METRICS_SYSTEM_NAME);
JvmMetrics.initSingleton(name, "");
+ // initialize hbase-metrics module based metric system as well. GlobalMetricRegistriesSource
+ // initialization depends on the metric system being already initialized, that is why we are
+ // doing it here. Once BaseSourceSourceImpl is removed, we should do the initialization of
+ // these elsewhere.
+ GlobalMetricRegistriesAdapter.init();
}
}
+ /**
+ * @deprecated Use hbase-metrics/hbase-metrics-api module interfaces for new metrics.
+ * Defining BaseSources for new metric groups (WAL, RPC, etc) is not needed anymore, however,
+ * for existing BaseSource implemetnations, please use the field named "registry" which is a
+ * MetricRegistry instance together with the HBaseMetrics2HadoopMetricsAdapter.
+ */
+ @Deprecated
protected final DynamicMetricsRegistry metricsRegistry;
protected final String metricsName;
protected final String metricsDescription;
protected final String metricsContext;
protected final String metricsJmxContext;
+ /**
+ * Note that there are at least 4 MetricRegistry definitions in the source code. The first one is
+ * Hadoop Metrics2 MetricRegistry, second one is DynamicMetricsRegistry which is HBase's fork
+ * of the Hadoop metrics2 class. The third one is the dropwizard metrics implementation of
+ * MetricRegistry, and finally a new API abstraction in HBase that is the
+ * o.a.h.h.metrics.MetricRegistry class. This last one is the new way to use metrics within the
+ * HBase code. However, the others are in play because of existing metrics2 based code still
+ * needs to coexists until we get rid of all of our BaseSource and convert them to the new
+ * framework. Until that happens, new metrics can use the new API, but will be collected
+ * through the HBaseMetrics2HadoopMetricsAdapter class.
+ *
+ * BaseSourceImpl has two MetricRegistries. metricRegistry is for hadoop Metrics2 based
+ * metrics, while the registry is for hbase-metrics based metrics.
+ */
+ protected final MetricRegistry registry;
+
+ /**
+ * The adapter from hbase-metrics module to metrics2. This adepter is the connection between the
+ * Metrics in the MetricRegistry and the Hadoop Metrics2 system. Using this adapter, existing
+ * BaseSource implementations can define new metrics using the hbase-metrics/hbase-metrics-api
+ * module interfaces and still be able to make use of metrics2 sinks (including JMX). Existing
+ * BaseSources should call metricsAdapter.snapshotAllMetrics() in getMetrics() method. See
+ * {@link MetricsRegionServerSourceImpl}.
+ */
+ protected final HBaseMetrics2HadoopMetricsAdapter metricsAdapter;
+
public BaseSourceImpl(
String metricsName,
String metricsDescription,
@@ -72,6 +113,11 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
//Register this instance.
DefaultMetricsSystem.instance().register(metricsJmxContext, metricsDescription, this);
+
+ // hbase-metrics module based metrics are registered in the hbase MetricsRegistry.
+ registry = MetricRegistries.global().create(this.getMetricRegistryInfo());
+ metricsAdapter = new HBaseMetrics2HadoopMetricsAdapter();
+
init();
}
@@ -166,4 +212,10 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
return metricsName;
}
+ @Override
+ public MetricRegistryInfo getMetricRegistryInfo() {
+ return new MetricRegistryInfo(getMetricsName(), getMetricsDescription(),
+ getMetricsContext(), getMetricsJmxContext(), true);
+ }
+
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/GlobalMetricRegistriesAdapter.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/GlobalMetricRegistriesAdapter.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/GlobalMetricRegistriesAdapter.java
new file mode 100644
index 0000000..40a358f
--- /dev/null
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/GlobalMetricRegistriesAdapter.java
@@ -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.metrics.impl;
+
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.lang.reflect.FieldUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.metrics.MetricRegistries;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsExecutor;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.impl.JmxCacheBuster;
+import org.apache.hadoop.metrics2.impl.MetricsSystemImpl;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper;
+import org.apache.hadoop.metrics2.lib.MetricsExecutorImpl;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+
+/**
+ * This class acts as an adapter to export the MetricRegistry's in the global registry. Each
+ * MetricRegistry will be registered or unregistered from the metric2 system. The collection will
+ * be performed via the MetricsSourceAdapter and the MetricRegistry will collected like a
+ * BaseSource instance for a group of metrics (like WAL, RPC, etc) with the MetricRegistryInfo's
+ * JMX context.
+ *
+ * <p>Developer note:
+ * Unlike the current metrics2 based approach, the new metrics approach
+ * (hbase-metrics-api and hbase-metrics modules) work by having different MetricRegistries that are
+ * initialized and used from the code that lives in their respective modules (hbase-server, etc).
+ * There is no need to define BaseSource classes and do a lot of indirection. The MetricRegistry'es
+ * will be in the global MetricRegistriesImpl, and this class will iterate over
+ * MetricRegistries.global() and register adapters to the metrics2 subsystem. These adapters then
+ * report the actual values by delegating to
+ * {@link HBaseMetrics2HadoopMetricsAdapter#snapshotAllMetrics(MetricRegistry, MetricsCollector)}.
+ *
+ * We do not initialize the Hadoop Metrics2 system assuming that other BaseSources already do so
+ * (see BaseSourceImpl). Once the last BaseSource is moved to the new system, the metric2
+ * initialization should be moved here.
+ * </p>
+ */
+public class GlobalMetricRegistriesAdapter {
+
+ private static final Log LOG = LogFactory.getLog(GlobalMetricRegistriesAdapter.class);
+
+ private class MetricsSourceAdapter implements MetricsSource {
+ private final MetricRegistry registry;
+ MetricsSourceAdapter(MetricRegistry registry) {
+ this.registry = registry;
+ }
+
+ @Override
+ public void getMetrics(MetricsCollector collector, boolean all) {
+ metricsAdapter.snapshotAllMetrics(registry, collector);
+ }
+ }
+
+ private final MetricsExecutor executor;
+ private final AtomicBoolean stopped;
+ private final DefaultMetricsSystemHelper helper;
+ private final HBaseMetrics2HadoopMetricsAdapter metricsAdapter;
+ private final HashMap<MetricRegistryInfo, MetricsSourceAdapter> registeredSources;
+
+ private GlobalMetricRegistriesAdapter() {
+ this.executor = new MetricsExecutorImpl();
+ this.stopped = new AtomicBoolean(false);
+ this.metricsAdapter = new HBaseMetrics2HadoopMetricsAdapter();
+ this.registeredSources = new HashMap<>();
+ this.helper = new DefaultMetricsSystemHelper();
+ executor.getExecutor().scheduleAtFixedRate(new Runnable(){
+ @Override
+ public void run() {
+ doRun();
+
+ }}, 10, 10, TimeUnit.SECONDS);
+ }
+
+ /**
+ * Make sure that this global MetricSource for hbase-metrics module based metrics are initialized.
+ * This should be called only once.
+ */
+ public static GlobalMetricRegistriesAdapter init() {
+ return new GlobalMetricRegistriesAdapter();
+ }
+
+ @VisibleForTesting
+ public void stop() {
+ stopped.set(true);
+ }
+
+ private void doRun() {
+ if (stopped.get()) {
+ executor.stop();
+ return;
+ }
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("doRun called: " + registeredSources);
+ }
+
+ Collection<MetricRegistry> registries = MetricRegistries.global().getMetricRegistries();
+ for (MetricRegistry registry : registries) {
+ MetricRegistryInfo info = registry.getMetricRegistryInfo();
+
+ if (info.isExistingSource()) {
+ // If there is an already existing BaseSource for this MetricRegistry, skip it here. These
+ // types of registries are there only due to existing BaseSource implementations in the
+ // source code (like MetricsRegionServer, etc). This is to make sure that we can transition
+ // iteratively to the new hbase-metrics system. These type of MetricRegistry metrics will be
+ // exported from the BaseSource.getMetrics() call directly because there is already a
+ // MetricRecordBuilder there (see MetricsRegionServerSourceImpl).
+ continue;
+ }
+
+ if (!registeredSources.containsKey(info)) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Registering adapter for the MetricRegistry: " + info.getMetricsJmxContext());
+ }
+ // register this as a MetricSource under different JMX Context'es.
+ MetricsSourceAdapter adapter = new MetricsSourceAdapter(registry);
+ LOG.info("Registering " + info.getMetricsJmxContext() + " " + info.getMetricsDescription());
+ DefaultMetricsSystem.instance().register(info.getMetricsJmxContext(),
+ info.getMetricsDescription(), adapter);
+ registeredSources.put(info, adapter);
+ // next collection will collect the newly registered MetricSource. Doing this here leads to
+ // ConcurrentModificationException.
+ }
+ }
+
+ boolean removed = false;
+ // Remove registered sources if it is removed from the global registry
+ for (Iterator<Entry<MetricRegistryInfo, MetricsSourceAdapter>> it =
+ registeredSources.entrySet().iterator(); it.hasNext();) {
+ Entry<MetricRegistryInfo, MetricsSourceAdapter> entry = it.next();
+ MetricRegistryInfo info = entry.getKey();
+ Optional<MetricRegistry> found = MetricRegistries.global().get(info);
+ if (!found.isPresent()) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Removing adapter for the MetricRegistry: " + info.getMetricsJmxContext());
+ }
+ synchronized(DefaultMetricsSystem.instance()) {
+ unregisterSource(info);
+ helper.removeSourceName(info.getMetricsJmxContext());
+ helper.removeObjectName(info.getMetricsJmxContext());
+ it.remove();
+ removed = true;
+ }
+ }
+ }
+ if (removed) {
+ JmxCacheBuster.clearJmxCache();
+ }
+ }
+
+ /**
+ * Use reflection to unregister the Hadoop metric source, since MetricsSystem#unregisterSource()
+ * is only available in Hadoop 2.6+ (HADOOP-10839)
+ */
+ @VisibleForTesting
+ protected void unregisterSource(MetricRegistryInfo info) {
+ // unregisterSource is only available in Hadoop 2.6+ (HADOOP-10839). Don't unregister for now
+ MetricsSystem metricsSystem = DefaultMetricsSystem.instance();
+ if (metricsSystem instanceof MetricsSystemImpl) {
+ try {
+ // it's actually a Map<String, MetricsSourceAdapter> , but MetricsSourceAdapter isn't
+ // accessible
+ @SuppressWarnings("unchecked")
+ Map<String, Object> sources =
+ (Map<String, Object>) FieldUtils.readField(metricsSystem, "sources", true);
+ String sourceName = info.getMetricsJmxContext();
+ if (sources.containsKey(sourceName)) {
+ Object sourceAdapter = sources.get(sourceName);
+ Method method = null;
+ try {
+ method = sourceAdapter.getClass().getDeclaredMethod("stop");
+ } catch (NoSuchMethodException e) {
+ LOG.info("Stop method not found on MetricsSourceAdapter");
+ } catch (SecurityException e) {
+ LOG.info("Don't have access to call stop method not found on MetricsSourceAdapter", e);
+ }
+ if (method != null) {
+ method.setAccessible(true);
+ try {
+ method.invoke(sourceAdapter);
+ } catch (IllegalArgumentException | InvocationTargetException e) {
+ LOG.warn("Couldn't invoke stop on metrics source adapter: " + sourceName);
+ e.printStackTrace();
+ }
+ }
+ sources.remove(sourceName);
+
+ }
+ @SuppressWarnings("unchecked")
+ Map<String, MetricsSource> allSources =
+ (Map<String, MetricsSource>) FieldUtils.readField(metricsSystem, "allSources", true);
+ if (allSources.containsKey(sourceName)) {
+ allSources.remove(sourceName);
+ }
+ } catch (IllegalAccessException e) {
+ LOG.warn("Error unregistering metric source " + info.getMetricsJmxContext());
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/HBaseMetrics2HadoopMetricsAdapter.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/HBaseMetrics2HadoopMetricsAdapter.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/HBaseMetrics2HadoopMetricsAdapter.java
new file mode 100644
index 0000000..ec4a1a7
--- /dev/null
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/HBaseMetrics2HadoopMetricsAdapter.java
@@ -0,0 +1,169 @@
+/**
+ * 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.
+ */
+/*
+ * Copyright 2016 Josh Elser
+ *
+ * Licensed 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.impl;
+
+import java.util.Map;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.metrics.Gauge;
+import org.apache.hadoop.hbase.metrics.Histogram;
+import org.apache.hadoop.hbase.metrics.Meter;
+import org.apache.hadoop.hbase.metrics.Metric;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+import org.apache.hadoop.hbase.metrics.Timer;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsInfo;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.lib.Interns;
+import org.apache.hadoop.metrics2.lib.MutableHistogram;
+
+/**
+ * This is the adapter from "HBase Metrics Framework", implemented in hbase-metrics-api and
+ * hbase-metrics modules to the Hadoop Metrics2 framework. This adapter is not a metric source,
+ * but a helper to be able to collect all of the Metric's in the MetricRegistry using the
+ * MetricsCollector and MetricsRecordBuilder.
+ *
+ * Some of the code is forked from https://github.com/joshelser/dropwizard-hadoop-metrics2.
+ */
+public class HBaseMetrics2HadoopMetricsAdapter {
+ private static final Log LOG
+ = LogFactory.getLog(HBaseMetrics2HadoopMetricsAdapter.class);
+ private static final String EMPTY_STRING = "";
+
+ public HBaseMetrics2HadoopMetricsAdapter() {
+ }
+
+ /**
+ * Iterates over the MetricRegistry and adds them to the {@code collector}.
+ *
+ * @param collector A metrics collector
+ */
+ public void snapshotAllMetrics(MetricRegistry metricRegistry,
+ MetricsCollector collector) {
+ MetricRegistryInfo info = metricRegistry.getMetricRegistryInfo();
+ MetricsRecordBuilder builder = collector.addRecord(Interns.info(info.getMetricsName(),
+ info.getMetricsDescription()));
+ builder.setContext(info.getMetricsContext());
+
+ snapshotAllMetrics(metricRegistry, builder);
+ }
+
+ /**
+ * Iterates over the MetricRegistry and adds them to the {@code builder}.
+ *
+ * @param builder A record builder
+ */
+ public void snapshotAllMetrics(MetricRegistry metricRegistry, MetricsRecordBuilder builder) {
+ Map<String, Metric> metrics = metricRegistry.getMetrics();
+
+ for (Map.Entry<String, Metric> e: metrics.entrySet()) {
+ // Always capitalize the name
+ String name = StringUtils.capitalize(e.getKey());
+ Metric metric = e.getValue();
+
+ if (metric instanceof Gauge) {
+ addGauge(name, (Gauge<?>) metric, builder);
+ } else if (metric instanceof Counter) {
+ addCounter(name, (Counter)metric, builder);
+ } else if (metric instanceof Histogram) {
+ addHistogram(name, (Histogram)metric, builder);
+ } else if (metric instanceof Meter) {
+ addMeter(name, (Meter)metric, builder);
+ } else if (metric instanceof Timer) {
+ addTimer(name, (Timer)metric, builder);
+ } else {
+ LOG.info("Ignoring unknown Metric class " + metric.getClass().getName());
+ }
+ }
+ }
+
+ private void addGauge(String name, Gauge<?> gauge, MetricsRecordBuilder builder) {
+ final MetricsInfo info = Interns.info(name, EMPTY_STRING);
+ final Object o = gauge.getValue();
+
+ // Figure out which gauge types metrics2 supports and call the right method
+ if (o instanceof Integer) {
+ builder.addGauge(info, (int) o);
+ } else if (o instanceof Long) {
+ builder.addGauge(info, (long) o);
+ } else if (o instanceof Float) {
+ builder.addGauge(info, (float) o);
+ } else if (o instanceof Double) {
+ builder.addGauge(info, (double) o);
+ } else {
+ LOG.warn("Ignoring Gauge (" + name + ") with unhandled type: " + o.getClass());
+ }
+ }
+
+ private void addCounter(String name, Counter counter, MetricsRecordBuilder builder) {
+ MetricsInfo info = Interns.info(name, EMPTY_STRING);
+ builder.addCounter(info, counter.getCount());
+ }
+
+ /**
+ * Add Histogram value-distribution data to a Hadoop-Metrics2 record building.
+ *
+ * @param builder A Hadoop-Metrics2 record builder.
+ * @param name A base name for this record.
+ * @param desc A description for this record.
+ * @param snapshot The distribution of measured values.
+ */
+ private void addHistogram(String name, Histogram histogram, MetricsRecordBuilder builder) {
+ MutableHistogram.snapshot(name, EMPTY_STRING, histogram, builder, true);
+ }
+
+ /**
+ * Add Dropwizard-Metrics rate information to a Hadoop-Metrics2 record builder, converting the
+ * rates to the appropriate unit.
+ *
+ * @param builder A Hadoop-Metrics2 record builder.
+ * @param name A base name for this record.
+ */
+ private void addMeter(String name, Meter meter, MetricsRecordBuilder builder) {
+ builder.addGauge(Interns.info(name + "_count", EMPTY_STRING), meter.getCount());
+ builder.addGauge(Interns.info(name + "_mean_rate", EMPTY_STRING), meter.getMeanRate());
+ builder.addGauge(Interns.info(name + "_1min_rate", EMPTY_STRING), meter.getOneMinuteRate());
+ builder.addGauge(Interns.info(name + "_5min_rate", EMPTY_STRING), meter.getFiveMinuteRate());
+ builder.addGauge(Interns.info(name + "_15min_rate", EMPTY_STRING),
+ meter.getFifteenMinuteRate());
+ }
+
+ private void addTimer(String name, Timer timer, MetricsRecordBuilder builder) {
+ addMeter(name, timer.getMeter(), builder);
+ addHistogram(name, timer.getHistogram(), builder);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
index f1bb0da..d3329db 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
@@ -473,6 +473,12 @@ public class MetricsRegionServerSourceImpl
}
metricsRegistry.snapshot(mrb, all);
+
+ // source is registered in supers constructor, sometimes called before the whole initialization.
+ if (metricsAdapter != null) {
+ // snapshot MetricRegistry as well
+ metricsAdapter.snapshotAllMetrics(registry, mrb);
+ }
}
@Override
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
index 7a34e45..0e5c07f 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.hbase.replication.regionserver;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
import org.apache.hadoop.metrics2.lib.MutableHistogram;
@@ -243,4 +244,10 @@ public class MetricsReplicationGlobalSourceSource implements MetricsReplicationS
public String getMetricsName() {
return rms.getMetricsName();
}
+
+ @Override
+ public MetricRegistryInfo getMetricRegistryInfo() {
+ return new MetricRegistryInfo(getMetricsName(), getMetricsDescription(),
+ getMetricsContext(), getMetricsJmxContext(), true);
+ }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
index 0b6a1e1..9838e42 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
@@ -17,6 +17,7 @@
*/
package org.apache.hadoop.hbase.replication.regionserver;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
import org.apache.hadoop.metrics2.lib.MutableHistogram;
@@ -306,4 +307,10 @@ public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSou
public String getMetricsName() {
return rms.getMetricsName();
}
+
+ @Override
+ public MetricRegistryInfo getMetricRegistryInfo() {
+ return new MetricRegistryInfo(getMetricsName(), getMetricsDescription(),
+ getMetricsContext(), getMetricsJmxContext(), true);
+ }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DefaultMetricsSystemHelper.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DefaultMetricsSystemHelper.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DefaultMetricsSystemHelper.java
index 832e220..eb465c3 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DefaultMetricsSystemHelper.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DefaultMetricsSystemHelper.java
@@ -17,26 +17,44 @@
*/
package org.apache.hadoop.metrics2.lib;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.util.HashMap;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import java.lang.reflect.Method;
-
public class DefaultMetricsSystemHelper {
private static final Log LOG = LogFactory.getLog(DefaultMetricsSystemHelper.class);
private final Method removeObjectMethod;
+ private final Field sourceNamesField;
+ private final Field mapField;
public DefaultMetricsSystemHelper() {
+ Class<? extends DefaultMetricsSystem> clazz = DefaultMetricsSystem.INSTANCE.getClass();
Method m;
try {
- Class<? extends DefaultMetricsSystem> clazz = DefaultMetricsSystem.INSTANCE.getClass();
m = clazz.getDeclaredMethod("removeObjectName", String.class);
m.setAccessible(true);
} catch (NoSuchMethodException e) {
m = null;
}
removeObjectMethod = m;
+
+ Field f1, f2;
+ try {
+ f1 = clazz.getDeclaredField("sourceNames");
+ f1.setAccessible(true);
+ f2 = UniqueNames.class.getDeclaredField("map");
+ f2.setAccessible(true);
+ } catch (NoSuchFieldException e) {
+ LOG.trace(e);
+ f1 = null;
+ f2 = null;
+ }
+ sourceNamesField = f1;
+ mapField = f2;
}
public boolean removeObjectName(final String name) {
@@ -52,4 +70,30 @@ public class DefaultMetricsSystemHelper {
}
return false;
}
+
+ /**
+ * Unfortunately Hadoop tries to be too-clever and permanently keeps track of all names registered
+ * so far as a Source, thus preventing further re-registration of the source with the same name.
+ * In case of dynamic metrics tied to region-lifecycles, this becomes a problem because we would
+ * like to be able to re-register and remove with the same name. Otherwise, it is resource leak.
+ * This ugly code manually removes the name from the UniqueNames map.
+ * TODO: May not be needed for Hadoop versions after YARN-5190.
+ */
+ public void removeSourceName(String name) {
+ if (sourceNamesField == null || mapField == null) {
+ return;
+ }
+ try {
+ Object sourceNames = sourceNamesField.get(DefaultMetricsSystem.INSTANCE);
+ HashMap map = (HashMap) mapField.get(sourceNames);
+ synchronized (sourceNames) {
+ map.remove(name);
+ }
+ } catch (Exception ex) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Received exception while trying to access Hadoop Metrics classes via reflection.",
+ ex);
+ }
+ }
+ }
}
[3/4] hbase git commit: HBASE-18060 Backport to branch-1 HBASE-9774
HBase native metrics and metric collection for coprocessors
Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java
index 1b8dab8..6a2f203 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java
@@ -20,9 +20,10 @@ package org.apache.hadoop.metrics2.lib;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.metrics.Histogram;
import org.apache.hadoop.hbase.metrics.Interns;
-import org.apache.hadoop.hbase.util.Counter;
-import org.apache.hadoop.hbase.util.FastLongHistogram;
+import org.apache.hadoop.hbase.metrics.Snapshot;
+import org.apache.hadoop.hbase.metrics.impl.HistogramImpl;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.MetricsInfo;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
@@ -32,30 +33,10 @@ import org.apache.hadoop.metrics2.MetricsRecordBuilder;
*/
@InterfaceAudience.Private
public class MutableHistogram extends MutableMetric implements MetricHistogram {
- // Double buffer the two FastLongHistograms.
- // As they are reset they learn how the buckets should be spaced
- // So keep two around and use them
- protected final FastLongHistogram histogram;
-
+ protected HistogramImpl histogram;
protected final String name;
protected final String desc;
- protected final Counter counter = new Counter(0);
-
- private boolean metricsInfoStringInited = false;
- private String NUM_OPS_METRIC;
- private String MIN_METRIC;
- private String MAX_METRIC;
- private String MEAN_METRIC;
- private String MEDIAN_METRIC;
- private String TWENTY_FIFTH_PERCENTILE_METRIC;
- private String SEVENTY_FIFTH_PERCENTILE_METRIC;
- private String NINETIETH_PERCENTILE_METRIC;
- private String NINETY_FIFTH_PERCENTILE_METRIC;
- private String NINETY_EIGHTH_PERCENTILE_METRIC;
- private String NINETY_NINETH_PERCENTILE_METRIC;
- private String NINETY_NINE_POINT_NINETH_PERCENTILE_METRIC;
-
public MutableHistogram(MetricsInfo info) {
this(info.name(), info.description());
}
@@ -67,13 +48,11 @@ public class MutableHistogram extends MutableMetric implements MetricHistogram {
protected MutableHistogram(String name, String description, long maxExpected) {
this.name = StringUtils.capitalize(name);
this.desc = StringUtils.uncapitalize(description);
-
- this.histogram = new FastLongHistogram(FastLongHistogram.DEFAULT_NBINS, 1, maxExpected);
+ this.histogram = new HistogramImpl();
}
public void add(final long val) {
- counter.increment();
- histogram.add(val, 1);
+ histogram.update(val);
}
public long getMax() {
@@ -82,56 +61,42 @@ public class MutableHistogram extends MutableMetric implements MetricHistogram {
@Override
public synchronized void snapshot(MetricsRecordBuilder metricsRecordBuilder, boolean all) {
- // Get a reference to the old histogram.
- FastLongHistogram histo = histogram.reset();
- if (histo != null) {
- updateSnapshotMetrics(metricsRecordBuilder, histo);
- }
+ snapshot(name, desc, histogram, metricsRecordBuilder, all);
}
- protected void updateSnapshotMetrics(MetricsRecordBuilder metricsRecordBuilder,
- FastLongHistogram histo) {
- if (!metricsInfoStringInited) {
- NUM_OPS_METRIC = name + NUM_OPS_METRIC_NAME;
- MIN_METRIC = name + MIN_METRIC_NAME;
- MAX_METRIC = name + MAX_METRIC_NAME;
- MEAN_METRIC = name + MEAN_METRIC_NAME;
- MEDIAN_METRIC = name + MEDIAN_METRIC_NAME;
- TWENTY_FIFTH_PERCENTILE_METRIC = name + TWENTY_FIFTH_PERCENTILE_METRIC_NAME;
- SEVENTY_FIFTH_PERCENTILE_METRIC = name + SEVENTY_FIFTH_PERCENTILE_METRIC_NAME;
- NINETIETH_PERCENTILE_METRIC = name + NINETIETH_PERCENTILE_METRIC_NAME;
- NINETY_FIFTH_PERCENTILE_METRIC = name + NINETY_FIFTH_PERCENTILE_METRIC_NAME;
- NINETY_EIGHTH_PERCENTILE_METRIC = name + NINETY_EIGHTH_PERCENTILE_METRIC_NAME;
- NINETY_NINETH_PERCENTILE_METRIC = name + NINETY_NINETH_PERCENTILE_METRIC_NAME;
- NINETY_NINE_POINT_NINETH_PERCENTILE_METRIC = name +
- NINETY_NINE_POINT_NINETH_PERCENTILE_METRIC_NAME;
-
- metricsInfoStringInited = true;
+ public static void snapshot(String name, String desc, Histogram histogram,
+ MetricsRecordBuilder metricsRecordBuilder, boolean all) {
+ // Get a reference to the old histogram.
+ Snapshot snapshot = histogram.snapshot();
+ if (snapshot != null) {
+ updateSnapshotMetrics(name, desc, histogram, snapshot, metricsRecordBuilder);
}
+ }
- metricsRecordBuilder.addCounter(Interns.info(NUM_OPS_METRIC, desc), counter.get());
- metricsRecordBuilder.addGauge(Interns.info(MIN_METRIC, desc), histo.getMin());
- metricsRecordBuilder.addGauge(Interns.info(MAX_METRIC, desc), histo.getMax());
- metricsRecordBuilder.addGauge(Interns.info(MEAN_METRIC, desc), histo.getMean());
-
- long[] percentiles = histo.getQuantiles();
-
- metricsRecordBuilder.addGauge(Interns.info(TWENTY_FIFTH_PERCENTILE_METRIC, desc),
- percentiles[0]);
- metricsRecordBuilder.addGauge(Interns.info(MEDIAN_METRIC, desc),
- percentiles[1]);
- metricsRecordBuilder.addGauge(Interns.info(SEVENTY_FIFTH_PERCENTILE_METRIC, desc),
- percentiles[2]);
- metricsRecordBuilder.addGauge(Interns.info(NINETIETH_PERCENTILE_METRIC, desc),
- percentiles[3]);
- metricsRecordBuilder.addGauge(Interns.info(NINETY_FIFTH_PERCENTILE_METRIC, desc),
- percentiles[4]);
- metricsRecordBuilder.addGauge(Interns.info(NINETY_EIGHTH_PERCENTILE_METRIC, desc),
- percentiles[5]);
- metricsRecordBuilder.addGauge(Interns.info(NINETY_NINETH_PERCENTILE_METRIC, desc),
- percentiles[6]);
+ protected static void updateSnapshotMetrics(String name, String desc, Histogram histogram,
+ Snapshot snapshot, MetricsRecordBuilder metricsRecordBuilder) {
+ metricsRecordBuilder.addCounter(Interns.info(name + NUM_OPS_METRIC_NAME, desc),
+ histogram.getCount());
+ metricsRecordBuilder.addGauge(Interns.info(name + MIN_METRIC_NAME, desc), snapshot.getMin());
+ metricsRecordBuilder.addGauge(Interns.info(name + MAX_METRIC_NAME, desc), snapshot.getMax());
+ metricsRecordBuilder.addGauge(Interns.info(name + MEAN_METRIC_NAME, desc), snapshot.getMean());
+
+ metricsRecordBuilder.addGauge(Interns.info(name + TWENTY_FIFTH_PERCENTILE_METRIC_NAME, desc),
+ snapshot.get25thPercentile());
+ metricsRecordBuilder.addGauge(Interns.info(name + MEDIAN_METRIC_NAME, desc),
+ snapshot.getMedian());
+ metricsRecordBuilder.addGauge(Interns.info(name + SEVENTY_FIFTH_PERCENTILE_METRIC_NAME, desc),
+ snapshot.get75thPercentile());
+ metricsRecordBuilder.addGauge(Interns.info(name + NINETIETH_PERCENTILE_METRIC_NAME, desc),
+ snapshot.get90thPercentile());
+ metricsRecordBuilder.addGauge(Interns.info(name + NINETY_FIFTH_PERCENTILE_METRIC_NAME, desc),
+ snapshot.get95thPercentile());
+ metricsRecordBuilder.addGauge(Interns.info(name + NINETY_EIGHTH_PERCENTILE_METRIC_NAME, desc),
+ snapshot.get98thPercentile());
+ metricsRecordBuilder.addGauge(Interns.info(name + NINETY_NINETH_PERCENTILE_METRIC_NAME, desc),
+ snapshot.get99thPercentile());
metricsRecordBuilder.addGauge(
- Interns.info(NINETY_NINE_POINT_NINETH_PERCENTILE_METRIC, desc),
- percentiles[7]);
+ Interns.info(name + NINETY_NINE_POINT_NINETH_PERCENTILE_METRIC_NAME, desc),
+ snapshot.get999thPercentile());
}
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java
index 94bcdaa..a12dc27 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.metrics2.lib;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.Interns;
-import org.apache.hadoop.hbase.util.FastLongHistogram;
+import org.apache.hadoop.hbase.metrics.Snapshot;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.MetricsInfo;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
@@ -56,22 +56,22 @@ public abstract class MutableRangeHistogram extends MutableHistogram implements
@Override
public synchronized void snapshot(MetricsRecordBuilder metricsRecordBuilder, boolean all) {
// Get a reference to the old histogram.
- FastLongHistogram histo = histogram.reset();
- if (histo != null) {
- updateSnapshotMetrics(metricsRecordBuilder, histo);
- updateSnapshotRangeMetrics(metricsRecordBuilder, histo);
+ Snapshot snapshot = histogram.snapshot();
+ if (snapshot != null) {
+ updateSnapshotMetrics(name, desc, histogram, snapshot, metricsRecordBuilder);
+ updateSnapshotRangeMetrics(metricsRecordBuilder, snapshot);
}
}
public void updateSnapshotRangeMetrics(MetricsRecordBuilder metricsRecordBuilder,
- FastLongHistogram histogram) {
+ Snapshot snapshot) {
long priorRange = 0;
long cumNum = 0;
final long[] ranges = getRanges();
final String rangeType = getRangeType();
for (int i = 0; i < ranges.length - 1; i++) {
- long val = histogram.getNumAtOrBelow(ranges[i]);
+ long val = snapshot.getCountAtOrBelow(ranges[i]);
if (val - cumNum > 0) {
metricsRecordBuilder.addCounter(
Interns.info(name + "_" + rangeType + "_" + priorRange + "-" + ranges[i], desc),
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestGlobalMetricRegistriesAdapter.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestGlobalMetricRegistriesAdapter.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestGlobalMetricRegistriesAdapter.java
new file mode 100644
index 0000000..e20a1f3
--- /dev/null
+++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestGlobalMetricRegistriesAdapter.java
@@ -0,0 +1,86 @@
+/**
+ * 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.impl;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+import org.apache.hadoop.hbase.testclassification.MetricsTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
+import org.apache.hadoop.metrics2.lib.MutableRate;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+@Category({ MetricsTests.class, SmallTests.class })
+public class TestGlobalMetricRegistriesAdapter {
+
+ /**
+ * Tests that using reflection to unregister the Hadoop metrics source works properly
+ */
+ @Test
+ public void testUnregisterSource() {
+ GlobalMetricRegistriesAdapter adapter = GlobalMetricRegistriesAdapter.init();
+ // we'll configure the sources manually, so disable the executor
+ adapter.stop();
+ TestSource ts1 = new TestSource("ts1");
+ TestSource ts2 = new TestSource("ts2");
+ MetricsSystem metricsSystem = DefaultMetricsSystem.instance();
+ metricsSystem.register("ts1", "", ts1);
+ metricsSystem.register("ts2", "", ts2);
+ MetricsSource s1 = metricsSystem.getSource("ts1");
+ assertNotNull(s1);
+ MetricRegistryInfo mockRegistryInfo = Mockito.mock(MetricRegistryInfo.class);
+ Mockito.when(mockRegistryInfo.getMetricsJmxContext()).thenReturn("ts1");
+ adapter.unregisterSource(mockRegistryInfo);
+ s1 = metricsSystem.getSource("ts1");
+ assertNull(s1);
+ MetricsSource s2 = metricsSystem.getSource("ts2");
+ assertNotNull(s2);
+ }
+
+ @Metrics(context = "test")
+ private static class TestSource {
+ @Metric("C1 desc")
+ MutableCounterLong c1;
+ @Metric("XXX desc")
+ MutableCounterLong xxx;
+ @Metric("G1 desc")
+ MutableGaugeLong g1;
+ @Metric("YYY desc")
+ MutableGaugeLong yyy;
+ @Metric
+ MutableRate s1;
+ @SuppressWarnings("unused")
+ final MetricsRegistry registry;
+
+ TestSource(String recName) {
+ registry = new MetricsRegistry(recName);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics-api/README.txt
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/README.txt b/hbase-metrics-api/README.txt
new file mode 100644
index 0000000..dfaa29f
--- /dev/null
+++ b/hbase-metrics-api/README.txt
@@ -0,0 +1,78 @@
+Overview
+========
+hbase-metrics and hbase-metrics-api are two modules that define and implement the "new" metric
+system used internally within HBase. These two modules (and some other code in hbase-hadoop2-compat)
+module are referred as "HBase metrics framework".
+
+HBase-metrics-api Module
+========================
+HBase Metrics API (hbase-metrics-api) contains the interface
+that HBase exposes internally and to third party code (including coprocessors). It is a thin
+abstraction over the actual implementation for backwards compatibility guarantees. The source
+/ binary and other compatibility guarantees are for "LimitedPrivate API" (see [1] for an
+explanation).
+
+The metrics API in this hbase-metrics-api module is inspired by the Dropwizard metrics 3.1 API
+(See [2]). It is a subset of the API only containing metrics collection. However, the implementation
+is HBase-specific and provided in hbase-metrics module. All of the classes in this module is
+HBase-internal. See the latest documentation of Dropwizard metrics for examples of defining / using
+metrics.
+
+
+HBase-metrics Module
+====================
+hbase-metrics module contains implementation of the "HBase Metrics API", including MetricRegistry,
+Counter, Histogram, etc. These are highly concurrent implementations of the Metric interfaces.
+Metrics in HBase are grouped into different sets (like WAL, RPC, RegionServer, etc). Each group of
+metrics should be tracked via a MetricRegistry specific to that group. Metrics can be dynamically
+added or removed from the registry with a name. Each Registry is independent of the other
+registries and will have it's own JMX context and MetricRecord (when used with Metrics2).
+
+
+MetricRegistry's themselves are tracked via a global registry (of MetricRegistries) called
+MetricRegistries. MetricRegistries.global() can be used to obtain the global instance.
+MetricRegistry instances can also be dynamically registered and removed. However, unlike the
+MetricRegistry, MetricRegistries does reference counting of the MetricRegistry instances. Only
+Metrics in the MetricRegistry instances that are in the global MetricRegistry are exported to the
+metric sinks or JMX.
+
+
+Coprocessor Metrics
+===================
+HBase allows custom coprocessors to track and export metrics using the new framework.
+Coprocessors and other third party code should only use the classes and interfaces from
+hbase-metrics-api module and only the classes that are marked with InterfaceAudience.LimitedPrivate
+annotation. There is no guarantee on the compatibility requirements for other classes.
+
+Coprocessors can obtain the MetricRegistry to register their custom metrics via corresponding
+CoprocessorEnvironment context. See ExampleRegionObserverWithMetrics and
+ExampleMasterObserverWithMetrics classes in hbase-examples module for usage.
+
+
+Developer Notes
+===============
+Historically, HBase has been using Hadoop's Metrics2 framework [3] for collecting and reporting the
+metrics internally. However, due to the difficultly of dealing with the Metrics2 framework, HBase is
+moving away from Hadoop's metrics implementation to its custom implementation. The move will happen
+incrementally, and during the time, both Hadoop Metrics2-based metrics and hbase-metrics module
+based classes will be in the source code. All new implementations for metrics SHOULD use the new
+API and framework.
+
+Examples of the new framework can be found in MetricsCoprocessor and MetricsRegionServerSourceImpl
+classes. See HBASE-9774 [4] for more context.
+
+hbase-metrics module right now only deals with metrics tracking and collection. It does not do JMX
+reporting or reporting to console, ganglia, opentsdb, etc. We use Hadoop's Metrics2 for reporting
+metrics to different sinks or exporting via JMX. However, this is contained within the
+hbase-hadoop2-compat module completely, so that rest of the code does not know anything about the
+Metrics2 dependency. HBaseMetrics2HadoopMetricsAdapter is the adapter that can collect metrics
+in a MetricRegistry using the metric2 MetricsCollector / MetricRecordBuilder interfaces.
+GlobalMetricRegistriesSource is the global Metrics2 Source that collects all of the metrics in all
+of the metric registries in the MetricRegistries.global() instance.
+
+
+References
+1. https://hbase.apache.org/book.html#hbase.versioning
+2. http://metrics.dropwizard.io/
+3. https://hadoop.apache.org/docs/r2.7.2/api/org/apache/hadoop/metrics2/package-summary.html
+4. https://issues.apache.org/jira/browse/HBASE-9774
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics-api/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/pom.xml b/hbase-metrics-api/pom.xml
new file mode 100644
index 0000000..cf55905
--- /dev/null
+++ b/hbase-metrics-api/pom.xml
@@ -0,0 +1,112 @@
+<?xml version="1.0"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <!--
+ /**
+ * 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.
+ */
+ -->
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <artifactId>hbase</artifactId>
+ <groupId>org.apache.hbase</groupId>
+ <version>1.4.0-SNAPSHOT</version>
+ <relativePath>..</relativePath>
+ </parent>
+
+ <artifactId>hbase-metrics-api</artifactId>
+ <name>Apache HBase - Metrics API</name>
+ <description>HBase Metrics API descriptions</description>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-site-plugin</artifactId>
+ <configuration>
+ <skip>true</skip>
+ </configuration>
+ </plugin>
+ <!-- Make a jar and put the sources in the jar -->
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-source-plugin</artifactId>
+ </plugin>
+ <plugin>
+ <!--Make it so assembly:single does nothing in here-->
+ <artifactId>maven-assembly-plugin</artifactId>
+ <version>${maven.assembly.version}</version>
+ <configuration>
+ <skipAssembly>true</skipAssembly>
+ </configuration>
+ </plugin>
+ <plugin>
+ <artifactId>maven-surefire-plugin</artifactId>
+ <!-- Always skip the second part executions, since we only run simple unit tests in this module -->
+ <executions>
+ <execution>
+ <id>secondPartTestsExecution</id>
+ <phase>test</phase>
+ <goals>
+ <goal>test</goal>
+ </goals>
+ <configuration>
+ <skip>true</skip>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+
+ <dependencies>
+ <!-- Intra-project dependencies -->
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-annotations</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-annotations</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-common</artifactId>
+ </dependency>
+ <!-- General dependencies -->
+ <dependency>
+ <groupId>commons-logging</groupId>
+ <artifactId>commons-logging</artifactId>
+ </dependency>
+ </dependencies>
+
+ <profiles>
+ <!-- Skip the tests in this module -->
+ <profile>
+ <id>skip-metrics-api-tests</id>
+ <activation>
+ <property>
+ <name>skip-metrics-api-tests</name>
+ </property>
+ </activation>
+ <properties>
+ <surefire.skipFirstPart>true</surefire.skipFirstPart>
+ </properties>
+ </profile>
+ </profiles>
+</project>
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Counter.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Counter.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Counter.java
new file mode 100644
index 0000000..2e4147e
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Counter.java
@@ -0,0 +1,60 @@
+/**
+ * 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.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * A mutable number optimized for high concurrency counting.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
+public interface Counter extends Metric {
+
+ /**
+ * Increment {@code this} by 1.
+ */
+ void increment();
+
+ /**
+ * Increment {@code this} by {@code n}.
+ *
+ * @param n The amount to increment.
+ */
+ void increment(long n);
+
+ /**
+ * Decrement {@code this} by 1.
+ */
+ void decrement();
+
+ /**
+ * Decrement {@code this} by {@code n}.
+ *
+ * @param n The amount to decrement.
+ */
+ void decrement(long n);
+
+ /**
+ * Returns the current value.
+ * @return the current value.
+ */
+ long getCount();
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Gauge.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Gauge.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Gauge.java
new file mode 100644
index 0000000..90df8e0
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Gauge.java
@@ -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.metrics;
+
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * A metrics which measures a discrete value.
+ *
+ * @param <T> The value of the Gauge.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
+public interface Gauge<T> extends Metric {
+
+ T getValue();
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Histogram.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Histogram.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Histogram.java
new file mode 100644
index 0000000..b5b54c7
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Histogram.java
@@ -0,0 +1,58 @@
+/**
+ * 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.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * A metric which measures the distribution of values.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
+public interface Histogram extends Metric {
+
+ /**
+ * Adds a new value to the distribution.
+ *
+ * @param value The value to add
+ */
+ void update(int value);
+
+ /**
+ * Adds a new value to the distribution.
+ *
+ * @param value The value to add
+ */
+ void update(long value);
+
+ /**
+ * Return the total number of values added to the histogram.
+ * @return the total number of values.
+ */
+ long getCount();
+
+ /**
+ * Snapshot the current values in the Histogram
+ * @return a Snapshot of the distribution.
+ */
+ @InterfaceAudience.Private
+ Snapshot snapshot();
+
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Meter.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Meter.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Meter.java
new file mode 100644
index 0000000..fccaa38
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Meter.java
@@ -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.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * A metric which measure the rate at which some operation is invoked.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
+public interface Meter extends Metric {
+
+ /**
+ * Records one occurrence.
+ */
+ void mark();
+
+ /**
+ * Records {@code events} occurrences.
+ *
+ * @param events Number of occurrences to record.
+ */
+ void mark(long events);
+
+ /**
+ * Returns the number of events.
+ * @return the number of events.
+ */
+ long getCount();
+
+ /**
+ * Returns the mean rate at which events have occurred since the meter was created.
+ * @return the mean rate at which events have occurred since the meter was created
+ */
+ double getMeanRate();
+
+ /**
+ * Returns the one-minute exponentially-weighted moving average rate at which events have
+ * occurred since the meter was created.
+ * <p/>
+ * This rate has the same exponential decay factor as the one-minute load average in the {@code
+ * top} Unix command.
+ *
+ * @return the one-minute exponentially-weighted moving average rate at which events have
+ * occurred since the meter was created
+ */
+ double getOneMinuteRate();
+
+ /**
+ * Returns the five-minute exponentially-weighted moving average rate at which events have
+ * occurred since the meter was created.
+ * <p/>
+ * This rate has the same exponential decay factor as the five-minute load average in the {@code
+ * top} Unix command.
+ *
+ * @return the five-minute exponentially-weighted moving average rate at which events have
+ * occurred since the meter was created
+ */
+ double getFiveMinuteRate();
+
+ /**
+ * Returns the fifteen-minute exponentially-weighted moving average rate at which events have
+ * occurred since the meter was created.
+ * <p/>
+ * This rate has the same exponential decay factor as the fifteen-minute load average in the
+ * {@code top} Unix command.
+ *
+ * @return the fifteen-minute exponentially-weighted moving average rate at which events have
+ * occurred since the meter was created
+ */
+ double getFifteenMinuteRate();
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Metric.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Metric.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Metric.java
new file mode 100644
index 0000000..0a31803
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Metric.java
@@ -0,0 +1,30 @@
+/**
+ * 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.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Parent interface for all metrics.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
+public interface Metric {
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistries.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistries.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistries.java
new file mode 100644
index 0000000..f2fb261
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistries.java
@@ -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 java.util.Collection;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+import com.google.common.base.Optional;
+
+/**
+ * MetricRegistries is collection of MetricRegistry's. MetricsRegistries implementations should do
+ * ref-counting of MetricRegistry's via create() and remove() methods.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
+public abstract class MetricRegistries {
+
+ private static final class LazyHolder {
+ private static final MetricRegistries GLOBAL = MetricRegistriesLoader.load();
+ }
+
+ /**
+ * Return the global singleton instance for the MetricRegistries.
+ * @return MetricRegistries implementation.
+ */
+ public static MetricRegistries global() {
+ return LazyHolder.GLOBAL;
+ }
+
+ /**
+ * Removes all the MetricRegisties.
+ */
+ public abstract void clear();
+
+ /**
+ * Create or return MetricRegistry with the given info. MetricRegistry will only be created
+ * if current reference count is 0. Otherwise ref counted is incremented, and an existing instance
+ * will be returned.
+ * @param info the info object for the MetricRegistrytry.
+ * @return created or existing MetricRegistry.
+ */
+ public abstract MetricRegistry create(MetricRegistryInfo info);
+
+ /**
+ * Decrements the ref count of the MetricRegistry, and removes if ref count == 0.
+ * @param key the info object for the MetricRegistrytry.
+ * @return true if metric registry is removed.
+ */
+ public abstract boolean remove(MetricRegistryInfo key);
+
+ /**
+ * Returns the MetricRegistry if found.
+ * @param info the info for the registry.
+ * @return a MetricRegistry optional.
+ */
+ public abstract Optional<MetricRegistry> get(MetricRegistryInfo info);
+
+ /**
+ * Returns MetricRegistryInfo's for the MetricRegistry's created.
+ * @return MetricRegistryInfo's for the MetricRegistry's created.
+ */
+ public abstract Set<MetricRegistryInfo> getMetricRegistryInfos();
+
+ /**
+ * Returns MetricRegistry's created.
+ * @return MetricRegistry's created.
+ */
+ public abstract Collection<MetricRegistry> getMetricRegistries();
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistriesLoader.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistriesLoader.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistriesLoader.java
new file mode 100644
index 0000000..4fef10c
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistriesLoader.java
@@ -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.metrics;
+
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.ServiceLoader;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+
+import com.google.common.annotations.VisibleForTesting;
+
+@InterfaceAudience.Private
+public class MetricRegistriesLoader {
+ private static final Log LOG = LogFactory.getLog(MetricRegistries.class);
+
+ private static final String defaultClass
+ = "org.apache.hadoop.hbase.metrics.impl.MetricRegistriesImpl";
+
+ /**
+ * Creates a {@link MetricRegistries} instance using the corresponding {@link MetricRegistries}
+ * available to {@link ServiceLoader} on the classpath. If no instance is found, then default
+ * implementation will be loaded.
+ * @return A {@link MetricRegistries} implementation.
+ */
+ public static MetricRegistries load() {
+ List<MetricRegistries> availableImplementations = getDefinedImplemantations();
+ return load(availableImplementations);
+ }
+
+ /**
+ * Creates a {@link MetricRegistries} instance using the corresponding {@link MetricRegistries}
+ * available to {@link ServiceLoader} on the classpath. If no instance is found, then default
+ * implementation will be loaded.
+ * @return A {@link MetricRegistries} implementation.
+ */
+ @VisibleForTesting
+ static MetricRegistries load(List<MetricRegistries> availableImplementations) {
+
+ if (availableImplementations.size() == 1) {
+ // One and only one instance -- what we want/expect
+ MetricRegistries impl = availableImplementations.get(0);
+ LOG.info("Loaded MetricRegistries " + impl.getClass());
+ return impl;
+ } else if (availableImplementations.isEmpty()) {
+ try {
+ return ReflectionUtils.newInstance((Class<MetricRegistries>)Class.forName(defaultClass));
+ } catch (ClassNotFoundException e) {
+ throw new RuntimeException(e);
+ }
+ } else {
+ // Tell the user they're doing something wrong, and choose the first impl.
+ StringBuilder sb = new StringBuilder();
+ for (MetricRegistries factory : availableImplementations) {
+ if (sb.length() > 0) {
+ sb.append(", ");
+ }
+ sb.append(factory.getClass());
+ }
+ LOG.warn("Found multiple MetricRegistries implementations: " + sb
+ + ". Using first found implementation: " + availableImplementations.get(0));
+ return availableImplementations.get(0);
+ }
+ }
+
+ private static List<MetricRegistries> getDefinedImplemantations() {
+ ServiceLoader<MetricRegistries> loader = ServiceLoader.load(MetricRegistries.class);
+ List<MetricRegistries> availableFactories = new ArrayList<>();
+ for (MetricRegistries impl : loader) {
+ availableFactories.add(impl);
+ }
+ return availableFactories;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistry.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistry.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistry.java
new file mode 100644
index 0000000..8404b43
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistry.java
@@ -0,0 +1,112 @@
+/**
+ * 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.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+import com.google.common.base.Optional;
+
+/**
+ * General purpose factory for creating various metrics.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
+public interface MetricRegistry extends MetricSet {
+
+ /**
+ * Get or construct a {@link Timer} used to measure durations and report rates.
+ *
+ * @param name the name of the timer.
+ * @return An instance of {@link Timer}.
+ */
+ Timer timer(String name);
+
+ /**
+ * Get or construct a {@link Histogram} used to measure a distribution of values.
+ *
+ * @param name The name of the Histogram.
+ * @return An instance of {@link Histogram}.
+ */
+ Histogram histogram(String name);
+
+ /**
+ * Get or construct a {@link Meter} used to measure durations and report distributions (a
+ * combination of a {@link Timer} and a {@link Histogram}.
+ *
+ * @param name The name of the Meter.
+ * @return An instance of {@link Meter}.
+ */
+ Meter meter(String name);
+
+ /**
+ * Get or construct a {@link Counter} used to track a mutable number.
+ *
+ * @param name The name of the Counter
+ * @return An instance of {@link Counter}.
+ */
+ Counter counter(String name);
+
+ /**
+ * Register a {@link Gauge}. The Gauge will be invoked at a period defined by the implementation
+ * of {@link MetricRegistry}.
+ * @param name The name of the Gauge.
+ * @param gauge A callback to compute the current value.
+ * @return the registered gauge, or the existing gauge
+ */
+ <T> Gauge<T> register(String name, Gauge<T> gauge);
+
+ /**
+ * Registers the {@link Metric} with the given name if there does not exist one with the same
+ * name. Returns the newly registered or existing Metric.
+ * @param name The name of the Metric.
+ * @param metric the metric to register
+ * @return the registered metric, or the existing metrid
+ */
+ Metric register(String name, Metric metric);
+
+ /**
+ * Registers the {@link Metric}s in the given MetricSet.
+ * @param metricSet set of metrics to register.
+ */
+ void registerAll(MetricSet metricSet);
+
+ /**
+ * Returns previously registered metric with the name if any.
+ * @param name the name of the metric
+ * @return previously registered metric
+ */
+ Optional<Metric> get(String name);
+
+ /**
+ * Removes the metric with the given name.
+ *
+ * @param name the name of the metric
+ * @return true if the metric is removed.
+ */
+ boolean remove(String name);
+
+ /**
+ * Return the MetricRegistryInfo object for this registry.
+ * @return MetricRegistryInfo describing the registry.
+ */
+ @InterfaceAudience.Private
+ MetricRegistryInfo getMetricRegistryInfo();
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistryFactory.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistryFactory.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistryFactory.java
new file mode 100644
index 0000000..b161d20
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistryFactory.java
@@ -0,0 +1,36 @@
+/**
+ *
+ * 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.hbase.classification.InterfaceAudience;
+
+/**
+ * A Factory for creating MetricRegistries. This is the main plugin point for metrics implementation
+ */
+@InterfaceAudience.Private
+public interface MetricRegistryFactory {
+ /**
+ * Create a MetricRegistry from the given MetricRegistryInfo
+ * @param info the descriptor for MetricRegistry
+ * @return a MetricRegistry implementation
+ */
+ MetricRegistry create(MetricRegistryInfo info);
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistryInfo.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistryInfo.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistryInfo.java
new file mode 100644
index 0000000..58fcba7
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistryInfo.java
@@ -0,0 +1,112 @@
+/**
+ * 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.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * HBase Metrics are grouped in different MetricRegistry'ies. All metrics that correspond to a
+ * subcomponent (like RPC, GC, WAL) are managed in a single MetricRegistry.
+ * This class holds the name and description and JMX related context names for such group of
+ * metrics.
+ */
+@InterfaceAudience.Private
+public class MetricRegistryInfo {
+
+ protected final String metricsName;
+ protected final String metricsDescription;
+ protected final String metricsContext;
+ protected final String metricsJmxContext;
+ protected final boolean existingSource;
+
+ public MetricRegistryInfo(
+ String metricsName,
+ String metricsDescription,
+ String metricsJmxContext,
+ String metricsContext,
+ boolean existingSource) {
+ this.metricsName = metricsName;
+ this.metricsDescription = metricsDescription;
+ this.metricsContext = metricsContext;
+ this.metricsJmxContext = metricsJmxContext;
+ this.existingSource = existingSource;
+ }
+
+ /**
+ * Get the metrics context. For hadoop metrics2 system this is usually an all lowercased string.
+ * eg. regionserver, master, thriftserver
+ *
+ * @return The string context used to register this source to hadoop's metrics2 system.
+ */
+ public String getMetricsContext() {
+ return metricsContext;
+ }
+
+ /**
+ * Get the description of what this source exposes.
+ */
+ public String getMetricsDescription() {
+ return metricsDescription;
+ }
+
+ /**
+ * Get the name of the context in JMX that this source will be exposed through.
+ * This is in ObjectName format. With the default context being Hadoop -> HBase
+ */
+ public String getMetricsJmxContext() {
+ return metricsJmxContext;
+ }
+
+ /**
+ * Get the name of the metrics that are being exported by this source.
+ * Eg. IPC, GC, WAL
+ */
+ public String getMetricsName() {
+ return metricsName;
+ }
+
+ /**
+ * Returns whether or not this MetricRegistry is for an existing BaseSource
+ * @return true if this MetricRegistry is for an existing BaseSource.
+ */
+ public boolean isExistingSource() {
+ return existingSource;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof MetricRegistryInfo) {
+ return this.hashCode() == obj.hashCode();
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return new HashCodeBuilder()
+ .append(metricsName)
+ .append(metricsDescription)
+ .append(metricsContext)
+ .append(metricsJmxContext)
+ .hashCode();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricSet.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricSet.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricSet.java
new file mode 100644
index 0000000..9cf2378
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricSet.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.metrics;
+
+import java.util.Map;
+
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * A set of named metrics.
+ *
+ * @see MetricRegistry#registerAll(MetricSet)
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
+public interface MetricSet extends Metric {
+ /**
+ * A map of metric names to metrics.
+ *
+ * @return the metrics
+ */
+ Map<String, Metric> getMetrics();
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/PackageMarker.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/PackageMarker.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/PackageMarker.java
new file mode 100644
index 0000000..a5f6f60
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/PackageMarker.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.metrics;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * This is a dummy annotation that forces javac to produce output for
+ * otherwise empty package-info.java.
+ *
+ * <p>The result is maven-compiler-plugin can properly identify the scope of
+ * changed files
+ *
+ * <p>See more details in
+ * <a href="https://jira.codehaus.org/browse/MCOMPILER-205">
+ * maven-compiler-plugin: incremental compilation broken</a>
+ */
+@InterfaceAudience.Private
+@Retention(RetentionPolicy.SOURCE)
+public @interface PackageMarker {
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Snapshot.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Snapshot.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Snapshot.java
new file mode 100644
index 0000000..56ee8ae
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Snapshot.java
@@ -0,0 +1,136 @@
+/**
+ *
+ * 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.hbase.classification.InterfaceAudience;
+
+/**
+ * A statictical sample of histogram values.
+ */
+@InterfaceAudience.Private
+public interface Snapshot {
+
+ /**
+ * Return the values with the given quantiles.
+ * @param quantiles the requested quantiles.
+ * @return the value for the quantiles.
+ */
+ long[] getQuantiles(double[] quantiles);
+
+ /**
+ * Return the values with the default quantiles.
+ * @return the value for default the quantiles.
+ */
+ long[] getQuantiles();
+
+ /**
+ * Returns the number of values in the snapshot.
+ *
+ * @return the number of values
+ */
+ long getCount();
+
+ /**
+ * Returns the total count below the given value
+ * @param val the value
+ * @return the total count below the given value
+ */
+ long getCountAtOrBelow(long val);
+
+ /**
+ * Returns the value at the 25th percentile in the distribution.
+ *
+ * @return the value at the 25th percentile
+ */
+ long get25thPercentile();
+
+ /**
+ * Returns the value at the 75th percentile in the distribution.
+ *
+ * @return the value at the 75th percentile
+ */
+ long get75thPercentile();
+
+ /**
+ * Returns the value at the 90th percentile in the distribution.
+ *
+ * @return the value at the 90th percentile
+ */
+ long get90thPercentile();
+
+ /**
+ * Returns the value at the 95th percentile in the distribution.
+ *
+ * @return the value at the 95th percentile
+ */
+ long get95thPercentile();
+
+ /**
+ * Returns the value at the 98th percentile in the distribution.
+ *
+ * @return the value at the 98th percentile
+ */
+ long get98thPercentile();
+
+ /**
+ * Returns the value at the 99th percentile in the distribution.
+ *
+ * @return the value at the 99th percentile
+ */
+ long get99thPercentile();
+
+ /**
+ * Returns the value at the 99.9th percentile in the distribution.
+ *
+ * @return the value at the 99.9th percentile
+ */
+ long get999thPercentile();
+
+ /**
+ * Returns the median value in the distribution.
+ *
+ * @return the median value
+ */
+ long getMedian();
+
+ /**
+ * Returns the highest value in the snapshot.
+ *
+ * @return the highest value
+ */
+ long getMax();
+
+ /**
+ * Returns the arithmetic mean of the values in the snapshot.
+ *
+ * @return the arithmetic mean
+ */
+ long getMean();
+
+ /**
+ * Returns the lowest value in the snapshot.
+ *
+ * @return the lowest value
+ */
+ long getMin();
+
+ // TODO: Dropwizard histograms also track stddev
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Timer.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Timer.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Timer.java
new file mode 100644
index 0000000..4fcb636
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Timer.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.metrics;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * A metric which encompasses a {@link Histogram} and {@link Meter}.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
+public interface Timer extends Metric {
+ /**
+ * Update the timer with the given duration in given time unit.
+ * @param duration the duration of the event
+ * @param unit the time unit for the duration
+ */
+ void update(long duration, TimeUnit unit);
+
+ /**
+ * Update the timer with the given duration in milliseconds
+ * @param durationMillis the duration of the event in ms
+ */
+ void updateMillis(long durationMillis);
+
+ /**
+ * Update the timer with the given duration in microseconds
+ * @param durationMicros the duration of the event in microseconds
+ */
+ void updateMicros(long durationMicros);
+
+ /**
+ * Update the timer with the given duration in nanoseconds
+ * @param durationNanos the duration of the event in ns
+ */
+ void updateNanos(long durationNanos);
+
+ @InterfaceAudience.Private
+ Histogram getHistogram();
+
+ @InterfaceAudience.Private
+ Meter getMeter();
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/package-info.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/package-info.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/package-info.java
new file mode 100644
index 0000000..e79451f
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/package-info.java
@@ -0,0 +1,25 @@
+/**
+ * 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.
+ */
+
+/**
+ * Metrics API for HBase.
+ */
+@PackageMarker
+package org.apache.hadoop.hbase.metrics;
+
+// End package-info.java
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics-api/src/test/java/org/apache/hadoop/hbase/metrics/TestMetricRegistriesLoader.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/test/java/org/apache/hadoop/hbase/metrics/TestMetricRegistriesLoader.java b/hbase-metrics-api/src/test/java/org/apache/hadoop/hbase/metrics/TestMetricRegistriesLoader.java
new file mode 100644
index 0000000..8746146
--- /dev/null
+++ b/hbase-metrics-api/src/test/java/org/apache/hadoop/hbase/metrics/TestMetricRegistriesLoader.java
@@ -0,0 +1,56 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.mockito.Mockito.mock;
+
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Test class for {@link MetricRegistriesLoader}.
+ */
+@Category(SmallTests.class)
+public class TestMetricRegistriesLoader {
+
+ @Test
+ public void testLoadSinleInstance() {
+ MetricRegistries loader = mock(MetricRegistries.class);
+ MetricRegistries instance = MetricRegistriesLoader.load(Lists.newArrayList(loader));
+ assertEquals(loader, instance);
+ }
+
+ @Test
+ public void testLoadMultipleInstances() {
+ MetricRegistries loader1 = mock(MetricRegistries.class);
+ MetricRegistries loader2 = mock(MetricRegistries.class);
+ MetricRegistries loader3 = mock(MetricRegistries.class);
+ MetricRegistries instance = MetricRegistriesLoader.load(Lists.newArrayList(loader1, loader2,
+ loader3));
+
+ // the load() returns the first instance
+ assertEquals(loader1, instance);
+ assertNotEquals(loader2, instance);
+ assertNotEquals(loader3, instance);
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics/README.txt
----------------------------------------------------------------------
diff --git a/hbase-metrics/README.txt b/hbase-metrics/README.txt
new file mode 100644
index 0000000..d80064c
--- /dev/null
+++ b/hbase-metrics/README.txt
@@ -0,0 +1 @@
+See the documentation at hbase-metrics-api/README.
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-metrics/pom.xml b/hbase-metrics/pom.xml
new file mode 100644
index 0000000..e32db77
--- /dev/null
+++ b/hbase-metrics/pom.xml
@@ -0,0 +1,136 @@
+<?xml version="1.0"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <!--
+ /**
+ * 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.
+ */
+ -->
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <artifactId>hbase</artifactId>
+ <groupId>org.apache.hbase</groupId>
+ <version>1.4.0-SNAPSHOT</version>
+ <relativePath>..</relativePath>
+ </parent>
+
+ <artifactId>hbase-metrics</artifactId>
+ <name>Apache HBase - Metrics Implementation</name>
+ <description>HBase Metrics Implementation</description>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-site-plugin</artifactId>
+ <configuration>
+ <skip>true</skip>
+ </configuration>
+ </plugin>
+ <!-- Make a jar and put the sources in the jar -->
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-source-plugin</artifactId>
+ </plugin>
+ <plugin>
+ <!--Make it so assembly:single does nothing in here-->
+ <artifactId>maven-assembly-plugin</artifactId>
+ <version>${maven.assembly.version}</version>
+ <configuration>
+ <skipAssembly>true</skipAssembly>
+ </configuration>
+ </plugin>
+ <plugin>
+ <artifactId>maven-surefire-plugin</artifactId>
+ <!-- Always skip the second part executions, since we only run simple unit tests in this module -->
+ <executions>
+ <execution>
+ <id>secondPartTestsExecution</id>
+ <phase>test</phase>
+ <goals>
+ <goal>test</goal>
+ </goals>
+ <configuration>
+ <skip>true</skip>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+
+ <dependencies>
+ <!-- Intra-project dependencies -->
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-annotations</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-annotations</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-metrics-api</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-metrics-api</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <!-- General dependencies -->
+ <dependency>
+ <groupId>commons-logging</groupId>
+ <artifactId>commons-logging</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>io.dropwizard.metrics</groupId>
+ <artifactId>metrics-core</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+
+ <profiles>
+ <!-- Skip the tests in this module -->
+ <profile>
+ <id>skip-metrics-tests</id>
+ <activation>
+ <property>
+ <name>skip-metrics-tests</name>
+ </property>
+ </activation>
+ <properties>
+ <surefire.skipFirstPart>true</surefire.skipFirstPart>
+ </properties>
+ </profile>
+ </profiles>
+</project>
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/CounterImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/CounterImpl.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/CounterImpl.java
new file mode 100644
index 0000000..28ac08b
--- /dev/null
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/CounterImpl.java
@@ -0,0 +1,60 @@
+/**
+ * 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.impl;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.util.LongAdder;
+
+/**
+ * Custom implementation of {@link org.apache.hadoop.hbase.metrics.Counter} using LongAdder.
+ */
+@InterfaceAudience.Private
+public class CounterImpl implements Counter {
+
+ private final LongAdder counter;
+
+ public CounterImpl() {
+ this.counter = new LongAdder();
+ }
+
+ @Override
+ public void increment() {
+ this.counter.increment();
+ }
+
+ @Override
+ public void increment(long n) {
+ this.counter.add(n);
+ }
+
+ @Override
+ public void decrement() {
+ this.counter.decrement();
+ }
+
+ @Override
+ public void decrement(long n) {
+ this.counter.add(-n);
+ }
+
+ @Override
+ public long getCount() {
+ return this.counter.sum();
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/DropwizardMeter.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/DropwizardMeter.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/DropwizardMeter.java
new file mode 100644
index 0000000..fc92483
--- /dev/null
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/DropwizardMeter.java
@@ -0,0 +1,74 @@
+/**
+ * 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.impl;
+
+import java.util.Objects;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+import com.codahale.metrics.Meter;
+
+/**
+ * Dropwizard metrics implementation of {@link org.apache.hadoop.hbase.metrics.Meter}.
+ */
+@InterfaceAudience.Private
+public class DropwizardMeter implements org.apache.hadoop.hbase.metrics.Meter {
+
+ private final Meter meter;
+
+ public DropwizardMeter() {
+ this.meter = new Meter();
+ }
+
+ public DropwizardMeter(Meter meter) {
+ this.meter = Objects.requireNonNull(meter);
+ }
+
+ @Override public void mark() {
+ this.meter.mark();
+ }
+
+ @Override public void mark(long count) {
+ this.meter.mark(count);
+ }
+
+ @Override
+ public long getCount() {
+ return meter.getCount();
+ }
+
+ @Override
+ public double getMeanRate() {
+ return meter.getMeanRate();
+ }
+
+ @Override
+ public double getOneMinuteRate() {
+ return meter.getOneMinuteRate();
+ }
+
+ @Override
+ public double getFiveMinuteRate() {
+ return meter.getFiveMinuteRate();
+ }
+
+ @Override
+ public double getFifteenMinuteRate() {
+ return meter.getFifteenMinuteRate();
+ }
+}
\ No newline at end of file
[2/4] hbase git commit: HBASE-18060 Backport to branch-1 HBASE-9774
HBase native metrics and metric collection for coprocessors
Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/FastLongHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/FastLongHistogram.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/FastLongHistogram.java
new file mode 100644
index 0000000..4e83e1b
--- /dev/null
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/FastLongHistogram.java
@@ -0,0 +1,406 @@
+/**
+ * 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.impl;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.metrics.Snapshot;
+import org.apache.hadoop.hbase.util.AtomicUtils;
+import org.apache.hadoop.hbase.util.LongAdder;
+
+/**
+ * FastLongHistogram is a thread-safe class that estimate distribution of data and computes the
+ * quantiles.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class FastLongHistogram {
+
+ /**
+ * Default number of bins.
+ */
+ public static final int DEFAULT_NBINS = 255;
+
+ public static final double[] DEFAULT_QUANTILES =
+ new double[]{0.25, 0.5, 0.75, 0.90, 0.95, 0.98, 0.99, 0.999};
+
+ /**
+ * Bins is a class containing a list of buckets(or bins) for estimation histogram of some data.
+ */
+ private static class Bins {
+ private final LongAdder[] counts;
+ // inclusive
+ private final long binsMin;
+ // exclusive
+ private final long binsMax;
+ private final long bins10XMax;
+ private final AtomicLong min = new AtomicLong(Long.MAX_VALUE);
+ private final AtomicLong max = new AtomicLong(0L);
+
+ private final LongAdder count = new LongAdder();
+ private final LongAdder total = new LongAdder();
+
+ // set to true when any of data has been inserted to the Bins. It is set after the counts are
+ // updated.
+ private final AtomicBoolean hasData = new AtomicBoolean(false);
+
+ /**
+ * The constructor for creating a Bins without any prior data.
+ */
+ public Bins(int numBins) {
+ counts = createCounters(numBins + 3);
+ this.binsMin = 1L;
+
+ // These two numbers are total guesses
+ // and should be treated as highly suspect.
+ this.binsMax = 1000;
+ this.bins10XMax = binsMax * 10;
+ }
+
+ /**
+ * The constructor for creating a Bins with last Bins.
+ */
+ public Bins(Bins last, int numOfBins, double minQ, double maxQ) {
+ long[] values = last.getQuantiles(new double[] { minQ, maxQ });
+ long wd = values[1] - values[0] + 1;
+ // expand minQ and maxQ in two ends back assuming uniform distribution
+ this.binsMin = Math.max(0L, (long) (values[0] - wd * minQ));
+ long binsMax = (long) (values[1] + wd * (1 - maxQ)) + 1;
+ // make sure each of bins is at least of width 1
+ this.binsMax = Math.max(binsMax, this.binsMin + numOfBins);
+ this.bins10XMax = Math.max((long) (values[1] + (binsMax - 1) * 9), this.binsMax + 1);
+
+ this.counts = createCounters(numOfBins + 3);
+ }
+
+ private LongAdder[] createCounters(int num) {
+ LongAdder[] counters = new LongAdder[num];
+ for (int i = 0; i < num; i++) {
+ counters[i] = new LongAdder();
+ }
+ return counters;
+ }
+
+ private int getIndex(long value) {
+ if (value < this.binsMin) {
+ return 0;
+ } else if (value > this.bins10XMax) {
+ return this.counts.length - 1;
+ } else if (value >= this.binsMax) {
+ return this.counts.length - 2;
+ }
+ // compute the position
+ return 1 + (int) ((value - this.binsMin) * (this.counts.length - 3) /
+ (this.binsMax - this.binsMin));
+
+ }
+
+ /**
+ * Adds a value to the histogram.
+ */
+ public void add(long value, long count) {
+ if (value < 0) {
+ // The whole computation is completely thrown off if there are negative numbers
+ //
+ // Normally we would throw an IllegalArgumentException however this is the metrics
+ // system and it should be completely safe at all times.
+ // So silently throw it away.
+ return;
+ }
+ AtomicUtils.updateMin(min, value);
+ AtomicUtils.updateMax(max, value);
+
+ this.count.add(count);
+ this.total.add(value * count);
+
+ int pos = getIndex(value);
+ this.counts[pos].add(count);
+
+ // hasData needs to be updated as last
+ this.hasData.set(true);
+ }
+
+ /**
+ * Computes the quantiles give the ratios.
+ */
+ public long[] getQuantiles(double[] quantiles) {
+ if (!this.hasData.get()) {
+ // No data yet.
+ return new long[quantiles.length];
+ }
+
+ // Make a snapshot of lowerCounter, higherCounter and bins.counts to counts.
+ // This is not synchronized, but since the counter are accumulating, the result is a good
+ // estimation of a snapshot.
+ long[] counts = new long[this.counts.length];
+ long total = 0L;
+ for (int i = 0; i < this.counts.length; i++) {
+ counts[i] = this.counts[i].sum();
+ total += counts[i];
+ }
+
+ int rIndex = 0;
+ double qCount = total * quantiles[0];
+ long cum = 0L;
+
+ long[] res = new long[quantiles.length];
+ countsLoop: for (int i = 0; i < counts.length; i++) {
+ // mn and mx define a value range
+ long mn, mx;
+ if (i == 0) {
+ mn = this.min.get();
+ mx = this.binsMin;
+ } else if (i == counts.length - 1) {
+ mn = this.bins10XMax;
+ mx = this.max.get();
+ } else if (i == counts.length - 2) {
+ mn = this.binsMax;
+ mx = this.bins10XMax;
+ } else {
+ mn = this.binsMin + (i - 1) * (this.binsMax - this.binsMin) / (this.counts.length - 3);
+ mx = this.binsMin + i * (this.binsMax - this.binsMin) / (this.counts.length - 3);
+ }
+
+ if (mx < this.min.get()) {
+ continue;
+ }
+ if (mn > this.max.get()) {
+ break;
+ }
+ mn = Math.max(mn, this.min.get());
+ mx = Math.min(mx, this.max.get());
+
+ // lastCum/cum are the corresponding counts to mn/mx
+ double lastCum = cum;
+ cum += counts[i];
+
+ // fill the results for qCount is within current range.
+ while (qCount <= cum) {
+ if (cum == lastCum) {
+ res[rIndex] = mn;
+ } else {
+ res[rIndex] = (long) ((qCount - lastCum) * (mx - mn) / (cum - lastCum) + mn);
+ }
+
+ // move to next quantile
+ rIndex++;
+ if (rIndex >= quantiles.length) {
+ break countsLoop;
+ }
+ qCount = total * quantiles[rIndex];
+ }
+ }
+ // In case quantiles contains values >= 100%
+ for (; rIndex < quantiles.length; rIndex++) {
+ res[rIndex] = this.max.get();
+ }
+
+ return res;
+ }
+
+
+ long getNumAtOrBelow(long val) {
+ final int targetIndex = getIndex(val);
+ long totalToCurrentIndex = 0;
+ for (int i = 0; i <= targetIndex; i++) {
+ totalToCurrentIndex += this.counts[i].sum();
+ }
+ return totalToCurrentIndex;
+ }
+
+ public long getMin() {
+ long min = this.min.get();
+ return min == Long.MAX_VALUE ? 0 : min; // in case it is not initialized
+ }
+
+ public long getMean() {
+ long count = this.count.sum();
+ long total = this.total.sum();
+ if (count == 0) {
+ return 0;
+ }
+ return total / count;
+ }
+ }
+
+ // The bins counting values. It is replaced with a new one in calling of reset().
+ private volatile Bins bins;
+
+ /**
+ * Constructor.
+ */
+ public FastLongHistogram() {
+ this(DEFAULT_NBINS);
+ }
+
+ /**
+ * Constructor.
+ * @param numOfBins the number of bins for the histogram. A larger value results in more precise
+ * results but with lower efficiency, and vice versus.
+ */
+ public FastLongHistogram(int numOfBins) {
+ this.bins = new Bins(numOfBins);
+ }
+
+ /**
+ * Constructor setting the bins assuming a uniform distribution within a range.
+ * @param numOfBins the number of bins for the histogram. A larger value results in more precise
+ * results but with lower efficiency, and vice versus.
+ * @param min lower bound of the region, inclusive.
+ * @param max higher bound of the region, inclusive.
+ */
+ public FastLongHistogram(int numOfBins, long min, long max) {
+ this(numOfBins);
+ Bins bins = new Bins(numOfBins);
+ bins.add(min, 1);
+ bins.add(max, 1);
+ this.bins = new Bins(bins, numOfBins, 0.01, 0.999);
+ }
+
+ private FastLongHistogram(Bins bins) {
+ this.bins = bins;
+ }
+
+ /**
+ * Adds a value to the histogram.
+ */
+ public void add(long value, long count) {
+ this.bins.add(value, count);
+ }
+
+ /**
+ * Computes the quantiles give the ratios.
+ */
+ public long[] getQuantiles(double[] quantiles) {
+ return this.bins.getQuantiles(quantiles);
+ }
+
+ public long[] getQuantiles() {
+ return this.bins.getQuantiles(DEFAULT_QUANTILES);
+ }
+
+ public long getMin() {
+ return this.bins.getMin();
+ }
+
+ public long getMax() {
+ return this.bins.max.get();
+ }
+
+ public long getCount() {
+ return this.bins.count.sum();
+ }
+
+ public long getMean() {
+ return this.bins.getMean();
+ }
+
+ public long getNumAtOrBelow(long value) {
+ return this.bins.getNumAtOrBelow(value);
+ }
+
+ /**
+ * Resets the histogram for new counting.
+ */
+ public Snapshot snapshotAndReset() {
+ final Bins oldBins = this.bins;
+ this.bins = new Bins(this.bins, this.bins.counts.length - 3, 0.01, 0.99);
+ final long[] percentiles = oldBins.getQuantiles(DEFAULT_QUANTILES);
+ final long count = oldBins.count.sum();
+
+ return new Snapshot() {
+ @Override
+ public long[] getQuantiles(double[] quantiles) {
+ return oldBins.getQuantiles(quantiles);
+ }
+
+ @Override
+ public long[] getQuantiles() {
+ return percentiles;
+ }
+
+ @Override
+ public long getCount() {
+ return count;
+ }
+
+ @Override
+ public long getCountAtOrBelow(long val) {
+ return oldBins.getNumAtOrBelow(val);
+ }
+
+ @Override
+ public long get25thPercentile() {
+ return percentiles[0];
+ }
+
+ @Override
+ public long get75thPercentile() {
+ return percentiles[2];
+ }
+
+ @Override
+ public long get90thPercentile() {
+ return percentiles[3];
+ }
+
+ @Override
+ public long get95thPercentile() {
+ return percentiles[4];
+ }
+
+ @Override
+ public long get98thPercentile() {
+ return percentiles[5];
+ }
+
+ @Override
+ public long get99thPercentile() {
+ return percentiles[6];
+ }
+
+ @Override
+ public long get999thPercentile() {
+ return percentiles[7];
+ }
+
+ @Override
+ public long getMedian() {
+ return percentiles[1];
+ }
+
+ @Override
+ public long getMax() {
+ return oldBins.max.get();
+ }
+
+ @Override
+ public long getMean() {
+ return oldBins.getMean();
+ }
+
+ @Override
+ public long getMin() {
+ return oldBins.getMin();
+ }
+ };
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/HistogramImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/HistogramImpl.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/HistogramImpl.java
new file mode 100644
index 0000000..b52caf8
--- /dev/null
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/HistogramImpl.java
@@ -0,0 +1,81 @@
+/**
+ *
+ * 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.impl;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.metrics.Histogram;
+import org.apache.hadoop.hbase.metrics.Snapshot;
+
+/**
+ * Custom histogram implementation based on FastLongHistogram. Dropwizard-based histograms are
+ * slow compared to this implementation, so we are using our implementation here.
+ * See HBASE-15222.
+ */
+@InterfaceAudience.Private
+public class HistogramImpl implements Histogram {
+ // Double buffer the two FastLongHistograms.
+ // As they are reset they learn how the buckets should be spaced
+ // So keep two around and use them
+ protected final FastLongHistogram histogram;
+ private final CounterImpl counter;
+
+ public HistogramImpl() {
+ this(Integer.MAX_VALUE << 2);
+ }
+
+ public HistogramImpl(long maxExpected) {
+ this(FastLongHistogram.DEFAULT_NBINS, 1, maxExpected);
+ }
+
+ public HistogramImpl(int numBins, long min, long maxExpected) {
+ this.counter = new CounterImpl();
+ this.histogram = new FastLongHistogram(numBins, min, maxExpected);
+ }
+
+ protected HistogramImpl(CounterImpl counter, FastLongHistogram histogram) {
+ this.counter = counter;
+ this.histogram = histogram;
+ }
+
+ @Override
+ public void update(int value) {
+ counter.increment();
+ histogram.add(value, 1);
+ }
+
+ @Override
+ public void update(long value) {
+ counter.increment();
+ histogram.add(value, 1);
+ }
+
+ public long getCount() {
+ return counter.getCount();
+ }
+
+ public long getMax() {
+ return this.histogram.getMax();
+ }
+
+ public Snapshot snapshot() {
+ return histogram.snapshotAndReset();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistriesImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistriesImpl.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistriesImpl.java
new file mode 100644
index 0000000..3788bd1
--- /dev/null
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistriesImpl.java
@@ -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.metrics.impl;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.metrics.MetricRegistries;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.MetricRegistryFactory;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Supplier;
+
+/**
+ * Implementation of MetricRegistries that does ref-counting.
+ */
+@InterfaceAudience.Private
+public class MetricRegistriesImpl extends MetricRegistries {
+ private final MetricRegistryFactory factory;
+ private final RefCountingMap<MetricRegistryInfo, MetricRegistry> registries;
+
+ public MetricRegistriesImpl() {
+ this(new MetricRegistryFactoryImpl());
+ }
+
+ public MetricRegistriesImpl(MetricRegistryFactory factory) {
+ this.factory = factory;
+ this.registries = new RefCountingMap<>();
+ }
+
+ @Override
+ public MetricRegistry create(final MetricRegistryInfo info) {
+ return registries.put(info, new Supplier<MetricRegistry>() {
+ @Override
+ public MetricRegistry get() {
+ return factory.create(info);
+ }
+ });
+ }
+
+ public boolean remove(MetricRegistryInfo key) {
+ return registries.remove(key) == null;
+ }
+
+ public Optional<MetricRegistry> get(MetricRegistryInfo info) {
+ return Optional.fromNullable(registries.get(info));
+ }
+
+ public Collection<MetricRegistry> getMetricRegistries() {
+ return Collections.unmodifiableCollection(registries.values());
+ }
+
+ public void clear() {
+ registries.clear();
+ }
+
+ public Set<MetricRegistryInfo> getMetricRegistryInfos() {
+ return Collections.unmodifiableSet(registries.keySet());
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryFactoryImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryFactoryImpl.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryFactoryImpl.java
new file mode 100644
index 0000000..b47153e
--- /dev/null
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryFactoryImpl.java
@@ -0,0 +1,34 @@
+/**
+ *
+ * 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.impl;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.MetricRegistryFactory;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+
+@InterfaceAudience.Private
+public class MetricRegistryFactoryImpl implements MetricRegistryFactory {
+ @Override
+ public MetricRegistry create(MetricRegistryInfo info) {
+ return new MetricRegistryImpl(info);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryImpl.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryImpl.java
new file mode 100644
index 0000000..c29e2b5
--- /dev/null
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryImpl.java
@@ -0,0 +1,183 @@
+/**
+ * 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.impl;
+
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.metrics.Gauge;
+import org.apache.hadoop.hbase.metrics.Histogram;
+import org.apache.hadoop.hbase.metrics.Meter;
+import org.apache.hadoop.hbase.metrics.Metric;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+import org.apache.hadoop.hbase.metrics.MetricSet;
+import org.apache.hadoop.hbase.metrics.Timer;
+import org.apache.hadoop.hbase.util.CollectionUtils;
+
+import com.google.common.base.Optional;
+
+/**
+ * Custom implementation of {@link MetricRegistry}.
+ */
+@InterfaceAudience.Private
+public class MetricRegistryImpl implements MetricRegistry {
+
+ private final MetricRegistryInfo info;
+
+ private final ConcurrentMap<String, Metric> metrics;
+
+ public MetricRegistryImpl(MetricRegistryInfo info) {
+ this.info = info;
+ this.metrics = new ConcurrentHashMap<>();
+ }
+
+ @Override
+ public Timer timer(String name) {
+ Timer metric = (Timer) metrics.get(name);
+ if (metric != null) {
+ return metric;
+ }
+
+ Timer newTimer = createTimer();
+ metric = (Timer) metrics.putIfAbsent(name, newTimer);
+ if (metric != null) {
+ return metric;
+ }
+
+ return newTimer;
+ }
+
+ protected Timer createTimer() {
+ return new TimerImpl();
+ }
+
+ @Override
+ public Histogram histogram(String name) {
+ Histogram metric = (Histogram) metrics.get(name);
+ if (metric != null) {
+ return metric;
+ }
+
+ Histogram newHistogram = createHistogram();
+ metric = (Histogram) metrics.putIfAbsent(name, newHistogram);
+ if (metric != null) {
+ return metric;
+ }
+
+ return newHistogram;
+ }
+
+ protected Histogram createHistogram() {
+ return new HistogramImpl();
+ }
+
+ @Override
+ public Meter meter(String name) {
+ Meter metric = (Meter) metrics.get(name);
+ if (metric != null) {
+ return metric;
+ }
+
+ Meter newmeter = createMeter();
+ metric = (Meter) metrics.putIfAbsent(name, newmeter);
+ if (metric != null) {
+ return metric;
+ }
+
+ return newmeter;
+ }
+
+ protected Meter createMeter() {
+ return new DropwizardMeter();
+ }
+
+ @Override
+ public Counter counter(String name) {
+ Counter metric = (Counter) metrics.get(name);
+ if (metric != null) {
+ return metric;
+ }
+
+ Counter newCounter = createCounter();
+ metric = (Counter) metrics.putIfAbsent(name, newCounter);
+ if (metric != null) {
+ return metric;
+ }
+
+ return newCounter;
+ }
+
+ protected Counter createCounter() {
+ return new CounterImpl();
+ }
+
+ @Override
+ public Optional<Metric> get(String name) {
+
+ return Optional.fromNullable(metrics.get(name));
+ }
+
+ @Override
+ public Metric register(String name, Metric metric) {
+ Metric m = metrics.get(name);
+ if (m != null) {
+ return m;
+ }
+
+ Metric oldMetric = metrics.putIfAbsent(name, metric);
+ if (oldMetric != null) {
+ return oldMetric;
+ }
+
+ return metric;
+ }
+
+ @Override
+ public <T> Gauge<T> register(String name, Gauge<T> gauge) {
+ return (Gauge) register(name, (Metric)gauge);
+ }
+
+ @Override
+ public void registerAll(MetricSet metricSet) {
+ Set<Entry<String,Metric>> entrySet = metricSet.getMetrics().entrySet();
+ for (Entry<String, Metric> entry : entrySet) {
+ register(entry.getKey(), entry.getValue());
+ }
+ }
+
+ @Override
+ public Map<String, Metric> getMetrics() {
+ return metrics;
+ }
+
+ @Override
+ public boolean remove(String name) {
+ return metrics.remove(name) != null;
+ }
+
+ @Override
+ public MetricRegistryInfo getMetricRegistryInfo() {
+ return info;
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/RefCountingMap.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/RefCountingMap.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/RefCountingMap.java
new file mode 100644
index 0000000..889b026
--- /dev/null
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/RefCountingMap.java
@@ -0,0 +1,108 @@
+/**
+ *
+ * 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.impl;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+import com.google.common.base.Supplier;
+
+/**
+ * A map of K to V, but does ref counting for added and removed values. The values are
+ * not added directly, but instead requested from the given Supplier if ref count == 0. Each put()
+ * call will increment the ref count, and each remove() will decrement it. The values are removed
+ * from the map iff ref count == 0.
+ */
+@InterfaceAudience.Private
+class RefCountingMap<K, V> {
+
+ private ConcurrentHashMap<K, Payload<V>> map = new ConcurrentHashMap<>();
+ private static class Payload<V> {
+ V v;
+ volatile int refCount;
+ Payload(V v) {
+ this.v = v;
+ this.refCount = 1; // create with ref count = 1
+ }
+ }
+
+ V put(K key, Supplier<V> supplier) {
+ synchronized (map) {
+ Payload<V> oldValue = map.get(key);
+ if (oldValue == null) {
+ oldValue = new Payload<V>(supplier.get());
+ map.put(key, oldValue);
+ return oldValue.v;
+ }
+ oldValue.refCount++;
+ return oldValue.v;
+ }
+ }
+
+ V get(K k) {
+ Payload<V> p = map.get(k);
+ return p == null ? null : p.v;
+ }
+
+ /**
+ * Decrements the ref count of k, and removes from map if ref count == 0.
+ * @param k the key to remove
+ * @return the value associated with the specified key or null if key is removed from map.
+ */
+ V remove(K key) {
+ synchronized (map) {
+ Payload<V> oldValue = map.get(key);
+ if (oldValue != null) {
+ if (--oldValue.refCount == 0) {
+ map.remove(key);
+ return null;
+ }
+ return oldValue.v;
+ }
+ }
+ return null;
+ }
+
+ void clear() {
+ map.clear();
+ }
+
+ Set<K> keySet() {
+ return map.keySet();
+ }
+
+ Collection<V> values() {
+ ArrayList<V> values = new ArrayList<V>(map.size());
+ for (Payload<V> v : map.values()) {
+ values.add(v.v);
+ }
+ return values;
+ }
+
+ int size() {
+ return map.size();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/TimerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/TimerImpl.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/TimerImpl.java
new file mode 100644
index 0000000..1cd8b17
--- /dev/null
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/TimerImpl.java
@@ -0,0 +1,74 @@
+/**
+ * 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.impl;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.metrics.Timer;
+
+/**
+ * Custom implementation of {@link Timer}.
+ */
+@InterfaceAudience.Private
+public class TimerImpl implements Timer {
+ private final HistogramImpl histogram;
+ private final DropwizardMeter meter;
+
+ // track time events in micros
+ private static final TimeUnit DEFAULT_UNIT = TimeUnit.MICROSECONDS;
+
+ public TimerImpl() {
+ this.histogram = new HistogramImpl();
+ this.meter = new DropwizardMeter();
+ }
+
+ @Override
+ public void update(long duration, TimeUnit unit) {
+ if (duration >= 0) {
+ histogram.update(DEFAULT_UNIT.convert(duration, unit));
+ meter.mark();
+ }
+ }
+
+ @Override
+ public HistogramImpl getHistogram() {
+ return histogram;
+ }
+
+ @Override
+ public DropwizardMeter getMeter() {
+ return meter;
+ }
+
+ @Override
+ public void updateMillis(long durationMillis) {
+ update(durationMillis, TimeUnit.NANOSECONDS);
+ }
+
+ @Override
+ public void updateMicros(long durationMicros) {
+ update(durationMicros, TimeUnit.MICROSECONDS);
+ }
+
+ @Override
+ public void updateNanos(long durationNanos) {
+ update(durationNanos, TimeUnit.NANOSECONDS);
+
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/package-info.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/package-info.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/package-info.java
new file mode 100644
index 0000000..9c2e952
--- /dev/null
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/package-info.java
@@ -0,0 +1,25 @@
+/**
+ * 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.
+ */
+
+/**
+ * Implementation of the HBase Metrics framework.
+ */
+@PackageMarker
+package org.apache.hadoop.hbase.metrics.impl;
+
+import org.apache.hadoop.hbase.metrics.PackageMarker;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics/src/main/resources/META-INF/services/org.apache.hadoop.hbase.metrics.MetricRegistries
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/resources/META-INF/services/org.apache.hadoop.hbase.metrics.MetricRegistries b/hbase-metrics/src/main/resources/META-INF/services/org.apache.hadoop.hbase.metrics.MetricRegistries
new file mode 100644
index 0000000..02edf2e
--- /dev/null
+++ b/hbase-metrics/src/main/resources/META-INF/services/org.apache.hadoop.hbase.metrics.MetricRegistries
@@ -0,0 +1,18 @@
+#
+# 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.
+#
+org.apache.hadoop.hbase.metrics.impl.MetricRegistriesImpl
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestCounterImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestCounterImpl.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestCounterImpl.java
new file mode 100644
index 0000000..f0b4f8c
--- /dev/null
+++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestCounterImpl.java
@@ -0,0 +1,59 @@
+/**
+ * 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.impl;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test class for {@link CounterImpl}.
+ */
+@Category(SmallTests.class)
+public class TestCounterImpl {
+
+ private Counter counter;
+
+ @Before public void setup() {
+ this.counter = new CounterImpl();
+ }
+
+ @Test public void testCounting() {
+ counter.increment();
+ assertEquals(1L, counter.getCount());
+ counter.increment();
+ assertEquals(2L, counter.getCount());
+ counter.increment(2L);
+ assertEquals(4L, counter.getCount());
+ counter.increment(-1L);
+ assertEquals(3L, counter.getCount());
+
+ counter.decrement();
+ assertEquals(2L, counter.getCount());
+ counter.decrement();
+ assertEquals(1L, counter.getCount());
+ counter.decrement(4L);
+ assertEquals(-3L, counter.getCount());
+ counter.decrement(-3L);
+ assertEquals(0L, counter.getCount());
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestDropwizardMeter.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestDropwizardMeter.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestDropwizardMeter.java
new file mode 100644
index 0000000..3ba9821
--- /dev/null
+++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestDropwizardMeter.java
@@ -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.metrics.impl;
+
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+import com.codahale.metrics.Meter;
+
+/**
+ * Test class for {@link DropwizardMeter}.
+ */
+@Category(SmallTests.class)
+public class TestDropwizardMeter {
+
+ private Meter meter;
+
+ @Before public void setup() {
+ this.meter = Mockito.mock(Meter.class);
+ }
+
+ @Test public void test() {
+ DropwizardMeter dwMeter = new DropwizardMeter(this.meter);
+
+ dwMeter.mark();
+ dwMeter.mark(10L);
+ dwMeter.mark();
+ dwMeter.mark();
+
+ Mockito.verify(meter, Mockito.times(3)).mark();
+ Mockito.verify(meter).mark(10L);
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestFastLongHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestFastLongHistogram.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestFastLongHistogram.java
new file mode 100644
index 0000000..3dcd4fe
--- /dev/null
+++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestFastLongHistogram.java
@@ -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.metrics.impl;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+import java.util.Random;
+
+
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Testcases for FastLongHistogram.
+ */
+@Category(SmallTests.class)
+public class TestFastLongHistogram {
+
+ private static void doTestUniform(FastLongHistogram hist) {
+ long[] VALUES = { 0, 10, 20, 30, 40, 50 };
+ double[] qs = new double[VALUES.length];
+ for (int i = 0; i < qs.length; i++) {
+ qs[i] = (double) VALUES[i] / VALUES[VALUES.length - 1];
+ }
+
+ for (int i = 0; i < 10; i++) {
+ for (long v : VALUES) {
+ hist.add(v, 1);
+ }
+ long[] vals = hist.getQuantiles(qs);
+ System.out.println(Arrays.toString(vals));
+ for (int j = 0; j < qs.length; j++) {
+ Assert.assertTrue(j + "-th element org: " + VALUES[j] + ", act: " + vals[j],
+ Math.abs(vals[j] - VALUES[j]) <= 10);
+ }
+ hist.snapshotAndReset();
+ }
+ }
+
+ @Test
+ public void testUniform() {
+ FastLongHistogram hist = new FastLongHistogram(100, 0, 50);
+ doTestUniform(hist);
+ }
+
+ @Test
+ public void testAdaptionOfChange() {
+ // assumes the uniform distribution
+ FastLongHistogram hist = new FastLongHistogram(100, 0, 100);
+
+ Random rand = new Random();
+
+ for (int n = 0; n < 10; n++) {
+ for (int i = 0; i < 900; i++) {
+ hist.add(rand.nextInt(100), 1);
+ }
+
+ // add 10% outliers, this breaks the assumption, hope bin10xMax works
+ for (int i = 0; i < 100; i++) {
+ hist.add(1000 + rand.nextInt(100), 1);
+ }
+
+ long[] vals = hist.getQuantiles(new double[] { 0.25, 0.75, 0.95 });
+ System.out.println(Arrays.toString(vals));
+ if (n == 0) {
+ Assert.assertTrue("Out of possible value", vals[0] >= 0 && vals[0] <= 50);
+ Assert.assertTrue("Out of possible value", vals[1] >= 50 && vals[1] <= 100);
+ Assert.assertTrue("Out of possible value", vals[2] >= 900 && vals[2] <= 1100);
+ }
+
+ hist.snapshotAndReset();
+ }
+ }
+
+
+ @Test
+ public void testGetNumAtOrBelow() {
+ long[] VALUES = { 1, 10, 20, 30, 40, 50 };
+
+ FastLongHistogram h = new FastLongHistogram();
+ for (long v : VALUES) {
+ for (int i = 0; i < 100; i++) {
+ h.add(v, 1);
+ }
+ }
+
+ h.add(Integer.MAX_VALUE, 1);
+
+ h.snapshotAndReset();
+
+ for (long v : VALUES) {
+ for (int i = 0; i < 100; i++) {
+ h.add(v, 1);
+ }
+ }
+ // Add something way out there to make sure it doesn't throw off the counts.
+ h.add(Integer.MAX_VALUE, 1);
+
+ assertEquals(100, h.getNumAtOrBelow(1));
+ assertEquals(200, h.getNumAtOrBelow(11));
+ assertEquals(601, h.getNumAtOrBelow(Long.MAX_VALUE));
+ }
+
+
+ @Test
+ public void testSameValues() {
+ FastLongHistogram hist = new FastLongHistogram(100);
+
+ hist.add(50, 100);
+
+ hist.snapshotAndReset();
+ doTestUniform(hist);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestGauge.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestGauge.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestGauge.java
new file mode 100644
index 0000000..7927110
--- /dev/null
+++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestGauge.java
@@ -0,0 +1,61 @@
+/**
+ * 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.impl;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.hbase.metrics.Gauge;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test class for {@link Gauge}.
+ */
+@Category(SmallTests.class)
+public class TestGauge {
+
+ @Test
+ public void testGetValue() {
+ SimpleGauge gauge = new SimpleGauge();
+
+ assertEquals(0, (long)gauge.getValue());
+
+ gauge.setValue(1000L);
+
+ assertEquals(1000L, (long)gauge.getValue());
+ }
+
+ /**
+ * Gauge implementation with a setter.
+ */
+ private static class SimpleGauge implements Gauge<Long> {
+
+ private final AtomicLong value = new AtomicLong(0L);
+
+ @Override public Long getValue() {
+ return this.value.get();
+ }
+
+ public void setValue(long value) {
+ this.value.set(value);
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestHistogramImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestHistogramImpl.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestHistogramImpl.java
new file mode 100644
index 0000000..5d3b1fa
--- /dev/null
+++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestHistogramImpl.java
@@ -0,0 +1,103 @@
+/**
+ *
+ * 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.impl;
+
+import static org.junit.Assert.assertEquals;
+
+
+import org.apache.hadoop.hbase.metrics.Snapshot;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test case for {@link HistogramImpl}
+ */
+@Category(SmallTests.class)
+public class TestHistogramImpl {
+
+ @Test
+ public void testUpdate() {
+ HistogramImpl histogram = new HistogramImpl();
+
+ assertEquals(0, histogram.getCount());
+
+ histogram.update(0);
+ assertEquals(1, histogram.getCount());
+
+ histogram.update(10);
+ assertEquals(2, histogram.getCount());
+
+ histogram.update(20);
+ histogram.update(30);
+
+ assertEquals(4, histogram.getCount());
+ }
+
+ @Test
+ public void testSnapshot() {
+ HistogramImpl histogram = new HistogramImpl();
+ for (int i = 0; i < 100; i++) {
+ histogram.update(i);
+ }
+
+ Snapshot snapshot = histogram.snapshot();
+
+ assertEquals(100, snapshot.getCount());
+ assertEquals(50, snapshot.getMedian());
+ assertEquals(49, snapshot.getMean());
+ assertEquals(0, snapshot.getMin());
+ assertEquals(99, snapshot.getMax());
+ assertEquals(25, snapshot.get25thPercentile());
+ assertEquals(75, snapshot.get75thPercentile());
+ assertEquals(90, snapshot.get90thPercentile());
+ assertEquals(95, snapshot.get95thPercentile());
+ assertEquals(98, snapshot.get98thPercentile());
+ assertEquals(99, snapshot.get99thPercentile());
+ assertEquals(99, snapshot.get999thPercentile());
+
+ assertEquals(51, snapshot.getCountAtOrBelow(50));
+
+ // check that histogram is reset.
+ assertEquals(100, histogram.getCount()); // count does not reset
+
+ // put more data after reset
+ for (int i = 100; i < 200; i++) {
+ histogram.update(i);
+ }
+
+ assertEquals(200, histogram.getCount());
+
+ snapshot = histogram.snapshot();
+ assertEquals(100, snapshot.getCount()); // only 100 more events
+ assertEquals(150, snapshot.getMedian());
+ assertEquals(149, snapshot.getMean());
+ assertEquals(100, snapshot.getMin());
+ assertEquals(199, snapshot.getMax());
+ assertEquals(125, snapshot.get25thPercentile());
+ assertEquals(175, snapshot.get75thPercentile());
+ assertEquals(190, snapshot.get90thPercentile());
+ assertEquals(195, snapshot.get95thPercentile());
+ assertEquals(198, snapshot.get98thPercentile());
+ assertEquals(199, snapshot.get99thPercentile());
+ assertEquals(199, snapshot.get999thPercentile());
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestMetricRegistryImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestMetricRegistryImpl.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestMetricRegistryImpl.java
new file mode 100644
index 0000000..1917ed4
--- /dev/null
+++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestMetricRegistryImpl.java
@@ -0,0 +1,164 @@
+/**
+ *
+ * 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.impl;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Map;
+
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.metrics.Gauge;
+import org.apache.hadoop.hbase.metrics.Meter;
+import org.apache.hadoop.hbase.metrics.Metric;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+import org.apache.hadoop.hbase.metrics.Timer;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.base.Optional;
+
+@Category(SmallTests.class)
+public class TestMetricRegistryImpl {
+ private MetricRegistryInfo info;
+ private MetricRegistryImpl registry;
+
+ @Before
+ public void setUp() {
+ info = new MetricRegistryInfo("foo", "bar", "baz", "foobar", false);
+ registry = new MetricRegistryImpl(info);
+ }
+
+ @Test
+ public void testCounter() {
+ Counter counter = registry.counter("mycounter");
+ assertNotNull(counter);
+ counter.increment(42L);
+ Optional<Metric> metric = registry.get("mycounter");
+ assertTrue(metric.isPresent());
+ assertEquals(42L, (long)((Counter)metric.get()).getCount());
+ }
+
+ @Test
+ public void testRegisterGauge() {
+ registry.register("mygauge", new Gauge<Long>() {
+ @Override
+ public Long getValue() {
+ return 42L;
+ }
+ });
+ Optional<Metric> metric = registry.get("mygauge");
+ assertTrue(metric.isPresent());
+ assertEquals(42L, (long)((Gauge<Long>)metric.get()).getValue());
+ }
+
+ @Test
+ public void testRegisterGaugeLambda() {
+ // register a Gauge using lambda expression
+ registry.register("gaugeLambda", new Gauge<Long>(){
+
+ @Override
+ public Long getValue() {
+ return 42L;
+ }
+ });
+ Optional<Metric> metric = registry.get("gaugeLambda");
+ assertTrue(metric.isPresent());
+ assertEquals(42L, (long)((Gauge<Long>)metric.get()).getValue());
+ }
+
+ @Test
+ public void testTimer() {
+ Timer timer = registry.timer("mytimer");
+ assertNotNull(timer);
+ timer.updateNanos(100);
+ }
+
+ @Test
+ public void testMeter() {
+ Meter meter = registry.meter("mymeter");
+ assertNotNull(meter);
+ meter.mark();
+ }
+
+ @Test
+ public void testRegister() {
+ CounterImpl counter = new CounterImpl();
+ registry.register("mycounter", counter);
+ counter.increment(42L);
+
+ Optional<Metric> metric = registry.get("mycounter");
+ assertTrue(metric.isPresent());
+ assertEquals(42L, (long)((Counter)metric.get()).getCount());
+ }
+
+ @Test
+ public void testDoubleRegister() {
+ Gauge g1 = registry.register("mygauge", new Gauge<Long>(){
+
+ @Override
+ public Long getValue() {
+ return 42L;
+ }});
+ Gauge g2 = registry.register("mygauge", new Gauge<Long>(){
+
+ @Override
+ public Long getValue() {
+ return 52L;
+ }});
+
+ // second gauge is ignored if it exists
+ assertEquals(g1, g2);
+
+ Optional<Metric> metric = registry.get("mygauge");
+ assertTrue(metric.isPresent());
+ assertEquals(42L, (long)((Gauge<Long>)metric.get()).getValue());
+
+
+ Counter c1 = registry.counter("mycounter");
+ Counter c2 = registry.counter("mycounter");
+
+ assertEquals(c1, c2);
+ }
+
+ @Test
+ public void testGetMetrics() {
+ CounterImpl counter = new CounterImpl();
+ registry.register("mycounter", counter);
+ Gauge gauge = registry.register("mygauge", new Gauge<Long>(){
+
+ @Override
+ public Long getValue() {
+ return 42L;
+ }});
+ Timer timer = registry.timer("mytimer");
+
+ Map<String, Metric> metrics = registry.getMetrics();
+ assertEquals(3, metrics.size());
+
+ assertEquals(counter, metrics.get("mycounter"));
+ assertEquals(gauge, metrics.get("mygauge"));
+ assertEquals(timer, metrics.get("mytimer"));
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestRefCountingMap.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestRefCountingMap.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestRefCountingMap.java
new file mode 100644
index 0000000..128f615
--- /dev/null
+++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestRefCountingMap.java
@@ -0,0 +1,284 @@
+/**
+ *
+ * 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.impl;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Collection;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.base.Supplier;
+import com.google.common.collect.Lists;
+
+@Category(SmallTests.class)
+public class TestRefCountingMap {
+
+ private RefCountingMap<String, String> map;
+
+ @Before
+ public void setUp() {
+ map = new RefCountingMap<>();
+ }
+
+ @Test
+ public void testPutGet() {
+ map.put("foo", new Supplier<String>() {
+
+ @Override
+ public String get() {
+ return "foovalue";
+ }
+ });
+
+ String v = map.get("foo");
+ assertNotNull(v);
+ assertEquals("foovalue", v);
+ }
+
+ @Test
+ public void testPutMulti() {
+ String v1 = map.put("foo", new Supplier<String>() {
+
+ @Override
+ public String get() {
+ return "foovalue";
+ }
+ });
+ String v2 = map.put("foo", new Supplier<String>() {
+
+ @Override
+ public String get() {
+ return "foovalue2";
+ }
+ });
+ String v3 = map.put("foo", new Supplier<String>() {
+
+ @Override
+ public String get() {
+ return "foovalue3";
+ }
+ });
+
+ String v = map.get("foo");
+ assertEquals("foovalue", v);
+ assertEquals(v, v1);
+ assertEquals(v, v2);
+ assertEquals(v, v3);
+ }
+
+ @Test
+ public void testPutRemove() {
+ map.put("foo", new Supplier<String>() {
+
+ @Override
+ public String get() {
+ return "foovalue";
+ }
+ });
+ String v = map.remove("foo");
+ assertNull(v);
+ v = map.get("foo");
+ assertNull(v);
+ }
+
+ @Test
+ public void testPutRemoveMulti() {
+ map.put("foo", new Supplier<String>() {
+
+ @Override
+ public String get() {
+ return "foovalue";
+ }
+ });
+ map.put("foo", new Supplier<String>() {
+
+ @Override
+ public String get() {
+ return "foovalue2";
+ }
+ });
+ map.put("foo", new Supplier<String>() {
+
+ @Override
+ public String get() {
+ return "foovalue3";
+ }
+ });
+
+ // remove 1
+ String v = map.remove("foo");
+ assertEquals("foovalue", v);
+
+ // remove 2
+ v = map.remove("foo");
+ assertEquals("foovalue", v);
+
+ // remove 3
+ v = map.remove("foo");
+ assertNull(v);
+ v = map.get("foo");
+ assertNull(v);
+ }
+
+ @Test
+ public void testSize() {
+ assertEquals(0, map.size());
+
+ // put a key
+ map.put("foo", new Supplier<String>() {
+
+ @Override
+ public String get() {
+ return "foovalue";
+ }
+ });
+ assertEquals(1, map.size());
+
+ // put a different key
+ map.put("bar", new Supplier<String>() {
+
+ @Override
+ public String get() {
+ return "foovalue2";
+ }
+ });
+ assertEquals(2, map.size());
+
+ // put the same key again
+ map.put("bar", new Supplier<String>() {
+
+ @Override
+ public String get() {
+ return "foovalue3";
+ }
+ });
+ assertEquals(2, map.size()); // map should be same size
+ }
+
+ @Test
+ public void testClear() {
+ map.put("foo", new Supplier<String>() {
+
+ @Override
+ public String get() {
+ return "foovalue";
+ }
+ });
+ map.put("bar", new Supplier<String>() {
+
+ @Override
+ public String get() {
+ return "foovalue2";
+ }
+ });
+ map.put("baz", new Supplier<String>() {
+
+ @Override
+ public String get() {
+ return "foovalue3";
+ }
+ });
+
+ map.clear();
+
+ assertEquals(0, map.size());
+ }
+
+
+ @Test
+ public void testKeySet() {
+ map.put("foo", new Supplier<String>() {
+
+ @Override
+ public String get() {
+ return "foovalue";
+ }
+ });
+ map.put("bar", new Supplier<String>() {
+
+ @Override
+ public String get() {
+ return "foovalue2";
+ }
+ });
+ map.put("baz", new Supplier<String>() {
+
+ @Override
+ public String get() {
+ return "foovalue3";
+ }
+ });
+
+ Set<String> keys = map.keySet();
+ assertEquals(3, keys.size());
+
+ for (String v : Lists.newArrayList("foo", "bar", "baz")) {
+ assertTrue(keys.contains(v));
+ }
+ }
+
+ @Test
+ public void testValues() {
+ map.put("foo", new Supplier<String>() {
+
+ @Override
+ public String get() {
+ return "foovalue";
+ }
+ });
+ map.put("foo", new Supplier<String>() {
+
+ @Override
+ public String get() {
+ return "foovalue2";
+ }
+ });
+ map.put("bar", new Supplier<String>() {
+
+ @Override
+ public String get() {
+ return "foovalue3";
+ }
+ });
+ map.put("baz", new Supplier<String>() {
+
+ @Override
+ public String get() {
+ return "foovalue4";
+ }
+ });
+
+ Collection<String> values = map.values();
+ assertEquals(3, values.size());
+
+ for (String v : Lists.newArrayList("foovalue", "foovalue3", "foovalue4")) {
+ assertTrue(values.contains(v));
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestTimerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestTimerImpl.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestTimerImpl.java
new file mode 100644
index 0000000..2b3dc8f
--- /dev/null
+++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestTimerImpl.java
@@ -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.metrics.impl;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.metrics.Timer;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test class for {@link TimerImpl}
+ */
+@Category(SmallTests.class)
+public class TestTimerImpl {
+
+ private Timer timer;
+
+ @Before
+ public void setup() {
+ this.timer = new TimerImpl();
+ }
+
+ @Test
+ public void testUpdate() {
+ timer.update(40, TimeUnit.MILLISECONDS);
+ timer.update(41, TimeUnit.MILLISECONDS);
+ timer.update(42, TimeUnit.MILLISECONDS);
+ assertEquals(3, timer.getHistogram().getCount());
+ assertEquals(3, timer.getMeter().getCount());
+
+ assertEquals(TimeUnit.MILLISECONDS.toMicros(41), timer.getHistogram().snapshot().getMedian());
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-resource-bundle/src/main/resources/supplemental-models.xml
----------------------------------------------------------------------
diff --git a/hbase-resource-bundle/src/main/resources/supplemental-models.xml b/hbase-resource-bundle/src/main/resources/supplemental-models.xml
index eac4cad..3f01f09 100644
--- a/hbase-resource-bundle/src/main/resources/supplemental-models.xml
+++ b/hbase-resource-bundle/src/main/resources/supplemental-models.xml
@@ -1065,6 +1065,20 @@ under the License.
</licenses>
</project>
</supplement>
+ <supplement>
+ <project>
+ <groupId>io.dropwizard.metrics</groupId>
+ <artifactId>metrics-core</artifactId>
+
+ <licenses>
+ <license>
+ <name>Apache License, Version 2.0</name>
+ <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+ <distribution>repo</distribution>
+ </license>
+ </licenses>
+ </project>
+ </supplement>
<supplement>
<project>
<groupId>org.codehaus.jettison</groupId>
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index c1a5329..e765b1d 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -397,6 +397,14 @@
<type>test-jar</type>
<scope>test</scope>
</dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-metrics-api</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase-metrics</artifactId>
+ </dependency>
<!-- resource bundle only needed at build time -->
<dependency>
<groupId>org.apache.hbase</groupId>
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterCoprocessorEnvironment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterCoprocessorEnvironment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterCoprocessorEnvironment.java
index abe934a..a6e54e0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterCoprocessorEnvironment.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterCoprocessorEnvironment.java
@@ -24,10 +24,20 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving
public interface MasterCoprocessorEnvironment extends CoprocessorEnvironment {
/** @return reference to the HMaster services */
MasterServices getMasterServices();
+
+ /**
+ * Returns a MetricRegistry that can be used to track metrics at the master level.
+ *
+ * <p>See ExampleMasterObserverWithMetrics class in the hbase-examples modules for examples
+ * of how metrics can be instantiated and used.</p>
+ * @return A MetricRegistry for the coprocessor class to track and export metrics.
+ */
+ MetricRegistry getMetricRegistryForMaster();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetricsCoprocessor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetricsCoprocessor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetricsCoprocessor.java
new file mode 100644
index 0000000..d564002
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetricsCoprocessor.java
@@ -0,0 +1,136 @@
+/**
+ *
+ * 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.coprocessor;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.metrics.MetricRegistries;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Utility class for tracking metrics for various types of coprocessors. Each coprocessor instance
+ * creates its own MetricRegistry which is exported as an individual MetricSource. MetricRegistries
+ * are ref counted using the hbase-metric module interfaces.
+ */
+@InterfaceAudience.Private
+public class MetricsCoprocessor {
+
+ // Master coprocessor metrics
+ private static final String MASTER_COPROC_METRICS_NAME = "Coprocessor.Master";
+ private static final String MASTER_COPROC_METRICS_CONTEXT = "master";
+ private static final String MASTER_COPROC_METRICS_DESCRIPTION
+ = "Metrics about HBase MasterObservers";
+ private static final String MASTER_COPROC_METRICS_JMX_CONTEXT
+ = "Master,sub=" + MASTER_COPROC_METRICS_NAME;
+
+ // RegionServer coprocessor metrics
+ private static final String RS_COPROC_METRICS_NAME = "Coprocessor.RegionServer";
+ private static final String RS_COPROC_METRICS_CONTEXT = "regionserver";
+ private static final String RS_COPROC_METRICS_DESCRIPTION
+ = "Metrics about HBase RegionServerObservers";
+ private static final String RS_COPROC_METRICS_JMX_CONTEXT = "RegionServer,sub="
+ + RS_COPROC_METRICS_NAME;
+
+ // Region coprocessor metrics
+ private static final String REGION_COPROC_METRICS_NAME = "Coprocessor.Region";
+ private static final String REGION_COPROC_METRICS_CONTEXT = "regionserver";
+ private static final String REGION_COPROC_METRICS_DESCRIPTION
+ = "Metrics about HBase RegionObservers";
+ private static final String REGION_COPROC_METRICS_JMX_CONTEXT
+ = "RegionServer,sub=" + REGION_COPROC_METRICS_NAME;
+
+ // WAL coprocessor metrics
+ private static final String WAL_COPROC_METRICS_NAME = "Coprocessor.WAL";
+ private static final String WAL_COPROC_METRICS_CONTEXT = "regionserver";
+ private static final String WAL_COPROC_METRICS_DESCRIPTION
+ = "Metrics about HBase WALObservers";
+ private static final String WAL_COPROC_METRICS_JMX_CONTEXT
+ = "RegionServer,sub=" + WAL_COPROC_METRICS_NAME;
+
+ private static String suffix(String metricName, String cpName) {
+ return new StringBuilder(metricName)
+ .append(".")
+ .append("CP_")
+ .append(cpName)
+ .toString();
+ }
+
+ @VisibleForTesting
+ static MetricRegistryInfo createRegistryInfoForMasterCoprocessor(String clazz) {
+ return new MetricRegistryInfo(
+ suffix(MASTER_COPROC_METRICS_NAME, clazz),
+ MASTER_COPROC_METRICS_DESCRIPTION,
+ suffix(MASTER_COPROC_METRICS_JMX_CONTEXT, clazz),
+ MASTER_COPROC_METRICS_CONTEXT, false);
+ }
+
+ public static MetricRegistry createRegistryForMasterCoprocessor(String clazz) {
+ return MetricRegistries.global().create(createRegistryInfoForMasterCoprocessor(clazz));
+ }
+
+ @VisibleForTesting
+ static MetricRegistryInfo createRegistryInfoForRSCoprocessor(String clazz) {
+ return new MetricRegistryInfo(
+ suffix(RS_COPROC_METRICS_NAME, clazz),
+ RS_COPROC_METRICS_DESCRIPTION,
+ suffix(RS_COPROC_METRICS_JMX_CONTEXT, clazz),
+ RS_COPROC_METRICS_CONTEXT, false);
+ }
+
+ public static MetricRegistry createRegistryForRSCoprocessor(String clazz) {
+ return MetricRegistries.global().create(createRegistryInfoForRSCoprocessor(clazz));
+ }
+
+ @VisibleForTesting
+ public static MetricRegistryInfo createRegistryInfoForRegionCoprocessor(String clazz) {
+ return new MetricRegistryInfo(
+ suffix(REGION_COPROC_METRICS_NAME, clazz),
+ REGION_COPROC_METRICS_DESCRIPTION,
+ suffix(REGION_COPROC_METRICS_JMX_CONTEXT, clazz),
+ REGION_COPROC_METRICS_CONTEXT, false);
+ }
+
+ public static MetricRegistry createRegistryForRegionCoprocessor(String clazz) {
+ return MetricRegistries.global().create(createRegistryInfoForRegionCoprocessor(clazz));
+ }
+
+ @VisibleForTesting
+ public static MetricRegistryInfo createRegistryInfoForWALCoprocessor(String clazz) {
+ return new MetricRegistryInfo(
+ suffix(WAL_COPROC_METRICS_NAME, clazz),
+ WAL_COPROC_METRICS_DESCRIPTION,
+ suffix(WAL_COPROC_METRICS_JMX_CONTEXT, clazz),
+ WAL_COPROC_METRICS_CONTEXT, false);
+ }
+
+ public static MetricRegistry createRegistryForWALCoprocessor(String clazz) {
+ return MetricRegistries.global().create(createRegistryInfoForWALCoprocessor(clazz));
+ }
+
+ public static void removeRegistry(MetricRegistry registry) {
+ if (registry == null) {
+ return;
+ }
+ MetricRegistries.global().remove(registry.getMetricRegistryInfo());
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java
index bdf88af..3566f06 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java
@@ -21,11 +21,12 @@ package org.apache.hadoop.hbase.coprocessor;
import java.util.concurrent.ConcurrentMap;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@@ -43,4 +44,23 @@ public interface RegionCoprocessorEnvironment extends CoprocessorEnvironment {
/** @return shared data between all instances of this coprocessor */
ConcurrentMap<String, Object> getSharedData();
+
+ /**
+ * Returns a MetricRegistry that can be used to track metrics at the region server level. All
+ * metrics tracked at this level will be shared by all the coprocessor instances
+ * of the same class in the same region server process. Note that there will be one
+ * region coprocessor environment per region in the server, but all of these instances will share
+ * the same MetricRegistry. The metric instances (like Counter, Timer, etc) will also be shared
+ * among all of the region coprocessor instances.
+ *
+ * <p>See ExampleRegionObserverWithMetrics class in the hbase-examples modules to see examples of how
+ * metrics can be instantiated and used.</p>
+ * @return A MetricRegistry for the coprocessor class to track and export metrics.
+ */
+ // Note: we are not exposing getMetricRegistryForRegion(). per-region metrics are already costly
+ // so we do not want to allow coprocessors to export metrics at the region level. We can allow
+ // getMetricRegistryForTable() to allow coprocessors to track metrics per-table, per-regionserver.
+ MetricRegistry getMetricRegistryForRegionServer();
+
+
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerCoprocessorEnvironment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerCoprocessorEnvironment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerCoprocessorEnvironment.java
index 527df45..f42556a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerCoprocessorEnvironment.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerCoprocessorEnvironment.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.hbase.coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
public interface RegionServerCoprocessorEnvironment extends CoprocessorEnvironment {
@@ -28,4 +29,13 @@ public interface RegionServerCoprocessorEnvironment extends CoprocessorEnvironme
* @return the region server services
*/
RegionServerServices getRegionServerServices();
+
+ /**
+ * Returns a MetricRegistry that can be used to track metrics at the region server level.
+ *
+ * <p>See ExampleMasterObserverWithMetrics class in the hbase-examples modules for examples
+ * of how metrics can be instantiated and used.</p>
+ * @return A MetricRegistry for the coprocessor class to track and export metrics.
+ */
+ MetricRegistry getMetricRegistryForRegionServer();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java
index a4ce5f1..0865d96 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
import org.apache.hadoop.hbase.wal.WAL;
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@@ -30,4 +31,13 @@ import org.apache.hadoop.hbase.wal.WAL;
public interface WALCoprocessorEnvironment extends CoprocessorEnvironment {
/** @return reference to the region server's WAL */
WAL getWAL();
+
+ /**
+ * Returns a MetricRegistry that can be used to track metrics at the region server level.
+ *
+ * <p>See ExampleRegionServerObserverWithMetrics class in the hbase-examples modules for examples
+ * of how metrics can be instantiated and used.</p>
+ * @return A MetricRegistry for the coprocessor class to track and export metrics.
+ */
+ MetricRegistry getMetricRegistryForRegionServer();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AgeSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AgeSnapshot.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AgeSnapshot.java
index 4c1ad23..dd3bf25 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AgeSnapshot.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AgeSnapshot.java
@@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hbase.io.hfile;
-import org.apache.hadoop.hbase.util.FastLongHistogram;
+import org.apache.hadoop.hbase.metrics.impl.FastLongHistogram;
import org.codehaus.jackson.annotate.JsonIgnoreProperties;
/**
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheUtil.java
index d868a1a..e5bb83b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheUtil.java
@@ -25,7 +25,7 @@ import java.util.concurrent.ConcurrentSkipListSet;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.util.FastLongHistogram;
+import org.apache.hadoop.hbase.metrics.impl.FastLongHistogram;
import org.codehaus.jackson.JsonGenerationException;
import org.codehaus.jackson.annotate.JsonIgnoreProperties;
import org.codehaus.jackson.map.JsonMappingException;
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java
index f38ec70..9249271 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java
@@ -21,9 +21,9 @@ package org.apache.hadoop.hbase.io.hfile;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.metrics.impl.FastLongHistogram;
import org.apache.hadoop.hbase.util.Counter;
-import org.apache.hadoop.hbase.util.FastLongHistogram;
/**
* Class that implements cache metrics.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c3f101/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index 9fb8d81..c7dd282 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -36,8 +36,14 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.coprocessor.*;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
+import org.apache.hadoop.hbase.coprocessor.MetricsCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDescription;
@@ -59,18 +65,33 @@ public class MasterCoprocessorHost
*/
static class MasterEnvironment extends CoprocessorHost.Environment
implements MasterCoprocessorEnvironment {
- private MasterServices masterServices;
+ private final MasterServices masterServices;
+ private final MetricRegistry metricRegistry;
public MasterEnvironment(final Class<?> implClass, final Coprocessor impl,
final int priority, final int seq, final Configuration conf,
final MasterServices services) {
super(impl, priority, seq, conf);
this.masterServices = services;
+ this.metricRegistry =
+ MetricsCoprocessor.createRegistryForMasterCoprocessor(implClass.getName());
}
+ @Override
public MasterServices getMasterServices() {
return masterServices;
}
+
+ @Override
+ public MetricRegistry getMetricRegistryForMaster() {
+ return metricRegistry;
+ }
+
+ @Override
+ protected void shutdown() {
+ super.shutdown();
+ MetricsCoprocessor.removeRegistry(this.metricRegistry);
+ }
}
private MasterServices masterServices;