http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
index 814fe07..3347e1c 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
@@ -26,6 +26,7 @@ import com.google.protobuf.ByteString;
 import com.google.protobuf.ServiceException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
@@ -37,7 +38,6 @@ import 
org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.Job;
@@ -289,7 +289,7 @@ public class TokenUtil {
    */
   private static Token<AuthenticationTokenIdentifier> 
getAuthToken(Configuration conf, User user)
       throws IOException, InterruptedException {
-    ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, 
"TokenUtil-getAuthToken", null);
+    ZKWatcher zkw = new ZKWatcher(conf, "TokenUtil-getAuthToken", null);
     try {
       String clusterId = ZKClusterId.readClusterIdZNode(zkw);
       if (clusterId == null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java
index 9f0da78..96502fd 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java
@@ -25,20 +25,20 @@ import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 
 /**
  * Synchronizes token encryption keys across cluster nodes.
  */
 @InterfaceAudience.Private
-public class ZKSecretWatcher extends ZooKeeperListener {
+public class ZKSecretWatcher extends ZKListener {
   private static final String DEFAULT_ROOT_NODE = "tokenauth";
   private static final String DEFAULT_KEYS_PARENT = "keys";
   private static final Log LOG = LogFactory.getLog(ZKSecretWatcher.class);
@@ -48,7 +48,7 @@ public class ZKSecretWatcher extends ZooKeeperListener {
   private String keysParentZNode;
 
   public ZKSecretWatcher(Configuration conf,
-      ZooKeeperWatcher watcher,
+      ZKWatcher watcher,
       AuthenticationTokenSecretManager secretManager) {
     super(watcher);
     this.secretManager = secretManager;
@@ -214,7 +214,7 @@ public class ZKSecretWatcher extends ZooKeeperListener {
       watcher.abort("Failed serializing key "+key.getKeyId(), ioe);
     }
   }
-  
+
   /**
    * refresh keys
    */
@@ -228,7 +228,7 @@ public class ZKSecretWatcher extends ZooKeeperListener {
       watcher.abort("Error reading changed keys from zookeeper", ke);
     }
   }
-  
+
   /**
    * get token keys parent node
    * @return token keys parent node

http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
index e913b21..8272da6 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
 import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.coprocessor.HasRegionServerServices;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Mutation;
@@ -67,7 +68,6 @@ import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 
 @InterfaceAudience.Private
 public class DefaultVisibilityLabelServiceImpl implements 
VisibilityLabelService {
@@ -120,7 +120,7 @@ public class DefaultVisibilityLabelServiceImpl implements 
VisibilityLabelService
     // This is a CoreCoprocessor. On creation, we should have gotten an 
environment that
     // implements HasRegionServerServices so we can get at RSS. FIX!!!! 
Integrate this CP as
     // native service.
-    ZooKeeperWatcher zk = 
((HasRegionServerServices)e).getRegionServerServices().getZooKeeper();
+    ZKWatcher zk = 
((HasRegionServerServices)e).getRegionServerServices().getZooKeeper();
     try {
       labelsCache = VisibilityLabelsCache.createAndGet(zk, this.conf);
     } catch (IOException ioe) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsCache.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsCache.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsCache.java
index 2edf636..85bc0d5 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsCache.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsCache.java
@@ -37,7 +37,7 @@ import 
org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.MultiUs
 import 
org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.UserAuthorizations;
 import 
org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.zookeeper.KeeperException;
 
 /**
@@ -64,7 +64,7 @@ public class VisibilityLabelsCache implements 
VisibilityLabelOrdinalProvider {
    */
   private ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
 
-  private VisibilityLabelsCache(ZooKeeperWatcher watcher, Configuration conf) 
throws IOException {
+  private VisibilityLabelsCache(ZKWatcher watcher, Configuration conf) throws 
IOException {
     zkVisibilityWatcher = new ZKVisibilityLabelWatcher(watcher, this, conf);
     try {
       zkVisibilityWatcher.start();
@@ -81,7 +81,7 @@ public class VisibilityLabelsCache implements 
VisibilityLabelOrdinalProvider {
    * @return Singleton instance of VisibilityLabelsCache
    * @throws IOException
    */
-  public synchronized static VisibilityLabelsCache 
createAndGet(ZooKeeperWatcher watcher,
+  public synchronized static VisibilityLabelsCache createAndGet(ZKWatcher 
watcher,
       Configuration conf) throws IOException {
     // VisibilityLabelService#init() for different regions (in same RS) passes 
same instance of
     // watcher as all get the instance from RS.
@@ -99,7 +99,7 @@ public class VisibilityLabelsCache implements 
VisibilityLabelOrdinalProvider {
    * @return Singleton instance of VisibilityLabelsCache
    * @throws IllegalStateException
    *           when this is called before calling
-   *           {@link #createAndGet(ZooKeeperWatcher, Configuration)}
+   *           {@link #createAndGet(ZKWatcher, Configuration)}
    */
   public static VisibilityLabelsCache get() {
     // By the time this method is called, the singleton instance of 
VisibilityLabelsCache should

http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ZKVisibilityLabelWatcher.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ZKVisibilityLabelWatcher.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ZKVisibilityLabelWatcher.java
index 5a6a414..5cc244c 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ZKVisibilityLabelWatcher.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ZKVisibilityLabelWatcher.java
@@ -21,12 +21,12 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 
 /**
@@ -34,12 +34,12 @@ import org.apache.zookeeper.KeeperException;
  * /hbase/visibility_labels and will have a serialized form of a set of labels 
in the system.
  */
 @InterfaceAudience.Private
-public class ZKVisibilityLabelWatcher extends ZooKeeperListener {
+public class ZKVisibilityLabelWatcher extends ZKListener {
 
   private static final Log LOG = 
LogFactory.getLog(ZKVisibilityLabelWatcher.class);
   private static final String VISIBILITY_LABEL_ZK_PATH = 
"zookeeper.znode.visibility.label.parent";
   private static final String DEFAULT_VISIBILITY_LABEL_NODE = 
"visibility/labels";
-  private static final String VISIBILITY_USER_AUTHS_ZK_PATH = 
+  private static final String VISIBILITY_USER_AUTHS_ZK_PATH =
       "zookeeper.znode.visibility.user.auths.parent";
   private static final String DEFAULT_VISIBILITY_USER_AUTHS_NODE = 
"visibility/user_auths";
 
@@ -47,8 +47,8 @@ public class ZKVisibilityLabelWatcher extends 
ZooKeeperListener {
   private String labelZnode;
   private String userAuthsZnode;
 
-  public ZKVisibilityLabelWatcher(ZooKeeperWatcher watcher, 
VisibilityLabelsCache labelsCache,
-      Configuration conf) {
+  public ZKVisibilityLabelWatcher(ZKWatcher watcher, VisibilityLabelsCache 
labelsCache,
+                                  Configuration conf) {
     super(watcher);
     this.labelsCache = labelsCache;
     String labelZnodeParent = conf.get(VISIBILITY_LABEL_ZK_PATH, 
DEFAULT_VISIBILITY_LABEL_NODE);
@@ -132,7 +132,7 @@ public class ZKVisibilityLabelWatcher extends 
ZooKeeperListener {
 
   /**
    * Write a labels mirror or user auths mirror into zookeeper
-   * 
+   *
    * @param data
    * @param labelsOrUserAuths true for writing labels and false for user auths.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index bf4478d..0879a69 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -130,8 +130,8 @@ import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.wal.WALSplitter;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -316,7 +316,7 @@ public class HBaseFsck extends Configured implements 
Closeable {
 
   private Map<TableName, Set<String>> skippedRegions = new HashMap<>();
 
-  private ZooKeeperWatcher zkw = null;
+  private ZKWatcher zkw = null;
   private String hbckEphemeralNodePath = null;
   private boolean hbckZodeCreated = false;
 
@@ -1918,8 +1918,8 @@ public class HBaseFsck extends Configured implements 
Closeable {
     return true;
   }
 
-  private ZooKeeperWatcher createZooKeeperWatcher() throws IOException {
-    return new ZooKeeperWatcher(getConf(), "hbase Fsck", new Abortable() {
+  private ZKWatcher createZooKeeperWatcher() throws IOException {
+    return new ZKWatcher(getConf(), "hbase Fsck", new Abortable() {
       @Override
       public void abort(String why, Throwable e) {
         LOG.error(why, e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
index 0f36a7b..b1c1f07 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
@@ -62,7 +62,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -900,7 +900,7 @@ public class RegionMover extends AbstractHBaseTool {
       return null;
     }
     if (region.isMetaRegion()) {
-      ZooKeeperWatcher zkw = new ZooKeeperWatcher(admin.getConfiguration(), 
"region_mover", null);
+      ZKWatcher zkw = new ZKWatcher(admin.getConfiguration(), "region_mover", 
null);
       MetaTableLocator locator = new MetaTableLocator();
       int maxWaitInSeconds =
           admin.getConfiguration().getInt(MOVE_WAIT_MAX_KEY, 
DEFAULT_MOVE_WAIT_MAX);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java
index e07861e..11327e8 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java
@@ -25,14 +25,14 @@ import java.util.Map;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 
 /**
@@ -52,7 +52,7 @@ public class ZKDataMigrator {
    * table descriptor based states.
    */
   @Deprecated
-  public static Map<TableName, TableState.State> 
queryForTableStates(ZooKeeperWatcher zkw)
+  public static Map<TableName, TableState.State> queryForTableStates(ZKWatcher 
zkw)
       throws KeeperException, InterruptedException {
     Map<TableName, TableState.State> rv = new HashMap<>();
     List<String> children = ZKUtil.listChildrenNoWatch(zkw, 
zkw.znodePaths.tableZNode);
@@ -86,14 +86,14 @@ public class ZKDataMigrator {
 
   /**
    * Gets table state from ZK.
-   * @param zkw ZooKeeperWatcher instance to use
+   * @param zkw ZKWatcher instance to use
    * @param tableName table we're checking
    * @return Null or {@link ZooKeeperProtos.DeprecatedTableState.State} found 
in znode.
    * @throws KeeperException
    */
   @Deprecated
   private static  ZooKeeperProtos.DeprecatedTableState.State getTableState(
-      final ZooKeeperWatcher zkw, final TableName tableName)
+          final ZKWatcher zkw, final TableName tableName)
       throws KeeperException, InterruptedException {
     String znode = ZNodePaths.joinZNode(zkw.znodePaths.tableZNode, 
tableName.getNameAsString());
     byte [] data = ZKUtil.getData(zkw, znode);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java
index 9fb8459..839b5ad 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java
@@ -27,13 +27,13 @@ import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.master.cleaner.ReplicationZKNodeCleaner;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.util.HBaseFsck;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 
 /*
  * Check and fix undeleted replication queues for removed peerId.
@@ -47,8 +47,8 @@ public class ReplicationChecker {
   private Set<String> undeletedHFileRefsQueueIds = new HashSet<>();
   private final ReplicationZKNodeCleaner cleaner;
 
-  public ReplicationChecker(Configuration conf, ZooKeeperWatcher zkw, 
ClusterConnection connection,
-      ErrorReporter errorReporter) throws IOException {
+  public ReplicationChecker(Configuration conf, ZKWatcher zkw, 
ClusterConnection connection,
+                            ErrorReporter errorReporter) throws IOException {
     this.cleaner = new ReplicationZKNodeCleaner(conf, zkw, connection);
     this.errorReporter = errorReporter;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java
deleted file mode 100644
index fd60765..0000000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java
+++ /dev/null
@@ -1,99 +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.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * Tracker on cluster settings up in zookeeper.
- * This is not related to {@link org.apache.hadoop.hbase.ClusterStatus}. That 
class 
- * is a data structure that holds snapshot of current view on cluster. This 
class 
- * is about tracking cluster attributes up in zookeeper.
- *
- */
-@InterfaceAudience.Private
-public class ClusterStatusTracker extends ZooKeeperNodeTracker {
-  private static final Log LOG = LogFactory.getLog(ClusterStatusTracker.class);
-
-  /**
-   * Creates a cluster status tracker.
-   *
-   * <p>After construction, use {@link #start} to kick off tracking.
-   *
-   * @param watcher
-   * @param abortable
-   */
-  public ClusterStatusTracker(ZooKeeperWatcher watcher, Abortable abortable) {
-    super(watcher, watcher.znodePaths.clusterStateZNode, abortable);
-  }
-
-  /**
-   * Checks if cluster is up.
-   * @return true if the cluster up ('shutdown' is its name up in zk) znode
-   * exists with data, false if not
-   */
-  public boolean isClusterUp() {
-    return super.getData(false) != null;
-  }
-
-  /**
-   * Sets the cluster as up.
-   * @throws KeeperException unexpected zk exception
-   */
-  public void setClusterUp()
-  throws KeeperException {
-    byte [] upData = toByteArray();
-    try {
-      ZKUtil.createAndWatch(watcher, watcher.znodePaths.clusterStateZNode, 
upData);
-    } catch(KeeperException.NodeExistsException nee) {
-      ZKUtil.setData(watcher, watcher.znodePaths.clusterStateZNode, upData);
-    }
-  }
-
-  /**
-   * Sets the cluster as down by deleting the znode.
-   * @throws KeeperException unexpected zk exception
-   */
-  public void setClusterDown()
-  throws KeeperException {
-    try {
-      ZKUtil.deleteNode(watcher, watcher.znodePaths.clusterStateZNode);
-    } catch(KeeperException.NoNodeException nne) {
-      LOG.warn("Attempted to set cluster as down but already down, cluster " +
-          "state node (" + watcher.znodePaths.clusterStateZNode + ") not 
found");
-    }
-  }
-
-  /**
-   * @return Content of the clusterup znode as a serialized pb with the pb
-   * magic as prefix.
-   */
-  static byte [] toByteArray() {
-    ZooKeeperProtos.ClusterUp.Builder builder =
-      ZooKeeperProtos.ClusterUp.newBuilder();
-    builder.setStartDate(new java.util.Date().toString());
-    return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java
deleted file mode 100644
index c58a840..0000000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/**
- * Copyright The Apache Software Foundation
- *
- * 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 java.util.concurrent.CountDownLatch;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * A ZooKeeper watcher meant to detect deletions of ZNodes.
- */
-@InterfaceAudience.Private
-public class DeletionListener extends ZooKeeperListener {
-
-  private static final Log LOG = LogFactory.getLog(DeletionListener.class);
-
-  private final String pathToWatch;
-  private final CountDownLatch deletedLatch;
-
-  private volatile Throwable exception;
-
-  /**
-   * Create a new instance of the deletion watcher.
-   * @param zkWatcher ZookeeperWatcher instance
-   * @param pathToWatch (Fully qualified) ZNode path that we are waiting to
-   *                    be deleted.
-   * @param deletedLatch Count down on this latch when deletion has occurred.
-   */
-  public DeletionListener(ZooKeeperWatcher zkWatcher, String pathToWatch,
-      CountDownLatch deletedLatch) {
-    super(zkWatcher);
-    this.pathToWatch = pathToWatch;
-    this.deletedLatch = deletedLatch;
-    exception = null;
-  }
-
-  /**
-   * Check if an exception has occurred when re-setting the watch.
-   * @return True if we were unable to re-set a watch on a ZNode due to
-   *         an exception.
-   */
-  public boolean hasException() {
-    return exception != null;
-  }
-
-  /**
-   * Get the last exception which has occurred when re-setting the watch.
-   * Use hasException() to check whether or not an exception has occurred.
-   * @return The last exception observed when re-setting the watch.
-   */
-  public Throwable getException() {
-    return exception;
-  }
-
-  @Override
-  public void nodeDataChanged(String path) {
-    if (!path.equals(pathToWatch)) {
-      return;
-    }
-    try {
-      if (!(ZKUtil.setWatchIfNodeExists(watcher, pathToWatch))) {
-        deletedLatch.countDown();
-      }
-    } catch (KeeperException ex) {
-      exception = ex;
-      deletedLatch.countDown();
-      LOG.error("Error when re-setting the watch on " + pathToWatch, ex);
-    }
-  }
-
-  @Override
-  public void nodeDeleted(String path) {
-    if (!path.equals(pathToWatch)) {
-      return;
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Processing delete on " + pathToWatch);
-    }
-    deletedLatch.countDown();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java
deleted file mode 100644
index cdc6794..0000000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java
+++ /dev/null
@@ -1,134 +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 java.io.IOException;
-import java.util.List;
-import java.util.NavigableSet;
-import java.util.TreeSet;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.master.ServerListener;
-import org.apache.hadoop.hbase.master.ServerManager;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * Tracks the list of draining region servers via ZK.
- *
- * <p>This class is responsible for watching for changes to the draining
- * servers list.  It handles adds/deletes in the draining RS list and
- * watches each node.
- *
- * <p>If an RS gets deleted from draining list, we call
- * {@link ServerManager#removeServerFromDrainList(ServerName)}
- *
- * <p>If an RS gets added to the draining list, we add a watcher to it and call
- * {@link ServerManager#addServerToDrainList(ServerName)}
- *
- * <p>This class is deprecated in 2.0 because decommission/draining API goes 
through
- * master in 2.0. Can remove this class in 3.0.
- *
- */
-@InterfaceAudience.Private
-public class DrainingServerTracker extends ZooKeeperListener {
-  private static final Log LOG = 
LogFactory.getLog(DrainingServerTracker.class);
-
-  private ServerManager serverManager;
-  private final NavigableSet<ServerName> drainingServers = new TreeSet<>();
-  private Abortable abortable;
-
-  public DrainingServerTracker(ZooKeeperWatcher watcher,
-      Abortable abortable, ServerManager serverManager) {
-    super(watcher);
-    this.abortable = abortable;
-    this.serverManager = serverManager;
-  }
-
-  /**
-   * Starts the tracking of draining RegionServers.
-   *
-   * <p>All Draining RSs will be tracked after this method is called.
-   *
-   * @throws KeeperException
-   */
-  public void start() throws KeeperException, IOException {
-    watcher.registerListener(this);
-    // Add a ServerListener to check if a server is draining when it's added.
-    serverManager.registerListener(new ServerListener() {
-      @Override
-      public void serverAdded(ServerName sn) {
-        if (drainingServers.contains(sn)){
-          serverManager.addServerToDrainList(sn);
-        }
-      }
-    });
-    List<String> servers =
-      ZKUtil.listChildrenAndWatchThem(watcher, 
watcher.znodePaths.drainingZNode);
-    add(servers);
-  }
-
-  private void add(final List<String> servers) throws IOException {
-    synchronized(this.drainingServers) {
-      this.drainingServers.clear();
-      for (String n: servers) {
-        final ServerName sn = ServerName.valueOf(ZKUtil.getNodeName(n));
-        this.drainingServers.add(sn);
-        this.serverManager.addServerToDrainList(sn);
-        LOG.info("Draining RS node created, adding to list [" +
-            sn + "]");
-
-      }
-    }
-  }
-
-  private void remove(final ServerName sn) {
-    synchronized(this.drainingServers) {
-      this.drainingServers.remove(sn);
-      this.serverManager.removeServerFromDrainList(sn);
-    }
-  }
-
-  @Override
-  public void nodeDeleted(final String path) {
-    if(path.startsWith(watcher.znodePaths.drainingZNode)) {
-      final ServerName sn = ServerName.valueOf(ZKUtil.getNodeName(path));
-      LOG.info("Draining RS node deleted, removing from list [" +
-          sn + "]");
-      remove(sn);
-    }
-  }
-
-  @Override
-  public void nodeChildrenChanged(final String path) {
-    if(path.equals(watcher.znodePaths.drainingZNode)) {
-      try {
-        final List<String> newNodes =
-          ZKUtil.listChildrenAndWatchThem(watcher, 
watcher.znodePaths.drainingZNode);
-        add(newNodes);
-      } catch (KeeperException e) {
-        abortable.abort("Unexpected zk exception getting RS nodes", e);
-      } catch (IOException e) {
-        abortable.abort("Unexpected zk exception getting RS nodes", e);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java
deleted file mode 100644
index 527bc17..0000000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java
+++ /dev/null
@@ -1,94 +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 java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * Tracks the load balancer state up in ZK
- */
-@InterfaceAudience.Private
-public class LoadBalancerTracker extends ZooKeeperNodeTracker {
-  private static final Log LOG = LogFactory.getLog(LoadBalancerTracker.class);
-
-  public LoadBalancerTracker(ZooKeeperWatcher watcher,
-      Abortable abortable) {
-    super(watcher, watcher.znodePaths.balancerZNode, abortable);
-  }
-
-  /**
-   * Return true if the balance switch is on, false otherwise
-   */
-  public boolean isBalancerOn() {
-    byte [] upData = super.getData(false);
-    try {
-      // if data in ZK is null, use default of on.
-      return upData == null || parseFrom(upData).getBalancerOn();
-    } catch (DeserializationException dex) {
-      LOG.error("ZK state for LoadBalancer could not be parsed " + 
Bytes.toStringBinary(upData));
-      // return false to be safe.
-      return false;
-    }
-  }
-
-  /**
-   * Set the balancer on/off
-   * @param balancerOn
-   * @throws KeeperException
-   */
-  public void setBalancerOn(boolean balancerOn) throws KeeperException {
-  byte [] upData = toByteArray(balancerOn);
-    try {
-      ZKUtil.setData(watcher, watcher.znodePaths.balancerZNode, upData);
-    } catch(KeeperException.NoNodeException nne) {
-      ZKUtil.createAndWatch(watcher, watcher.znodePaths.balancerZNode, upData);
-    }
-    super.nodeDataChanged(watcher.znodePaths.balancerZNode);
-  }
-
-  private byte [] toByteArray(boolean isBalancerOn) {
-    LoadBalancerProtos.LoadBalancerState.Builder builder =
-      LoadBalancerProtos.LoadBalancerState.newBuilder();
-    builder.setBalancerOn(isBalancerOn);
-    return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
-  }
-
-  private LoadBalancerProtos.LoadBalancerState parseFrom(byte [] pbBytes)
-  throws DeserializationException {
-    ProtobufUtil.expectPBMagicPrefix(pbBytes);
-    LoadBalancerProtos.LoadBalancerState.Builder builder =
-      LoadBalancerProtos.LoadBalancerState.newBuilder();
-    try {
-      int magicLen = ProtobufUtil.lengthOfPBMagic();
-      ProtobufUtil.mergeFrom(builder, pbBytes, magicLen, pbBytes.length - 
magicLen);
-    } catch (IOException e) {
-      throw new DeserializationException(e);
-    }
-    return builder.build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java
deleted file mode 100644
index 7d6f9fd..0000000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java
+++ /dev/null
@@ -1,81 +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 java.util.List;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * Tracks the master Maintenance Mode via ZK.
- */
-@InterfaceAudience.Private
-public class MasterMaintenanceModeTracker extends ZooKeeperListener {
-  private boolean hasChildren;
-
-  public MasterMaintenanceModeTracker(ZooKeeperWatcher watcher) {
-    super(watcher);
-    hasChildren = false;
-  }
-
-  public boolean isInMaintenanceMode() {
-    return hasChildren;
-  }
-
-  private void update(String path) {
-    if (path.startsWith(watcher.znodePaths.masterMaintZNode)) {
-      update();
-    }
-  }
-
-  private void update() {
-    try {
-      List<String> children =
-          ZKUtil.listChildrenAndWatchForNewChildren(watcher, 
watcher.znodePaths.masterMaintZNode);
-      hasChildren = (children != null && children.size() > 0);
-    } catch (KeeperException e) {
-      // Ignore the ZK keeper exception
-      hasChildren = false;
-    }
-  }
-
-  /**
-   * Starts the tracking of whether master is in Maintenance Mode.
-   */
-  public void start() {
-    watcher.registerListener(this);
-    update();
-  }
-
-  @Override
-  public void nodeCreated(String path) {
-    update(path);
-  }
-
-  @Override
-  public void nodeDeleted(String path) {
-    update(path);
-  }
-
-  @Override
-  public void nodeChildrenChanged(String path) {
-    update(path);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
deleted file mode 100644
index ef643bf..0000000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
+++ /dev/null
@@ -1,472 +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 java.io.BufferedReader;
-import java.io.File;
-import java.io.InterruptedIOException;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.io.Reader;
-import java.net.BindException;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.zookeeper.server.NIOServerCnxnFactory;
-import org.apache.zookeeper.server.ZooKeeperServer;
-import org.apache.zookeeper.server.persistence.FileTxnLog;
-
-import 
org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-
-/**
- * TODO: Most of the code in this class is ripped from ZooKeeper tests. Instead
- * of redoing it, we should contribute updates to their code which let us more
- * easily access testing helper objects.
- */
-@InterfaceAudience.Public
-public class MiniZooKeeperCluster {
-  private static final Log LOG = LogFactory.getLog(MiniZooKeeperCluster.class);
-
-  private static final int TICK_TIME = 2000;
-  private static final int DEFAULT_CONNECTION_TIMEOUT = 30000;
-  private int connectionTimeout;
-
-  private boolean started;
-
-  /** The default port. If zero, we use a random port. */
-  private int defaultClientPort = 0;
-
-  private List<NIOServerCnxnFactory> standaloneServerFactoryList;
-  private List<ZooKeeperServer> zooKeeperServers;
-  private List<Integer> clientPortList;
-
-  private int activeZKServerIndex;
-  private int tickTime = 0;
-
-  private Configuration configuration;
-
-  public MiniZooKeeperCluster() {
-    this(new Configuration());
-  }
-
-  public MiniZooKeeperCluster(Configuration configuration) {
-    this.started = false;
-    this.configuration = configuration;
-    activeZKServerIndex = -1;
-    zooKeeperServers = new ArrayList<>();
-    clientPortList = new ArrayList<>();
-    standaloneServerFactoryList = new ArrayList<>();
-    connectionTimeout = configuration.getInt(HConstants.ZK_SESSION_TIMEOUT + 
".localHBaseCluster",
-      DEFAULT_CONNECTION_TIMEOUT);
-  }
-
-  /**
-   * Add a client port to the list.
-   *
-   * @param clientPort the specified port
-   */
-  public void addClientPort(int clientPort) {
-    clientPortList.add(clientPort);
-  }
-
-  /**
-   * Get the list of client ports.
-   * @return clientPortList the client port list
-   */
-  @VisibleForTesting
-  public List<Integer> getClientPortList() {
-    return clientPortList;
-  }
-
-  /**
-   * Check whether the client port in a specific position of the client port 
list is valid.
-   *
-   * @param index the specified position
-   */
-  private boolean hasValidClientPortInList(int index) {
-    return (clientPortList.size() > index && clientPortList.get(index) > 0);
-  }
-
-  public void setDefaultClientPort(int clientPort) {
-    if (clientPort <= 0) {
-      throw new IllegalArgumentException("Invalid default ZK client port: "
-          + clientPort);
-    }
-    this.defaultClientPort = clientPort;
-  }
-
-  /**
-   * Selects a ZK client port.
-   *
-   * @param seedPort the seed port to start with; -1 means first time.
-   * @Returns a valid and unused client port
-   */
-  private int selectClientPort(int seedPort) {
-    int i;
-    int returnClientPort = seedPort + 1;
-    if (returnClientPort == 0) {
-      // If the new port is invalid, find one - starting with the default 
client port.
-      // If the default client port is not specified, starting with a random 
port.
-      // The random port is selected from the range between 49152 to 65535. 
These ports cannot be
-      // registered with IANA and are intended for dynamic allocation (see 
http://bit.ly/dynports).
-      if (defaultClientPort > 0) {
-        returnClientPort = defaultClientPort;
-      } else {
-        returnClientPort = 0xc000 + new Random().nextInt(0x3f00);
-      }
-    }
-    // Make sure that the port is unused.
-    while (true) {
-      for (i = 0; i < clientPortList.size(); i++) {
-        if (returnClientPort == clientPortList.get(i)) {
-          // Already used. Update the port and retry.
-          returnClientPort++;
-          break;
-        }
-      }
-      if (i == clientPortList.size()) {
-        break; // found a unused port, exit
-      }
-    }
-    return returnClientPort;
-  }
-
-  public void setTickTime(int tickTime) {
-    this.tickTime = tickTime;
-  }
-
-  public int getBackupZooKeeperServerNum() {
-    return zooKeeperServers.size()-1;
-  }
-
-  public int getZooKeeperServerNum() {
-    return zooKeeperServers.size();
-  }
-
-  // / XXX: From o.a.zk.t.ClientBase
-  private static void setupTestEnv() {
-    // during the tests we run with 100K prealloc in the logs.
-    // on windows systems prealloc of 64M was seen to take ~15seconds
-    // resulting in test failure (client timeout on first session).
-    // set env and directly in order to handle static init/gc issues
-    System.setProperty("zookeeper.preAllocSize", "100");
-    FileTxnLog.setPreallocSize(100 * 1024);
-  }
-
-  public int startup(File baseDir) throws IOException, InterruptedException {
-    int numZooKeeperServers = clientPortList.size();
-    if (numZooKeeperServers == 0) {
-      numZooKeeperServers = 1; // need at least 1 ZK server for testing
-    }
-    return startup(baseDir, numZooKeeperServers);
-  }
-
-  /**
-   * @param baseDir
-   * @param numZooKeeperServers
-   * @return ClientPort server bound to, -1 if there was a
-   *         binding problem and we couldn't pick another port.
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public int startup(File baseDir, int numZooKeeperServers) throws IOException,
-      InterruptedException {
-    if (numZooKeeperServers <= 0)
-      return -1;
-
-    setupTestEnv();
-    shutdown();
-
-    int tentativePort = -1; // the seed port
-    int currentClientPort;
-
-    // running all the ZK servers
-    for (int i = 0; i < numZooKeeperServers; i++) {
-      File dir = new File(baseDir, "zookeeper_"+i).getAbsoluteFile();
-      createDir(dir);
-      int tickTimeToUse;
-      if (this.tickTime > 0) {
-        tickTimeToUse = this.tickTime;
-      } else {
-        tickTimeToUse = TICK_TIME;
-      }
-
-      // Set up client port - if we have already had a list of valid ports, 
use it.
-      if (hasValidClientPortInList(i)) {
-        currentClientPort = clientPortList.get(i);
-      } else {
-        tentativePort = selectClientPort(tentativePort); // update the seed
-        currentClientPort = tentativePort;
-      }
-
-      ZooKeeperServer server = new ZooKeeperServer(dir, dir, tickTimeToUse);
-      // Setting {min,max}SessionTimeout defaults to be the same as in 
Zookeeper
-      
server.setMinSessionTimeout(configuration.getInt("hbase.zookeeper.property.minSessionTimeout",
 -1));
-      
server.setMaxSessionTimeout(configuration.getInt("hbase.zookeeper.property.maxSessionTimeout",
 -1));
-      NIOServerCnxnFactory standaloneServerFactory;
-      while (true) {
-        try {
-          standaloneServerFactory = new NIOServerCnxnFactory();
-          standaloneServerFactory.configure(
-            new InetSocketAddress(currentClientPort),
-            configuration.getInt(HConstants.ZOOKEEPER_MAX_CLIENT_CNXNS, 
HConstants.DEFAULT_ZOOKEPER_MAX_CLIENT_CNXNS));
-        } catch (BindException e) {
-          LOG.debug("Failed binding ZK Server to client port: " +
-              currentClientPort, e);
-          // We're told to use some port but it's occupied, fail
-          if (hasValidClientPortInList(i)) {
-            return -1;
-          }
-          // This port is already in use, try to use another.
-          tentativePort = selectClientPort(tentativePort);
-          currentClientPort = tentativePort;
-          continue;
-        }
-        break;
-      }
-
-      // Start up this ZK server
-      standaloneServerFactory.startup(server);
-      // Runs a 'stat' against the servers.
-      if (!waitForServerUp(currentClientPort, connectionTimeout)) {
-        throw new IOException("Waiting for startup of standalone server");
-      }
-
-      // We have selected a port as a client port.  Update clientPortList if 
necessary.
-      if (clientPortList.size() <= i) { // it is not in the list, add the port
-        clientPortList.add(currentClientPort);
-      }
-      else if (clientPortList.get(i) <= 0) { // the list has invalid port, 
update with valid port
-        clientPortList.remove(i);
-        clientPortList.add(i, currentClientPort);
-      }
-
-      standaloneServerFactoryList.add(standaloneServerFactory);
-      zooKeeperServers.add(server);
-    }
-
-    // set the first one to be active ZK; Others are backups
-    activeZKServerIndex = 0;
-    started = true;
-    int clientPort = clientPortList.get(activeZKServerIndex);
-    LOG.info("Started MiniZooKeeperCluster and ran successful 'stat' " +
-        "on client port=" + clientPort);
-    return clientPort;
-  }
-
-  private void createDir(File dir) throws IOException {
-    try {
-      if (!dir.exists()) {
-        dir.mkdirs();
-      }
-    } catch (SecurityException e) {
-      throw new IOException("creating dir: " + dir, e);
-    }
-  }
-
-  /**
-   * @throws IOException
-   */
-  public void shutdown() throws IOException {
-    // shut down all the zk servers
-    for (int i = 0; i < standaloneServerFactoryList.size(); i++) {
-      NIOServerCnxnFactory standaloneServerFactory =
-        standaloneServerFactoryList.get(i);
-      int clientPort = clientPortList.get(i);
-
-      standaloneServerFactory.shutdown();
-      if (!waitForServerDown(clientPort, connectionTimeout)) {
-        throw new IOException("Waiting for shutdown of standalone server");
-      }
-    }
-    standaloneServerFactoryList.clear();
-
-    for (ZooKeeperServer zkServer: zooKeeperServers) {
-      //explicitly close ZKDatabase since ZookeeperServer does not close them
-      zkServer.getZKDatabase().close();
-    }
-    zooKeeperServers.clear();
-
-    // clear everything
-    if (started) {
-      started = false;
-      activeZKServerIndex = 0;
-      clientPortList.clear();
-      LOG.info("Shutdown MiniZK cluster with all ZK servers");
-    }
-  }
-
-  /**@return clientPort return clientPort if there is another ZK backup can run
-   *         when killing the current active; return -1, if there is no 
backups.
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public int killCurrentActiveZooKeeperServer() throws IOException,
-                                        InterruptedException {
-    if (!started || activeZKServerIndex < 0) {
-      return -1;
-    }
-
-    // Shutdown the current active one
-    NIOServerCnxnFactory standaloneServerFactory =
-      standaloneServerFactoryList.get(activeZKServerIndex);
-    int clientPort = clientPortList.get(activeZKServerIndex);
-
-    standaloneServerFactory.shutdown();
-    if (!waitForServerDown(clientPort, connectionTimeout)) {
-      throw new IOException("Waiting for shutdown of standalone server");
-    }
-
-    zooKeeperServers.get(activeZKServerIndex).getZKDatabase().close();
-
-    // remove the current active zk server
-    standaloneServerFactoryList.remove(activeZKServerIndex);
-    clientPortList.remove(activeZKServerIndex);
-    zooKeeperServers.remove(activeZKServerIndex);
-    LOG.info("Kill the current active ZK servers in the cluster " +
-        "on client port: " + clientPort);
-
-    if (standaloneServerFactoryList.isEmpty()) {
-      // there is no backup servers;
-      return -1;
-    }
-    clientPort = clientPortList.get(activeZKServerIndex);
-    LOG.info("Activate a backup zk server in the cluster " +
-        "on client port: " + clientPort);
-    // return the next back zk server's port
-    return clientPort;
-  }
-
-  /**
-   * Kill one back up ZK servers
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public void killOneBackupZooKeeperServer() throws IOException,
-                                        InterruptedException {
-    if (!started || activeZKServerIndex < 0 ||
-        standaloneServerFactoryList.size() <= 1) {
-      return ;
-    }
-
-    int backupZKServerIndex = activeZKServerIndex+1;
-    // Shutdown the current active one
-    NIOServerCnxnFactory standaloneServerFactory =
-      standaloneServerFactoryList.get(backupZKServerIndex);
-    int clientPort = clientPortList.get(backupZKServerIndex);
-
-    standaloneServerFactory.shutdown();
-    if (!waitForServerDown(clientPort, connectionTimeout)) {
-      throw new IOException("Waiting for shutdown of standalone server");
-    }
-
-    zooKeeperServers.get(backupZKServerIndex).getZKDatabase().close();
-
-    // remove this backup zk server
-    standaloneServerFactoryList.remove(backupZKServerIndex);
-    clientPortList.remove(backupZKServerIndex);
-    zooKeeperServers.remove(backupZKServerIndex);
-    LOG.info("Kill one backup ZK servers in the cluster " +
-        "on client port: " + clientPort);
-  }
-
-  // XXX: From o.a.zk.t.ClientBase
-  private static boolean waitForServerDown(int port, long timeout) throws 
IOException {
-    long start = System.currentTimeMillis();
-    while (true) {
-      try {
-        Socket sock = new Socket("localhost", port);
-        try {
-          OutputStream outstream = sock.getOutputStream();
-          outstream.write("stat".getBytes());
-          outstream.flush();
-        } finally {
-          sock.close();
-        }
-      } catch (IOException e) {
-        return true;
-      }
-
-      if (System.currentTimeMillis() > start + timeout) {
-        break;
-      }
-      try {
-        Thread.sleep(250);
-      } catch (InterruptedException e) {
-        throw (InterruptedIOException)new 
InterruptedIOException().initCause(e);
-      }
-    }
-    return false;
-  }
-
-  // XXX: From o.a.zk.t.ClientBase
-  private static boolean waitForServerUp(int port, long timeout) throws 
IOException {
-    long start = System.currentTimeMillis();
-    while (true) {
-      try {
-        Socket sock = new Socket("localhost", port);
-        BufferedReader reader = null;
-        try {
-          OutputStream outstream = sock.getOutputStream();
-          outstream.write("stat".getBytes());
-          outstream.flush();
-
-          Reader isr = new InputStreamReader(sock.getInputStream());
-          reader = new BufferedReader(isr);
-          String line = reader.readLine();
-          if (line != null && line.startsWith("Zookeeper version:")) {
-            return true;
-          }
-        } finally {
-          sock.close();
-          if (reader != null) {
-            reader.close();
-          }
-        }
-      } catch (IOException e) {
-        // ignore as this is expected
-        LOG.info("server localhost:" + port + " not up " + e);
-      }
-
-      if (System.currentTimeMillis() > start + timeout) {
-        break;
-      }
-      try {
-        Thread.sleep(250);
-      } catch (InterruptedException e) {
-        throw (InterruptedIOException)new 
InterruptedIOException().initCause(e);
-      }
-    }
-    return false;
-  }
-
-  public int getClientPort() {
-    return activeZKServerIndex < 0 || activeZKServerIndex >= 
clientPortList.size() ? -1
-        : clientPortList.get(activeZKServerIndex);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java
deleted file mode 100644
index a993934..0000000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java
+++ /dev/null
@@ -1,94 +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.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.zookeeper.KeeperException;
-
-import java.io.IOException;
-
-/**
- * Tracks region normalizer state up in ZK
- */
-public class RegionNormalizerTracker extends ZooKeeperNodeTracker {
-  private static final Log LOG = 
LogFactory.getLog(RegionNormalizerTracker.class);
-
-  public RegionNormalizerTracker(ZooKeeperWatcher watcher,
-                             Abortable abortable) {
-    super(watcher, watcher.znodePaths.regionNormalizerZNode, abortable);
-  }
-
-  /**
-   * Return true if region normalizer is on, false otherwise
-   */
-  public boolean isNormalizerOn() {
-    byte [] upData = super.getData(false);
-    try {
-      // if data in ZK is null, use default of on.
-      return upData == null || parseFrom(upData).getNormalizerOn();
-    } catch (DeserializationException dex) {
-      LOG.error("ZK state for RegionNormalizer could not be parsed "
-        + Bytes.toStringBinary(upData));
-      // return false to be safe.
-      return false;
-    }
-  }
-
-  /**
-   * Set region normalizer on/off
-   * @param normalizerOn whether normalizer should be on or off
-   * @throws KeeperException
-   */
-  public void setNormalizerOn(boolean normalizerOn) throws KeeperException {
-    byte [] upData = toByteArray(normalizerOn);
-    try {
-      ZKUtil.setData(watcher, watcher.znodePaths.regionNormalizerZNode, 
upData);
-    } catch(KeeperException.NoNodeException nne) {
-      ZKUtil.createAndWatch(watcher, watcher.znodePaths.regionNormalizerZNode, 
upData);
-    }
-    super.nodeDataChanged(watcher.znodePaths.regionNormalizerZNode);
-  }
-
-  private byte [] toByteArray(boolean isNormalizerOn) {
-    RegionNormalizerProtos.RegionNormalizerState.Builder builder =
-      RegionNormalizerProtos.RegionNormalizerState.newBuilder();
-    builder.setNormalizerOn(isNormalizerOn);
-    return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
-  }
-
-  private RegionNormalizerProtos.RegionNormalizerState parseFrom(byte [] 
pbBytes)
-    throws DeserializationException {
-    ProtobufUtil.expectPBMagicPrefix(pbBytes);
-    RegionNormalizerProtos.RegionNormalizerState.Builder builder =
-      RegionNormalizerProtos.RegionNormalizerState.newBuilder();
-    try {
-      int magicLen = ProtobufUtil.lengthOfPBMagic();
-      ProtobufUtil.mergeFrom(builder, pbBytes, magicLen, pbBytes.length - 
magicLen);
-    } catch (IOException e) {
-      throw new DeserializationException(e);
-    }
-    return builder.build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
deleted file mode 100644
index c22876a..0000000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
+++ /dev/null
@@ -1,162 +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 java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.master.ServerManager;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * Tracks the online region servers via ZK.
- *
- * <p>Handling of new RSs checking in is done via RPC.  This class
- * is only responsible for watching for expired nodes.  It handles
- * listening for changes in the RS node list and watching each node.
- *
- * <p>If an RS node gets deleted, this automatically handles calling of
- * {@link ServerManager#expireServer(ServerName)}
- */
-@InterfaceAudience.Private
-public class RegionServerTracker extends ZooKeeperListener {
-  private static final Log LOG = LogFactory.getLog(RegionServerTracker.class);
-  private NavigableMap<ServerName, RegionServerInfo> regionServers = new 
TreeMap<>();
-  private ServerManager serverManager;
-  private MasterServices server;
-
-  public RegionServerTracker(ZooKeeperWatcher watcher,
-      MasterServices server, ServerManager serverManager) {
-    super(watcher);
-    this.server = server;
-    this.serverManager = serverManager;
-  }
-
-  /**
-   * Starts the tracking of online RegionServers.
-   *
-   * <p>All RSs will be tracked after this method is called.
-   *
-   * @throws KeeperException
-   * @throws IOException
-   */
-  public void start() throws KeeperException, IOException {
-    watcher.registerListener(this);
-    List<String> servers =
-      ZKUtil.listChildrenAndWatchThem(watcher, watcher.znodePaths.rsZNode);
-    refresh(servers);
-  }
-
-  private void refresh(final List<String> servers) throws IOException {
-    synchronized(this.regionServers) {
-      this.regionServers.clear();
-      for (String n: servers) {
-        ServerName sn = ServerName.parseServerName(ZKUtil.getNodeName(n));
-        if (regionServers.get(sn) == null) {
-          RegionServerInfo.Builder rsInfoBuilder = 
RegionServerInfo.newBuilder();
-          try {
-            String nodePath = ZNodePaths.joinZNode(watcher.znodePaths.rsZNode, 
n);
-            byte[] data = ZKUtil.getData(watcher, nodePath);
-            if (data != null && data.length > 0 && 
ProtobufUtil.isPBMagicPrefix(data)) {
-              int magicLen = ProtobufUtil.lengthOfPBMagic();
-              ProtobufUtil.mergeFrom(rsInfoBuilder, data, magicLen, 
data.length - magicLen);
-            }
-            if (LOG.isTraceEnabled()) {
-              LOG.trace("Added tracking of RS " + nodePath);
-            }
-          } catch (KeeperException e) {
-            LOG.warn("Get Rs info port from ephemeral node", e);
-          } catch (IOException e) {
-            LOG.warn("Illegal data from ephemeral node", e);
-          } catch (InterruptedException e) {
-            throw new InterruptedIOException();
-          }
-          this.regionServers.put(sn, rsInfoBuilder.build());
-        }
-      }
-    }
-    if (server.isInitialized()) {
-      server.checkIfShouldMoveSystemRegionAsync();
-    }
-  }
-
-  private void remove(final ServerName sn) {
-    synchronized(this.regionServers) {
-      this.regionServers.remove(sn);
-    }
-  }
-
-  @Override
-  public void nodeDeleted(String path) {
-    if (path.startsWith(watcher.znodePaths.rsZNode)) {
-      String serverName = ZKUtil.getNodeName(path);
-      LOG.info("RegionServer ephemeral node deleted, processing expiration [" +
-        serverName + "]");
-      ServerName sn = ServerName.parseServerName(serverName);
-      if (!serverManager.isServerOnline(sn)) {
-        LOG.warn(serverName.toString() + " is not online or isn't known to the 
master."+
-         "The latter could be caused by a DNS misconfiguration.");
-        return;
-      }
-      remove(sn);
-      this.serverManager.expireServer(sn);
-    }
-  }
-
-  @Override
-  public void nodeChildrenChanged(String path) {
-    if (path.equals(watcher.znodePaths.rsZNode)
-        && !server.isAborted() && !server.isStopped()) {
-      try {
-        List<String> servers =
-          ZKUtil.listChildrenAndWatchThem(watcher, watcher.znodePaths.rsZNode);
-        refresh(servers);
-      } catch (IOException e) {
-        server.abort("Unexpected zk exception getting RS nodes", e);
-      } catch (KeeperException e) {
-        server.abort("Unexpected zk exception getting RS nodes", e);
-      }
-    }
-  }
-
-  public RegionServerInfo getRegionServerInfo(final ServerName sn) {
-    return regionServers.get(sn);
-  }
-  
-  /**
-   * Gets the online servers.
-   * @return list of online servers
-   */
-  public List<ServerName> getOnlineServers() {
-    synchronized (this.regionServers) {
-      return new ArrayList<>(this.regionServers.keySet());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTracker.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTracker.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTracker.java
deleted file mode 100644
index 2c5ffba..0000000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTracker.java
+++ /dev/null
@@ -1,149 +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 java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.client.MasterSwitchType;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SwitchState;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.zookeeper.KeeperException;
-
-
-/**
- * Tracks the switch of split and merge states in ZK
- *
- */
-@InterfaceAudience.Private
-public class SplitOrMergeTracker {
-
-  private String splitZnode;
-  private String mergeZnode;
-
-  private SwitchStateTracker splitStateTracker;
-  private SwitchStateTracker mergeStateTracker;
-
-  public SplitOrMergeTracker(ZooKeeperWatcher watcher, Configuration conf,
-                             Abortable abortable) {
-    try {
-      if (ZKUtil.checkExists(watcher, watcher.znodePaths.switchZNode) < 0) {
-        ZKUtil.createAndFailSilent(watcher, watcher.znodePaths.switchZNode);
-      }
-    } catch (KeeperException e) {
-      throw new RuntimeException(e);
-    }
-    splitZnode = ZNodePaths.joinZNode(watcher.znodePaths.switchZNode,
-      conf.get("zookeeper.znode.switch.split", "split"));
-    mergeZnode = ZNodePaths.joinZNode(watcher.znodePaths.switchZNode,
-      conf.get("zookeeper.znode.switch.merge", "merge"));
-    splitStateTracker = new SwitchStateTracker(watcher, splitZnode, abortable);
-    mergeStateTracker = new SwitchStateTracker(watcher, mergeZnode, abortable);
-  }
-
-  public void start() {
-    splitStateTracker.start();
-    mergeStateTracker.start();
-  }
-
-  public boolean isSplitOrMergeEnabled(MasterSwitchType switchType) {
-    switch (switchType) {
-      case SPLIT:
-        return splitStateTracker.isSwitchEnabled();
-      case MERGE:
-        return mergeStateTracker.isSwitchEnabled();
-      default:
-        break;
-    }
-    return false;
-  }
-
-  public void setSplitOrMergeEnabled(boolean enabled, MasterSwitchType 
switchType)
-    throws KeeperException {
-    switch (switchType) {
-      case SPLIT:
-        splitStateTracker.setSwitchEnabled(enabled);
-        break;
-      case MERGE:
-        mergeStateTracker.setSwitchEnabled(enabled);
-        break;
-      default:
-        break;
-    }
-  }
-
-  private static class SwitchStateTracker extends ZooKeeperNodeTracker {
-
-    public SwitchStateTracker(ZooKeeperWatcher watcher, String node, Abortable 
abortable) {
-      super(watcher, node, abortable);
-    }
-
-    /**
-     * Return true if the switch is on, false otherwise
-     */
-    public boolean isSwitchEnabled() {
-      byte [] upData = super.getData(false);
-      try {
-        // if data in ZK is null, use default of on.
-        return upData == null || parseFrom(upData).getEnabled();
-      } catch (DeserializationException dex) {
-        LOG.error("ZK state for LoadBalancer could not be parsed " + 
Bytes.toStringBinary(upData));
-        // return false to be safe.
-        return false;
-      }
-    }
-
-    /**
-     * Set the switch on/off
-     * @param enabled switch enabled or not?
-     * @throws KeeperException keepException will be thrown out
-     */
-    public void setSwitchEnabled(boolean enabled) throws KeeperException {
-      byte [] upData = toByteArray(enabled);
-      try {
-        ZKUtil.setData(watcher, node, upData);
-      } catch(KeeperException.NoNodeException nne) {
-        ZKUtil.createAndWatch(watcher, node, upData);
-      }
-      super.nodeDataChanged(node);
-    }
-
-    private byte [] toByteArray(boolean enabled) {
-      SwitchState.Builder builder = SwitchState.newBuilder();
-      builder.setEnabled(enabled);
-      return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
-    }
-
-    private SwitchState parseFrom(byte [] bytes)
-      throws DeserializationException {
-      ProtobufUtil.expectPBMagicPrefix(bytes);
-      SwitchState.Builder builder = SwitchState.newBuilder();
-      try {
-        int magicLen = ProtobufUtil.lengthOfPBMagic();
-        ProtobufUtil.mergeFrom(builder, bytes, magicLen, bytes.length - 
magicLen);
-      } catch (IOException e) {
-        throw new DeserializationException(e);
-      }
-      return builder.build();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java
deleted file mode 100644
index 70e58f8..0000000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java
+++ /dev/null
@@ -1,65 +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.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.yetus.audience.InterfaceAudience;
-
-import java.util.LinkedList;
-import java.util.List;
-
-/**
- * Tool for reading ZooKeeper servers from HBase XML configuration and 
producing
- * a line-by-line list for use by bash scripts.
- */
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
-public class ZKServerTool {
-  public static ServerName[] readZKNodes(Configuration conf) {
-    List<ServerName> hosts = new LinkedList<>();
-    String quorum = conf.get(HConstants.ZOOKEEPER_QUORUM, 
HConstants.LOCALHOST);
-
-    String[] values = quorum.split(",");
-    for (String value : values) {
-      String[] parts = value.split(":");
-      String host = parts[0];
-      int port = HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT;
-      if (parts.length > 1) {
-        port = Integer.parseInt(parts[1]);
-      }
-      hosts.add(ServerName.valueOf(host, port, -1));
-    }
-    return hosts.toArray(new ServerName[hosts.size()]);
-  }
-
-  /**
-   * Run the tool.
-   * @param args Command line arguments.
-   */
-  public static void main(String args[]) {
-    for(ServerName server: readZKNodes(HBaseConfiguration.create())) {
-      // bin/zookeeper.sh relies on the "ZK host" string for grepping which is 
case sensitive.
-      System.out.println("ZK host: " + server.getHostname());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java
deleted file mode 100644
index 1376ba9..0000000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java
+++ /dev/null
@@ -1,124 +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 java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.net.URLDecoder;
-import java.net.URLEncoder;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * Common methods and attributes used by {@link 
org.apache.hadoop.hbase.master.SplitLogManager}
- * and {@link org.apache.hadoop.hbase.regionserver.SplitLogWorker}
- * running distributed splitting of WAL logs.
- */
-@InterfaceAudience.Private
-public class ZKSplitLog {
-  private static final Log LOG = LogFactory.getLog(ZKSplitLog.class);
-
-  /**
-   * Gets the full path node name for the log file being split.
-   * This method will url encode the filename.
-   * @param zkw zk reference
-   * @param filename log file name (only the basename)
-   */
-  public static String getEncodedNodeName(ZooKeeperWatcher zkw, String 
filename) {
-    return ZNodePaths.joinZNode(zkw.znodePaths.splitLogZNode, 
encode(filename));
-  }
-
-  public static String getFileName(String node) {
-    String basename = node.substring(node.lastIndexOf('/') + 1);
-    return decode(basename);
-  }
-
-  static String encode(String s) {
-    try {
-      return URLEncoder.encode(s, "UTF-8");
-    } catch (UnsupportedEncodingException e) {
-      throw new RuntimeException("URLENCODER doesn't support UTF-8");
-    }
-  }
-
-  static String decode(String s) {
-    try {
-      return URLDecoder.decode(s, "UTF-8");
-    } catch (UnsupportedEncodingException e) {
-      throw new RuntimeException("URLDecoder doesn't support UTF-8");
-    }
-  }
-
-  public static String getRescanNode(ZooKeeperWatcher zkw) {
-    return ZNodePaths.joinZNode(zkw.znodePaths.splitLogZNode, "RESCAN");
-  }
-
-  /**
-   * @param name the last part in path
-   * @return whether the node name represents a rescan node
-   */
-  public static boolean isRescanNode(String name) {
-    return name.startsWith("RESCAN");
-  }
-
-  /**
-   * @param zkw
-   * @param path the absolute path, starts with '/'
-   * @return whether the path represents a rescan node
-   */
-  public static boolean isRescanNode(ZooKeeperWatcher zkw, String path) {
-    String prefix = getRescanNode(zkw);
-    if (path.length() <= prefix.length()) {
-      return false;
-    }
-    for (int i = 0; i < prefix.length(); i++) {
-      if (prefix.charAt(i) != path.charAt(i)) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  public static Path getSplitLogDir(Path rootdir, String tmpname) {
-    return new Path(new Path(rootdir, HConstants.SPLIT_LOGDIR_NAME), tmpname);
-  }
-
-  public static void markCorrupted(Path rootdir, String logFileName,
-      FileSystem fs) {
-    Path file = new Path(getSplitLogDir(rootdir, logFileName), "corrupt");
-    try {
-      fs.createNewFile(file);
-    } catch (IOException e) {
-      LOG.warn("Could not flag a log file as corrupted. Failed to create " +
-          file, e);
-    }
-  }
-
-  public static boolean isCorrupted(Path rootdir, String logFileName,
-      FileSystem fs) throws IOException {
-    Path file = new Path(getSplitLogDir(rootdir, logFileName), "corrupt");
-    boolean isCorrupt;
-    isCorrupt = fs.exists(file);
-    return isCorrupt;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMainServer.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMainServer.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMainServer.java
deleted file mode 100644
index 341fbbd..0000000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMainServer.java
+++ /dev/null
@@ -1,126 +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 java.io.IOException;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.curator.shaded.com.google.common.base.Stopwatch;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooKeeperMain;
-
-/**
- * Tool for running ZookeeperMain from HBase by  reading a ZooKeeper server
- * from HBase XML configuration.
- */
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
-public class ZooKeeperMainServer {
-  private static final String SERVER_ARG = "-server";
-
-  public String parse(final Configuration c) {
-    return ZKConfig.getZKQuorumServersString(c);
-  }
-
-  /**
-   * ZooKeeper 3.4.6 broke being able to pass commands on command line.
-   * See ZOOKEEPER-1897.  This class is a hack to restore this faclity.
-   */
-  private static class HACK_UNTIL_ZOOKEEPER_1897_ZooKeeperMain extends 
ZooKeeperMain {
-    public HACK_UNTIL_ZOOKEEPER_1897_ZooKeeperMain(String[] args)
-    throws IOException, InterruptedException {
-      super(args);
-      // Make sure we are connected before we proceed. Can take a while on 
some systems. If we
-      // run the command without being connected, we get ConnectionLoss 
KeeperErrorConnection...
-      Stopwatch stopWatch = Stopwatch.createStarted();
-      while (!this.zk.getState().isConnected()) {
-        Thread.sleep(1);
-        if (stopWatch.elapsed(TimeUnit.SECONDS) > 10) {
-          throw new InterruptedException("Failed connect after waiting " +
-              stopWatch.elapsed(TimeUnit.SECONDS) + "seconds; state=" + 
this.zk.getState() +
-              "; " + this.zk);
-        }
-      }
-    }
-
-    /**
-     * Run the command-line args passed.  Calls System.exit when done.
-     * @throws KeeperException
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    void runCmdLine() throws KeeperException, IOException, 
InterruptedException {
-      processCmd(this.cl);
-      System.exit(0);
-    }
-  }
-
-  /**
-   * @param args
-   * @return True if argument strings have a '-server' in them.
-   */
-  private static boolean hasServer(final String args[]) {
-    return args.length > 0 && args[0].equals(SERVER_ARG);
-  }
-
-  /**
-   * @param args
-   * @return True if command-line arguments were passed.
-   */
-  private static boolean hasCommandLineArguments(final String args[]) {
-    if (hasServer(args)) {
-      if (args.length < 2) throw new IllegalStateException("-server param but 
no value");
-      return args.length > 2;
-    }
-    return args.length > 0;
-  }
-
-  /**
-   * Run the tool.
-   * @param args Command line arguments. First arg is path to zookeepers file.
-   */
-  public static void main(String args[]) throws Exception {
-    String [] newArgs = args;
-    if (!hasServer(args)) {
-      // Add the zk ensemble from configuration if none passed on command-line.
-      Configuration conf = HBaseConfiguration.create();
-      String hostport = new ZooKeeperMainServer().parse(conf);
-      if (hostport != null && hostport.length() > 0) {
-        newArgs = new String[args.length + 2];
-        System.arraycopy(args, 0, newArgs, 2, args.length);
-        newArgs[0] = "-server";
-        newArgs[1] = hostport;
-      }
-    }
-    // If command-line arguments, run our hack so they are executed.
-    // ZOOKEEPER-1897 was committed to zookeeper-3.4.6 but elsewhere in this 
class we say
-    // 3.4.6 breaks command-processing; TODO.
-    if (hasCommandLineArguments(args)) {
-      HACK_UNTIL_ZOOKEEPER_1897_ZooKeeperMain zkm =
-        new HACK_UNTIL_ZOOKEEPER_1897_ZooKeeperMain(newArgs);
-      zkm.runCmdLine();
-    } else {
-      ZooKeeperMain.main(newArgs);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e0c4f374/hbase-server/src/main/resources/hbase-webapps/master/zk.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/zk.jsp 
b/hbase-server/src/main/resources/hbase-webapps/master/zk.jsp
index 4465162..1951617 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/zk.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/zk.jsp
@@ -20,11 +20,11 @@
 <%@ page contentType="text/html;charset=UTF-8"
   import="org.apache.commons.lang3.StringEscapeUtils"
   import="org.apache.hadoop.hbase.zookeeper.ZKUtil"
-  import="org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher"
+  import="org.apache.hadoop.hbase.zookeeper.ZKWatcher"
   import="org.apache.hadoop.hbase.HBaseConfiguration"
   import="org.apache.hadoop.hbase.master.HMaster"%><%
   HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER);
-  ZooKeeperWatcher watcher = master.getZooKeeper();
+  ZKWatcher watcher = master.getZooKeeper();
 %>
 <!DOCTYPE html>
 <?xml version="1.0" encoding="UTF-8" ?>

Reply via email to