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/04/27 01:30:21 UTC

[2/2] hbase git commit: HBASE-17448 Export metrics from RecoverableZooKeeper

HBASE-17448 Export metrics from RecoverableZooKeeper

Added metrics for RecoverableZooKeeper related to specific exceptions,
total failed ZooKeeper API calls and latency histograms for read,
write and sync operations. Also added unit tests for the same. Added
service provider for the ZooKeeper metrics implementation inside the
hadoop compatibility module.

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/eb6ded48
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/eb6ded48
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/eb6ded48

Branch: refs/heads/master
Commit: eb6ded4849adfc33d68e97a0bc689e7b4ac8c355
Parents: 6bad35e
Author: ckulkarni <ck...@salesforce.com>
Authored: Fri Apr 21 18:41:15 2017 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Apr 26 18:30:13 2017 -0700

----------------------------------------------------------------------
 hbase-client/pom.xml                            |   8 +
 .../hbase/zookeeper/MetricsZooKeeper.java       | 110 +++++++++++++
 .../hbase/zookeeper/RecoverableZooKeeper.java   | 144 ++++++++++++++--
 .../zookeeper/ZooKeeperMetricsListener.java     |  91 ++++++++++
 .../hbase/zookeeper/TestMetricsZooKeeper.java   |  77 +++++++++
 .../hbase/zookeeper/MetricsZooKeeperSource.java | 139 ++++++++++++++++
 .../zookeeper/TestMetricsZooKeeperSource.java   |  34 ++++
 .../zookeeper/MetricsZooKeeperSourceImpl.java   | 164 +++++++++++++++++++
 ...adoop.hbase.zookeeper.MetricsZooKeeperSource |  18 ++
 .../TestMetricsZooKeeperSourceImpl.java         |  38 +++++
 .../hadoop/hbase/metrics/PackageMarker.java     |   3 +
 11 files changed, 812 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/eb6ded48/hbase-client/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml
index d76049f..928ba03 100644
--- a/hbase-client/pom.xml
+++ b/hbase-client/pom.xml
@@ -112,6 +112,14 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-hadoop-compat</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-hadoop2-compat</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-common</artifactId>
       <type>test-jar</type>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb6ded48/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeper.java
new file mode 100644
index 0000000..6b5e188
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeper.java
@@ -0,0 +1,110 @@
+/**
+ *
+ * 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.zookeeper;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.regionserver.wal.MetricsWALSource;
+import org.apache.hadoop.hbase.regionserver.wal.MetricsWALSourceImpl;
+
+/**
+ * Class used to push numbers about ZooKeeper into the metrics subsystem. This will take a
+ * single function call and turn it into multiple manipulations of the hadoop metrics system.
+ */
+@InterfaceAudience.Private
+public class MetricsZooKeeper implements ZooKeeperMetricsListener {
+  private final MetricsZooKeeperSource source;
+
+  public MetricsZooKeeper() {
+    this(CompatibilitySingletonFactory.getInstance(MetricsZooKeeperSource.class));
+  }
+
+  @VisibleForTesting
+  public MetricsZooKeeper(MetricsZooKeeperSource s) {
+    this.source = s;
+  }
+
+  @Override
+  public void registerAuthFailedException() {
+    source.incrementAuthFailedCount();
+  }
+
+  @Override
+  public void registerConnectionLossException() {
+    source.incrementConnectionLossCount();
+  }
+
+  @Override
+  public void registerDataInconsistencyException() {
+    source.incrementDataInconsistencyCount();
+  }
+
+  @Override
+  public void registerInvalidACLException() {
+    source.incrementInvalidACLCount();
+  }
+
+  @Override
+  public void registerNoAuthException() {
+    source.incrementNoAuthCount();
+  }
+
+  @Override
+  public void registerOperationTimeoutException() {
+    source.incrementOperationTimeoutCount();
+  }
+
+  @Override
+  public void registerRuntimeInconsistencyException() {
+    source.incrementRuntimeInconsistencyCount();
+  }
+
+  @Override
+  public void registerSessionExpiredException() {
+    source.incrementSessionExpiredCount();
+  }
+
+  @Override
+  public void registerSystemErrorException() {
+    source.incrementSystemErrorCount();
+  }
+
+  @Override
+  public void registerFailedZKCall() {
+    source.incrementTotalFailedZKCalls();
+  }
+
+  @Override
+  public void registerReadOperationLatency(long latency) {
+    source.recordReadOperationLatency(latency);
+  }
+
+  @Override
+  public void registerWriteOperationLatency(long latency) {
+    source.recordWriteOperationLatency(latency);
+  }
+
+  @Override
+  public void registerSyncOperationLatency(long latency) {
+    source.recordSyncOperationLatency(latency);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb6ded48/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
index 4f07d5b..e0a950d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.RetryCounter;
 import org.apache.hadoop.hbase.util.RetryCounterFactory;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
@@ -83,6 +84,7 @@ public class RecoverableZooKeeper {
   private int sessionTimeout;
   private String quorumServers;
   private final Random salter;
+  private final ZooKeeperMetricsListener metrics;
 
   // The metadata attached to each piece of data has the
   // format:
@@ -125,6 +127,7 @@ public class RecoverableZooKeeper {
     this.watcher = watcher;
     this.sessionTimeout = sessionTimeout;
     this.quorumServers = quorumServers;
+    this.metrics = new MetricsZooKeeper();
     try {checkZk();} catch (Exception x) {/* ignore */}
     salter = new Random();
   }
@@ -175,9 +178,12 @@ public class RecoverableZooKeeper {
       boolean isRetry = false; // False for first attempt, true for all retries.
       while (true) {
         try {
+          long startTime = EnvironmentEdgeManager.currentTime();
           checkZk().delete(path, version);
+          this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
           return;
         } catch (KeeperException e) {
+          this.metrics.registerFailedZKCall();
           switch (e.code()) {
             case NONODE:
               if (isRetry) {
@@ -189,7 +195,11 @@ public class RecoverableZooKeeper {
               throw e;
 
             case CONNECTIONLOSS:
+              this.metrics.registerConnectionLossException();
+              retryOrThrow(retryCounter, e, "delete");
+              break;
             case OPERATIONTIMEOUT:
+              this.metrics.registerOperationTimeoutException();
               retryOrThrow(retryCounter, e, "delete");
               break;
 
@@ -217,11 +227,19 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
-          return checkZk().exists(path, watcher);
+          long startTime = EnvironmentEdgeManager.currentTime();
+          Stat nodeStat = checkZk().exists(path, watcher);
+          this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
+          return nodeStat;
         } catch (KeeperException e) {
+          this.metrics.registerFailedZKCall();
           switch (e.code()) {
             case CONNECTIONLOSS:
+              this.metrics.registerConnectionLossException();
+              retryOrThrow(retryCounter, e, "exists");
+              break;
             case OPERATIONTIMEOUT:
+              this.metrics.registerOperationTimeoutException();
               retryOrThrow(retryCounter, e, "exists");
               break;
 
@@ -248,11 +266,19 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
-          return checkZk().exists(path, watch);
+          long startTime = EnvironmentEdgeManager.currentTime();
+          Stat nodeStat = checkZk().exists(path, watch);
+          this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
+          return nodeStat;
         } catch (KeeperException e) {
+          this.metrics.registerFailedZKCall();
           switch (e.code()) {
             case CONNECTIONLOSS:
+              this.metrics.registerConnectionLossException();
+              retryOrThrow(retryCounter, e, "exists");
+              break;
             case OPERATIONTIMEOUT:
+              this.metrics.registerOperationTimeoutException();
               retryOrThrow(retryCounter, e, "exists");
               break;
 
@@ -289,11 +315,19 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
-          return checkZk().getChildren(path, watcher);
+          long startTime = EnvironmentEdgeManager.currentTime();
+          List<String> children = checkZk().getChildren(path, watcher);
+          this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
+          return children;
         } catch (KeeperException e) {
+          this.metrics.registerFailedZKCall();
           switch (e.code()) {
             case CONNECTIONLOSS:
+              this.metrics.registerConnectionLossException();
+              retryOrThrow(retryCounter, e, "getChildren");
+              break;
             case OPERATIONTIMEOUT:
+              this.metrics.registerOperationTimeoutException();
               retryOrThrow(retryCounter, e, "getChildren");
               break;
 
@@ -320,11 +354,19 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
-          return checkZk().getChildren(path, watch);
+          long startTime = EnvironmentEdgeManager.currentTime();
+          List<String> children = checkZk().getChildren(path, watch);
+          this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
+          return children;
         } catch (KeeperException e) {
+          this.metrics.registerFailedZKCall();
           switch (e.code()) {
             case CONNECTIONLOSS:
+              this.metrics.registerConnectionLossException();
+              retryOrThrow(retryCounter, e, "getChildren");
+              break;
             case OPERATIONTIMEOUT:
+              this.metrics.registerOperationTimeoutException();
               retryOrThrow(retryCounter, e, "getChildren");
               break;
 
@@ -351,12 +393,19 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
+          long startTime = EnvironmentEdgeManager.currentTime();
           byte[] revData = checkZk().getData(path, watcher, stat);
+          this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
           return removeMetaData(revData);
         } catch (KeeperException e) {
+          this.metrics.registerFailedZKCall();
           switch (e.code()) {
             case CONNECTIONLOSS:
+              this.metrics.registerConnectionLossException();
+              retryOrThrow(retryCounter, e, "getData");
+              break;
             case OPERATIONTIMEOUT:
+              this.metrics.registerOperationTimeoutException();
               retryOrThrow(retryCounter, e, "getData");
               break;
 
@@ -372,7 +421,7 @@ public class RecoverableZooKeeper {
   }
 
   /**
-   * getData is an idemnpotent operation. Retry before throwing exception
+   * getData is an idempotent operation. Retry before throwing exception
    * @return Data
    */
   public byte[] getData(String path, boolean watch, Stat stat)
@@ -383,12 +432,19 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
+          long startTime = EnvironmentEdgeManager.currentTime();
           byte[] revData = checkZk().getData(path, watch, stat);
+          this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
           return removeMetaData(revData);
         } catch (KeeperException e) {
+          this.metrics.registerFailedZKCall();
           switch (e.code()) {
             case CONNECTIONLOSS:
+              this.metrics.registerConnectionLossException();
+              retryOrThrow(retryCounter, e, "getData");
+              break;
             case OPERATIONTIMEOUT:
+              this.metrics.registerOperationTimeoutException();
               retryOrThrow(retryCounter, e, "getData");
               break;
 
@@ -417,13 +473,22 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       byte[] newData = appendMetaData(data);
       boolean isRetry = false;
+      long startTime;
       while (true) {
         try {
-          return checkZk().setData(path, newData, version);
+          startTime = EnvironmentEdgeManager.currentTime();
+          Stat nodeStat = checkZk().setData(path, newData, version);
+          this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
+          return nodeStat;
         } catch (KeeperException e) {
+          this.metrics.registerFailedZKCall();
           switch (e.code()) {
             case CONNECTIONLOSS:
+              this.metrics.registerConnectionLossException();
+              retryOrThrow(retryCounter, e, "setData");
+              break;
             case OPERATIONTIMEOUT:
+              this.metrics.registerOperationTimeoutException();
               retryOrThrow(retryCounter, e, "setData");
               break;
             case BADVERSION:
@@ -431,12 +496,15 @@ public class RecoverableZooKeeper {
                 // try to verify whether the previous setData success or not
                 try{
                   Stat stat = new Stat();
+                  startTime = EnvironmentEdgeManager.currentTime();
                   byte[] revData = checkZk().getData(path, false, stat);
+                  this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
                   if(Bytes.compareTo(revData, newData) == 0) {
                     // the bad version is caused by previous successful setData
                     return stat;
                   }
                 } catch(KeeperException keeperException){
+                  this.metrics.registerFailedZKCall();
                   // the ZK is not reliable at this moment. just throwing exception
                   throw keeperException;
                 }
@@ -466,11 +534,19 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
-          return checkZk().getACL(path, stat);
+          long startTime = EnvironmentEdgeManager.currentTime();
+          List<ACL> nodeACL = checkZk().getACL(path, stat);
+          this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
+          return nodeACL;
         } catch (KeeperException e) {
+          this.metrics.registerFailedZKCall();
           switch (e.code()) {
             case CONNECTIONLOSS:
+              this.metrics.registerConnectionLossException();
+              retryOrThrow(retryCounter, e, "getAcl");
+              break;
             case OPERATIONTIMEOUT:
+              this.metrics.registerOperationTimeoutException();
               retryOrThrow(retryCounter, e, "getAcl");
               break;
 
@@ -497,11 +573,19 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
-          return checkZk().setACL(path, acls, version);
+          long startTime = EnvironmentEdgeManager.currentTime();
+          Stat nodeStat = checkZk().setACL(path, acls, version);
+          this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
+          return nodeStat;
         } catch (KeeperException e) {
+          this.metrics.registerFailedZKCall();
           switch (e.code()) {
             case CONNECTIONLOSS:
+              this.metrics.registerConnectionLossException();
+              retryOrThrow(retryCounter, e, "setAcl");
+              break;
             case OPERATIONTIMEOUT:
+              this.metrics.registerOperationTimeoutException();
               retryOrThrow(retryCounter, e, "setAcl");
               break;
 
@@ -560,17 +644,24 @@ public class RecoverableZooKeeper {
       CreateMode createMode) throws KeeperException, InterruptedException {
     RetryCounter retryCounter = retryCounterFactory.create();
     boolean isRetry = false; // False for first attempt, true for all retries.
+    long startTime;
     while (true) {
       try {
-        return checkZk().create(path, data, acl, createMode);
+        startTime = EnvironmentEdgeManager.currentTime();
+        String nodePath = checkZk().create(path, data, acl, createMode);
+        this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
+        return nodePath;
       } catch (KeeperException e) {
+        this.metrics.registerFailedZKCall();
         switch (e.code()) {
           case NODEEXISTS:
             if (isRetry) {
               // If the connection was lost, there is still a possibility that
               // we have successfully created the node at our previous attempt,
               // so we read the node and compare.
+              startTime = EnvironmentEdgeManager.currentTime();
               byte[] currentData = checkZk().getData(path, false, null);
+              this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
               if (currentData != null &&
                   Bytes.compareTo(currentData, data) == 0) {
                 // We successfully created a non-sequential node
@@ -585,7 +676,11 @@ public class RecoverableZooKeeper {
             throw e;
 
           case CONNECTIONLOSS:
+            this.metrics.registerConnectionLossException();
+            retryOrThrow(retryCounter, e, "create");
+            break;
           case OPERATIONTIMEOUT:
+            this.metrics.registerOperationTimeoutException();
             retryOrThrow(retryCounter, e, "create");
             break;
 
@@ -614,11 +709,19 @@ public class RecoverableZooKeeper {
           }
         }
         first = false;
-        return checkZk().create(newPath, data, acl, createMode);
+        long startTime = EnvironmentEdgeManager.currentTime();
+        String nodePath = checkZk().create(newPath, data, acl, createMode);
+        this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
+        return nodePath;
       } catch (KeeperException e) {
+        this.metrics.registerFailedZKCall();
         switch (e.code()) {
           case CONNECTIONLOSS:
+            this.metrics.registerConnectionLossException();
+            retryOrThrow(retryCounter, e, "create");
+            break;
           case OPERATIONTIMEOUT:
+            this.metrics.registerOperationTimeoutException();
             retryOrThrow(retryCounter, e, "create");
             break;
 
@@ -669,11 +772,19 @@ public class RecoverableZooKeeper {
       Iterable<Op> multiOps = prepareZKMulti(ops);
       while (true) {
         try {
-          return checkZk().multi(multiOps);
+          long startTime = EnvironmentEdgeManager.currentTime();
+          List<OpResult> opResults = checkZk().multi(multiOps);
+          this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
+          return opResults;
         } catch (KeeperException e) {
+          this.metrics.registerFailedZKCall();
           switch (e.code()) {
             case CONNECTIONLOSS:
+              this.metrics.registerConnectionLossException();
+              retryOrThrow(retryCounter, e, "multi");
+              break;
             case OPERATIONTIMEOUT:
+              this.metrics.registerOperationTimeoutException();
               retryOrThrow(retryCounter, e, "multi");
               break;
 
@@ -682,7 +793,7 @@ public class RecoverableZooKeeper {
           }
         }
         retryCounter.sleepUntilNextRetry();
-    }
+      }
     } finally {
       if (traceScope != null) traceScope.close();
     }
@@ -694,12 +805,15 @@ public class RecoverableZooKeeper {
     assert(lastSlashIdx != -1);
     String parent = path.substring(0, lastSlashIdx);
     String nodePrefix = path.substring(lastSlashIdx+1);
-
+    long startTime = EnvironmentEdgeManager.currentTime();
     List<String> nodes = checkZk().getChildren(parent, false);
+    this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
     List<String> matching = filterByPrefix(nodes, nodePrefix);
     for (String node : matching) {
       String nodePath = parent + "/" + node;
+      startTime = EnvironmentEdgeManager.currentTime();
       Stat stat = checkZk().exists(nodePath, false);
+      this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
       if (stat != null) {
         return nodePath;
       }
@@ -763,7 +877,9 @@ public class RecoverableZooKeeper {
   }
 
   public void sync(String path, AsyncCallback.VoidCallback cb, Object ctx) throws KeeperException {
+    long startTime = EnvironmentEdgeManager.currentTime();
     checkZk().sync(path, cb, null);
+    this.metrics.registerSyncOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
   }
 
   /**
@@ -792,4 +908,4 @@ public class RecoverableZooKeeper {
   public String getIdentifier() {
     return identifier;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb6ded48/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMetricsListener.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMetricsListener.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMetricsListener.java
new file mode 100644
index 0000000..222affc
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMetricsListener.java
@@ -0,0 +1,91 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.zookeeper;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public interface ZooKeeperMetricsListener {
+
+  /**
+   * An AUTHFAILED Exception was seen.
+   */
+  void registerAuthFailedException();
+
+  /**
+   * A CONNECTIONLOSS Exception was seen.
+   */
+  void registerConnectionLossException();
+
+  /**
+   * A DATAINCONSISTENCY Exception was seen.
+   */
+  void registerDataInconsistencyException();
+
+  /**
+   * An INVALIDACL Exception was seen.
+   */
+  void registerInvalidACLException();
+
+  /**
+   * A NOAUTH Exception was seen.
+   */
+  void registerNoAuthException();
+
+  /**
+   * A OPERATIONTIMEOUT Exception was seen.
+   */
+  void registerOperationTimeoutException();
+
+  /**
+   * A RUNTIMEINCONSISTENCY Exception was seen.
+   */
+  void registerRuntimeInconsistencyException();
+
+  /**
+   * A SESSIONEXPIRED Exception was seen.
+   */
+  void registerSessionExpiredException();
+
+  /**
+   * A SYSTEMERROR Exception was seen.
+   */
+  void registerSystemErrorException();
+
+  /**
+   * A ZooKeeper API Call failed.
+   */
+  void registerFailedZKCall();
+
+  /**
+   * Register the latency incurred for read operations.
+   */
+  void registerReadOperationLatency(long latency);
+
+  /**
+   * Register the latency incurred for write operations.
+   */
+  void registerWriteOperationLatency(long latency);
+
+  /**
+   * Register the latency incurred for sync operations.
+   */
+  void registerSyncOperationLatency(long latency);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb6ded48/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeper.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeper.java
new file mode 100644
index 0000000..9d7a495
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeper.java
@@ -0,0 +1,77 @@
+/**
+ *
+ * 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.zookeeper;
+
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import static org.mockito.Mockito.*;
+
+@Category(SmallTests.class)
+public class TestMetricsZooKeeper {
+
+  @Test
+  public void testRegisterExceptions() {
+    MetricsZooKeeperSource zkSource = mock(MetricsZooKeeperSourceImpl.class);
+    MetricsZooKeeper metricsZK = new MetricsZooKeeper(zkSource);
+    metricsZK.registerAuthFailedException();
+    metricsZK.registerConnectionLossException();
+    metricsZK.registerConnectionLossException();
+    metricsZK.registerDataInconsistencyException();
+    metricsZK.registerInvalidACLException();
+    metricsZK.registerNoAuthException();
+    metricsZK.registerOperationTimeoutException();
+    metricsZK.registerOperationTimeoutException();
+    metricsZK.registerRuntimeInconsistencyException();
+    metricsZK.registerSessionExpiredException();
+    metricsZK.registerSystemErrorException();
+    metricsZK.registerSystemErrorException();
+    metricsZK.registerFailedZKCall();
+
+    verify(zkSource, times(1)).incrementAuthFailedCount();
+    // ConnectionLoss Exception was registered twice.
+    verify(zkSource, times(2)).incrementConnectionLossCount();
+    verify(zkSource, times(1)).incrementDataInconsistencyCount();
+    verify(zkSource, times(1)).incrementInvalidACLCount();
+    verify(zkSource, times(1)).incrementNoAuthCount();
+    // OperationTimeout Exception was registered twice.
+    verify(zkSource, times(2)).incrementOperationTimeoutCount();
+    verify(zkSource, times(1)).incrementRuntimeInconsistencyCount();
+    verify(zkSource, times(1)).incrementSessionExpiredCount();
+    // SystemError Exception was registered twice.
+    verify(zkSource, times(2)).incrementSystemErrorCount();
+    verify(zkSource, times(1)).incrementTotalFailedZKCalls();
+  }
+
+  @Test
+  public void testLatencyHistogramUpdates() {
+    MetricsZooKeeperSource zkSource = mock(MetricsZooKeeperSourceImpl.class);
+    MetricsZooKeeper metricsZK = new MetricsZooKeeper(zkSource);
+    long latency = 100;
+
+    metricsZK.registerReadOperationLatency(latency);
+    metricsZK.registerReadOperationLatency(latency);
+    metricsZK.registerWriteOperationLatency(latency);
+    metricsZK.registerSyncOperationLatency(latency);
+    // Read Operation Latency update was registered twice.
+    verify(zkSource, times(2)).recordReadOperationLatency(latency);
+    verify(zkSource, times(1)).recordWriteOperationLatency(latency);
+    verify(zkSource, times(1)).recordSyncOperationLatency(latency);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb6ded48/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeperSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeperSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeperSource.java
new file mode 100644
index 0000000..0b54b7b
--- /dev/null
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeperSource.java
@@ -0,0 +1,139 @@
+/**
+ * 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.zookeeper;
+
+import org.apache.hadoop.hbase.metrics.BaseSource;
+
+/**
+ * Interface of the source that will export metrics about the ZooKeeper.
+ */
+public interface MetricsZooKeeperSource extends BaseSource {
+
+  /**
+   * The name of the metrics
+   */
+  String METRICS_NAME = "ZOOKEEPER";
+
+  /**
+   * The name of the metrics context that metrics will be under.
+   */
+  String METRICS_CONTEXT = "zookeeper";
+
+  /**
+   * Description
+   */
+  String METRICS_DESCRIPTION = "Metrics about ZooKeeper";
+
+  /**
+   * The name of the metrics context that metrics will be under in jmx.
+   */
+  String METRICS_JMX_CONTEXT = "ZooKeeper,sub=" + METRICS_NAME;
+
+  String EXCEPTION_AUTHFAILED = "AUTHFAILED Exception";
+  String EXCEPTION_AUTHFAILED_DESC = "Number of failed ops due to an AUTHFAILED exception,";
+  String EXCEPTION_CONNECTIONLOSS = "CONNECTIONLOSS Exception";
+  String EXCEPTION_CONNECTIONLOSS_DESC = "Number of failed ops due to a CONNECTIONLOSS exception.";
+  String EXCEPTION_DATAINCONSISTENCY = "DATAINCONSISTENCY Exception";
+  String EXCEPTION_DATAINCONSISTENCY_DESC = "Number of failed ops due to a DATAINCONSISTENCY exception.";
+  String EXCEPTION_INVALIDACL = "INVALIDACL Exception";
+  String EXCEPTION_INVALIDACL_DESC = "Number of failed ops due to an INVALIDACL exception";
+  String EXCEPTION_NOAUTH = "NOAUTH Exception";
+  String EXCEPTION_NOAUTH_DESC = "Number of failed ops due to a NOAUTH exception.";
+  String EXCEPTION_OPERATIONTIMEOUT = "OPERATIONTIMEOUT Exception";
+  String EXCEPTION_OPERATIONTIMEOUT_DESC = "Number of failed ops due to an OPERATIONTIMEOUT exception.";
+  String EXCEPTION_RUNTIMEINCONSISTENCY = "RUNTIMEINCONSISTENCY Exception";
+  String EXCEPTION_RUNTIMEINCONSISTENCY_DESC = "Number of failed ops due to a RUNTIMEINCONSISTENCY exception.";
+  String EXCEPTION_SESSIONEXPIRED = "SESSIONEXPIRED Exception";
+  String EXCEPTION_SESSIONEXPIRED_DESC = "Number of failed ops due to a SESSIONEXPIRED exception.";
+  String EXCEPTION_SYSTEMERROR = "SYSTEMERROR Exception";
+  String EXCEPTION_SYSTEMERROR_DESC = "Number of failed ops due to a SYSTEMERROR exception.";
+  String TOTAL_FAILED_ZK_CALLS = "TotalFailedZKCalls";
+  String TOTAL_FAILED_ZK_CALLS_DESC = "Total number of failed ZooKeeper API Calls";
+
+  String READ_OPERATION_LATENCY_NAME = "ReadOperationLatency";
+  String READ_OPERATION_LATENCY_DESC = "Latency histogram for read operations.";
+  String WRITE_OPERATION_LATENCY_NAME = "WriteOperationLatency";
+  String WRITE_OPERATION_LATENCY_DESC = "Latency histogram for write operations.";
+  String SYNC_OPERATION_LATENCY_NAME = "SyncOperationLatency";
+  String SYNC_OPERATION_LATENCY_DESC = "Latency histogram for sync operations.";
+
+  /**
+   * Increment the count of failed ops due to AUTHFAILED Exception.
+   */
+  void incrementAuthFailedCount();
+
+  /**
+   * Increment the count of failed ops due to a CONNECTIONLOSS Exception.
+   */
+  void incrementConnectionLossCount();
+
+  /**
+   * Increment the count of failed ops due to a DATAINCONSISTENCY Exception.
+   */
+  void incrementDataInconsistencyCount();
+
+  /**
+   * Increment the count of failed ops due to INVALIDACL Exception.
+   */
+  void incrementInvalidACLCount();
+
+  /**
+   * Increment the count of failed ops due to NOAUTH Exception.
+   */
+  void incrementNoAuthCount();
+
+  /**
+   * Increment the count of failed ops due to an OPERATIONTIMEOUT Exception.
+   */
+  void incrementOperationTimeoutCount();
+
+  /**
+   * Increment the count of failed ops due to RUNTIMEINCONSISTENCY Exception.
+   */
+  void incrementRuntimeInconsistencyCount();
+
+  /**
+   * Increment the count of failed ops due to a SESSIONEXPIRED Exception.
+   */
+  void incrementSessionExpiredCount();
+
+  /**
+   * Increment the count of failed ops due to a SYSTEMERROR Exception.
+   */
+  void incrementSystemErrorCount();
+
+  /**
+   * Record the latency incurred for read operations.
+   */
+  void recordReadOperationLatency(long latency);
+
+  /**
+   * Record the latency incurred for write operations.
+   */
+  void recordWriteOperationLatency(long latency);
+
+  /**
+   * Record the latency incurred for sync operations.
+   */
+  void recordSyncOperationLatency(long latency);
+
+  /**
+   * Record the total number of failed ZooKeeper API calls.
+   */
+  void incrementTotalFailedZKCalls();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb6ded48/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeperSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeperSource.java b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeperSource.java
new file mode 100644
index 0000000..ba45f0a
--- /dev/null
+++ b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeperSource.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.zookeeper;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.hbase.testclassification.MetricsTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MetricsTests.class, SmallTests.class})
+public class TestMetricsZooKeeperSource {
+
+  @Test(expected=RuntimeException.class)
+  public void testGetInstanceNoHadoopCompat() throws Exception {
+    //This should throw an exception because there is no compat lib on the class path.
+    CompatibilitySingletonFactory.getInstance(MetricsZooKeeperSource.class);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb6ded48/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeperSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeperSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeperSourceImpl.java
new file mode 100644
index 0000000..13ab130
--- /dev/null
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeperSourceImpl.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.zookeeper;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
+import org.apache.hadoop.metrics2.lib.MutableHistogram;
+
+/**
+ * Class that transitions metrics from MetricsZooKeeper into the metrics subsystem.
+ *
+ * Implements BaseSource through BaseSourceImpl, following the pattern.
+ */
+@InterfaceAudience.Private
+public class MetricsZooKeeperSourceImpl extends BaseSourceImpl implements MetricsZooKeeperSource {
+
+  private final MutableGaugeLong authFailedFailedOpCount;
+  private final MutableGaugeLong connectionLossFailedOpCount;
+  private final MutableGaugeLong dataInconsistencyFailedOpCount;
+  private final MutableGaugeLong invalidACLFailedOpCount;
+  private final MutableGaugeLong noAuthFailedOpCount;
+  private final MutableGaugeLong operationTimeOutFailedOpCount;
+  private final MutableGaugeLong runtimeInconsistencyFailedOpCount;
+  private final MutableGaugeLong sessionExpiredFailedOpCount;
+  private final MutableGaugeLong systemErrorFailedOpCount;
+  private final MutableGaugeLong totalFailedZKCalls;
+
+  private MutableHistogram readOpLatency;
+  private MutableHistogram writeOpLatency;
+  private MutableHistogram syncOpLatency;
+
+  public MetricsZooKeeperSourceImpl() {
+    this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);
+  }
+
+  public MetricsZooKeeperSourceImpl(String metricsName, String metricsDescription, String metricsContext,
+                                    String metricsJmxContext) {
+    super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
+
+    //Create and store the metrics that will be used.
+    authFailedFailedOpCount = this.getMetricsRegistry().newGauge(EXCEPTION_AUTHFAILED, EXCEPTION_AUTHFAILED_DESC, 0L);
+    connectionLossFailedOpCount = this.getMetricsRegistry().newGauge(EXCEPTION_CONNECTIONLOSS, EXCEPTION_CONNECTIONLOSS_DESC, 0L);
+    dataInconsistencyFailedOpCount = this.getMetricsRegistry().newGauge(EXCEPTION_DATAINCONSISTENCY, EXCEPTION_DATAINCONSISTENCY_DESC, 0L);
+    invalidACLFailedOpCount = this.getMetricsRegistry().newGauge(EXCEPTION_INVALIDACL, EXCEPTION_INVALIDACL_DESC, 0L);
+    noAuthFailedOpCount = this.getMetricsRegistry().newGauge(EXCEPTION_NOAUTH, EXCEPTION_NOAUTH_DESC, 0L);
+    operationTimeOutFailedOpCount = this.getMetricsRegistry().newGauge(EXCEPTION_OPERATIONTIMEOUT, EXCEPTION_OPERATIONTIMEOUT_DESC, 0L);
+    runtimeInconsistencyFailedOpCount = this.getMetricsRegistry().newGauge(EXCEPTION_RUNTIMEINCONSISTENCY, EXCEPTION_RUNTIMEINCONSISTENCY_DESC, 0L);
+    sessionExpiredFailedOpCount = this.getMetricsRegistry().newGauge(EXCEPTION_SESSIONEXPIRED, EXCEPTION_SESSIONEXPIRED_DESC, 0L);
+    systemErrorFailedOpCount = this.getMetricsRegistry().newGauge(EXCEPTION_SYSTEMERROR, EXCEPTION_SYSTEMERROR_DESC, 0L);
+    totalFailedZKCalls = this.getMetricsRegistry().newGauge(TOTAL_FAILED_ZK_CALLS, TOTAL_FAILED_ZK_CALLS_DESC, 0L);
+
+    readOpLatency = this.getMetricsRegistry().newHistogram(READ_OPERATION_LATENCY_NAME, READ_OPERATION_LATENCY_DESC);
+    writeOpLatency = this.getMetricsRegistry().newHistogram(WRITE_OPERATION_LATENCY_NAME, WRITE_OPERATION_LATENCY_DESC);
+    syncOpLatency = this.getMetricsRegistry().newHistogram(SYNC_OPERATION_LATENCY_NAME, SYNC_OPERATION_LATENCY_DESC);
+  }
+
+  public void getMetrics(MetricsCollector metricsCollector, boolean all) {
+    super.getMetrics(metricsCollector, all);
+    clearZKExceptionMetrics();
+  }
+
+  private void clearZKExceptionMetrics() {
+    //Reset the exception metrics.
+    clearMetricIfNotNull(authFailedFailedOpCount);
+    clearMetricIfNotNull(connectionLossFailedOpCount);
+    clearMetricIfNotNull(dataInconsistencyFailedOpCount);
+    clearMetricIfNotNull(invalidACLFailedOpCount);
+    clearMetricIfNotNull(noAuthFailedOpCount);
+    clearMetricIfNotNull(operationTimeOutFailedOpCount);
+    clearMetricIfNotNull(runtimeInconsistencyFailedOpCount);
+    clearMetricIfNotNull(sessionExpiredFailedOpCount);
+    clearMetricIfNotNull(systemErrorFailedOpCount);
+    clearMetricIfNotNull(totalFailedZKCalls);
+  }
+
+  private static void clearMetricIfNotNull(MutableGaugeLong metric) {
+    if (metric != null) {
+      metric.set(0L);
+    }
+  }
+
+  @Override
+  public void incrementAuthFailedCount() {
+    authFailedFailedOpCount.incr();
+  }
+
+  @Override
+  public void incrementConnectionLossCount() {
+    connectionLossFailedOpCount.incr();
+  }
+
+  @Override
+  public void incrementDataInconsistencyCount() {
+    dataInconsistencyFailedOpCount.incr();
+  }
+
+  @Override
+  public void incrementInvalidACLCount() {
+    invalidACLFailedOpCount.incr();
+  }
+
+  @Override
+  public void incrementNoAuthCount() {
+    noAuthFailedOpCount.incr();
+  }
+
+  @Override
+  public void incrementOperationTimeoutCount() {
+    operationTimeOutFailedOpCount.incr();
+  }
+
+  @Override
+  public void incrementRuntimeInconsistencyCount() {
+    runtimeInconsistencyFailedOpCount.incr();
+  }
+
+  @Override
+  public void incrementSessionExpiredCount() {
+    sessionExpiredFailedOpCount.incr();
+  }
+
+  @Override
+  public void incrementSystemErrorCount() {
+    systemErrorFailedOpCount.incr();
+  }
+
+  @Override
+  public void incrementTotalFailedZKCalls() {
+    totalFailedZKCalls.incr();
+  }
+
+  @Override
+  public void recordReadOperationLatency(long latency) {
+    readOpLatency.add(latency);
+  }
+
+  @Override
+  public void recordWriteOperationLatency(long latency) {
+    writeOpLatency.add(latency);
+  }
+
+  @Override
+  public void recordSyncOperationLatency(long latency) {
+    syncOpLatency.add(latency);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb6ded48/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.zookeeper.MetricsZooKeeperSource
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.zookeeper.MetricsZooKeeperSource b/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.zookeeper.MetricsZooKeeperSource
new file mode 100644
index 0000000..43f9a22
--- /dev/null
+++ b/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.zookeeper.MetricsZooKeeperSource
@@ -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.zookeeper.MetricsZooKeeperSourceImpl

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb6ded48/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeperSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeperSourceImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeperSourceImpl.java
new file mode 100644
index 0000000..ce1a62e
--- /dev/null
+++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeperSourceImpl.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.zookeeper;
+
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.hbase.testclassification.MetricsTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MetricsTests.class, SmallTests.class})
+public class TestMetricsZooKeeperSourceImpl {
+
+  @Test
+  public void testGetInstance() throws Exception {
+    MetricsZooKeeperSource zkSource = CompatibilitySingletonFactory.getInstance(MetricsZooKeeperSource.class);
+    assertTrue(zkSource instanceof MetricsZooKeeperSourceImpl);
+    assertSame(zkSource, CompatibilitySingletonFactory.getInstance(MetricsZooKeeperSource.class));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb6ded48/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
index 8278d0c..f0be2ce 100644
--- 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
@@ -20,6 +20,8 @@ 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.
@@ -32,5 +34,6 @@ import java.lang.annotation.RetentionPolicy;
  *   maven-compiler-plugin: incremental compilation broken</a>
  */
 @Retention(RetentionPolicy.SOURCE)
+@InterfaceAudience.Private
 public @interface PackageMarker {
 }