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 2018/03/02 00:29:20 UTC

[2/2] hbase git commit: HBASE-19769 IllegalAccessError on package-private Hadoop metrics2 classes in MapReduce jobs

HBASE-19769 IllegalAccessError on package-private Hadoop metrics2 classes in MapReduce jobs

Client-side ZooKeeper metrics cause issues when launching MapReduce
jobs via 'yarn jar' on the command line. This stems from ClassLoader
separation issues that YARN implements. It was chosen that the
easiest solution was to remove these ZooKeeper metrics entirely.

Revert "HBASE-17448 Export metrics from RecoverableZooKeeper"

This reverts commit defc25c6d109c4c8782477cc517d71206e7d3aca


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/f38f390f
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/f38f390f
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/f38f390f

Branch: refs/heads/branch-1.4
Commit: f38f390f0e85f1bf842c5730e587aca5bab58834
Parents: 87654d2
Author: Andrew Purtell <ap...@apache.org>
Authored: Thu Mar 1 15:28:39 2018 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Thu Mar 1 15:30:53 2018 -0800

----------------------------------------------------------------------
 hbase-client/pom.xml                            |   8 -
 .../hbase/zookeeper/MetricsZooKeeper.java       | 110 -------------
 .../hbase/zookeeper/RecoverableZooKeeper.java   | 146 ++---------------
 .../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 -----
 10 files changed, 15 insertions(+), 810 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f38f390f/hbase-client/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml
index a30db8a..2c16140 100644
--- a/hbase-client/pom.xml
+++ b/hbase-client/pom.xml
@@ -128,14 +128,6 @@
     </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/f38f390f/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
deleted file mode 100644
index 6b5e188..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeper.java
+++ /dev/null
@@ -1,110 +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.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/f38f390f/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 75cee0a..36fb5f9 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,7 +31,6 @@ 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;
@@ -84,7 +83,6 @@ 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:
@@ -127,7 +125,6 @@ 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();
   }
@@ -178,12 +175,9 @@ 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) {
@@ -195,11 +189,7 @@ 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;
 
@@ -227,19 +217,11 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
-          long startTime = EnvironmentEdgeManager.currentTime();
-          Stat nodeStat = checkZk().exists(path, watcher);
-          this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime, 1));
-          return nodeStat;
+          return checkZk().exists(path, watcher);
         } 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;
 
@@ -266,19 +248,11 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
-          long startTime = EnvironmentEdgeManager.currentTime();
-          Stat nodeStat = checkZk().exists(path, watch);
-          this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
-          return nodeStat;
+          return checkZk().exists(path, watch);
         } 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;
 
@@ -315,19 +289,11 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
-          long startTime = EnvironmentEdgeManager.currentTime();
-          List<String> children = checkZk().getChildren(path, watcher);
-          this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
-          return children;
+          return checkZk().getChildren(path, watcher);
         } 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;
 
@@ -354,19 +320,11 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
-          long startTime = EnvironmentEdgeManager.currentTime();
-          List<String> children = checkZk().getChildren(path, watch);
-          this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
-          return children;
+          return checkZk().getChildren(path, watch);
         } 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;
 
@@ -393,19 +351,12 @@ 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);
+          return this.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;
 
@@ -421,7 +372,7 @@ public class RecoverableZooKeeper {
   }
 
   /**
-   * getData is an idempotent operation. Retry before throwing exception
+   * getData is an idemnpotent operation. Retry before throwing exception
    * @return Data
    */
   public byte[] getData(String path, boolean watch, Stat stat)
@@ -432,19 +383,12 @@ 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);
+          return this.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;
 
@@ -473,22 +417,13 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       byte[] newData = appendMetaData(data);
       boolean isRetry = false;
-      long startTime;
       while (true) {
         try {
-          startTime = EnvironmentEdgeManager.currentTime();
-          Stat nodeStat = checkZk().setData(path, newData, version);
-          this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
-          return nodeStat;
+          return checkZk().setData(path, newData, version);
         } 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:
@@ -496,15 +431,12 @@ 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;
                 }
@@ -534,19 +466,11 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
-          long startTime = EnvironmentEdgeManager.currentTime();
-          List<ACL> nodeACL = checkZk().getACL(path, stat);
-          this.metrics.registerReadOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
-          return nodeACL;
+          return checkZk().getACL(path, stat);
         } 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;
 
@@ -573,19 +497,11 @@ public class RecoverableZooKeeper {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
-          long startTime = EnvironmentEdgeManager.currentTime();
-          Stat nodeStat = checkZk().setACL(path, acls, version);
-          this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
-          return nodeStat;
+          return checkZk().setACL(path, acls, version);
         } 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;
 
@@ -644,24 +560,17 @@ 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 {
-        startTime = EnvironmentEdgeManager.currentTime();
-        String nodePath = checkZk().create(path, data, acl, createMode);
-        this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
-        return nodePath;
+        return checkZk().create(path, data, acl, createMode);
       } 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
@@ -676,11 +585,7 @@ 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;
 
@@ -709,19 +614,11 @@ public class RecoverableZooKeeper {
           }
         }
         first = false;
-        long startTime = EnvironmentEdgeManager.currentTime();
-        String nodePath = checkZk().create(newPath, data, acl, createMode);
-        this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
-        return nodePath;
+        return checkZk().create(newPath, data, acl, createMode);
       } 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;
 
@@ -772,19 +669,11 @@ public class RecoverableZooKeeper {
       Iterable<Op> multiOps = prepareZKMulti(ops);
       while (true) {
         try {
-          long startTime = EnvironmentEdgeManager.currentTime();
-          List<OpResult> opResults = checkZk().multi(multiOps);
-          this.metrics.registerWriteOperationLatency(Math.min(EnvironmentEdgeManager.currentTime() - startTime,  1));
-          return opResults;
+          return checkZk().multi(multiOps);
         } 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;
 
@@ -793,7 +682,7 @@ public class RecoverableZooKeeper {
           }
         }
         retryCounter.sleepUntilNextRetry();
-      }
+    }
     } finally {
       if (traceScope != null) traceScope.close();
     }
@@ -805,15 +694,12 @@ 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;
       }
@@ -877,9 +763,7 @@ 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));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/f38f390f/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
deleted file mode 100644
index 222affc..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMetricsListener.java
+++ /dev/null
@@ -1,91 +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.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/f38f390f/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
deleted file mode 100644
index 9d7a495..0000000
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeper.java
+++ /dev/null
@@ -1,77 +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.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/f38f390f/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
deleted file mode 100644
index 0b54b7b..0000000
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeperSource.java
+++ /dev/null
@@ -1,139 +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.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/f38f390f/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
deleted file mode 100644
index ba45f0a..0000000
--- a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeperSource.java
+++ /dev/null
@@ -1,34 +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.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/f38f390f/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
deleted file mode 100644
index 13ab130..0000000
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeperSourceImpl.java
+++ /dev/null
@@ -1,164 +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.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/f38f390f/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
deleted file mode 100644
index 43f9a22..0000000
--- a/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.zookeeper.MetricsZooKeeperSource
+++ /dev/null
@@ -1,18 +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.
-#
-org.apache.hadoop.hbase.zookeeper.MetricsZooKeeperSourceImpl

http://git-wip-us.apache.org/repos/asf/hbase/blob/f38f390f/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
deleted file mode 100644
index ce1a62e..0000000
--- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeperSourceImpl.java
+++ /dev/null
@@ -1,38 +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.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