Return-Path: X-Original-To: apmail-hbase-commits-archive@www.apache.org Delivered-To: apmail-hbase-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id D7D0817792 for ; Tue, 11 Aug 2015 00:12:10 +0000 (UTC) Received: (qmail 73566 invoked by uid 500); 11 Aug 2015 00:12:10 -0000 Delivered-To: apmail-hbase-commits-archive@hbase.apache.org Received: (qmail 73526 invoked by uid 500); 11 Aug 2015 00:12:10 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 73517 invoked by uid 99); 11 Aug 2015 00:12:10 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 11 Aug 2015 00:12:10 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 8AA75E045F; Tue, 11 Aug 2015 00:12:10 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: ndimiduk@apache.org To: commits@hbase.apache.org Message-Id: <737291b8b8e649d3932e81ade1b795ba@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: hbase git commit: HBASE-12911 Client-side metrics Date: Tue, 11 Aug 2015 00:12:10 +0000 (UTC) Repository: hbase Updated Branches: refs/heads/master ae35f65e9 -> 06989fd1f HBASE-12911 Client-side metrics First pass. Plumbs up metrics for each connection instance. Exposes static information about those connections (zk quorum and root node, cluster id, user). Exposes basic thread pool utilization gauges for "meta lookup" and "batch" pools. Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/06989fd1 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/06989fd1 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/06989fd1 Branch: refs/heads/master Commit: 06989fd1f936f905a94e6e98e462ba72704d05c4 Parents: ae35f65 Author: Nick Dimiduk Authored: Mon Aug 10 16:48:48 2015 -0700 Committer: Nick Dimiduk Committed: Mon Aug 10 16:48:48 2015 -0700 ---------------------------------------------------------------------- hbase-client/pom.xml | 4 + .../apache/hadoop/hbase/client/Connection.java | 7 ++ .../hadoop/hbase/client/ConnectionAdapter.java | 5 + .../hbase/client/ConnectionImplementation.java | 14 ++- .../apache/hadoop/hbase/client/MetaCache.java | 9 ++ .../hadoop/hbase/client/MetricsConnection.java | 48 ++++++++++ .../client/MetricsConnectionWrapperImpl.java | 99 ++++++++++++++++++++ .../hbase/client/MetricsConnectionSource.java | 87 +++++++++++++++++ .../client/MetricsConnectionSourceFactory.java | 32 +++++++ .../hbase/client/MetricsConnectionWrapper.java | 50 ++++++++++ .../MetricsConnectionSourceFactoryImpl.java | 28 ++++++ .../client/MetricsConnectionSourceImpl.java | 86 +++++++++++++++++ ....hbase.client.MetricsConnectionSourceFactory | 18 ++++ 13 files changed, 484 insertions(+), 3 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/06989fd1/hbase-client/pom.xml ---------------------------------------------------------------------- diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml index dc0da77..11eea17 100644 --- a/hbase-client/pom.xml +++ b/hbase-client/pom.xml @@ -128,6 +128,10 @@ org.apache.hbase + hbase-hadoop-compat + + + org.apache.hbase hbase-common test-jar http://git-wip-us.apache.org/repos/asf/hbase/blob/06989fd1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java index dab4905..cfc9d56 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java @@ -175,4 +175,11 @@ public interface Connection extends Abortable, Closeable { */ boolean isClosed(); + /** + * Retrieve the metrics instance for this connection. + * + * @return a MetricsConnection instance for examining connection metrics. + */ + public MetricsConnection getConnectionMetrics(); + } http://git-wip-us.apache.org/repos/asf/hbase/blob/06989fd1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java index 1d8a793..aace6ef 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java @@ -131,6 +131,11 @@ abstract class ConnectionAdapter implements ClusterConnection { } @Override + public MetricsConnection getConnectionMetrics() { + return wrappedConnection.getConnectionMetrics(); + } + + @Override public boolean isMasterRunning() throws MasterNotRunningException, ZooKeeperConnectionException { return wrappedConnection.isMasterRunning(); http://git-wip-us.apache.org/repos/asf/hbase/blob/06989fd1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java index 2754997..beb2f66 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java @@ -118,6 +118,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable { private final AsyncProcess asyncProcess; // single tracker per connection private final ServerStatisticTracker stats; + private final MetricsConnection metrics; private volatile boolean closed; private volatile boolean aborted; @@ -154,11 +155,11 @@ class ConnectionImplementation implements ClusterConnection, Closeable { // Client rpc instance. private RpcClient rpcClient; - private MetaCache metaCache = new MetaCache(); + private final MetaCache metaCache; private int refCount; - private User user; + protected User user; private RpcRetryingCallerFactory rpcCallerFactory; @@ -236,11 +237,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable { } else { nonceGenerator = new NoNonceGenerator(); } - stats = ServerStatisticTracker.create(conf); + this.stats = ServerStatisticTracker.create(conf); this.asyncProcess = createAsyncProcess(this.conf); this.interceptor = (new RetryingCallerInterceptorFactory(conf)).build(); this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf, interceptor, this.stats); this.backoffPolicy = ClientBackoffPolicyFactory.create(conf); + this.metrics = new MetricsConnection(new MetricsConnectionWrapperImpl(this)); + this.metaCache = new MetaCache(this.metrics); } /** @@ -362,6 +365,11 @@ class ConnectionImplementation implements ClusterConnection, Closeable { return new HBaseAdmin(this); } + @Override + public MetricsConnection getConnectionMetrics() { + return this.metrics; + } + private ExecutorService getBatchPool() { if (batchPool == null) { synchronized (this) { http://git-wip-us.apache.org/repos/asf/hbase/blob/06989fd1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaCache.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaCache.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaCache.java index 8e1c93c..e763dd9 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaCache.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaCache.java @@ -59,6 +59,12 @@ public class MetaCache { // The access to this attribute must be protected by a lock on cachedRegionLocations private final Set cachedServers = new ConcurrentSkipListSet(); + private final MetricsConnection metrics; + + public MetaCache(MetricsConnection metrics) { + this.metrics = metrics; + } + /** * Search the cache for a location that fits our table and row key. * Return null if no suitable region is located. @@ -74,6 +80,7 @@ public class MetaCache { Entry e = tableLocations.floorEntry(row); if (e == null) { + metrics.incrMetaCacheMiss(); return null; } RegionLocations possibleRegion = e.getValue(); @@ -94,10 +101,12 @@ public class MetaCache { // HConstants.EMPTY_END_ROW) check itself will pass. if (Bytes.equals(endKey, HConstants.EMPTY_END_ROW) || Bytes.compareTo(endKey, 0, endKey.length, row, 0, row.length) > 0) { + metrics.incrMetaCacheHit(); return possibleRegion; } // Passed all the way through, so we got nothing - complete cache miss + metrics.incrMetaCacheMiss(); return null; } http://git-wip-us.apache.org/repos/asf/hbase/blob/06989fd1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java new file mode 100644 index 0000000..2c34c9c --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java @@ -0,0 +1,48 @@ +/** + * 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.client; + +import org.apache.hadoop.hbase.CompatibilitySingletonFactory; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; + +/** + * This class is for maintaining the various connection statistics and publishing them through + * the metrics interfaces. + */ +@InterfaceStability.Evolving +@InterfaceAudience.Private +public class MetricsConnection { + + private final MetricsConnectionWrapper wrapper; + private final MetricsConnectionSource source; + + public MetricsConnection(MetricsConnectionWrapper wrapper) { + this.wrapper = wrapper; + this.source = CompatibilitySingletonFactory.getInstance(MetricsConnectionSourceFactory.class) + .createConnection(this.wrapper); + } + + public void incrMetaCacheHit() { + source.incrMetaCacheHit(); + } + + public void incrMetaCacheMiss() { + source.incrMetaCacheMiss(); + } +} http://git-wip-us.apache.org/repos/asf/hbase/blob/06989fd1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionWrapperImpl.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionWrapperImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionWrapperImpl.java new file mode 100644 index 0000000..e6056ec --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionWrapperImpl.java @@ -0,0 +1,99 @@ +/** + * 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.client; + +import com.google.common.base.Preconditions; + +import java.util.concurrent.ThreadPoolExecutor; + +public class MetricsConnectionWrapperImpl implements MetricsConnectionWrapper { + + private final ConnectionImplementation conn; + + public MetricsConnectionWrapperImpl(ConnectionImplementation connection) { + Preconditions.checkNotNull(connection); + this.conn = connection; + } + + @Override public String getId() { + return conn.toString(); + } + + @Override public String getUserName() { + return conn.user == null ? "" : conn.user.toString(); + } + + @Override public String getClusterId() { + return conn.clusterId; + } + + @Override public String getZookeeperQuorum() { + try { + return conn.getKeepAliveZooKeeperWatcher().getQuorum(); + } catch (Exception e) { + return ""; + } + } + + @Override public String getZookeeperBaseNode() { + try { + return conn.getKeepAliveZooKeeperWatcher().getBaseZNode(); + } catch (Exception e) { + return ""; + } + } + + @Override public int getMetaLookupPoolActiveCount() { + if (conn.getCurrentMetaLookupPool() == null) { + return 0; + } + ThreadPoolExecutor tpe = (ThreadPoolExecutor) conn.getCurrentMetaLookupPool(); + return tpe.getActiveCount(); + } + + @Override public int getMetaLookupPoolLargestPoolSize() { + if (conn.getCurrentMetaLookupPool() == null) { + return 0; + } + ThreadPoolExecutor tpe = (ThreadPoolExecutor) conn.getCurrentMetaLookupPool(); + return tpe.getLargestPoolSize(); + } + + @Override public String getBatchPoolId() { + if (conn.getCurrentBatchPool() == null) { + return ""; + } + return Integer.toHexString(conn.getCurrentBatchPool().hashCode()); + } + + @Override public int getBatchPoolActiveCount() { + if (conn.getCurrentBatchPool() == null) { + return 0; + } + ThreadPoolExecutor tpe = (ThreadPoolExecutor) conn.getCurrentBatchPool(); + return tpe.getActiveCount(); + } + + @Override public int getBatchPoolLargestPoolSize() { + if (conn.getCurrentBatchPool() == null) { + return 0; + } + ThreadPoolExecutor tpe = (ThreadPoolExecutor) conn.getCurrentBatchPool(); + return tpe.getLargestPoolSize(); + } +} http://git-wip-us.apache.org/repos/asf/hbase/blob/06989fd1/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionSource.java ---------------------------------------------------------------------- diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionSource.java new file mode 100644 index 0000000..53f9f11 --- /dev/null +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionSource.java @@ -0,0 +1,87 @@ +/** + * 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.client; + +import org.apache.hadoop.hbase.metrics.BaseSource; + +public interface MetricsConnectionSource extends BaseSource { + + /** + * The name of the metrics + */ + String METRICS_NAME = "Connection"; + + /** + * The name of the metrics context that metrics will be under. + */ + String METRICS_CONTEXT = "connection"; + + /** + * Description + */ + String METRICS_DESCRIPTION = "Metrics about HBase Connection"; + + /** + * The name of the metrics context that metrics will be under in jmx + */ + String METRICS_JMX_CONTEXT = "Client,sub="; + + /** + * Increment number of meta cache hits + */ + void incrMetaCacheHit(); + + /** + * Increment number of meta cache misses + */ + void incrMetaCacheMiss(); + + // Strings used for exporting to metrics system. + String CONNECTION_ID_NAME = "connectionId"; + String CONNECTION_ID_DESC = "The connection's process-unique identifier."; + String USER_NAME_NAME = "userName"; + String USER_NAME_DESC = "The user on behalf of whom the Connection is acting."; + String CLUSTER_ID_NAME = "clusterId"; + String CLUSTER_ID_DESC = "Cluster Id"; + String ZOOKEEPER_QUORUM_NAME = "zookeeperQuorum"; + String ZOOKEEPER_QUORUM_DESC = "Zookeeper Quorum"; + String ZOOKEEPER_ZNODE_NAME = "zookeeperBaseZNode"; + String ZOOKEEPER_ZNODE_DESC = "Base ZNode for this cluster."; + + String META_CACHE_HIT_NAME = "metaCacheHit"; + String META_CACHE_HIT_DESC = + "A counter on the number of times this connection's meta cache has a valid region location."; + String META_CACHE_MISS_NAME = "metaCacheMiss"; + String META_CACHE_MISS_DESC = + "A counter on the number of times this connection does not know where to find a region."; + + String META_LOOKUP_POOL_ACTIVE_THREAD_NAME = "metaLookupPoolActiveThreads"; + String META_LOOKUP_POOL_ACTIVE_THREAD_DESC = + "The approximate number of threads actively resolving region locations from META."; + String META_LOOKUP_POOL_LARGEST_SIZE_NAME = "metaLookupPoolLargestSize"; + String META_LOOKUP_POOL_LARGEST_SIZE_DESC = + "The largest number of threads that have ever simultaneously been in the pool."; + String BATCH_POOL_ID_NAME = "batchPoolId"; + String BATCH_POOL_ID_DESC = "The connection's batch pool's unique identifier."; + String BATCH_POOL_ACTIVE_THREAD_NAME = "batchPoolActiveThreads"; + String BATCH_POOL_ACTIVE_THREAD_DESC = + "The approximate number of threads executing table operations."; + String BATCH_POOL_LARGEST_SIZE_NAME = "batchPoolLargestSize"; + String BATCH_POOL_LARGEST_SIZE_DESC = + "The largest number of threads that have ever simultaneously been in the pool."; +} http://git-wip-us.apache.org/repos/asf/hbase/blob/06989fd1/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionSourceFactory.java ---------------------------------------------------------------------- diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionSourceFactory.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionSourceFactory.java new file mode 100644 index 0000000..9b30421 --- /dev/null +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionSourceFactory.java @@ -0,0 +1,32 @@ +/** + * 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.client; + +/** + * Interface of a factory to create Metrics Sources used inside of Connections. + */ +public interface MetricsConnectionSourceFactory { + + /** + * Given a wrapper create a {@link MetricsConnectionSource}. + * + * @param wrapper The wrapped Connection + * @return a Metrics Source. + */ + public MetricsConnectionSource createConnection(MetricsConnectionWrapper wrapper); +} http://git-wip-us.apache.org/repos/asf/hbase/blob/06989fd1/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionWrapper.java ---------------------------------------------------------------------- diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionWrapper.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionWrapper.java new file mode 100644 index 0000000..43410a8 --- /dev/null +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionWrapper.java @@ -0,0 +1,50 @@ +/** + * 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.client; + +/** + * This is the interface that will expose Connection information to hadoop1/hadoop2 + * implementations of the {@link MetricsConnectionSource}. + */ +public interface MetricsConnectionWrapper { + + /** Get the connection's unique identifier */ + String getId(); + + /** Get the User's name. */ + String getUserName(); + + /** Get the Cluster ID */ + String getClusterId(); + + /** Get the Zookeeper Quorum Info */ + String getZookeeperQuorum(); + + /** Get the base ZNode for this cluster. */ + String getZookeeperBaseNode(); + + int getMetaLookupPoolActiveCount(); + + int getMetaLookupPoolLargestPoolSize(); + + String getBatchPoolId(); + + int getBatchPoolActiveCount(); + + int getBatchPoolLargestPoolSize(); +} http://git-wip-us.apache.org/repos/asf/hbase/blob/06989fd1/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionSourceFactoryImpl.java ---------------------------------------------------------------------- diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionSourceFactoryImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionSourceFactoryImpl.java new file mode 100644 index 0000000..faa3295 --- /dev/null +++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionSourceFactoryImpl.java @@ -0,0 +1,28 @@ +/** + * 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.client; + +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +@InterfaceAudience.Private +public class MetricsConnectionSourceFactoryImpl implements MetricsConnectionSourceFactory { + + @Override public MetricsConnectionSource createConnection(MetricsConnectionWrapper wrapper) { + return new MetricsConnectionSourceImpl(wrapper); + } +} http://git-wip-us.apache.org/repos/asf/hbase/blob/06989fd1/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionSourceImpl.java ---------------------------------------------------------------------- diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionSourceImpl.java new file mode 100644 index 0000000..06da639 --- /dev/null +++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/client/MetricsConnectionSourceImpl.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.client; + +import org.apache.hadoop.hbase.metrics.BaseSourceImpl; +import org.apache.hadoop.metrics2.MetricsCollector; +import org.apache.hadoop.metrics2.MetricsRecordBuilder; +import org.apache.hadoop.metrics2.lib.Interns; +import org.apache.hadoop.metrics2.lib.MutableCounterLong; + +public class MetricsConnectionSourceImpl + extends BaseSourceImpl implements MetricsConnectionSource { + + // wrapper for access statistics collected in Connection instance + private final MetricsConnectionWrapper wrapper; + + // Hadoop Metric2 objects for additional monitoring + + private final MutableCounterLong metaCacheHit; + private final MutableCounterLong metaCacheMiss; + + public MetricsConnectionSourceImpl(MetricsConnectionWrapper wrapper) { + this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, + METRICS_JMX_CONTEXT + wrapper.getId(), wrapper); + } + + public MetricsConnectionSourceImpl(String metricsName, String metricsDescription, + String metricsContext, String metricsJmxContext, MetricsConnectionWrapper wrapper) { + super(metricsName, metricsDescription, metricsContext, metricsJmxContext); + this.wrapper = wrapper; + + metaCacheHit = getMetricsRegistry().newCounter(META_CACHE_HIT_NAME, META_CACHE_HIT_DESC, 0l); + metaCacheMiss = + getMetricsRegistry().newCounter(META_CACHE_MISS_NAME, META_CACHE_MISS_DESC, 0l); + } + + @Override + public void getMetrics(MetricsCollector metricsCollector, boolean all) { + + MetricsRecordBuilder mrb = metricsCollector.addRecord(metricsName); + + if (wrapper != null) { + mrb.addGauge(Interns.info(META_LOOKUP_POOL_LARGEST_SIZE_NAME, + META_LOOKUP_POOL_LARGEST_SIZE_DESC), wrapper.getMetaLookupPoolLargestPoolSize()) + .addGauge(Interns.info(META_LOOKUP_POOL_ACTIVE_THREAD_NAME, + META_LOOKUP_POOL_ACTIVE_THREAD_DESC), wrapper.getMetaLookupPoolActiveCount()) + .tag(Interns.info(BATCH_POOL_ID_NAME, BATCH_POOL_ID_DESC), wrapper.getBatchPoolId()) + .addGauge(Interns.info(BATCH_POOL_ACTIVE_THREAD_NAME, BATCH_POOL_ACTIVE_THREAD_DESC), + wrapper.getBatchPoolActiveCount()) + .addGauge(Interns.info(BATCH_POOL_LARGEST_SIZE_NAME, BATCH_POOL_LARGEST_SIZE_DESC), + wrapper.getBatchPoolLargestPoolSize()) + .tag(Interns.info(CONNECTION_ID_NAME, CONNECTION_ID_DESC), wrapper.getId()) + .tag(Interns.info(USER_NAME_NAME, USER_NAME_DESC), wrapper.getUserName()) + .tag(Interns.info(CLUSTER_ID_NAME, CLUSTER_ID_DESC), wrapper.getClusterId()) + .tag(Interns.info(ZOOKEEPER_QUORUM_NAME, ZOOKEEPER_QUORUM_DESC), + wrapper.getZookeeperQuorum()) + .tag(Interns.info(ZOOKEEPER_ZNODE_NAME, ZOOKEEPER_ZNODE_DESC), + wrapper.getZookeeperBaseNode()); + } + + metricsRegistry.snapshot(mrb, all); + } + + @Override public void incrMetaCacheHit() { + metaCacheHit.incr(); + } + + @Override public void incrMetaCacheMiss() { + metaCacheMiss.incr(); + } +} http://git-wip-us.apache.org/repos/asf/hbase/blob/06989fd1/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.client.MetricsConnectionSourceFactory ---------------------------------------------------------------------- diff --git a/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.client.MetricsConnectionSourceFactory b/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.client.MetricsConnectionSourceFactory new file mode 100644 index 0000000..4adf4c8 --- /dev/null +++ b/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.client.MetricsConnectionSourceFactory @@ -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.client.MetricsConnectionSourceFactoryImpl